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 4b84ee1b22fd..a28585a48472 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.cpp +++ b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.cpp @@ -54,181 +54,6 @@ 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 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 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 { - -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, "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"); -} -} // 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"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -RemoteSplit::RemoteSplit() noexcept { - _type = "$remote"; -} - -void to_json(json& j, const RemoteSplit& p) { - j = json::object(); - 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, RemoteSplit& p) { - p._type = j["@type"]; - from_json_key( - j, "location", p.location, "RemoteSplit", "Location", "location"); - from_json_key( - j, - "remoteSourceTaskId", - p.remoteSourceTaskId, - "RemoteSplit", - "TaskId", - "remoteSourceTaskId"); -} -} // 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. @@ -290,6 +115,12 @@ void to_json(json& j, const std::shared_ptr& p) { j = *std::static_pointer_cast(p); return; } + + if (getConnectorKey(type) == "hive-iceberg") { + j = *std::static_pointer_cast(p); + return; + } + if (getConnectorKey(type) == "tpch") { j = *std::static_pointer_cast(p); return; @@ -325,6 +156,12 @@ void from_json(const json& j, std::shared_ptr& p) { p = k; return; } + if (getConnectorKey(type) == "hive-iceberg") { + 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); @@ -349,4544 +186,4659 @@ void from_json(const json& j, std::shared_ptr& 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 ConnectorInsertTableHandle "); + if (getConnectorKey(type) == "hive-iceberg") { + 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) { +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"); + 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 ConnectorInsertTableHandle "); + if (getConnectorKey(type) == "hive-iceberg") { + 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 +/* + * 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 InsertTableHandle& 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"); -} - -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"); - from_json_key( - j, - "connectorHandle", - p.connectorHandle, - "InsertTableHandle", - "ConnectorInsertTableHandle", - "connectorHandle"); -} -} // 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 -namespace facebook::presto::protocol { -InsertHandle::InsertHandle() noexcept { - _type = "InsertHandle"; -} - -void to_json(json& j, const InsertHandle& p) { - j = json::object(); - 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, InsertHandle& p) { - p._type = j["@type"]; - 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 { - -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"); -} - -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 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); + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); return; } - throw TypeError(type + " no abstract type RowExpression "); -} -} // 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) { + if (getConnectorKey(type) == "hive-iceberg") { + j = *std::static_pointer_cast(p); 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); + j = *std::static_pointer_cast(p); return; } - throw TypeError(type + " no abstract type ConnectorPartitioningHandle"); + 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()) + " ConnectorPartitioningHandle"); + throw ParseError( + std::string(e.what()) + + " ConnectorTableLayoutHandle ConnectorTableLayoutHandle"); } - if (type == "$remote") { - auto k = std::make_shared(); + if (getConnectorKey(type) == "hive") { + auto k = std::make_shared(); j.get_to(*k); p = k; return; } - if (getConnectorKey(type) == "hive") { - auto k = std::make_shared(); + + if (getConnectorKey(type) == "hive-iceberg") { + auto k = std::make_shared(); j.get_to(*k); p = k; return; } + if (getConnectorKey(type) == "tpch") { - auto k = std::make_shared(); + auto k = std::make_shared(); j.get_to(*k); p = k; return; } - throw TypeError(type + " no abstract type ConnectorPartitioningHandle"); + throw TypeError(type + " no abstract type ConnectorTableLayoutHandle"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const PartitioningHandle& p) { +void to_json(json& j, const TableHandle& p) { j = json::object(); to_json_key( j, "connectorId", p.connectorId, - "PartitioningHandle", + "TableHandle", "ConnectorId", "connectorId"); - to_json_key( - j, - "transactionHandle", - p.transactionHandle, - "PartitioningHandle", - "ConnectorTransactionHandle", - "transactionHandle"); to_json_key( j, "connectorHandle", p.connectorHandle, - "PartitioningHandle", - "ConnectorPartitioningHandle", + "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, PartitioningHandle& p) { +void from_json(const json& j, TableHandle& p) { from_json_key( j, "connectorId", p.connectorId, - "PartitioningHandle", + "TableHandle", "ConnectorId", "connectorId"); - from_json_key( - j, - "transactionHandle", - p.transactionHandle, - "PartitioningHandle", - "ConnectorTransactionHandle", - "transactionHandle"); from_json_key( j, "connectorHandle", p.connectorHandle, - "PartitioningHandle", - "ConnectorPartitioningHandle", + "TableHandle", + "ConnectorTableHandle", "connectorHandle"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Partitioning& p) { - j = json::object(); - to_json_key( - j, "handle", p.handle, "Partitioning", "PartitioningHandle", "handle"); - 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"); + j, + "transaction", + p.transaction, + "TableHandle", + "ConnectorTransactionHandle", + "transaction"); from_json_key( j, - "arguments", - p.arguments, - "Partitioning", - "List>", - "arguments"); + "connectorTableLayout", + p.connectorTableLayout, + "TableHandle", + "ConnectorTableLayoutHandle", + "connectorTableLayout"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const DwrfEncryptionMetadata& p) { +void to_json(json& j, const AnalyzeTableHandle& p) { j = json::object(); to_json_key( j, - "fieldToKeyData", - p.fieldToKeyData, - "DwrfEncryptionMetadata", - "Map", - "fieldToKeyData"); - to_json_key( - j, - "extraMetadata", - p.extraMetadata, - "DwrfEncryptionMetadata", - "Map", - "extraMetadata"); + "connectorId", + p.connectorId, + "AnalyzeTableHandle", + "ConnectorId", + "connectorId"); to_json_key( j, - "encryptionAlgorithm", - p.encryptionAlgorithm, - "DwrfEncryptionMetadata", - "String", - "encryptionAlgorithm"); + "transactionHandle", + p.transactionHandle, + "AnalyzeTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); to_json_key( j, - "encryptionProvider", - p.encryptionProvider, - "DwrfEncryptionMetadata", - "String", - "encryptionProvider"); + "connectorHandle", + p.connectorHandle, + "AnalyzeTableHandle", + "ConnectorTableHandle", + "connectorHandle"); } -void from_json(const json& j, DwrfEncryptionMetadata& p) { - from_json_key( - j, - "fieldToKeyData", - p.fieldToKeyData, - "DwrfEncryptionMetadata", - "Map", - "fieldToKeyData"); +void from_json(const json& j, AnalyzeTableHandle& p) { from_json_key( j, - "extraMetadata", - p.extraMetadata, - "DwrfEncryptionMetadata", - "Map", - "extraMetadata"); + "connectorId", + p.connectorId, + "AnalyzeTableHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "encryptionAlgorithm", - p.encryptionAlgorithm, - "DwrfEncryptionMetadata", - "String", - "encryptionAlgorithm"); + "transactionHandle", + p.transactionHandle, + "AnalyzeTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); from_json_key( j, - "encryptionProvider", - p.encryptionProvider, - "DwrfEncryptionMetadata", - "String", - "encryptionProvider"); + "connectorHandle", + p.connectorHandle, + "AnalyzeTableHandle", + "ConnectorTableHandle", + "connectorHandle"); +} +} // 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(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 { +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() -void to_json(json& j, const EncryptionInformation& p) { - j = json::object(); - to_json_key( - j, - "dwrfEncryptionMetadata", - p.dwrfEncryptionMetadata, - "EncryptionInformation", - "DwrfEncryptionMetadata", - "dwrfEncryptionMetadata"); +// 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_assert( + std::is_enum::value, "BlockedReason 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 { + return ej_pair.first == e; + }); + j = ((it != std::end(BlockedReason_enum_table)) + ? it + : std::begin(BlockedReason_enum_table)) + ->second; +} +void from_json(const json& j, BlockedReason& e) { + static_assert( + std::is_enum::value, "BlockedReason 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 { + return ej_pair.second == j; + }); + e = ((it != std::end(BlockedReason_enum_table)) + ? it + : 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. + */ -void from_json(const json& j, EncryptionInformation& p) { - from_json_key( - j, - "dwrfEncryptionMetadata", - p.dwrfEncryptionMetadata, - "EncryptionInformation", - "DwrfEncryptionMetadata", - "dwrfEncryptionMetadata"); +namespace facebook::presto::protocol { + +void to_json(json& j, const Duration& p) { + j = p.toString(); +} + +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 namespace facebook::presto::protocol { -void to_json(json& j, const PageBufferInfo& p) { +void to_json(json& j, const OperatorStats& p) { j = json::object(); - to_json_key( - j, "partition", p.partition, "PageBufferInfo", "int", "partition"); + to_json_key(j, "stageId", p.stageId, "OperatorStats", "int", "stageId"); to_json_key( j, - "bufferedPages", - p.bufferedPages, - "PageBufferInfo", - "int64_t", - "bufferedPages"); + "stageExecutionId", + p.stageExecutionId, + "OperatorStats", + "int", + "stageExecutionId"); to_json_key( - j, - "bufferedBytes", - p.bufferedBytes, - "PageBufferInfo", - "int64_t", - "bufferedBytes"); + j, "pipelineId", p.pipelineId, "OperatorStats", "int", "pipelineId"); to_json_key( - j, "rowsAdded", p.rowsAdded, "PageBufferInfo", "int64_t", "rowsAdded"); + j, "operatorId", p.operatorId, "OperatorStats", "int", "operatorId"); to_json_key( - j, "pagesAdded", p.pagesAdded, "PageBufferInfo", "int64_t", "pagesAdded"); -} - -void from_json(const json& j, PageBufferInfo& p) { - from_json_key( - j, "partition", p.partition, "PageBufferInfo", "int", "partition"); - from_json_key( j, - "bufferedPages", - p.bufferedPages, - "PageBufferInfo", + "planNodeId", + p.planNodeId, + "OperatorStats", + "PlanNodeId", + "planNodeId"); + to_json_key( + j, + "operatorType", + p.operatorType, + "OperatorStats", + "String", + "operatorType"); + to_json_key( + j, + "totalDrivers", + p.totalDrivers, + "OperatorStats", "int64_t", - "bufferedPages"); - from_json_key( + "totalDrivers"); + to_json_key( j, - "bufferedBytes", - p.bufferedBytes, - "PageBufferInfo", + "addInputCalls", + p.addInputCalls, + "OperatorStats", "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 { -LambdaDefinitionExpression::LambdaDefinitionExpression() noexcept { - _type = "lambda"; -} - -void to_json(json& j, const LambdaDefinitionExpression& p) { - j = json::object(); - j["@type"] = "lambda"; + "addInputCalls"); to_json_key( j, - "sourceLocation", - p.sourceLocation, - "LambdaDefinitionExpression", - "SourceLocation", - "sourceLocation"); + "addInputWall", + p.addInputWall, + "OperatorStats", + "Duration", + "addInputWall"); to_json_key( j, - "argumentTypes", - p.argumentTypes, - "LambdaDefinitionExpression", - "List", - "argumentTypes"); + "addInputCpu", + p.addInputCpu, + "OperatorStats", + "Duration", + "addInputCpu"); to_json_key( j, - "arguments", - p.arguments, - "LambdaDefinitionExpression", - "List", - "arguments"); + "addInputAllocation", + p.addInputAllocation, + "OperatorStats", + "DataSize", + "addInputAllocation"); to_json_key( - j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); -} - -void from_json(const json& j, LambdaDefinitionExpression& p) { - p._type = j["@type"]; - from_json_key( j, - "sourceLocation", - p.sourceLocation, - "LambdaDefinitionExpression", - "SourceLocation", - "sourceLocation"); - from_json_key( + "rawInputDataSize", + p.rawInputDataSize, + "OperatorStats", + "DataSize", + "rawInputDataSize"); + to_json_key( j, - "argumentTypes", - p.argumentTypes, - "LambdaDefinitionExpression", - "List", - "argumentTypes"); - from_json_key( + "rawInputPositions", + p.rawInputPositions, + "OperatorStats", + "int64_t", + "rawInputPositions"); + to_json_key( j, - "arguments", - p.arguments, - "LambdaDefinitionExpression", - "List", - "arguments"); - from_json_key( - j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); -} -} // 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; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -VariableReferenceExpression::VariableReferenceExpression() noexcept { - _type = "variable"; -} - -void to_json(json& j, const VariableReferenceExpression& p) { - j = json::object(); - j["@type"] = "variable"; + "inputDataSize", + p.inputDataSize, + "OperatorStats", + "DataSize", + "inputDataSize"); to_json_key( j, - "sourceLocation", - p.sourceLocation, - "VariableReferenceExpression", - "SourceLocation", - "sourceLocation"); + "inputPositions", + p.inputPositions, + "OperatorStats", + "int64_t", + "inputPositions"); to_json_key( - j, "name", p.name, "VariableReferenceExpression", "String", "name"); - to_json_key(j, "type", p.type, "VariableReferenceExpression", "Type", "type"); -} - -void from_json(const json& j, VariableReferenceExpression& p) { - p._type = j["@type"]; - from_json_key( j, - "sourceLocation", - p.sourceLocation, - "VariableReferenceExpression", - "SourceLocation", - "sourceLocation"); - from_json_key( - j, "name", p.name, "VariableReferenceExpression", "String", "name"); - from_json_key( - j, "type", p.type, "VariableReferenceExpression", "Type", "type"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Assignments& p) { - j = json::object(); + "sumSquaredInputPositions", + p.sumSquaredInputPositions, + "OperatorStats", + "double", + "sumSquaredInputPositions"); to_json_key( j, - "assignments", - p.assignments, - "Assignments", - "Map>", - "assignments"); -} - -void from_json(const json& j, Assignments& p) { - from_json_key( + "getOutputCalls", + p.getOutputCalls, + "OperatorStats", + "int64_t", + "getOutputCalls"); + to_json_key( j, - "assignments", - p.assignments, - "Assignments", - "Map>", - "assignments"); -} -} // 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 == ".MarkDistinctNode") { - 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 == ".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.SampleNode") { - 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 == "com.facebook.presto.sql.planner.plan.TableWriterMergeNode") { - 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.TopNRowNumberNode") { - 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 == ".MarkDistinctNode") { - 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 == ".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.SampleNode") { - 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 == "com.facebook.presto.sql.planner.plan.TableWriterMergeNode") { - 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.TopNRowNumberNode") { - 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 { -ProjectNode::ProjectNode() noexcept { - _type = ".ProjectNode"; + "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"); + to_json_key( + j, + "nullJoinBuildKeyCount", + p.nullJoinBuildKeyCount, + "OperatorStats", + "int64_t", + "nullJoinBuildKeyCount"); + to_json_key( + j, + "joinBuildKeyCount", + p.joinBuildKeyCount, + "OperatorStats", + "int64_t", + "joinBuildKeyCount"); + to_json_key( + j, + "nullJoinProbeKeyCount", + p.nullJoinProbeKeyCount, + "OperatorStats", + "int64_t", + "nullJoinProbeKeyCount"); + to_json_key( + j, + "joinProbeKeyCount", + p.joinProbeKeyCount, + "OperatorStats", + "int64_t", + "joinProbeKeyCount"); } -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( +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"); + from_json_key( + j, "operatorId", p.operatorId, "OperatorStats", "int", "operatorId"); + from_json_key( + j, + "planNodeId", + p.planNodeId, + "OperatorStats", + "PlanNodeId", + "planNodeId"); + from_json_key( + j, + "operatorType", + p.operatorType, + "OperatorStats", + "String", + "operatorType"); + from_json_key( + j, + "totalDrivers", + p.totalDrivers, + "OperatorStats", + "int64_t", + "totalDrivers"); + from_json_key( + j, + "addInputCalls", + p.addInputCalls, + "OperatorStats", + "int64_t", + "addInputCalls"); + from_json_key( j, - "assignments", - p.assignments, - "ProjectNode", - "Assignments", - "assignments"); - to_json_key(j, "locality", p.locality, "ProjectNode", "Locality", "locality"); -} - -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"); + "addInputWall", + p.addInputWall, + "OperatorStats", + "Duration", + "addInputWall"); from_json_key( j, - "assignments", - p.assignments, - "ProjectNode", - "Assignments", - "assignments"); + "addInputCpu", + p.addInputCpu, + "OperatorStats", + "Duration", + "addInputCpu"); from_json_key( - j, "locality", p.locality, "ProjectNode", "Locality", "locality"); -} -} // 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 "); -} - -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; - } - - throw TypeError(type + " no abstract type ValueSet "); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Domain& p) { - j = json::object(); - 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, Domain& p) { - from_json_key(j, "values", p.values, "Domain", "ValueSet", "values"); + j, + "addInputAllocation", + p.addInputAllocation, + "OperatorStats", + "DataSize", + "addInputAllocation"); from_json_key( - j, "nullAllowed", p.nullAllowed, "Domain", "bool", "nullAllowed"); -} -} // 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 { - -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 from_json(const json& j, SortingColumn& p) { + j, + "rawInputDataSize", + p.rawInputDataSize, + "OperatorStats", + "DataSize", + "rawInputDataSize"); from_json_key( - j, "columnName", p.columnName, "SortingColumn", "String", "columnName"); - from_json_key(j, "order", p.order, "SortingColumn", "Order", "order"); -} -} // 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"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const MetadataUpdates& p) { - j = json::object(); - to_json_key( j, - "connectorId", - p.connectorId, - "MetadataUpdates", - "ConnectorId", - "connectorId"); - to_json_key( + "rawInputPositions", + p.rawInputPositions, + "OperatorStats", + "int64_t", + "rawInputPositions"); + from_json_key( j, - "metadataUpdates", - p.metadataUpdates, - "MetadataUpdates", - "List>", - "metadataUpdates"); -} - -void from_json(const json& j, MetadataUpdates& p) { + "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, - "connectorId", - p.connectorId, - "MetadataUpdates", - "ConnectorId", - "connectorId"); + "getOutputWall", + p.getOutputWall, + "OperatorStats", + "Duration", + "getOutputWall"); from_json_key( j, - "metadataUpdates", - p.metadataUpdates, - "MetadataUpdates", - "List>", - "metadataUpdates"); -} -} // 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 ColumnHandle "); -} - -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()) + " ColumnHandle ColumnHandle"); - } - - if (getConnectorKey(type) == "hive") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - 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 -namespace facebook::presto::protocol { -TpchTableHandle::TpchTableHandle() noexcept { - _type = "tpch"; -} - -void to_json(json& j, const TpchTableHandle& p) { - j = json::object(); - j["@type"] = "tpch"; - to_json_key( - j, "tableName", p.tableName, "TpchTableHandle", "String", "tableName"); - to_json_key( + "getOutputCpu", + p.getOutputCpu, + "OperatorStats", + "Duration", + "getOutputCpu"); + from_json_key( j, - "scaleFactor", - p.scaleFactor, - "TpchTableHandle", - "double", - "scaleFactor"); -} - -void from_json(const json& j, TpchTableHandle& p) { - p._type = j["@type"]; + "getOutputAllocation", + p.getOutputAllocation, + "OperatorStats", + "DataSize", + "getOutputAllocation"); from_json_key( - j, "tableName", p.tableName, "TpchTableHandle", "String", "tableName"); + j, + "outputDataSize", + p.outputDataSize, + "OperatorStats", + "DataSize", + "outputDataSize"); 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( + "outputPositions", + p.outputPositions, + "OperatorStats", + "int64_t", + "outputPositions"); + from_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.totalParts, "TpchSplit", "int", "totalParts"); - to_json_key( + "physicalWrittenDataSize", + p.physicalWrittenDataSize, + "OperatorStats", + "DataSize", + "physicalWrittenDataSize"); + from_json_key( j, - "addresses", - p.addresses, - "TpchSplit", - "List", - "addresses"); - to_json_key( + "additionalCpu", + p.additionalCpu, + "OperatorStats", + "Duration", + "additionalCpu"); + from_json_key( j, - "predicate", - p.predicate, - "TpchSplit", - "TupleDomain>", - "predicate"); -} - -void from_json(const json& j, TpchSplit& p) { - p._type = j["@type"]; + "blockedWall", + p.blockedWall, + "OperatorStats", + "Duration", + "blockedWall"); from_json_key( j, - "tableHandle", - p.tableHandle, - "TpchSplit", - "TpchTableHandle", - "tableHandle"); + "finishCalls", + p.finishCalls, + "OperatorStats", + "int64_t", + "finishCalls"); from_json_key( - j, "partNumber", p.partNumber, "TpchSplit", "int", "partNumber"); + j, "finishWall", p.finishWall, "OperatorStats", "Duration", "finishWall"); from_json_key( - j, "totalParts", p.totalParts, "TpchSplit", "int", "totalParts"); + j, "finishCpu", p.finishCpu, "OperatorStats", "Duration", "finishCpu"); from_json_key( j, - "addresses", - p.addresses, - "TpchSplit", - "List", - "addresses"); + "finishAllocation", + p.finishAllocation, + "OperatorStats", + "DataSize", + "finishAllocation"); from_json_key( j, - "predicate", - p.predicate, - "TpchSplit", - "TupleDomain>", - "predicate"); -} -} // 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) { + "userMemoryReservation", + p.userMemoryReservation, + "OperatorStats", + "DataSize", + "userMemoryReservation"); from_json_key( - j, "cpuCost", p.cpuCost, "PlanCostEstimate", "double", "cpuCost"); + j, + "revocableMemoryReservation", + p.revocableMemoryReservation, + "OperatorStats", + "DataSize", + "revocableMemoryReservation"); from_json_key( - j, "maxMemory", p.maxMemory, "PlanCostEstimate", "double", "maxMemory"); + j, + "systemMemoryReservation", + p.systemMemoryReservation, + "OperatorStats", + "DataSize", + "systemMemoryReservation"); from_json_key( j, - "maxMemoryWhenOutputting", - p.maxMemoryWhenOutputting, - "PlanCostEstimate", - "double", - "maxMemoryWhenOutputting"); + "peakUserMemoryReservation", + p.peakUserMemoryReservation, + "OperatorStats", + "DataSize", + "peakUserMemoryReservation"); from_json_key( j, - "networkCost", - p.networkCost, - "PlanCostEstimate", - "double", - "networkCost"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const JoinNodeStatsEstimate& p) { - j = json::object(); - to_json_key( + "peakSystemMemoryReservation", + p.peakSystemMemoryReservation, + "OperatorStats", + "DataSize", + "peakSystemMemoryReservation"); + from_json_key( j, - "nullJoinBuildKeyCount", - p.nullJoinBuildKeyCount, - "JoinNodeStatsEstimate", - "double", - "nullJoinBuildKeyCount"); - to_json_key( + "peakTotalMemoryReservation", + p.peakTotalMemoryReservation, + "OperatorStats", + "DataSize", + "peakTotalMemoryReservation"); + from_json_key( j, - "joinBuildKeyCount", - p.joinBuildKeyCount, - "JoinNodeStatsEstimate", - "double", - "joinBuildKeyCount"); - to_json_key( + "spilledDataSize", + p.spilledDataSize, + "OperatorStats", + "DataSize", + "spilledDataSize"); + from_json_key( j, - "nullJoinProbeKeyCount", - p.nullJoinProbeKeyCount, - "JoinNodeStatsEstimate", - "double", - "nullJoinProbeKeyCount"); - to_json_key( + "blockedReason", + p.blockedReason, + "OperatorStats", + "BlockedReason", + "blockedReason"); + from_json_key(j, "info", p.info, "OperatorStats", "OperatorInfo", "info"); + from_json_key( j, - "joinProbeKeyCount", - p.joinProbeKeyCount, - "JoinNodeStatsEstimate", - "double", - "joinProbeKeyCount"); -} - -void from_json(const json& j, JoinNodeStatsEstimate& p) { + "runtimeStats", + p.runtimeStats, + "OperatorStats", + "RuntimeStats", + "runtimeStats"); from_json_key( j, "nullJoinBuildKeyCount", p.nullJoinBuildKeyCount, - "JoinNodeStatsEstimate", - "double", + "OperatorStats", + "int64_t", "nullJoinBuildKeyCount"); from_json_key( j, "joinBuildKeyCount", p.joinBuildKeyCount, - "JoinNodeStatsEstimate", - "double", + "OperatorStats", + "int64_t", "joinBuildKeyCount"); from_json_key( j, "nullJoinProbeKeyCount", p.nullJoinProbeKeyCount, - "JoinNodeStatsEstimate", - "double", + "OperatorStats", + "int64_t", "nullJoinProbeKeyCount"); from_json_key( j, "joinProbeKeyCount", p.joinProbeKeyCount, - "JoinNodeStatsEstimate", - "double", + "OperatorStats", + "int64_t", "joinProbeKeyCount"); } } // 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 VariableStatsEstimate& 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"); -} - -void from_json(const json& j, VariableStatsEstimate& p) { - from_json_key( - j, "lowValue", p.lowValue, "VariableStatsEstimate", "double", "lowValue"); - from_json_key( - j, - "highValue", - p.highValue, - "VariableStatsEstimate", - "double", - "highValue"); - from_json_key( - j, - "nullsFraction", - p.nullsFraction, - "VariableStatsEstimate", - "double", - "nullsFraction"); - from_json_key( - j, - "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 PartialAggregationStatsEstimate& p) { - j = json::object(); - to_json_key( - j, - "inputBytes", - p.inputBytes, - "PartialAggregationStatsEstimate", - "double", - "inputBytes"); - to_json_key( - j, - "outputBytes", - p.outputBytes, - "PartialAggregationStatsEstimate", - "double", - "outputBytes"); - to_json_key( - j, - "inputRowCount", - p.inputRowCount, - "PartialAggregationStatsEstimate", - "double", - "inputRowCount"); - to_json_key( - j, - "outputRowCount", - p.outputRowCount, - "PartialAggregationStatsEstimate", - "double", - "outputRowCount"); +void to_json(json& j, const Lifespan& p) { + if (p.isgroup) { + j = "Group" + std::to_string(p.groupid); + } else { + j = "TaskWide"; + } } -void from_json(const json& j, PartialAggregationStatsEstimate& p) { - from_json_key( - j, - "inputBytes", - p.inputBytes, - "PartialAggregationStatsEstimate", - "double", - "inputBytes"); - from_json_key( - j, - "outputBytes", - p.outputBytes, - "PartialAggregationStatsEstimate", - "double", - "outputBytes"); - from_json_key( - j, - "inputRowCount", - p.inputRowCount, - "PartialAggregationStatsEstimate", - "double", - "inputRowCount"); - from_json_key( - j, - "outputRowCount", - p.outputRowCount, - "PartialAggregationStatsEstimate", - "double", - "outputRowCount"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { +void from_json(const json& j, Lifespan& p) { + String lifespan = j; -void to_json(json& j, const TableWriterNodeStatsEstimate& p) { - j = json::object(); - to_json_key( - j, - "taskCountIfScaledWriter", - p.taskCountIfScaledWriter, - "TableWriterNodeStatsEstimate", - "double", - "taskCountIfScaledWriter"); + if (lifespan == "TaskWide") { + p.isgroup = false; + p.groupid = 0; + } else { + if (lifespan != "Group") { + // fail... + } + p.isgroup = true; + p.groupid = std::stoi(lifespan.substr(strlen("Group"))); + } } -void from_json(const json& j, TableWriterNodeStatsEstimate& p) { - from_json_key( - j, - "taskCountIfScaledWriter", - p.taskCountIfScaledWriter, - "TableWriterNodeStatsEstimate", - "double", - "taskCountIfScaledWriter"); -} } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const PlanNodeStatsEstimate& p) { +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, - "outputRowCount", - p.outputRowCount, - "PlanNodeStatsEstimate", - "double", - "outputRowCount"); + j, "createTime", p.createTime, "DriverStats", "DateTime", "createTime"); to_json_key( - j, - "totalSize", - p.totalSize, - "PlanNodeStatsEstimate", - "double", - "totalSize"); + j, "startTime", p.startTime, "DriverStats", "DateTime", "startTime"); + to_json_key(j, "endTime", p.endTime, "DriverStats", "DateTime", "endTime"); to_json_key( - j, - "confident", - p.confident, - "PlanNodeStatsEstimate", - "bool", - "confident"); + j, "queuedTime", p.queuedTime, "DriverStats", "Duration", "queuedTime"); to_json_key( j, - "variableStatistics", - p.variableStatistics, - "PlanNodeStatsEstimate", - "Map", - "variableStatistics"); + "elapsedTime", + p.elapsedTime, + "DriverStats", + "Duration", + "elapsedTime"); to_json_key( j, - "joinNodeStatsEstimate", - p.joinNodeStatsEstimate, - "PlanNodeStatsEstimate", - "JoinNodeStatsEstimate", - "joinNodeStatsEstimate"); + "userMemoryReservation", + p.userMemoryReservation, + "DriverStats", + "DataSize", + "userMemoryReservation"); to_json_key( j, - "tableWriterNodeStatsEstimate", - p.tableWriterNodeStatsEstimate, - "PlanNodeStatsEstimate", - "TableWriterNodeStatsEstimate", - "tableWriterNodeStatsEstimate"); + "revocableMemoryReservation", + p.revocableMemoryReservation, + "DriverStats", + "DataSize", + "revocableMemoryReservation"); to_json_key( j, - "partialAggregationStatsEstimate", - p.partialAggregationStatsEstimate, - "PlanNodeStatsEstimate", - "PartialAggregationStatsEstimate", - "partialAggregationStatsEstimate"); -} - -void from_json(const json& j, PlanNodeStatsEstimate& p) { - from_json_key( - j, - "outputRowCount", - p.outputRowCount, - "PlanNodeStatsEstimate", - "double", - "outputRowCount"); - from_json_key( - j, - "totalSize", - p.totalSize, - "PlanNodeStatsEstimate", - "double", - "totalSize"); - from_json_key( + "systemMemoryReservation", + p.systemMemoryReservation, + "DriverStats", + "DataSize", + "systemMemoryReservation"); + to_json_key( j, - "confident", - p.confident, - "PlanNodeStatsEstimate", - "bool", - "confident"); - from_json_key( + "totalScheduledTime", + p.totalScheduledTime, + "DriverStats", + "Duration", + "totalScheduledTime"); + to_json_key( j, - "variableStatistics", - p.variableStatistics, - "PlanNodeStatsEstimate", - "Map", - "variableStatistics"); - from_json_key( + "totalCpuTime", + p.totalCpuTime, + "DriverStats", + "Duration", + "totalCpuTime"); + to_json_key( j, - "joinNodeStatsEstimate", - p.joinNodeStatsEstimate, - "PlanNodeStatsEstimate", - "JoinNodeStatsEstimate", - "joinNodeStatsEstimate"); - from_json_key( + "totalBlockedTime", + p.totalBlockedTime, + "DriverStats", + "Duration", + "totalBlockedTime"); + to_json_key( + j, "fullyBlocked", p.fullyBlocked, "DriverStats", "bool", "fullyBlocked"); + to_json_key( j, - "tableWriterNodeStatsEstimate", - p.tableWriterNodeStatsEstimate, - "PlanNodeStatsEstimate", - "TableWriterNodeStatsEstimate", - "tableWriterNodeStatsEstimate"); - from_json_key( + "blockedReasons", + p.blockedReasons, + "DriverStats", + "List", + "blockedReasons"); + to_json_key( j, - "partialAggregationStatsEstimate", - p.partialAggregationStatsEstimate, - "PlanNodeStatsEstimate", - "PartialAggregationStatsEstimate", - "partialAggregationStatsEstimate"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const StatsAndCosts& p) { - j = json::object(); + "totalAllocation", + p.totalAllocation, + "DriverStats", + "DataSize", + "totalAllocation"); to_json_key( j, - "stats", - p.stats, - "StatsAndCosts", - "Map", - "stats"); + "rawInputDataSize", + p.rawInputDataSize, + "DriverStats", + "DataSize", + "rawInputDataSize"); to_json_key( j, - "costs", - p.costs, - "StatsAndCosts", - "Map", - "costs"); -} - -void from_json(const json& j, StatsAndCosts& p) { - from_json_key( + "rawInputPositions", + p.rawInputPositions, + "DriverStats", + "int64_t", + "rawInputPositions"); + to_json_key( j, - "stats", - p.stats, - "StatsAndCosts", - "Map", - "stats"); - from_json_key( + "rawInputReadTime", + p.rawInputReadTime, + "DriverStats", + "Duration", + "rawInputReadTime"); + to_json_key( j, - "costs", - p.costs, - "StatsAndCosts", - "Map", - "costs"); -} -} // 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; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const StageExecutionDescriptor& p) { - j = json::object(); + "processedInputDataSize", + p.processedInputDataSize, + "DriverStats", + "DataSize", + "processedInputDataSize"); to_json_key( j, - "stageExecutionStrategy", - p.stageExecutionStrategy, - "StageExecutionDescriptor", - "StageExecutionStrategy", - "stageExecutionStrategy"); + "processedInputPositions", + p.processedInputPositions, + "DriverStats", + "int64_t", + "processedInputPositions"); to_json_key( j, - "groupedExecutionScanNodes", - p.groupedExecutionScanNodes, - "StageExecutionDescriptor", - "List", - "groupedExecutionScanNodes"); + "outputDataSize", + p.outputDataSize, + "DriverStats", + "DataSize", + "outputDataSize"); to_json_key( j, - "totalLifespans", - p.totalLifespans, - "StageExecutionDescriptor", - "int", - "totalLifespans"); + "outputPositions", + p.outputPositions, + "DriverStats", + "int64_t", + "outputPositions"); + to_json_key( + j, + "physicalWrittenDataSize", + p.physicalWrittenDataSize, + "DriverStats", + "DataSize", + "physicalWrittenDataSize"); + to_json_key( + j, + "operatorStats", + p.operatorStats, + "DriverStats", + "List", + "operatorStats"); } -void from_json(const json& j, StageExecutionDescriptor& p) { +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"); from_json_key( j, - "stageExecutionStrategy", - p.stageExecutionStrategy, - "StageExecutionDescriptor", - "StageExecutionStrategy", - "stageExecutionStrategy"); + "elapsedTime", + p.elapsedTime, + "DriverStats", + "Duration", + "elapsedTime"); from_json_key( j, - "groupedExecutionScanNodes", - p.groupedExecutionScanNodes, - "StageExecutionDescriptor", - "List", - "groupedExecutionScanNodes"); + "userMemoryReservation", + p.userMemoryReservation, + "DriverStats", + "DataSize", + "userMemoryReservation"); from_json_key( j, - "totalLifespans", - p.totalLifespans, - "StageExecutionDescriptor", - "int", - "totalLifespans"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const PartitioningScheme& p) { - j = json::object(); - to_json_key( + "revocableMemoryReservation", + p.revocableMemoryReservation, + "DriverStats", + "DataSize", + "revocableMemoryReservation"); + from_json_key( j, - "partitioning", - p.partitioning, - "PartitioningScheme", - "Partitioning", - "partitioning"); - to_json_key( + "systemMemoryReservation", + p.systemMemoryReservation, + "DriverStats", + "DataSize", + "systemMemoryReservation"); + from_json_key( j, - "outputLayout", - p.outputLayout, - "PartitioningScheme", - "List", - "outputLayout"); - to_json_key( + "totalScheduledTime", + p.totalScheduledTime, + "DriverStats", + "Duration", + "totalScheduledTime"); + from_json_key( j, - "hashColumn", - p.hashColumn, - "PartitioningScheme", - "VariableReferenceExpression", - "hashColumn"); - to_json_key( + "totalCpuTime", + p.totalCpuTime, + "DriverStats", + "Duration", + "totalCpuTime"); + from_json_key( j, - "replicateNullsAndAny", - p.replicateNullsAndAny, - "PartitioningScheme", - "bool", - "replicateNullsAndAny"); - to_json_key( + "totalBlockedTime", + p.totalBlockedTime, + "DriverStats", + "Duration", + "totalBlockedTime"); + from_json_key( + j, "fullyBlocked", p.fullyBlocked, "DriverStats", "bool", "fullyBlocked"); + from_json_key( j, - "bucketToPartition", - p.bucketToPartition, - "PartitioningScheme", - "List", - "bucketToPartition"); -} - -void from_json(const json& j, PartitioningScheme& p) { + "blockedReasons", + p.blockedReasons, + "DriverStats", + "List", + "blockedReasons"); from_json_key( j, - "partitioning", - p.partitioning, - "PartitioningScheme", - "Partitioning", - "partitioning"); + "totalAllocation", + p.totalAllocation, + "DriverStats", + "DataSize", + "totalAllocation"); from_json_key( j, - "outputLayout", - p.outputLayout, - "PartitioningScheme", - "List", - "outputLayout"); + "rawInputDataSize", + p.rawInputDataSize, + "DriverStats", + "DataSize", + "rawInputDataSize"); from_json_key( j, - "hashColumn", - p.hashColumn, - "PartitioningScheme", - "VariableReferenceExpression", - "hashColumn"); + "rawInputPositions", + p.rawInputPositions, + "DriverStats", + "int64_t", + "rawInputPositions"); from_json_key( j, - "replicateNullsAndAny", - p.replicateNullsAndAny, - "PartitioningScheme", - "bool", - "replicateNullsAndAny"); + "rawInputReadTime", + p.rawInputReadTime, + "DriverStats", + "Duration", + "rawInputReadTime"); + from_json_key( + j, + "processedInputDataSize", + p.processedInputDataSize, + "DriverStats", + "DataSize", + "processedInputDataSize"); + from_json_key( + j, + "processedInputPositions", + p.processedInputPositions, + "DriverStats", + "int64_t", + "processedInputPositions"); + from_json_key( + j, + "outputDataSize", + p.outputDataSize, + "DriverStats", + "DataSize", + "outputDataSize"); + from_json_key( + j, + "outputPositions", + p.outputPositions, + "DriverStats", + "int64_t", + "outputPositions"); + from_json_key( + j, + "physicalWrittenDataSize", + p.physicalWrittenDataSize, + "DriverStats", + "DataSize", + "physicalWrittenDataSize"); + from_json_key( + j, + "operatorStats", + p.operatorStats, + "DriverStats", + "List", + "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, - "bucketToPartition", - p.bucketToPartition, - "PartitioningScheme", - "List", - "bucketToPartition"); + 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 -/* - * 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) { +void to_json(json& j, const PipelineStats& 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"); + j, "pipelineId", p.pipelineId, "PipelineStats", "int", "pipelineId"); to_json_key( j, - "partitioning", - p.partitioning, - "PlanFragment", - "PartitioningHandle", - "partitioning"); + "firstStartTime", + p.firstStartTime, + "PipelineStats", + "DateTime", + "firstStartTime"); to_json_key( j, - "tableScanSchedulingOrder", - p.tableScanSchedulingOrder, - "PlanFragment", - "List", - "tableScanSchedulingOrder"); + "lastStartTime", + p.lastStartTime, + "PipelineStats", + "DateTime", + "lastStartTime"); to_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "PlanFragment", - "PartitioningScheme", - "partitioningScheme"); + "lastEndTime", + p.lastEndTime, + "PipelineStats", + "DateTime", + "lastEndTime"); to_json_key( j, - "stageExecutionDescriptor", - p.stageExecutionDescriptor, - "PlanFragment", - "StageExecutionDescriptor", - "stageExecutionDescriptor"); + "inputPipeline", + p.inputPipeline, + "PipelineStats", + "bool", + "inputPipeline"); to_json_key( j, - "outputTableWriterFragment", - p.outputTableWriterFragment, - "PlanFragment", + "outputPipeline", + p.outputPipeline, + "PipelineStats", "bool", - "outputTableWriterFragment"); + "outputPipeline"); 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( + "totalDrivers", + p.totalDrivers, + "PipelineStats", + "int", + "totalDrivers"); + to_json_key( j, - "variables", - p.variables, - "PlanFragment", - "List", - "variables"); - from_json_key( + "queuedDrivers", + p.queuedDrivers, + "PipelineStats", + "int", + "queuedDrivers"); + to_json_key( j, - "partitioning", - p.partitioning, - "PlanFragment", - "PartitioningHandle", - "partitioning"); - from_json_key( + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "PipelineStats", + "int", + "queuedPartitionedDrivers"); + to_json_key( j, - "tableScanSchedulingOrder", - p.tableScanSchedulingOrder, - "PlanFragment", - "List", - "tableScanSchedulingOrder"); - from_json_key( + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "PipelineStats", + "int64_t", + "queuedPartitionedSplitsWeight"); + to_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "PlanFragment", - "PartitioningScheme", - "partitioningScheme"); - from_json_key( + "runningDrivers", + p.runningDrivers, + "PipelineStats", + "int", + "runningDrivers"); + to_json_key( j, - "stageExecutionDescriptor", - p.stageExecutionDescriptor, - "PlanFragment", - "StageExecutionDescriptor", - "stageExecutionDescriptor"); - from_json_key( + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "PipelineStats", + "int", + "runningPartitionedDrivers"); + to_json_key( j, - "outputTableWriterFragment", - p.outputTableWriterFragment, - "PlanFragment", - "bool", - "outputTableWriterFragment"); - from_json_key( + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "PipelineStats", + "int64_t", + "runningPartitionedSplitsWeight"); + to_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 -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(); + "blockedDrivers", + p.blockedDrivers, + "PipelineStats", + "int", + "blockedDrivers"); + to_json_key( + j, + "completedDrivers", + p.completedDrivers, + "PipelineStats", + "int", + "completedDrivers"); + to_json_key( + j, + "userMemoryReservationInBytes", + p.userMemoryReservationInBytes, + "PipelineStats", + "int64_t", + "userMemoryReservationInBytes"); to_json_key( j, - "variable", - p.variable, - "Ordering", - "VariableReferenceExpression", - "variable"); + "revocableMemoryReservationInBytes", + p.revocableMemoryReservationInBytes, + "PipelineStats", + "int64_t", + "revocableMemoryReservationInBytes"); 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(); + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "PipelineStats", + "int64_t", + "systemMemoryReservationInBytes"); 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 std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (type == "$static") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "json_file") { - 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; - } - if (type == "json_file") { - 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"; + j, + "queuedTime", + p.queuedTime, + "PipelineStats", + "DistributionSnapshot", + "queuedTime"); to_json_key( j, - "sourceLocation", - p.sourceLocation, - "CallExpression", - "SourceLocation", - "sourceLocation"); + "elapsedTime", + p.elapsedTime, + "PipelineStats", + "DistributionSnapshot", + "elapsedTime"); to_json_key( j, - "displayName", - p.displayName, - "CallExpression", - "String", - "displayName"); + "totalScheduledTimeInNanos", + p.totalScheduledTimeInNanos, + "PipelineStats", + "int64_t", + "totalScheduledTimeInNanos"); to_json_key( j, - "functionHandle", - p.functionHandle, - "CallExpression", - "FunctionHandle", - "functionHandle"); + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "PipelineStats", + "int64_t", + "totalCpuTimeInNanos"); to_json_key( - j, "returnType", p.returnType, "CallExpression", "Type", "returnType"); + j, + "totalBlockedTimeInNanos", + p.totalBlockedTimeInNanos, + "PipelineStats", + "int64_t", + "totalBlockedTimeInNanos"); 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( + "fullyBlocked", + p.fullyBlocked, + "PipelineStats", + "bool", + "fullyBlocked"); + to_json_key( j, - "sourceLocation", - p.sourceLocation, - "CallExpression", - "SourceLocation", - "sourceLocation"); - from_json_key( + "blockedReasons", + p.blockedReasons, + "PipelineStats", + "List", + "blockedReasons"); + to_json_key( j, - "displayName", - p.displayName, - "CallExpression", - "String", - "displayName"); - from_json_key( + "totalAllocationInBytes", + p.totalAllocationInBytes, + "PipelineStats", + "int64_t", + "totalAllocationInBytes"); + to_json_key( j, - "functionHandle", - p.functionHandle, - "CallExpression", - "FunctionHandle", - "functionHandle"); - from_json_key( - j, "returnType", p.returnType, "CallExpression", "Type", "returnType"); - from_json_key( + "rawInputDataSizeInBytes", + p.rawInputDataSizeInBytes, + "PipelineStats", + "int64_t", + "rawInputDataSizeInBytes"); + to_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(); - to_json_key(j, "call", p.call, "Aggregation", "CallExpression", "call"); + "rawInputPositions", + p.rawInputPositions, + "PipelineStats", + "int64_t", + "rawInputPositions"); to_json_key( j, - "filter", - p.filter, - "Aggregation", - "std::shared_ptr", - "filter"); + "processedInputDataSizeInBytes", + p.processedInputDataSizeInBytes, + "PipelineStats", + "int64_t", + "processedInputDataSizeInBytes"); to_json_key( - j, "orderBy", p.orderBy, "Aggregation", "OrderingScheme", "orderBy"); - to_json_key(j, "distinct", p.distinct, "Aggregation", "bool", "distinct"); + j, + "processedInputPositions", + p.processedInputPositions, + "PipelineStats", + "int64_t", + "processedInputPositions"); to_json_key( - j, "mask", p.mask, "Aggregation", "VariableReferenceExpression", "mask"); + j, + "outputDataSizeInBytes", + p.outputDataSizeInBytes, + "PipelineStats", + "int64_t", + "outputDataSizeInBytes"); to_json_key( j, - "functionHandle", - p.functionHandle, - "Aggregation", - "FunctionHandle", - "functionHandle"); + "outputPositions", + p.outputPositions, + "PipelineStats", + "int64_t", + "outputPositions"); + to_json_key( + j, + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "PipelineStats", + "int64_t", + "physicalWrittenDataSizeInBytes"); + to_json_key( + j, + "operatorSummaries", + p.operatorSummaries, + "PipelineStats", + "List", + "operatorSummaries"); to_json_key( - j, - "arguments", - p.arguments, - "Aggregation", - "List>", - "arguments"); + j, "drivers", p.drivers, "PipelineStats", "List", "drivers"); } -void from_json(const json& j, Aggregation& p) { - from_json_key(j, "call", p.call, "Aggregation", "CallExpression", "call"); +void from_json(const json& j, PipelineStats& p) { + from_json_key( + j, "pipelineId", p.pipelineId, "PipelineStats", "int", "pipelineId"); from_json_key( j, - "filter", - p.filter, - "Aggregation", - "std::shared_ptr", - "filter"); + "firstStartTime", + p.firstStartTime, + "PipelineStats", + "DateTime", + "firstStartTime"); from_json_key( - j, "orderBy", p.orderBy, "Aggregation", "OrderingScheme", "orderBy"); - from_json_key(j, "distinct", p.distinct, "Aggregation", "bool", "distinct"); + j, + "lastStartTime", + p.lastStartTime, + "PipelineStats", + "DateTime", + "lastStartTime"); from_json_key( - j, "mask", p.mask, "Aggregation", "VariableReferenceExpression", "mask"); + j, + "lastEndTime", + p.lastEndTime, + "PipelineStats", + "DateTime", + "lastEndTime"); from_json_key( j, - "functionHandle", - p.functionHandle, - "Aggregation", - "FunctionHandle", - "functionHandle"); + "inputPipeline", + p.inputPipeline, + "PipelineStats", + "bool", + "inputPipeline"); from_json_key( j, - "arguments", - p.arguments, - "Aggregation", - "List>", - "arguments"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const GroupingSetDescriptor& p) { - j = json::object(); - to_json_key( + "outputPipeline", + p.outputPipeline, + "PipelineStats", + "bool", + "outputPipeline"); + from_json_key( j, - "groupingKeys", - p.groupingKeys, - "GroupingSetDescriptor", - "List", - "groupingKeys"); - to_json_key( + "totalDrivers", + p.totalDrivers, + "PipelineStats", + "int", + "totalDrivers"); + from_json_key( j, - "groupingSetCount", - p.groupingSetCount, - "GroupingSetDescriptor", + "queuedDrivers", + p.queuedDrivers, + "PipelineStats", "int", - "groupingSetCount"); - to_json_key( + "queuedDrivers"); + from_json_key( j, - "globalGroupingSets", - p.globalGroupingSets, - "GroupingSetDescriptor", - "List", - "globalGroupingSets"); -} - -void from_json(const json& j, GroupingSetDescriptor& p) { + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "PipelineStats", + "int", + "queuedPartitionedDrivers"); from_json_key( j, - "groupingKeys", - p.groupingKeys, - "GroupingSetDescriptor", - "List", - "groupingKeys"); + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "PipelineStats", + "int64_t", + "queuedPartitionedSplitsWeight"); from_json_key( j, - "groupingSetCount", - p.groupingSetCount, - "GroupingSetDescriptor", + "runningDrivers", + p.runningDrivers, + "PipelineStats", "int", - "groupingSetCount"); + "runningDrivers"); from_json_key( j, - "globalGroupingSets", - p.globalGroupingSets, - "GroupingSetDescriptor", - "List", - "globalGroupingSets"); -} -} // 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 { -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( - j, - "aggregations", - p.aggregations, - "AggregationNode", - "Map", - "aggregations"); - to_json_key( + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "PipelineStats", + "int", + "runningPartitionedDrivers"); + from_json_key( j, - "groupingSets", - p.groupingSets, - "AggregationNode", - "GroupingSetDescriptor", - "groupingSets"); - to_json_key( + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "PipelineStats", + "int64_t", + "runningPartitionedSplitsWeight"); + from_json_key( j, - "preGroupedVariables", - p.preGroupedVariables, - "AggregationNode", - "List", - "preGroupedVariables"); - to_json_key( - j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); - to_json_key( + "blockedDrivers", + p.blockedDrivers, + "PipelineStats", + "int", + "blockedDrivers"); + from_json_key( j, - "hashVariable", - p.hashVariable, - "AggregationNode", - "VariableReferenceExpression", - "hashVariable"); - to_json_key( + "completedDrivers", + p.completedDrivers, + "PipelineStats", + "int", + "completedDrivers"); + from_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "AggregationNode", - "VariableReferenceExpression", - "groupIdVariable"); - to_json_key( + "userMemoryReservationInBytes", + p.userMemoryReservationInBytes, + "PipelineStats", + "int64_t", + "userMemoryReservationInBytes"); + from_json_key( j, - "aggregationId", - p.aggregationId, - "AggregationNode", - "Integer", - "aggregationId"); -} - -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"); + "revocableMemoryReservationInBytes", + p.revocableMemoryReservationInBytes, + "PipelineStats", + "int64_t", + "revocableMemoryReservationInBytes"); from_json_key( j, - "aggregations", - p.aggregations, - "AggregationNode", - "Map", - "aggregations"); + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "PipelineStats", + "int64_t", + "systemMemoryReservationInBytes"); from_json_key( j, - "groupingSets", - p.groupingSets, - "AggregationNode", - "GroupingSetDescriptor", - "groupingSets"); + "queuedTime", + p.queuedTime, + "PipelineStats", + "DistributionSnapshot", + "queuedTime"); from_json_key( j, - "preGroupedVariables", - p.preGroupedVariables, - "AggregationNode", - "List", - "preGroupedVariables"); + "elapsedTime", + p.elapsedTime, + "PipelineStats", + "DistributionSnapshot", + "elapsedTime"); from_json_key( - j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); + j, + "totalScheduledTimeInNanos", + p.totalScheduledTimeInNanos, + "PipelineStats", + "int64_t", + "totalScheduledTimeInNanos"); from_json_key( j, - "hashVariable", - p.hashVariable, - "AggregationNode", - "VariableReferenceExpression", - "hashVariable"); + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "PipelineStats", + "int64_t", + "totalCpuTimeInNanos"); from_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "AggregationNode", - "VariableReferenceExpression", - "groupIdVariable"); + "totalBlockedTimeInNanos", + p.totalBlockedTimeInNanos, + "PipelineStats", + "int64_t", + "totalBlockedTimeInNanos"); from_json_key( j, - "aggregationId", - p.aggregationId, - "AggregationNode", - "Integer", - "aggregationId"); -} -} // 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_assert( - std::is_enum::value, "BlockedReason 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 { - return ej_pair.first == e; - }); - j = ((it != std::end(BlockedReason_enum_table)) - ? it - : std::begin(BlockedReason_enum_table)) - ->second; -} -void from_json(const json& j, BlockedReason& e) { - static_assert( - std::is_enum::value, "BlockedReason 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 { - return ej_pair.second == j; - }); - e = ((it != std::end(BlockedReason_enum_table)) - ? it - : std::begin(BlockedReason_enum_table)) - ->first; -} -} // 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) { + "fullyBlocked", + p.fullyBlocked, + "PipelineStats", + "bool", + "fullyBlocked"); 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"); -} -} // 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 Lifespan& p) { - if (p.isgroup) { - j = "Group" + std::to_string(p.groupid); - } else { - j = "TaskWide"; - } -} - -void from_json(const json& j, Lifespan& p) { - String lifespan = j; - - if (lifespan == "TaskWide") { - p.isgroup = false; - p.groupid = 0; - } else { - if (lifespan != "Group") { - // fail... - } - p.isgroup = true; - p.groupid = std::stoi(lifespan.substr(strlen("Group"))); - } -} - -} // 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 Duration& p) { - j = p.toString(); -} - -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(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(); + j, + "blockedReasons", + p.blockedReasons, + "PipelineStats", + "List", + "blockedReasons"); + from_json_key( + j, + "totalAllocationInBytes", + p.totalAllocationInBytes, + "PipelineStats", + "int64_t", + "totalAllocationInBytes"); + from_json_key( + j, + "rawInputDataSizeInBytes", + p.rawInputDataSizeInBytes, + "PipelineStats", + "int64_t", + "rawInputDataSizeInBytes"); + from_json_key( + j, + "rawInputPositions", + p.rawInputPositions, + "PipelineStats", + "int64_t", + "rawInputPositions"); + from_json_key( + j, + "processedInputDataSizeInBytes", + p.processedInputDataSizeInBytes, + "PipelineStats", + "int64_t", + "processedInputDataSizeInBytes"); + from_json_key( + j, + "processedInputPositions", + p.processedInputPositions, + "PipelineStats", + "int64_t", + "processedInputPositions"); + from_json_key( + j, + "outputDataSizeInBytes", + p.outputDataSizeInBytes, + "PipelineStats", + "int64_t", + "outputDataSizeInBytes"); + from_json_key( + j, + "outputPositions", + p.outputPositions, + "PipelineStats", + "int64_t", + "outputPositions"); + from_json_key( + j, + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "PipelineStats", + "int64_t", + "physicalWrittenDataSizeInBytes"); + from_json_key( + j, + "operatorSummaries", + p.operatorSummaries, + "PipelineStats", + "List", + "operatorSummaries"); + from_json_key( + j, "drivers", p.drivers, "PipelineStats", "List", "drivers"); } - -} // 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 { -void to_json(json& j, const OperatorStats& p) { +void to_json(json& j, const TaskStats& p) { j = json::object(); - to_json_key(j, "stageId", p.stageId, "OperatorStats", "int", "stageId"); + to_json_key( + j, "createTime", p.createTime, "TaskStats", "DateTime", "createTime"); to_json_key( j, - "stageExecutionId", - p.stageExecutionId, - "OperatorStats", - "int", - "stageExecutionId"); + "firstStartTime", + p.firstStartTime, + "TaskStats", + "DateTime", + "firstStartTime"); to_json_key( - j, "pipelineId", p.pipelineId, "OperatorStats", "int", "pipelineId"); + j, + "lastStartTime", + p.lastStartTime, + "TaskStats", + "DateTime", + "lastStartTime"); to_json_key( - j, "operatorId", p.operatorId, "OperatorStats", "int", "operatorId"); + j, "lastEndTime", p.lastEndTime, "TaskStats", "DateTime", "lastEndTime"); + to_json_key(j, "endTime", p.endTime, "TaskStats", "DateTime", "endTime"); to_json_key( j, - "planNodeId", - p.planNodeId, - "OperatorStats", - "PlanNodeId", - "planNodeId"); + "elapsedTimeInNanos", + p.elapsedTimeInNanos, + "TaskStats", + "int64_t", + "elapsedTimeInNanos"); to_json_key( j, - "operatorType", - p.operatorType, - "OperatorStats", - "String", - "operatorType"); + "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, - "totalDrivers", - p.totalDrivers, - "OperatorStats", - "int64_t", - "totalDrivers"); + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "TaskStats", + "int", + "queuedPartitionedDrivers"); to_json_key( j, - "addInputCalls", - p.addInputCalls, - "OperatorStats", + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "TaskStats", "int64_t", - "addInputCalls"); + "queuedPartitionedSplitsWeight"); to_json_key( j, - "addInputWall", - p.addInputWall, - "OperatorStats", - "Duration", - "addInputWall"); + "runningDrivers", + p.runningDrivers, + "TaskStats", + "int", + "runningDrivers"); to_json_key( j, - "addInputCpu", - p.addInputCpu, - "OperatorStats", - "Duration", - "addInputCpu"); + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "TaskStats", + "int", + "runningPartitionedDrivers"); to_json_key( j, - "addInputAllocation", - p.addInputAllocation, - "OperatorStats", - "DataSize", - "addInputAllocation"); + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "TaskStats", + "int64_t", + "runningPartitionedSplitsWeight"); to_json_key( j, - "rawInputDataSize", - p.rawInputDataSize, - "OperatorStats", - "DataSize", - "rawInputDataSize"); + "blockedDrivers", + p.blockedDrivers, + "TaskStats", + "int", + "blockedDrivers"); to_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "OperatorStats", - "int64_t", - "rawInputPositions"); + "completedDrivers", + p.completedDrivers, + "TaskStats", + "int", + "completedDrivers"); to_json_key( j, - "inputDataSize", - p.inputDataSize, - "OperatorStats", - "DataSize", - "inputDataSize"); + "cumulativeUserMemory", + p.cumulativeUserMemory, + "TaskStats", + "double", + "cumulativeUserMemory"); to_json_key( j, - "inputPositions", - p.inputPositions, - "OperatorStats", + "cumulativeTotalMemory", + p.cumulativeTotalMemory, + "TaskStats", + "double", + "cumulativeTotalMemory"); + to_json_key( + j, + "userMemoryReservationInBytes", + p.userMemoryReservationInBytes, + "TaskStats", "int64_t", - "inputPositions"); + "userMemoryReservationInBytes"); to_json_key( j, - "sumSquaredInputPositions", - p.sumSquaredInputPositions, - "OperatorStats", - "double", - "sumSquaredInputPositions"); + "revocableMemoryReservationInBytes", + p.revocableMemoryReservationInBytes, + "TaskStats", + "int64_t", + "revocableMemoryReservationInBytes"); to_json_key( j, - "getOutputCalls", - p.getOutputCalls, - "OperatorStats", + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "TaskStats", "int64_t", - "getOutputCalls"); + "systemMemoryReservationInBytes"); to_json_key( j, - "getOutputWall", - p.getOutputWall, - "OperatorStats", - "Duration", - "getOutputWall"); + "peakTotalMemoryInBytes", + p.peakTotalMemoryInBytes, + "TaskStats", + "int64_t", + "peakTotalMemoryInBytes"); to_json_key( j, - "getOutputCpu", - p.getOutputCpu, - "OperatorStats", - "Duration", - "getOutputCpu"); + "peakUserMemoryInBytes", + p.peakUserMemoryInBytes, + "TaskStats", + "int64_t", + "peakUserMemoryInBytes"); to_json_key( j, - "getOutputAllocation", - p.getOutputAllocation, - "OperatorStats", - "DataSize", - "getOutputAllocation"); + "peakNodeTotalMemoryInBytes", + p.peakNodeTotalMemoryInBytes, + "TaskStats", + "int64_t", + "peakNodeTotalMemoryInBytes"); to_json_key( j, - "outputDataSize", - p.outputDataSize, - "OperatorStats", - "DataSize", - "outputDataSize"); + "totalScheduledTimeInNanos", + p.totalScheduledTimeInNanos, + "TaskStats", + "int64_t", + "totalScheduledTimeInNanos"); to_json_key( j, - "outputPositions", - p.outputPositions, - "OperatorStats", + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "TaskStats", "int64_t", - "outputPositions"); + "totalCpuTimeInNanos"); to_json_key( j, - "physicalWrittenDataSize", - p.physicalWrittenDataSize, - "OperatorStats", - "DataSize", - "physicalWrittenDataSize"); + "totalBlockedTimeInNanos", + p.totalBlockedTimeInNanos, + "TaskStats", + "int64_t", + "totalBlockedTimeInNanos"); to_json_key( - j, - "additionalCpu", - p.additionalCpu, - "OperatorStats", - "Duration", - "additionalCpu"); + j, "fullyBlocked", p.fullyBlocked, "TaskStats", "bool", "fullyBlocked"); to_json_key( j, - "blockedWall", - p.blockedWall, - "OperatorStats", - "Duration", - "blockedWall"); + "blockedReasons", + p.blockedReasons, + "TaskStats", + "List", + "blockedReasons"); to_json_key( j, - "finishCalls", - p.finishCalls, - "OperatorStats", + "totalAllocationInBytes", + p.totalAllocationInBytes, + "TaskStats", "int64_t", - "finishCalls"); - to_json_key( - j, "finishWall", p.finishWall, "OperatorStats", "Duration", "finishWall"); - to_json_key( - j, "finishCpu", p.finishCpu, "OperatorStats", "Duration", "finishCpu"); + "totalAllocationInBytes"); to_json_key( j, - "finishAllocation", - p.finishAllocation, - "OperatorStats", - "DataSize", - "finishAllocation"); + "rawInputDataSizeInBytes", + p.rawInputDataSizeInBytes, + "TaskStats", + "int64_t", + "rawInputDataSizeInBytes"); to_json_key( j, - "userMemoryReservation", - p.userMemoryReservation, - "OperatorStats", - "DataSize", - "userMemoryReservation"); + "rawInputPositions", + p.rawInputPositions, + "TaskStats", + "int64_t", + "rawInputPositions"); to_json_key( j, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "OperatorStats", - "DataSize", - "revocableMemoryReservation"); + "processedInputDataSizeInBytes", + p.processedInputDataSizeInBytes, + "TaskStats", + "int64_t", + "processedInputDataSizeInBytes"); to_json_key( j, - "systemMemoryReservation", - p.systemMemoryReservation, - "OperatorStats", - "DataSize", - "systemMemoryReservation"); + "processedInputPositions", + p.processedInputPositions, + "TaskStats", + "int64_t", + "processedInputPositions"); to_json_key( j, - "peakUserMemoryReservation", - p.peakUserMemoryReservation, - "OperatorStats", - "DataSize", - "peakUserMemoryReservation"); + "outputDataSizeInBytes", + p.outputDataSizeInBytes, + "TaskStats", + "int64_t", + "outputDataSizeInBytes"); to_json_key( j, - "peakSystemMemoryReservation", - p.peakSystemMemoryReservation, - "OperatorStats", - "DataSize", - "peakSystemMemoryReservation"); + "outputPositions", + p.outputPositions, + "TaskStats", + "int64_t", + "outputPositions"); to_json_key( j, - "peakTotalMemoryReservation", - p.peakTotalMemoryReservation, - "OperatorStats", - "DataSize", - "peakTotalMemoryReservation"); + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "TaskStats", + "int64_t", + "physicalWrittenDataSizeInBytes"); + to_json_key( + j, "fullGcCount", p.fullGcCount, "TaskStats", "int", "fullGcCount"); to_json_key( j, - "spilledDataSize", - p.spilledDataSize, - "OperatorStats", - "DataSize", - "spilledDataSize"); + "fullGcTimeInMillis", + p.fullGcTimeInMillis, + "TaskStats", + "int64_t", + "fullGcTimeInMillis"); to_json_key( j, - "blockedReason", - p.blockedReason, - "OperatorStats", - "BlockedReason", - "blockedReason"); - to_json_key(j, "info", p.info, "OperatorStats", "OperatorInfo", "info"); + "pipelines", + p.pipelines, + "TaskStats", + "List", + "pipelines"); to_json_key( j, "runtimeStats", p.runtimeStats, - "OperatorStats", + "TaskStats", "RuntimeStats", "runtimeStats"); - to_json_key( +} + +void from_json(const json& j, TaskStats& p) { + from_json_key( + j, "createTime", p.createTime, "TaskStats", "DateTime", "createTime"); + from_json_key( j, - "nullJoinBuildKeyCount", - p.nullJoinBuildKeyCount, - "OperatorStats", + "firstStartTime", + p.firstStartTime, + "TaskStats", + "DateTime", + "firstStartTime"); + from_json_key( + j, + "lastStartTime", + p.lastStartTime, + "TaskStats", + "DateTime", + "lastStartTime"); + from_json_key( + j, "lastEndTime", p.lastEndTime, "TaskStats", "DateTime", "lastEndTime"); + from_json_key(j, "endTime", p.endTime, "TaskStats", "DateTime", "endTime"); + from_json_key( + j, + "elapsedTimeInNanos", + p.elapsedTimeInNanos, + "TaskStats", "int64_t", - "nullJoinBuildKeyCount"); - to_json_key( + "elapsedTimeInNanos"); + from_json_key( j, - "joinBuildKeyCount", - p.joinBuildKeyCount, - "OperatorStats", + "queuedTimeInNanos", + p.queuedTimeInNanos, + "TaskStats", "int64_t", - "joinBuildKeyCount"); - to_json_key( + "queuedTimeInNanos"); + from_json_key( + j, "totalDrivers", p.totalDrivers, "TaskStats", "int", "totalDrivers"); + from_json_key( + j, "queuedDrivers", p.queuedDrivers, "TaskStats", "int", "queuedDrivers"); + from_json_key( j, - "nullJoinProbeKeyCount", - p.nullJoinProbeKeyCount, - "OperatorStats", + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "TaskStats", + "int", + "queuedPartitionedDrivers"); + from_json_key( + j, + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "TaskStats", "int64_t", - "nullJoinProbeKeyCount"); - to_json_key( + "queuedPartitionedSplitsWeight"); + from_json_key( j, - "joinProbeKeyCount", - p.joinProbeKeyCount, - "OperatorStats", + "runningDrivers", + p.runningDrivers, + "TaskStats", + "int", + "runningDrivers"); + from_json_key( + j, + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "TaskStats", + "int", + "runningPartitionedDrivers"); + from_json_key( + j, + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "TaskStats", "int64_t", - "joinProbeKeyCount"); -} - -void from_json(const json& j, OperatorStats& p) { - from_json_key(j, "stageId", p.stageId, "OperatorStats", "int", "stageId"); + "runningPartitionedSplitsWeight"); + from_json_key( + j, + "blockedDrivers", + p.blockedDrivers, + "TaskStats", + "int", + "blockedDrivers"); from_json_key( j, - "stageExecutionId", - p.stageExecutionId, - "OperatorStats", + "completedDrivers", + p.completedDrivers, + "TaskStats", "int", - "stageExecutionId"); - from_json_key( - j, "pipelineId", p.pipelineId, "OperatorStats", "int", "pipelineId"); - from_json_key( - j, "operatorId", p.operatorId, "OperatorStats", "int", "operatorId"); + "completedDrivers"); from_json_key( j, - "planNodeId", - p.planNodeId, - "OperatorStats", - "PlanNodeId", - "planNodeId"); + "cumulativeUserMemory", + p.cumulativeUserMemory, + "TaskStats", + "double", + "cumulativeUserMemory"); from_json_key( j, - "operatorType", - p.operatorType, - "OperatorStats", - "String", - "operatorType"); + "cumulativeTotalMemory", + p.cumulativeTotalMemory, + "TaskStats", + "double", + "cumulativeTotalMemory"); from_json_key( j, - "totalDrivers", - p.totalDrivers, - "OperatorStats", + "userMemoryReservationInBytes", + p.userMemoryReservationInBytes, + "TaskStats", "int64_t", - "totalDrivers"); + "userMemoryReservationInBytes"); from_json_key( j, - "addInputCalls", - p.addInputCalls, - "OperatorStats", + "revocableMemoryReservationInBytes", + p.revocableMemoryReservationInBytes, + "TaskStats", "int64_t", - "addInputCalls"); + "revocableMemoryReservationInBytes"); from_json_key( j, - "addInputWall", - p.addInputWall, - "OperatorStats", - "Duration", - "addInputWall"); + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "TaskStats", + "int64_t", + "systemMemoryReservationInBytes"); from_json_key( j, - "addInputCpu", - p.addInputCpu, - "OperatorStats", - "Duration", - "addInputCpu"); + "peakTotalMemoryInBytes", + p.peakTotalMemoryInBytes, + "TaskStats", + "int64_t", + "peakTotalMemoryInBytes"); from_json_key( j, - "addInputAllocation", - p.addInputAllocation, - "OperatorStats", - "DataSize", - "addInputAllocation"); + "peakUserMemoryInBytes", + p.peakUserMemoryInBytes, + "TaskStats", + "int64_t", + "peakUserMemoryInBytes"); from_json_key( j, - "rawInputDataSize", - p.rawInputDataSize, - "OperatorStats", - "DataSize", - "rawInputDataSize"); + "peakNodeTotalMemoryInBytes", + p.peakNodeTotalMemoryInBytes, + "TaskStats", + "int64_t", + "peakNodeTotalMemoryInBytes"); from_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "OperatorStats", + "totalScheduledTimeInNanos", + p.totalScheduledTimeInNanos, + "TaskStats", "int64_t", - "rawInputPositions"); + "totalScheduledTimeInNanos"); from_json_key( j, - "inputDataSize", - p.inputDataSize, - "OperatorStats", - "DataSize", - "inputDataSize"); + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "TaskStats", + "int64_t", + "totalCpuTimeInNanos"); from_json_key( j, - "inputPositions", - p.inputPositions, - "OperatorStats", + "totalBlockedTimeInNanos", + p.totalBlockedTimeInNanos, + "TaskStats", "int64_t", - "inputPositions"); + "totalBlockedTimeInNanos"); + from_json_key( + j, "fullyBlocked", p.fullyBlocked, "TaskStats", "bool", "fullyBlocked"); from_json_key( j, - "sumSquaredInputPositions", - p.sumSquaredInputPositions, - "OperatorStats", - "double", - "sumSquaredInputPositions"); + "blockedReasons", + p.blockedReasons, + "TaskStats", + "List", + "blockedReasons"); from_json_key( j, - "getOutputCalls", - p.getOutputCalls, - "OperatorStats", + "totalAllocationInBytes", + p.totalAllocationInBytes, + "TaskStats", "int64_t", - "getOutputCalls"); + "totalAllocationInBytes"); from_json_key( j, - "getOutputWall", - p.getOutputWall, - "OperatorStats", - "Duration", - "getOutputWall"); + "rawInputDataSizeInBytes", + p.rawInputDataSizeInBytes, + "TaskStats", + "int64_t", + "rawInputDataSizeInBytes"); from_json_key( j, - "getOutputCpu", - p.getOutputCpu, - "OperatorStats", - "Duration", - "getOutputCpu"); + "rawInputPositions", + p.rawInputPositions, + "TaskStats", + "int64_t", + "rawInputPositions"); from_json_key( j, - "getOutputAllocation", - p.getOutputAllocation, - "OperatorStats", - "DataSize", - "getOutputAllocation"); + "processedInputDataSizeInBytes", + p.processedInputDataSizeInBytes, + "TaskStats", + "int64_t", + "processedInputDataSizeInBytes"); from_json_key( j, - "outputDataSize", - p.outputDataSize, - "OperatorStats", - "DataSize", - "outputDataSize"); + "processedInputPositions", + p.processedInputPositions, + "TaskStats", + "int64_t", + "processedInputPositions"); + from_json_key( + j, + "outputDataSizeInBytes", + p.outputDataSizeInBytes, + "TaskStats", + "int64_t", + "outputDataSizeInBytes"); from_json_key( j, "outputPositions", p.outputPositions, - "OperatorStats", + "TaskStats", "int64_t", "outputPositions"); from_json_key( j, - "physicalWrittenDataSize", - p.physicalWrittenDataSize, - "OperatorStats", - "DataSize", - "physicalWrittenDataSize"); + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "TaskStats", + "int64_t", + "physicalWrittenDataSizeInBytes"); + from_json_key( + j, "fullGcCount", p.fullGcCount, "TaskStats", "int", "fullGcCount"); from_json_key( j, - "additionalCpu", - p.additionalCpu, - "OperatorStats", - "Duration", - "additionalCpu"); + "fullGcTimeInMillis", + p.fullGcTimeInMillis, + "TaskStats", + "int64_t", + "fullGcTimeInMillis"); from_json_key( j, - "blockedWall", - p.blockedWall, - "OperatorStats", - "Duration", - "blockedWall"); + "pipelines", + p.pipelines, + "TaskStats", + "List", + "pipelines"); from_json_key( j, - "finishCalls", - p.finishCalls, - "OperatorStats", - "int64_t", - "finishCalls"); - from_json_key( - j, "finishWall", p.finishWall, "OperatorStats", "Duration", "finishWall"); + "runtimeStats", + p.runtimeStats, + "TaskStats", + "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 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 { +// 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 { + +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, "finishCpu", p.finishCpu, "OperatorStats", "Duration", "finishCpu"); + j, "lineNumber", p.lineNumber, "ErrorLocation", "int", "lineNumber"); from_json_key( j, - "finishAllocation", - p.finishAllocation, - "OperatorStats", - "DataSize", - "finishAllocation"); - from_json_key( + "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 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, "message", p.message, "ExecutionFailureInfo", "String", "message"); + to_json_key( j, - "userMemoryReservation", - p.userMemoryReservation, - "OperatorStats", - "DataSize", - "userMemoryReservation"); - from_json_key( + "cause", + p.cause, + "ExecutionFailureInfo", + "ExecutionFailureInfo", + "cause"); + to_json_key( j, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "OperatorStats", - "DataSize", - "revocableMemoryReservation"); - from_json_key( + "suppressed", + p.suppressed, + "ExecutionFailureInfo", + "List", + "suppressed"); + to_json_key( + j, "stack", p.stack, "ExecutionFailureInfo", "List", "stack"); + to_json_key( j, - "systemMemoryReservation", - p.systemMemoryReservation, - "OperatorStats", - "DataSize", - "systemMemoryReservation"); - from_json_key( + "errorLocation", + p.errorLocation, + "ExecutionFailureInfo", + "ErrorLocation", + "errorLocation"); + to_json_key( j, - "peakUserMemoryReservation", - p.peakUserMemoryReservation, - "OperatorStats", - "DataSize", - "peakUserMemoryReservation"); - from_json_key( + "errorCode", + p.errorCode, + "ExecutionFailureInfo", + "ErrorCode", + "errorCode"); + to_json_key( j, - "peakSystemMemoryReservation", - p.peakSystemMemoryReservation, - "OperatorStats", - "DataSize", - "peakSystemMemoryReservation"); - from_json_key( + "remoteHost", + p.remoteHost, + "ExecutionFailureInfo", + "HostAddress", + "remoteHost"); + to_json_key( j, - "peakTotalMemoryReservation", - p.peakTotalMemoryReservation, - "OperatorStats", - "DataSize", - "peakTotalMemoryReservation"); + "errorCause", + p.errorCause, + "ExecutionFailureInfo", + "ErrorCause", + "errorCause"); +} + +void from_json(const json& j, ExecutionFailureInfo& p) { + from_json_key(j, "type", p.type, "ExecutionFailureInfo", "String", "type"); from_json_key( - j, - "spilledDataSize", - p.spilledDataSize, - "OperatorStats", - "DataSize", - "spilledDataSize"); + j, "message", p.message, "ExecutionFailureInfo", "String", "message"); from_json_key( j, - "blockedReason", - p.blockedReason, - "OperatorStats", - "BlockedReason", - "blockedReason"); - from_json_key(j, "info", p.info, "OperatorStats", "OperatorInfo", "info"); + "cause", + p.cause, + "ExecutionFailureInfo", + "ExecutionFailureInfo", + "cause"); from_json_key( j, - "runtimeStats", - p.runtimeStats, - "OperatorStats", - "RuntimeStats", - "runtimeStats"); + "suppressed", + p.suppressed, + "ExecutionFailureInfo", + "List", + "suppressed"); from_json_key( - j, - "nullJoinBuildKeyCount", - p.nullJoinBuildKeyCount, - "OperatorStats", - "int64_t", - "nullJoinBuildKeyCount"); + j, "stack", p.stack, "ExecutionFailureInfo", "List", "stack"); from_json_key( j, - "joinBuildKeyCount", - p.joinBuildKeyCount, - "OperatorStats", - "int64_t", - "joinBuildKeyCount"); + "errorLocation", + p.errorLocation, + "ExecutionFailureInfo", + "ErrorLocation", + "errorLocation"); from_json_key( j, - "nullJoinProbeKeyCount", - p.nullJoinProbeKeyCount, - "OperatorStats", - "int64_t", - "nullJoinProbeKeyCount"); + "errorCode", + p.errorCode, + "ExecutionFailureInfo", + "ErrorCode", + "errorCode"); from_json_key( j, - "joinProbeKeyCount", - p.joinProbeKeyCount, - "OperatorStats", - "int64_t", - "joinProbeKeyCount"); + "remoteHost", + p.remoteHost, + "ExecutionFailureInfo", + "HostAddress", + "remoteHost"); + from_json_key( + j, + "errorCause", + p.errorCause, + "ExecutionFailureInfo", + "ErrorCause", + "errorCause"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const DriverStats& p) { +void to_json(json& j, const TaskStatus& 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"); + "taskInstanceIdLeastSignificantBits", + p.taskInstanceIdLeastSignificantBits, + "TaskStatus", + "int64_t", + "taskInstanceIdLeastSignificantBits"); to_json_key( j, - "userMemoryReservation", - p.userMemoryReservation, - "DriverStats", - "DataSize", - "userMemoryReservation"); + "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, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "DriverStats", - "DataSize", - "revocableMemoryReservation"); + "completedDriverGroups", + p.completedDriverGroups, + "TaskStatus", + "List", + "completedDriverGroups"); to_json_key( j, - "systemMemoryReservation", - p.systemMemoryReservation, - "DriverStats", - "DataSize", - "systemMemoryReservation"); + "failures", + p.failures, + "TaskStatus", + "List", + "failures"); to_json_key( j, - "totalScheduledTime", - p.totalScheduledTime, - "DriverStats", - "Duration", - "totalScheduledTime"); + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "TaskStatus", + "int", + "queuedPartitionedDrivers"); to_json_key( j, - "totalCpuTime", - p.totalCpuTime, - "DriverStats", - "Duration", - "totalCpuTime"); + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "TaskStatus", + "int", + "runningPartitionedDrivers"); to_json_key( j, - "totalBlockedTime", - p.totalBlockedTime, - "DriverStats", - "Duration", - "totalBlockedTime"); - to_json_key( - j, "fullyBlocked", p.fullyBlocked, "DriverStats", "bool", "fullyBlocked"); + "outputBufferUtilization", + p.outputBufferUtilization, + "TaskStatus", + "double", + "outputBufferUtilization"); to_json_key( j, - "blockedReasons", - p.blockedReasons, - "DriverStats", - "List", - "blockedReasons"); + "outputBufferOverutilized", + p.outputBufferOverutilized, + "TaskStatus", + "bool", + "outputBufferOverutilized"); to_json_key( j, - "totalAllocation", - p.totalAllocation, - "DriverStats", - "DataSize", - "totalAllocation"); + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "TaskStatus", + "int64_t", + "physicalWrittenDataSizeInBytes"); to_json_key( j, - "rawInputDataSize", - p.rawInputDataSize, - "DriverStats", - "DataSize", - "rawInputDataSize"); + "memoryReservationInBytes", + p.memoryReservationInBytes, + "TaskStatus", + "int64_t", + "memoryReservationInBytes"); to_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "DriverStats", + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "TaskStatus", "int64_t", - "rawInputPositions"); + "systemMemoryReservationInBytes"); to_json_key( j, - "rawInputReadTime", - p.rawInputReadTime, - "DriverStats", - "Duration", - "rawInputReadTime"); + "peakNodeTotalMemoryReservationInBytes", + p.peakNodeTotalMemoryReservationInBytes, + "TaskStatus", + "int64_t", + "peakNodeTotalMemoryReservationInBytes"); to_json_key( - j, - "processedInputDataSize", - p.processedInputDataSize, - "DriverStats", - "DataSize", - "processedInputDataSize"); + j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); to_json_key( j, - "processedInputPositions", - p.processedInputPositions, - "DriverStats", + "fullGcTimeInMillis", + p.fullGcTimeInMillis, + "TaskStatus", "int64_t", - "processedInputPositions"); + "fullGcTimeInMillis"); to_json_key( j, - "outputDataSize", - p.outputDataSize, - "DriverStats", - "DataSize", - "outputDataSize"); + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "TaskStatus", + "int64_t", + "totalCpuTimeInNanos"); to_json_key( j, - "outputPositions", - p.outputPositions, - "DriverStats", + "taskAgeInMillis", + p.taskAgeInMillis, + "TaskStatus", "int64_t", - "outputPositions"); + "taskAgeInMillis"); to_json_key( j, - "physicalWrittenDataSize", - p.physicalWrittenDataSize, - "DriverStats", - "DataSize", - "physicalWrittenDataSize"); + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "TaskStatus", + "int64_t", + "queuedPartitionedSplitsWeight"); to_json_key( j, - "operatorStats", - p.operatorStats, - "DriverStats", - "List", - "operatorStats"); + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "TaskStatus", + "int64_t", + "runningPartitionedSplitsWeight"); } -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, TaskStatus& p) { from_json_key( j, - "elapsedTime", - p.elapsedTime, - "DriverStats", - "Duration", - "elapsedTime"); + "taskInstanceIdLeastSignificantBits", + p.taskInstanceIdLeastSignificantBits, + "TaskStatus", + "int64_t", + "taskInstanceIdLeastSignificantBits"); from_json_key( j, - "userMemoryReservation", - p.userMemoryReservation, - "DriverStats", - "DataSize", - "userMemoryReservation"); + "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"); from_json_key( j, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "DriverStats", - "DataSize", - "revocableMemoryReservation"); + "completedDriverGroups", + p.completedDriverGroups, + "TaskStatus", + "List", + "completedDriverGroups"); from_json_key( j, - "systemMemoryReservation", - p.systemMemoryReservation, - "DriverStats", - "DataSize", - "systemMemoryReservation"); + "failures", + p.failures, + "TaskStatus", + "List", + "failures"); from_json_key( j, - "totalScheduledTime", - p.totalScheduledTime, - "DriverStats", - "Duration", - "totalScheduledTime"); + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "TaskStatus", + "int", + "queuedPartitionedDrivers"); from_json_key( j, - "totalCpuTime", - p.totalCpuTime, - "DriverStats", - "Duration", - "totalCpuTime"); + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "TaskStatus", + "int", + "runningPartitionedDrivers"); from_json_key( j, - "totalBlockedTime", - p.totalBlockedTime, - "DriverStats", - "Duration", - "totalBlockedTime"); - from_json_key( - j, "fullyBlocked", p.fullyBlocked, "DriverStats", "bool", "fullyBlocked"); + "outputBufferUtilization", + p.outputBufferUtilization, + "TaskStatus", + "double", + "outputBufferUtilization"); from_json_key( j, - "blockedReasons", - p.blockedReasons, - "DriverStats", - "List", - "blockedReasons"); + "outputBufferOverutilized", + p.outputBufferOverutilized, + "TaskStatus", + "bool", + "outputBufferOverutilized"); from_json_key( j, - "totalAllocation", - p.totalAllocation, - "DriverStats", - "DataSize", - "totalAllocation"); + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "TaskStatus", + "int64_t", + "physicalWrittenDataSizeInBytes"); from_json_key( j, - "rawInputDataSize", - p.rawInputDataSize, - "DriverStats", - "DataSize", - "rawInputDataSize"); + "memoryReservationInBytes", + p.memoryReservationInBytes, + "TaskStatus", + "int64_t", + "memoryReservationInBytes"); from_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "DriverStats", + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "TaskStatus", "int64_t", - "rawInputPositions"); + "systemMemoryReservationInBytes"); from_json_key( j, - "rawInputReadTime", - p.rawInputReadTime, - "DriverStats", - "Duration", - "rawInputReadTime"); + "peakNodeTotalMemoryReservationInBytes", + p.peakNodeTotalMemoryReservationInBytes, + "TaskStatus", + "int64_t", + "peakNodeTotalMemoryReservationInBytes"); from_json_key( - j, - "processedInputDataSize", - p.processedInputDataSize, - "DriverStats", - "DataSize", - "processedInputDataSize"); + j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); from_json_key( j, - "processedInputPositions", - p.processedInputPositions, - "DriverStats", + "fullGcTimeInMillis", + p.fullGcTimeInMillis, + "TaskStatus", "int64_t", - "processedInputPositions"); + "fullGcTimeInMillis"); from_json_key( j, - "outputDataSize", - p.outputDataSize, - "DriverStats", - "DataSize", - "outputDataSize"); + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "TaskStatus", + "int64_t", + "totalCpuTimeInNanos"); from_json_key( j, - "outputPositions", - p.outputPositions, - "DriverStats", + "taskAgeInMillis", + p.taskAgeInMillis, + "TaskStatus", "int64_t", - "outputPositions"); + "taskAgeInMillis"); from_json_key( j, - "physicalWrittenDataSize", - p.physicalWrittenDataSize, - "DriverStats", - "DataSize", - "physicalWrittenDataSize"); + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "TaskStatus", + "int64_t", + "queuedPartitionedSplitsWeight"); from_json_key( j, - "operatorStats", - p.operatorStats, - "DriverStats", - "List", - "operatorStats"); + "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; -void to_json(json& j, const PipelineStats& p) { - j = json::object(); - to_json_key( - j, "pipelineId", p.pipelineId, "PipelineStats", "int", "pipelineId"); - to_json_key( - j, - "firstStartTime", - p.firstStartTime, - "PipelineStats", - "DateTime", - "firstStartTime"); - to_json_key( - j, - "lastStartTime", - p.lastStartTime, - "PipelineStats", - "DateTime", - "lastStartTime"); - to_json_key( - j, - "lastEndTime", - p.lastEndTime, - "PipelineStats", - "DateTime", - "lastEndTime"); - to_json_key( - j, - "inputPipeline", - p.inputPipeline, - "PipelineStats", - "bool", - "inputPipeline"); - to_json_key( - j, - "outputPipeline", - p.outputPipeline, - "PipelineStats", - "bool", - "outputPipeline"); - to_json_key( - j, - "totalDrivers", - p.totalDrivers, - "PipelineStats", - "int", - "totalDrivers"); - to_json_key( - j, - "queuedDrivers", - p.queuedDrivers, - "PipelineStats", - "int", - "queuedDrivers"); - to_json_key( - j, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "PipelineStats", - "int", - "queuedPartitionedDrivers"); - to_json_key( - j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "PipelineStats", - "int64_t", - "queuedPartitionedSplitsWeight"); - to_json_key( - j, - "runningDrivers", - p.runningDrivers, - "PipelineStats", - "int", - "runningDrivers"); + 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"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const MetadataUpdates& p) { + j = json::object(); to_json_key( j, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "PipelineStats", - "int", - "runningPartitionedDrivers"); + "connectorId", + p.connectorId, + "MetadataUpdates", + "ConnectorId", + "connectorId"); to_json_key( j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "PipelineStats", - "int64_t", - "runningPartitionedSplitsWeight"); - to_json_key( + "metadataUpdates", + p.metadataUpdates, + "MetadataUpdates", + "List>", + "metadataUpdates"); +} + +void from_json(const json& j, MetadataUpdates& p) { + from_json_key( j, - "blockedDrivers", - p.blockedDrivers, - "PipelineStats", - "int", - "blockedDrivers"); - to_json_key( + "connectorId", + p.connectorId, + "MetadataUpdates", + "ConnectorId", + "connectorId"); + from_json_key( j, - "completedDrivers", - p.completedDrivers, - "PipelineStats", - "int", - "completedDrivers"); + "metadataUpdates", + p.metadataUpdates, + "MetadataUpdates", + "List>", + "metadataUpdates"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const PageBufferInfo& p) { + j = json::object(); to_json_key( - j, - "userMemoryReservationInBytes", - p.userMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "userMemoryReservationInBytes"); + j, "partition", p.partition, "PageBufferInfo", "int", "partition"); to_json_key( j, - "revocableMemoryReservationInBytes", - p.revocableMemoryReservationInBytes, - "PipelineStats", + "bufferedPages", + p.bufferedPages, + "PageBufferInfo", "int64_t", - "revocableMemoryReservationInBytes"); + "bufferedPages"); to_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "PipelineStats", + "bufferedBytes", + p.bufferedBytes, + "PageBufferInfo", "int64_t", - "systemMemoryReservationInBytes"); - to_json_key( - j, - "queuedTime", - p.queuedTime, - "PipelineStats", - "DistributionSnapshot", - "queuedTime"); + "bufferedBytes"); to_json_key( - j, - "elapsedTime", - p.elapsedTime, - "PipelineStats", - "DistributionSnapshot", - "elapsedTime"); + j, "rowsAdded", p.rowsAdded, "PageBufferInfo", "int64_t", "rowsAdded"); to_json_key( + j, "pagesAdded", p.pagesAdded, "PageBufferInfo", "int64_t", "pagesAdded"); +} + +void from_json(const json& j, PageBufferInfo& p) { + from_json_key( + j, "partition", p.partition, "PageBufferInfo", "int", "partition"); + from_json_key( j, - "totalScheduledTimeInNanos", - p.totalScheduledTimeInNanos, - "PipelineStats", + "bufferedPages", + p.bufferedPages, + "PageBufferInfo", "int64_t", - "totalScheduledTimeInNanos"); - to_json_key( + "bufferedPages"); + from_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "PipelineStats", + "bufferedBytes", + p.bufferedBytes, + "PageBufferInfo", "int64_t", - "totalCpuTimeInNanos"); + "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 BufferInfo& p) { + j = json::object(); to_json_key( - j, - "totalBlockedTimeInNanos", - p.totalBlockedTimeInNanos, - "PipelineStats", - "int64_t", - "totalBlockedTimeInNanos"); + j, "bufferId", p.bufferId, "BufferInfo", "OutputBufferId", "bufferId"); + to_json_key(j, "finished", p.finished, "BufferInfo", "bool", "finished"); to_json_key( j, - "fullyBlocked", - p.fullyBlocked, - "PipelineStats", - "bool", - "fullyBlocked"); + "bufferedPages", + p.bufferedPages, + "BufferInfo", + "int", + "bufferedPages"); to_json_key( - j, - "blockedReasons", - p.blockedReasons, - "PipelineStats", - "List", - "blockedReasons"); + j, "pagesSent", p.pagesSent, "BufferInfo", "int64_t", "pagesSent"); to_json_key( j, - "totalAllocationInBytes", - p.totalAllocationInBytes, - "PipelineStats", - "int64_t", - "totalAllocationInBytes"); - to_json_key( + "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, - "rawInputDataSizeInBytes", - p.rawInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "rawInputDataSizeInBytes"); + "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 { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair BufferState_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!"); + const auto* it = std::find_if( + 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(BufferState_enum_table)) + ? it + : std::begin(BufferState_enum_table)) + ->second; +} +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(BufferState_enum_table), + std::end(BufferState_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(BufferState_enum_table)) + ? it + : std::begin(BufferState_enum_table)) + ->first; +} +} // 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, - "rawInputPositions", - p.rawInputPositions, - "PipelineStats", - "int64_t", - "rawInputPositions"); + "canAddBuffers", + p.canAddBuffers, + "OutputBufferInfo", + "bool", + "canAddBuffers"); to_json_key( j, - "processedInputDataSizeInBytes", - p.processedInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "processedInputDataSizeInBytes"); + "canAddPages", + p.canAddPages, + "OutputBufferInfo", + "bool", + "canAddPages"); to_json_key( j, - "processedInputPositions", - p.processedInputPositions, - "PipelineStats", + "totalBufferedBytes", + p.totalBufferedBytes, + "OutputBufferInfo", "int64_t", - "processedInputPositions"); + "totalBufferedBytes"); to_json_key( j, - "outputDataSizeInBytes", - p.outputDataSizeInBytes, - "PipelineStats", + "totalBufferedPages", + p.totalBufferedPages, + "OutputBufferInfo", "int64_t", - "outputDataSizeInBytes"); + "totalBufferedPages"); to_json_key( j, - "outputPositions", - p.outputPositions, - "PipelineStats", + "totalRowsSent", + p.totalRowsSent, + "OutputBufferInfo", "int64_t", - "outputPositions"); + "totalRowsSent"); to_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "PipelineStats", + "totalPagesSent", + p.totalPagesSent, + "OutputBufferInfo", "int64_t", - "physicalWrittenDataSizeInBytes"); + "totalPagesSent"); to_json_key( j, - "operatorSummaries", - p.operatorSummaries, - "PipelineStats", - "List", - "operatorSummaries"); - to_json_key( - j, "drivers", p.drivers, "PipelineStats", "List", "drivers"); + "buffers", + p.buffers, + "OutputBufferInfo", + "List", + "buffers"); } -void from_json(const json& j, PipelineStats& p) { +void from_json(const json& j, OutputBufferInfo& p) { + from_json_key(j, "type", p.type, "OutputBufferInfo", "String", "type"); from_json_key( - j, "pipelineId", p.pipelineId, "PipelineStats", "int", "pipelineId"); + j, "state", p.state, "OutputBufferInfo", "BufferState", "state"); from_json_key( j, - "firstStartTime", - p.firstStartTime, - "PipelineStats", - "DateTime", - "firstStartTime"); + "canAddBuffers", + p.canAddBuffers, + "OutputBufferInfo", + "bool", + "canAddBuffers"); from_json_key( j, - "lastStartTime", - p.lastStartTime, - "PipelineStats", - "DateTime", - "lastStartTime"); + "canAddPages", + p.canAddPages, + "OutputBufferInfo", + "bool", + "canAddPages"); from_json_key( j, - "lastEndTime", - p.lastEndTime, - "PipelineStats", - "DateTime", - "lastEndTime"); + "totalBufferedBytes", + p.totalBufferedBytes, + "OutputBufferInfo", + "int64_t", + "totalBufferedBytes"); from_json_key( j, - "inputPipeline", - p.inputPipeline, - "PipelineStats", - "bool", - "inputPipeline"); + "totalBufferedPages", + p.totalBufferedPages, + "OutputBufferInfo", + "int64_t", + "totalBufferedPages"); from_json_key( j, - "outputPipeline", - p.outputPipeline, - "PipelineStats", - "bool", - "outputPipeline"); + "totalRowsSent", + p.totalRowsSent, + "OutputBufferInfo", + "int64_t", + "totalRowsSent"); from_json_key( j, - "totalDrivers", - p.totalDrivers, - "PipelineStats", - "int", - "totalDrivers"); + "totalPagesSent", + p.totalPagesSent, + "OutputBufferInfo", + "int64_t", + "totalPagesSent"); from_json_key( j, - "queuedDrivers", - p.queuedDrivers, - "PipelineStats", - "int", - "queuedDrivers"); + "buffers", + p.buffers, + "OutputBufferInfo", + "List", + "buffers"); +} +} // 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, + "lastHeartbeat", + p.lastHeartbeat, + "TaskInfo", + "DateTime", + "lastHeartbeat"); + to_json_key( + j, + "outputBuffers", + p.outputBuffers, + "TaskInfo", + "OutputBufferInfo", + "outputBuffers"); + 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"); + 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( j, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "PipelineStats", - "int", - "queuedPartitionedDrivers"); + "lastHeartbeat", + p.lastHeartbeat, + "TaskInfo", + "DateTime", + "lastHeartbeat"); from_json_key( j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "PipelineStats", - "int64_t", - "queuedPartitionedSplitsWeight"); + "outputBuffers", + p.outputBuffers, + "TaskInfo", + "OutputBufferInfo", + "outputBuffers"); from_json_key( j, - "runningDrivers", - p.runningDrivers, - "PipelineStats", - "int", - "runningDrivers"); + "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, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "PipelineStats", - "int", - "runningPartitionedDrivers"); - from_json_key( + "metadataUpdates", + p.metadataUpdates, + "TaskInfo", + "MetadataUpdates", + "metadataUpdates"); + from_json_key(j, "nodeId", p.nodeId, "TaskInfo", "String", "nodeId"); +} +} // 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 ConnectorOutputTableHandle "); +} + +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(); + to_json_key( + j, + "connectorId", + p.connectorId, + "OutputTableHandle", + "ConnectorId", + "connectorId"); + to_json_key( j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "PipelineStats", - "int64_t", - "runningPartitionedSplitsWeight"); - from_json_key( + "transactionHandle", + p.transactionHandle, + "OutputTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); + to_json_key( j, - "blockedDrivers", - p.blockedDrivers, - "PipelineStats", - "int", - "blockedDrivers"); + "connectorHandle", + p.connectorHandle, + "OutputTableHandle", + "ConnectorOutputTableHandle", + "connectorHandle"); +} + +void from_json(const json& j, OutputTableHandle& p) { from_json_key( j, - "completedDrivers", - p.completedDrivers, - "PipelineStats", - "int", - "completedDrivers"); + "connectorId", + p.connectorId, + "OutputTableHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "userMemoryReservationInBytes", - p.userMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "userMemoryReservationInBytes"); + "transactionHandle", + p.transactionHandle, + "OutputTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); from_json_key( j, - "revocableMemoryReservationInBytes", - p.revocableMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "revocableMemoryReservationInBytes"); - from_json_key( + "connectorHandle", + p.connectorHandle, + "OutputTableHandle", + "ConnectorOutputTableHandle", + "connectorHandle"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +AllOrNoneValueSet::AllOrNoneValueSet() noexcept { + _type = "allOrNone"; +} + +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 { + +void to_json(json& j, const VariableStatsEstimate& p) { + j = json::object(); + to_json_key( + j, "lowValue", p.lowValue, "VariableStatsEstimate", "double", "lowValue"); + to_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "systemMemoryReservationInBytes"); - from_json_key( + "highValue", + p.highValue, + "VariableStatsEstimate", + "double", + "highValue"); + to_json_key( j, - "queuedTime", - p.queuedTime, - "PipelineStats", - "DistributionSnapshot", - "queuedTime"); - from_json_key( + "nullsFraction", + p.nullsFraction, + "VariableStatsEstimate", + "double", + "nullsFraction"); + to_json_key( j, - "elapsedTime", - p.elapsedTime, - "PipelineStats", - "DistributionSnapshot", - "elapsedTime"); - from_json_key( + "averageRowSize", + p.averageRowSize, + "VariableStatsEstimate", + "double", + "averageRowSize"); + to_json_key( j, - "totalScheduledTimeInNanos", - p.totalScheduledTimeInNanos, - "PipelineStats", - "int64_t", - "totalScheduledTimeInNanos"); + "distinctValuesCount", + p.distinctValuesCount, + "VariableStatsEstimate", + "double", + "distinctValuesCount"); +} + +void from_json(const json& j, VariableStatsEstimate& p) { from_json_key( - j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "PipelineStats", - "int64_t", - "totalCpuTimeInNanos"); + j, "lowValue", p.lowValue, "VariableStatsEstimate", "double", "lowValue"); from_json_key( j, - "totalBlockedTimeInNanos", - p.totalBlockedTimeInNanos, - "PipelineStats", - "int64_t", - "totalBlockedTimeInNanos"); + "highValue", + p.highValue, + "VariableStatsEstimate", + "double", + "highValue"); from_json_key( j, - "fullyBlocked", - p.fullyBlocked, - "PipelineStats", - "bool", - "fullyBlocked"); + "nullsFraction", + p.nullsFraction, + "VariableStatsEstimate", + "double", + "nullsFraction"); from_json_key( j, - "blockedReasons", - p.blockedReasons, - "PipelineStats", - "List", - "blockedReasons"); + "averageRowSize", + p.averageRowSize, + "VariableStatsEstimate", + "double", + "averageRowSize"); from_json_key( j, - "totalAllocationInBytes", - p.totalAllocationInBytes, - "PipelineStats", - "int64_t", - "totalAllocationInBytes"); + "distinctValuesCount", + p.distinctValuesCount, + "VariableStatsEstimate", + "double", + "distinctValuesCount"); +} +} // 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 { + +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 from_json(const json& j, SortingColumn& p) { from_json_key( - j, - "rawInputDataSizeInBytes", - p.rawInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "rawInputDataSizeInBytes"); + j, "columnName", p.columnName, "SortingColumn", "String", "columnName"); + from_json_key(j, "order", p.order, "SortingColumn", "Order", "order"); +} +} // 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_assert( + 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(SelectedRoleType_enum_table)) + ? it + : std::begin(SelectedRoleType_enum_table)) + ->second; +} +void from_json(const json& j, SelectedRoleType& e) { + static_assert( + 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), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(SelectedRoleType_enum_table)) + ? it + : std::begin(SelectedRoleType_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 SplitContext& p) { + j = json::object(); + to_json_key(j, "cacheable", p.cacheable, "SplitContext", "bool", "cacheable"); +} + +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. + */ +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) == "hive-iceberg") { + 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) == "hive-iceberg") { + 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, "connectorId", p.connectorId, "Split", "ConnectorId", "connectorId"); + to_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "PipelineStats", - "int64_t", - "rawInputPositions"); - from_json_key( + "transactionHandle", + p.transactionHandle, + "Split", + "ConnectorTransactionHandle", + "transactionHandle"); + to_json_key( j, - "processedInputDataSizeInBytes", - p.processedInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "processedInputDataSizeInBytes"); - from_json_key( + "connectorSplit", + p.connectorSplit, + "Split", + "ConnectorSplit", + "connectorSplit"); + to_json_key(j, "lifespan", p.lifespan, "Split", "Lifespan", "lifespan"); + to_json_key( j, - "processedInputPositions", - p.processedInputPositions, - "PipelineStats", - "int64_t", - "processedInputPositions"); + "splitContext", + p.splitContext, + "Split", + "SplitContext", + "splitContext"); +} + +void from_json(const json& j, Split& p) { from_json_key( - j, - "outputDataSizeInBytes", - p.outputDataSizeInBytes, - "PipelineStats", - "int64_t", - "outputDataSizeInBytes"); + j, "connectorId", p.connectorId, "Split", "ConnectorId", "connectorId"); from_json_key( j, - "outputPositions", - p.outputPositions, - "PipelineStats", - "int64_t", - "outputPositions"); + "transactionHandle", + p.transactionHandle, + "Split", + "ConnectorTransactionHandle", + "transactionHandle"); from_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "PipelineStats", - "int64_t", - "physicalWrittenDataSizeInBytes"); + "connectorSplit", + p.connectorSplit, + "Split", + "ConnectorSplit", + "connectorSplit"); + from_json_key(j, "lifespan", p.lifespan, "Split", "Lifespan", "lifespan"); from_json_key( j, - "operatorSummaries", - p.operatorSummaries, - "PipelineStats", - "List", - "operatorSummaries"); - from_json_key( - j, "drivers", p.drivers, "PipelineStats", "List", "drivers"); + "splitContext", + p.splitContext, + "Split", + "SplitContext", + "splitContext"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +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"); +} + +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 std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; -void to_json(json& j, const StatisticAggregations& p) { + 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 { +VariableReferenceExpression::VariableReferenceExpression() noexcept { + _type = "variable"; +} + +void to_json(json& j, const VariableReferenceExpression& p) { j = json::object(); + j["@type"] = "variable"; to_json_key( j, - "outputVariables", - p.outputVariables, - "StatisticAggregations", - "List", - "outputVariables"); - to_json_key( - j, - "aggregations", - p.aggregations, - "StatisticAggregations", - "Map", - "aggregations"); + "sourceLocation", + p.sourceLocation, + "VariableReferenceExpression", + "SourceLocation", + "sourceLocation"); to_json_key( - j, - "groupingVariables", - p.groupingVariables, - "StatisticAggregations", - "List", - "groupingVariables"); + j, "name", p.name, "VariableReferenceExpression", "String", "name"); + to_json_key(j, "type", p.type, "VariableReferenceExpression", "Type", "type"); } -void from_json(const json& j, StatisticAggregations& p) { +void from_json(const json& j, VariableReferenceExpression& p) { + p._type = j["@type"]; from_json_key( j, - "outputVariables", - p.outputVariables, - "StatisticAggregations", - "List", - "outputVariables"); + "sourceLocation", + p.sourceLocation, + "VariableReferenceExpression", + "SourceLocation", + "sourceLocation"); from_json_key( + j, "name", p.name, "VariableReferenceExpression", "String", "name"); + from_json_key( + j, "type", p.type, "VariableReferenceExpression", "Type", "type"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const Assignments& p) { + j = json::object(); + to_json_key( j, - "aggregations", - p.aggregations, - "StatisticAggregations", - "Map", - "aggregations"); + "assignments", + p.assignments, + "Assignments", + "Map>", + "assignments"); +} + +void from_json(const json& j, Assignments& p) { from_json_key( j, - "groupingVariables", - p.groupingVariables, - "StatisticAggregations", - "List", - "groupingVariables"); + "assignments", + p.assignments, + "Assignments", + "Map>", + "assignments"); } } // 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 == ".MarkDistinctNode") { + 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 == ".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.SampleNode") { + 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 == "com.facebook.presto.sql.planner.plan.TableWriterMergeNode") { + 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.TopNRowNumberNode") { + 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 EquiJoinClause& p) { - j = json::object(); - to_json_key( - j, - "left", - p.left, - "EquiJoinClause", - "VariableReferenceExpression", - "left"); - to_json_key( - j, - "right", - p.right, - "EquiJoinClause", - "VariableReferenceExpression", - "right"); + throw TypeError(type + " no abstract type PlanNode "); } -void from_json(const json& j, EquiJoinClause& p) { - from_json_key( - j, - "left", - p.left, - "EquiJoinClause", - "VariableReferenceExpression", - "left"); - from_json_key( - j, - "right", - p.right, - "EquiJoinClause", - "VariableReferenceExpression", - "right"); +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 == ".MarkDistinctNode") { + 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 == ".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.SampleNode") { + 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 == "com.facebook.presto.sql.planner.plan.TableWriterMergeNode") { + 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.TopNRowNumberNode") { + 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 { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair DistributionType_enum_table[] = +static const std::pair SortOrder_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!"); + {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(DistributionType_enum_table), - std::end(DistributionType_enum_table), - [e](const std::pair& ej_pair) -> bool { + 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(DistributionType_enum_table)) + j = ((it != std::end(SortOrder_enum_table)) ? it - : std::begin(DistributionType_enum_table)) + : std::begin(SortOrder_enum_table)) ->second; } -void from_json(const json& j, DistributionType& e) { - static_assert( - std::is_enum::value, - "DistributionType must be an enum!"); +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(DistributionType_enum_table), - std::end(DistributionType_enum_table), - [&j](const std::pair& ej_pair) -> bool { + 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(DistributionType_enum_table)) + e = ((it != std::end(SortOrder_enum_table)) ? it - : std::begin(DistributionType_enum_table)) + : std::begin(SortOrder_enum_table)) ->first; } } // 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) { +void to_json(json& j, const Ordering& 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", + "variable", + p.variable, + "Ordering", "VariableReferenceExpression", - "sourceJoinVariable"); + "variable"); to_json_key( + j, "sortOrder", p.sortOrder, "Ordering", "SortOrder", "sortOrder"); +} + +void from_json(const json& j, Ordering& p) { + from_json_key( j, - "filteringSourceJoinVariable", - p.filteringSourceJoinVariable, - "SemiJoinNode", + "variable", + p.variable, + "Ordering", "VariableReferenceExpression", - "filteringSourceJoinVariable"); + "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, - "semiJoinOutput", - p.semiJoinOutput, - "SemiJoinNode", - "VariableReferenceExpression", - "semiJoinOutput"); + 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 std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + + if (type == "$static") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "json_file") { + 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; + } + if (type == "json_file") { + 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, - "sourceHashVariable", - p.sourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "sourceHashVariable"); + "sourceLocation", + p.sourceLocation, + "CallExpression", + "SourceLocation", + "sourceLocation"); to_json_key( j, - "filteringSourceHashVariable", - p.filteringSourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "filteringSourceHashVariable"); + "displayName", + p.displayName, + "CallExpression", + "String", + "displayName"); to_json_key( j, - "distributionType", - p.distributionType, - "SemiJoinNode", - "DistributionType", - "distributionType"); + "functionHandle", + p.functionHandle, + "CallExpression", + "FunctionHandle", + "functionHandle"); + to_json_key( + j, "returnType", p.returnType, "CallExpression", "Type", "returnType"); to_json_key( j, - "dynamicFilters", - p.dynamicFilters, - "SemiJoinNode", - "Map", - "dynamicFilters"); + "arguments", + p.arguments, + "CallExpression", + "List>", + "arguments"); } -void from_json(const json& j, SemiJoinNode& p) { +void from_json(const json& j, CallExpression& 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"); - from_json_key( - j, - "sourceHashVariable", - p.sourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "sourceHashVariable"); - from_json_key( - j, - "filteringSourceHashVariable", - p.filteringSourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "filteringSourceHashVariable"); from_json_key( j, - "distributionType", - p.distributionType, - "SemiJoinNode", - "DistributionType", - "distributionType"); + "sourceLocation", + p.sourceLocation, + "CallExpression", + "SourceLocation", + "sourceLocation"); from_json_key( j, - "dynamicFilters", - p.dynamicFilters, - "SemiJoinNode", - "Map", - "dynamicFilters"); -} -} // 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; + "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 CacheQuotaRequirement& p) { +void to_json(json& j, const Aggregation& p) { j = json::object(); + to_json_key(j, "call", p.call, "Aggregation", "CallExpression", "call"); to_json_key( j, - "cacheQuotaScope", - p.cacheQuotaScope, - "CacheQuotaRequirement", - "CacheQuotaScope", - "cacheQuotaScope"); + "filter", + p.filter, + "Aggregation", + "std::shared_ptr", + "filter"); to_json_key( - j, "quota", p.quota, "CacheQuotaRequirement", "DataSize", "quota"); + j, "orderBy", p.orderBy, "Aggregation", "OrderingScheme", "orderBy"); + to_json_key(j, "distinct", p.distinct, "Aggregation", "bool", "distinct"); + to_json_key( + j, "mask", p.mask, "Aggregation", "VariableReferenceExpression", "mask"); + to_json_key( + j, + "functionHandle", + p.functionHandle, + "Aggregation", + "FunctionHandle", + "functionHandle"); + to_json_key( + j, + "arguments", + p.arguments, + "Aggregation", + "List>", + "arguments"); } -void from_json(const json& j, CacheQuotaRequirement& p) { +void from_json(const json& j, Aggregation& p) { + from_json_key(j, "call", p.call, "Aggregation", "CallExpression", "call"); from_json_key( j, - "cacheQuotaScope", - p.cacheQuotaScope, - "CacheQuotaRequirement", - "CacheQuotaScope", - "cacheQuotaScope"); + "filter", + p.filter, + "Aggregation", + "std::shared_ptr", + "filter"); from_json_key( - j, "quota", p.quota, "CacheQuotaRequirement", "DataSize", "quota"); + j, "orderBy", p.orderBy, "Aggregation", "OrderingScheme", "orderBy"); + from_json_key(j, "distinct", p.distinct, "Aggregation", "bool", "distinct"); + from_json_key( + j, "mask", p.mask, "Aggregation", "VariableReferenceExpression", "mask"); + from_json_key( + j, + "functionHandle", + p.functionHandle, + "Aggregation", + "FunctionHandle", + "functionHandle"); + from_json_key( + j, + "arguments", + p.arguments, + "Aggregation", + "List>", + "arguments"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Column& p) { +void to_json(json& j, const StatisticAggregations& p) { j = json::object(); - to_json_key(j, "name", p.name, "Column", "String", "name"); - to_json_key(j, "type", p.type, "Column", "String", "type"); + to_json_key( + j, + "outputVariables", + p.outputVariables, + "StatisticAggregations", + "List", + "outputVariables"); + to_json_key( + j, + "aggregations", + p.aggregations, + "StatisticAggregations", + "Map", + "aggregations"); + to_json_key( + j, + "groupingVariables", + p.groupingVariables, + "StatisticAggregations", + "List", + "groupingVariables"); } -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"); +void from_json(const json& j, StatisticAggregations& p) { + from_json_key( + j, + "outputVariables", + p.outputVariables, + "StatisticAggregations", + "List", + "outputVariables"); + from_json_key( + j, + "aggregations", + p.aggregations, + "StatisticAggregations", + "Map", + "aggregations"); + from_json_key( + j, + "groupingVariables", + p.groupingVariables, + "StatisticAggregations", + "List", + "groupingVariables"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +TableWriterMergeNode::TableWriterMergeNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.TableWriterMergeNode"; +} -void to_json(json& j, const SplitContext& p) { +void to_json(json& j, const TableWriterMergeNode& p) { j = json::object(); - to_json_key(j, "cacheable", p.cacheable, "SplitContext", "bool", "cacheable"); + j["@type"] = "com.facebook.presto.sql.planner.plan.TableWriterMergeNode"; + to_json_key(j, "id", p.id, "TableWriterMergeNode", "PlanNodeId", "id"); + to_json_key( + j, "source", p.source, "TableWriterMergeNode", "PlanNode", "source"); + to_json_key( + j, + "rowCountVariable", + p.rowCountVariable, + "TableWriterMergeNode", + "VariableReferenceExpression", + "rowCountVariable"); + to_json_key( + j, + "fragmentVariable", + p.fragmentVariable, + "TableWriterMergeNode", + "VariableReferenceExpression", + "fragmentVariable"); + to_json_key( + j, + "tableCommitContextVariable", + p.tableCommitContextVariable, + "TableWriterMergeNode", + "VariableReferenceExpression", + "tableCommitContextVariable"); + to_json_key( + j, + "statisticsAggregation", + p.statisticsAggregation, + "TableWriterMergeNode", + "StatisticAggregations", + "statisticsAggregation"); } -void from_json(const json& j, SplitContext& p) { +void from_json(const json& j, TableWriterMergeNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "TableWriterMergeNode", "PlanNodeId", "id"); 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. - */ -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; - } + j, "source", p.source, "TableWriterMergeNode", "PlanNode", "source"); + from_json_key( + j, + "rowCountVariable", + p.rowCountVariable, + "TableWriterMergeNode", + "VariableReferenceExpression", + "rowCountVariable"); + from_json_key( + j, + "fragmentVariable", + p.fragmentVariable, + "TableWriterMergeNode", + "VariableReferenceExpression", + "fragmentVariable"); + from_json_key( + j, + "tableCommitContextVariable", + p.tableCommitContextVariable, + "TableWriterMergeNode", + "VariableReferenceExpression", + "tableCommitContextVariable"); + from_json_key( + j, + "statisticsAggregation", + p.statisticsAggregation, + "TableWriterMergeNode", + "StatisticAggregations", + "statisticsAggregation"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { - if (getConnectorKey(type) == "tpch") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } +void to_json(json& j, const NodeVersion& p) { + j = json::object(); + to_json_key(j, "version", p.version, "NodeVersion", "String", "version"); +} - throw TypeError(type + " no abstract type ConnectorSplit"); +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 Split& p) { +void to_json(json& j, const ServerInfo& p) { j = json::object(); - to_json_key( - j, "connectorId", p.connectorId, "Split", "ConnectorId", "connectorId"); to_json_key( j, - "transactionHandle", - p.transactionHandle, - "Split", - "ConnectorTransactionHandle", - "transactionHandle"); + "nodeVersion", + p.nodeVersion, + "ServerInfo", + "NodeVersion", + "nodeVersion"); to_json_key( - j, - "connectorSplit", - p.connectorSplit, - "Split", - "ConnectorSplit", - "connectorSplit"); - to_json_key(j, "lifespan", p.lifespan, "Split", "Lifespan", "lifespan"); + j, "environment", p.environment, "ServerInfo", "String", "environment"); to_json_key( - j, - "splitContext", - p.splitContext, - "Split", - "SplitContext", - "splitContext"); + 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, Split& p) { - from_json_key( - j, "connectorId", p.connectorId, "Split", "ConnectorId", "connectorId"); +void from_json(const json& j, ServerInfo& p) { from_json_key( j, - "transactionHandle", - p.transactionHandle, - "Split", - "ConnectorTransactionHandle", - "transactionHandle"); + "nodeVersion", + p.nodeVersion, + "ServerInfo", + "NodeVersion", + "nodeVersion"); from_json_key( - j, - "connectorSplit", - p.connectorSplit, - "Split", - "ConnectorSplit", - "connectorSplit"); - from_json_key(j, "lifespan", p.lifespan, "Split", "Lifespan", "lifespan"); + j, "environment", p.environment, "ServerInfo", "String", "environment"); from_json_key( - j, - "splitContext", - p.splitContext, - "Split", - "SplitContext", - "splitContext"); + 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 { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const ScheduledSplit& p) { +// 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) { j = json::object(); + j["@type"] = "special"; to_json_key( - j, "sequenceId", p.sequenceId, "ScheduledSplit", "int64_t", "sequenceId"); + j, + "sourceLocation", + p.sourceLocation, + "SpecialFormExpression", + "SourceLocation", + "sourceLocation"); + to_json_key(j, "form", p.form, "SpecialFormExpression", "Form", "form"); to_json_key( j, - "planNodeId", - p.planNodeId, - "ScheduledSplit", - "PlanNodeId", - "planNodeId"); - to_json_key(j, "split", p.split, "ScheduledSplit", "Split", "split"); + "returnType", + p.returnType, + "SpecialFormExpression", + "Type", + "returnType"); + to_json_key( + j, + "arguments", + p.arguments, + "SpecialFormExpression", + "List>", + "arguments"); } -void from_json(const json& j, ScheduledSplit& p) { +void from_json(const json& j, SpecialFormExpression& p) { + p._type = j["@type"]; from_json_key( - j, "sequenceId", p.sequenceId, "ScheduledSplit", "int64_t", "sequenceId"); + j, + "sourceLocation", + p.sourceLocation, + "SpecialFormExpression", + "SourceLocation", + "sourceLocation"); + from_json_key(j, "form", p.form, "SpecialFormExpression", "Form", "form"); from_json_key( j, - "planNodeId", - p.planNodeId, - "ScheduledSplit", - "PlanNodeId", - "planNodeId"); - from_json_key(j, "split", p.split, "ScheduledSplit", "Split", "split"); + "returnType", + p.returnType, + "SpecialFormExpression", + "Type", + "returnType"); + from_json_key( + j, + "arguments", + p.arguments, + "SpecialFormExpression", + "List>", + "arguments"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +MarkDistinctNode::MarkDistinctNode() noexcept { + _type = ".MarkDistinctNode"; +} -void to_json(json& j, const TaskSource& p) { +void to_json(json& j, const MarkDistinctNode& p) { j = json::object(); + j["@type"] = ".MarkDistinctNode"; + to_json_key(j, "id", p.id, "MarkDistinctNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "MarkDistinctNode", "PlanNode", "source"); to_json_key( - j, "planNodeId", p.planNodeId, "TaskSource", "PlanNodeId", "planNodeId"); - to_json_key( - j, "splits", p.splits, "TaskSource", "List", "splits"); + j, + "markerVariable", + p.markerVariable, + "MarkDistinctNode", + "VariableReferenceExpression", + "markerVariable"); to_json_key( j, - "noMoreSplitsForLifespan", - p.noMoreSplitsForLifespan, - "TaskSource", - "List", - "noMoreSplitsForLifespan"); + "distinctVariables", + p.distinctVariables, + "MarkDistinctNode", + "List", + "distinctVariables"); to_json_key( - j, "noMoreSplits", p.noMoreSplits, "TaskSource", "bool", "noMoreSplits"); + j, + "hashVariable", + p.hashVariable, + "MarkDistinctNode", + "VariableReferenceExpression", + "hashVariable"); } -void from_json(const json& j, TaskSource& p) { +void from_json(const json& j, MarkDistinctNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "MarkDistinctNode", "PlanNodeId", "id"); from_json_key( - j, "planNodeId", p.planNodeId, "TaskSource", "PlanNodeId", "planNodeId"); + j, "source", p.source, "MarkDistinctNode", "PlanNode", "source"); from_json_key( - j, "splits", p.splits, "TaskSource", "List", "splits"); + j, + "markerVariable", + p.markerVariable, + "MarkDistinctNode", + "VariableReferenceExpression", + "markerVariable"); from_json_key( j, - "noMoreSplitsForLifespan", - p.noMoreSplitsForLifespan, - "TaskSource", - "List", - "noMoreSplitsForLifespan"); + "distinctVariables", + p.distinctVariables, + "MarkDistinctNode", + "List", + "distinctVariables"); from_json_key( - j, "noMoreSplits", p.noMoreSplits, "TaskSource", "bool", "noMoreSplits"); + j, + "hashVariable", + p.hashVariable, + "MarkDistinctNode", + "VariableReferenceExpression", + "hashVariable"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { @@ -4960,2972 +4912,3044 @@ void from_json(const json& j, Marker& p) { } } // 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 { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair JoinType_enum_table[] = +static const std::pair FunctionKind_enum_table[] = { // NOLINT: cert-err58-cpp - {JoinType::INNER, "INNER"}, - {JoinType::LEFT, "LEFT"}, - {JoinType::RIGHT, "RIGHT"}, - {JoinType::FULL, "FULL"}}; -void to_json(json& j, const JoinType& e) { - static_assert(std::is_enum::value, "JoinType must be an enum!"); + {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(JoinType_enum_table), - std::end(JoinType_enum_table), - [e](const std::pair& ej_pair) -> bool { + 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(JoinType_enum_table)) ? it - : std::begin(JoinType_enum_table)) + j = ((it != std::end(FunctionKind_enum_table)) + ? it + : std::begin(FunctionKind_enum_table)) ->second; } -void from_json(const json& j, JoinType& e) { - static_assert(std::is_enum::value, "JoinType must be an enum!"); +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(JoinType_enum_table), - std::end(JoinType_enum_table), - [&j](const std::pair& ej_pair) -> bool { + 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(JoinType_enum_table)) ? it - : std::begin(JoinType_enum_table)) - ->first; -} -} // 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", "JoinType", "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", "JoinType", "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, - "criteria", - p.criteria, - "MergeJoinNode", - "List", - "criteria"); - from_json_key( - j, - "outputVariables", - p.outputVariables, - "MergeJoinNode", - "List", - "outputVariables"); - from_json_key( - j, - "filter", - p.filter, - "MergeJoinNode", - "std::shared_ptr", - "filter"); - from_json_key( - j, - "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 TypeVariableConstraint& p) { - j = json::object(); - to_json_key(j, "name", p.name, "TypeVariableConstraint", "String", "name"); - to_json_key( - j, - "comparableRequired", - p.comparableRequired, - "TypeVariableConstraint", - "bool", - "comparableRequired"); - to_json_key( - j, - "orderableRequired", - p.orderableRequired, - "TypeVariableConstraint", - "bool", - "orderableRequired"); - to_json_key( - j, - "variadicBound", - p.variadicBound, - "TypeVariableConstraint", - "String", - "variadicBound"); - to_json_key( - j, - "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, - "comparableRequired", - p.comparableRequired, - "TypeVariableConstraint", - "bool", - "comparableRequired"); - from_json_key( - j, - "orderableRequired", - p.orderableRequired, - "TypeVariableConstraint", - "bool", - "orderableRequired"); - from_json_key( - j, - "variadicBound", - p.variadicBound, - "TypeVariableConstraint", - "String", - "variadicBound"); - from_json_key( - j, - "nonDecimalNumericRequired", - p.nonDecimalNumericRequired, - "TypeVariableConstraint", - "bool", - "nonDecimalNumericRequired"); -} -} // 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 MemoryAllocation& 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"); -} - -void from_json(const json& j, MemoryAllocation& p) { - from_json_key(j, "tag", p.tag, "MemoryAllocation", "String", "tag"); - from_json_key( - j, - "allocation", - p.allocation, - "MemoryAllocation", - "int64_t", - "allocation"); + 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 MemoryPoolInfo& p) { +void to_json(json& j, const TypeVariableConstraint& p) { j = json::object(); - to_json_key( - j, "maxBytes", p.maxBytes, "MemoryPoolInfo", "int64_t", "maxBytes"); - to_json_key( - j, - "reservedBytes", - p.reservedBytes, - "MemoryPoolInfo", - "int64_t", - "reservedBytes"); + to_json_key(j, "name", p.name, "TypeVariableConstraint", "String", "name"); to_json_key( j, - "reservedRevocableBytes", - p.reservedRevocableBytes, - "MemoryPoolInfo", - "int64_t", - "reservedRevocableBytes"); + "comparableRequired", + p.comparableRequired, + "TypeVariableConstraint", + "bool", + "comparableRequired"); to_json_key( j, - "queryMemoryReservations", - p.queryMemoryReservations, - "MemoryPoolInfo", - "Map", - "queryMemoryReservations"); + "orderableRequired", + p.orderableRequired, + "TypeVariableConstraint", + "bool", + "orderableRequired"); to_json_key( j, - "queryMemoryAllocations", - p.queryMemoryAllocations, - "MemoryPoolInfo", - "Map>", - "queryMemoryAllocations"); + "variadicBound", + p.variadicBound, + "TypeVariableConstraint", + "String", + "variadicBound"); to_json_key( j, - "queryMemoryRevocableReservations", - p.queryMemoryRevocableReservations, - "MemoryPoolInfo", - "Map", - "queryMemoryRevocableReservations"); + "nonDecimalNumericRequired", + p.nonDecimalNumericRequired, + "TypeVariableConstraint", + "bool", + "nonDecimalNumericRequired"); } -void from_json(const json& j, MemoryPoolInfo& p) { - from_json_key( - j, "maxBytes", p.maxBytes, "MemoryPoolInfo", "int64_t", "maxBytes"); - from_json_key( - j, - "reservedBytes", - p.reservedBytes, - "MemoryPoolInfo", - "int64_t", - "reservedBytes"); +void from_json(const json& j, TypeVariableConstraint& p) { + from_json_key(j, "name", p.name, "TypeVariableConstraint", "String", "name"); from_json_key( j, - "reservedRevocableBytes", - p.reservedRevocableBytes, - "MemoryPoolInfo", - "int64_t", - "reservedRevocableBytes"); + "comparableRequired", + p.comparableRequired, + "TypeVariableConstraint", + "bool", + "comparableRequired"); from_json_key( j, - "queryMemoryReservations", - p.queryMemoryReservations, - "MemoryPoolInfo", - "Map", - "queryMemoryReservations"); + "orderableRequired", + p.orderableRequired, + "TypeVariableConstraint", + "bool", + "orderableRequired"); from_json_key( j, - "queryMemoryAllocations", - p.queryMemoryAllocations, - "MemoryPoolInfo", - "Map>", - "queryMemoryAllocations"); + "variadicBound", + p.variadicBound, + "TypeVariableConstraint", + "String", + "variadicBound"); from_json_key( j, - "queryMemoryRevocableReservations", - p.queryMemoryRevocableReservations, - "MemoryPoolInfo", - "Map", - "queryMemoryRevocableReservations"); + "nonDecimalNumericRequired", + p.nonDecimalNumericRequired, + "TypeVariableConstraint", + "bool", + "nonDecimalNumericRequired"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const MemoryInfo& p) { +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, - "totalNodeMemory", - p.totalNodeMemory, - "MemoryInfo", - "DataSize", - "totalNodeMemory"); - to_json_key( - j, - "pools", - p.pools, - "MemoryInfo", - "Map", - "pools"); + "expression", + p.expression, + "LongVariableConstraint", + "String", + "expression"); } -void from_json(const json& j, MemoryInfo& p) { - from_json_key( - j, - "totalNodeMemory", - p.totalNodeMemory, - "MemoryInfo", - "DataSize", - "totalNodeMemory"); +void from_json(const json& j, LongVariableConstraint& p) { + from_json_key(j, "name", p.name, "LongVariableConstraint", "String", "name"); from_json_key( j, - "pools", - p.pools, - "MemoryInfo", - "Map", - "pools"); + "expression", + p.expression, + "LongVariableConstraint", + "String", + "expression"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const NodeStatus& p) { +void to_json(json& j, const Signature& 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, "name", p.name, "Signature", "QualifiedObjectName", "name"); + to_json_key(j, "kind", p.kind, "Signature", "FunctionKind", "kind"); to_json_key( j, - "externalAddress", - p.externalAddress, - "NodeStatus", - "String", - "externalAddress"); + "typeVariableConstraints", + p.typeVariableConstraints, + "Signature", + "List", + "typeVariableConstraints"); to_json_key( j, - "internalAddress", - p.internalAddress, - "NodeStatus", - "String", - "internalAddress"); - to_json_key( - j, "memoryInfo", p.memoryInfo, "NodeStatus", "MemoryInfo", "memoryInfo"); - to_json_key(j, "processors", p.processors, "NodeStatus", "int", "processors"); + "longVariableConstraints", + p.longVariableConstraints, + "Signature", + "List", + "longVariableConstraints"); to_json_key( j, - "processCpuLoad", - p.processCpuLoad, - "NodeStatus", - "double", - "processCpuLoad"); + "returnType", + p.returnType, + "Signature", + "TypeSignature", + "returnType"); to_json_key( j, - "systemCpuLoad", - p.systemCpuLoad, - "NodeStatus", - "double", - "systemCpuLoad"); - to_json_key(j, "heapUsed", p.heapUsed, "NodeStatus", "int64_t", "heapUsed"); + "argumentTypes", + p.argumentTypes, + "Signature", + "List", + "argumentTypes"); to_json_key( j, - "heapAvailable", - p.heapAvailable, - "NodeStatus", - "int64_t", - "heapAvailable"); - to_json_key( - j, "nonHeapUsed", p.nonHeapUsed, "NodeStatus", "int64_t", "nonHeapUsed"); + "variableArity", + p.variableArity, + "Signature", + "bool", + "variableArity"); } -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, 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, - "externalAddress", - p.externalAddress, - "NodeStatus", - "String", - "externalAddress"); + "typeVariableConstraints", + p.typeVariableConstraints, + "Signature", + "List", + "typeVariableConstraints"); from_json_key( j, - "internalAddress", - p.internalAddress, - "NodeStatus", - "String", - "internalAddress"); - from_json_key( - j, "memoryInfo", p.memoryInfo, "NodeStatus", "MemoryInfo", "memoryInfo"); - from_json_key( - j, "processors", p.processors, "NodeStatus", "int", "processors"); + "longVariableConstraints", + p.longVariableConstraints, + "Signature", + "List", + "longVariableConstraints"); from_json_key( j, - "processCpuLoad", - p.processCpuLoad, - "NodeStatus", - "double", - "processCpuLoad"); + "returnType", + p.returnType, + "Signature", + "TypeSignature", + "returnType"); from_json_key( j, - "systemCpuLoad", - p.systemCpuLoad, - "NodeStatus", - "double", - "systemCpuLoad"); - from_json_key(j, "heapUsed", p.heapUsed, "NodeStatus", "int64_t", "heapUsed"); + "argumentTypes", + p.argumentTypes, + "Signature", + "List", + "argumentTypes"); from_json_key( j, - "heapAvailable", - p.heapAvailable, - "NodeStatus", - "int64_t", - "heapAvailable"); - from_json_key( - j, "nonHeapUsed", p.nonHeapUsed, "NodeStatus", "int64_t", "nonHeapUsed"); + "variableArity", + p.variableArity, + "Signature", + "bool", + "variableArity"); +} +} // 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 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!"); +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(ErrorType_enum_table), - std::end(ErrorType_enum_table), - [e](const std::pair& ej_pair) -> bool { + 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(ErrorType_enum_table)) + j = ((it != std::end(Determinism_enum_table)) ? it - : std::begin(ErrorType_enum_table)) + : std::begin(Determinism_enum_table)) ->second; } -void from_json(const json& j, ErrorType& e) { - static_assert(std::is_enum::value, "ErrorType must be an enum!"); +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(ErrorType_enum_table), - std::end(ErrorType_enum_table), - [&j](const std::pair& ej_pair) -> bool { + 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(ErrorType_enum_table)) + e = ((it != std::end(Determinism_enum_table)) ? it - : std::begin(ErrorType_enum_table)) + : std::begin(Determinism_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 Language& 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, "language", p.language, "Language", "String", "language"); } -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, 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 ErrorCause_enum_table[] = +static const std::pair NullCallClause_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!"); + {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(ErrorCause_enum_table), - std::end(ErrorCause_enum_table), - [e](const std::pair& ej_pair) -> bool { + 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(ErrorCause_enum_table)) + j = ((it != std::end(NullCallClause_enum_table)) ? it - : std::begin(ErrorCause_enum_table)) + : std::begin(NullCallClause_enum_table)) ->second; } -void from_json(const json& j, ErrorCause& e) { - static_assert(std::is_enum::value, "ErrorCause must be an enum!"); +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(ErrorCause_enum_table), - std::end(ErrorCause_enum_table), - [&j](const std::pair& ej_pair) -> bool { + 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(ErrorCause_enum_table)) + e = ((it != std::end(NullCallClause_enum_table)) ? it - : std::begin(ErrorCause_enum_table)) + : std::begin(NullCallClause_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const ErrorLocation& p) { +void to_json(json& j, const RoutineCharacteristics& p) { j = json::object(); to_json_key( - j, "lineNumber", p.lineNumber, "ErrorLocation", "int", "lineNumber"); + j, + "language", + p.language, + "RoutineCharacteristics", + "Language", + "language"); to_json_key( j, - "columnNumber", - p.columnNumber, - "ErrorLocation", - "int", - "columnNumber"); + "determinism", + p.determinism, + "RoutineCharacteristics", + "Determinism", + "determinism"); + to_json_key( + j, + "nullCallClause", + p.nullCallClause, + "RoutineCharacteristics", + "NullCallClause", + "nullCallClause"); } -void from_json(const json& j, ErrorLocation& p) { +void from_json(const json& j, RoutineCharacteristics& p) { + from_json_key( + j, + "language", + p.language, + "RoutineCharacteristics", + "Language", + "language"); + from_json_key( + j, + "determinism", + p.determinism, + "RoutineCharacteristics", + "Determinism", + "determinism"); + from_json_key( + j, + "nullCallClause", + p.nullCallClause, + "RoutineCharacteristics", + "NullCallClause", + "nullCallClause"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const SqlInvokedFunction& p) { + j = json::object(); + 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"); +} + +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"); from_json_key( - j, "lineNumber", p.lineNumber, "ErrorLocation", "int", "lineNumber"); + j, + "signature", + p.signature, + "SqlInvokedFunction", + "Signature", + "signature"); from_json_key( j, - "columnNumber", - p.columnNumber, - "ErrorLocation", - "int", - "columnNumber"); + "functionId", + p.functionId, + "SqlInvokedFunction", + "SqlFunctionId", + "functionId"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const ExecutionFailureInfo& p) { +void to_json(json& j, const ResourceEstimates& 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"); + "executionTime", + p.executionTime, + "ResourceEstimates", + "Duration", + "executionTime"); to_json_key( - j, - "errorCode", - p.errorCode, - "ExecutionFailureInfo", - "ErrorCode", - "errorCode"); + j, "cpuTime", p.cpuTime, "ResourceEstimates", "Duration", "cpuTime"); to_json_key( j, - "remoteHost", - p.remoteHost, - "ExecutionFailureInfo", - "HostAddress", - "remoteHost"); + "peakMemory", + p.peakMemory, + "ResourceEstimates", + "DataSize", + "peakMemory"); to_json_key( j, - "errorCause", - p.errorCause, - "ExecutionFailureInfo", - "ErrorCause", - "errorCause"); + "peakTaskMemory", + p.peakTaskMemory, + "ResourceEstimates", + "DataSize", + "peakTaskMemory"); } -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"); +void from_json(const json& j, ResourceEstimates& p) { from_json_key( j, - "errorLocation", - p.errorLocation, - "ExecutionFailureInfo", - "ErrorLocation", - "errorLocation"); + "executionTime", + p.executionTime, + "ResourceEstimates", + "Duration", + "executionTime"); from_json_key( - j, - "errorCode", - p.errorCode, - "ExecutionFailureInfo", - "ErrorCode", - "errorCode"); + j, "cpuTime", p.cpuTime, "ResourceEstimates", "Duration", "cpuTime"); from_json_key( j, - "remoteHost", - p.remoteHost, - "ExecutionFailureInfo", - "HostAddress", - "remoteHost"); + "peakMemory", + p.peakMemory, + "ResourceEstimates", + "DataSize", + "peakMemory"); from_json_key( j, - "errorCause", - p.errorCause, - "ExecutionFailureInfo", - "ErrorCause", - "errorCause"); -} -} // 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; + "peakTaskMemory", + p.peakTaskMemory, + "ResourceEstimates", + "DataSize", + "peakTaskMemory"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const TaskStatus& p) { +void to_json(json& j, const SessionRepresentation& p) { j = json::object(); to_json_key( - j, - "taskInstanceIdLeastSignificantBits", - p.taskInstanceIdLeastSignificantBits, - "TaskStatus", - "int64_t", - "taskInstanceIdLeastSignificantBits"); + j, "queryId", p.queryId, "SessionRepresentation", "String", "queryId"); 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"); + "transactionId", + p.transactionId, + "SessionRepresentation", + "TransactionId", + "transactionId"); to_json_key( j, - "completedDriverGroups", - p.completedDriverGroups, - "TaskStatus", - "List", - "completedDriverGroups"); + "clientTransactionSupport", + p.clientTransactionSupport, + "SessionRepresentation", + "bool", + "clientTransactionSupport"); + to_json_key(j, "user", p.user, "SessionRepresentation", "String", "user"); to_json_key( j, - "failures", - p.failures, - "TaskStatus", - "List", - "failures"); + "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, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "TaskStatus", - "int", - "queuedPartitionedDrivers"); + "traceToken", + p.traceToken, + "SessionRepresentation", + "String", + "traceToken"); to_json_key( j, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "TaskStatus", - "int", - "runningPartitionedDrivers"); + "timeZoneKey", + p.timeZoneKey, + "SessionRepresentation", + "TimeZoneKey", + "timeZoneKey"); + to_json_key( + j, "locale", p.locale, "SessionRepresentation", "Locale", "locale"); to_json_key( j, - "outputBufferUtilization", - p.outputBufferUtilization, - "TaskStatus", - "double", - "outputBufferUtilization"); + "remoteUserAddress", + p.remoteUserAddress, + "SessionRepresentation", + "String", + "remoteUserAddress"); to_json_key( j, - "outputBufferOverutilized", - p.outputBufferOverutilized, - "TaskStatus", - "bool", - "outputBufferOverutilized"); + "userAgent", + p.userAgent, + "SessionRepresentation", + "String", + "userAgent"); to_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "TaskStatus", - "int64_t", - "physicalWrittenDataSizeInBytes"); + "clientInfo", + p.clientInfo, + "SessionRepresentation", + "String", + "clientInfo"); to_json_key( j, - "memoryReservationInBytes", - p.memoryReservationInBytes, - "TaskStatus", - "int64_t", - "memoryReservationInBytes"); + "clientTags", + p.clientTags, + "SessionRepresentation", + "List", + "clientTags"); to_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "TaskStatus", - "int64_t", - "systemMemoryReservationInBytes"); + "resourceEstimates", + p.resourceEstimates, + "SessionRepresentation", + "ResourceEstimates", + "resourceEstimates"); to_json_key( j, - "peakNodeTotalMemoryReservationInBytes", - p.peakNodeTotalMemoryReservationInBytes, - "TaskStatus", + "startTime", + p.startTime, + "SessionRepresentation", "int64_t", - "peakNodeTotalMemoryReservationInBytes"); + "startTime"); to_json_key( - j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); + j, + "systemProperties", + p.systemProperties, + "SessionRepresentation", + "Map", + "systemProperties"); to_json_key( j, - "fullGcTimeInMillis", - p.fullGcTimeInMillis, - "TaskStatus", - "int64_t", - "fullGcTimeInMillis"); + "catalogProperties", + p.catalogProperties, + "SessionRepresentation", + "Map>", + "catalogProperties"); to_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "TaskStatus", - "int64_t", - "totalCpuTimeInNanos"); + "unprocessedCatalogProperties", + p.unprocessedCatalogProperties, + "SessionRepresentation", + "Map>", + "unprocessedCatalogProperties"); to_json_key( j, - "taskAgeInMillis", - p.taskAgeInMillis, - "TaskStatus", - "int64_t", - "taskAgeInMillis"); + "roles", + p.roles, + "SessionRepresentation", + "Map", + "roles"); to_json_key( j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "TaskStatus", - "int64_t", - "queuedPartitionedSplitsWeight"); + "preparedStatements", + p.preparedStatements, + "SessionRepresentation", + "Map", + "preparedStatements"); to_json_key( j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "TaskStatus", - "int64_t", - "runningPartitionedSplitsWeight"); + "sessionFunctions", + p.sessionFunctions, + "SessionRepresentation", + "Map", + "sessionFunctions"); } -void from_json(const json& j, TaskStatus& p) { +void from_json(const json& j, SessionRepresentation& p) { + from_json_key( + j, "queryId", p.queryId, "SessionRepresentation", "String", "queryId"); from_json_key( j, - "taskInstanceIdLeastSignificantBits", - p.taskInstanceIdLeastSignificantBits, - "TaskStatus", - "int64_t", - "taskInstanceIdLeastSignificantBits"); + "transactionId", + p.transactionId, + "SessionRepresentation", + "TransactionId", + "transactionId"); 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"); + "clientTransactionSupport", + p.clientTransactionSupport, + "SessionRepresentation", + "bool", + "clientTransactionSupport"); + from_json_key(j, "user", p.user, "SessionRepresentation", "String", "user"); from_json_key( j, - "completedDriverGroups", - p.completedDriverGroups, - "TaskStatus", - "List", - "completedDriverGroups"); + "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, - "failures", - p.failures, - "TaskStatus", - "List", - "failures"); + "traceToken", + p.traceToken, + "SessionRepresentation", + "String", + "traceToken"); from_json_key( j, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "TaskStatus", - "int", - "queuedPartitionedDrivers"); + "timeZoneKey", + p.timeZoneKey, + "SessionRepresentation", + "TimeZoneKey", + "timeZoneKey"); + from_json_key( + j, "locale", p.locale, "SessionRepresentation", "Locale", "locale"); from_json_key( j, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "TaskStatus", - "int", - "runningPartitionedDrivers"); + "remoteUserAddress", + p.remoteUserAddress, + "SessionRepresentation", + "String", + "remoteUserAddress"); from_json_key( j, - "outputBufferUtilization", - p.outputBufferUtilization, - "TaskStatus", - "double", - "outputBufferUtilization"); + "userAgent", + p.userAgent, + "SessionRepresentation", + "String", + "userAgent"); from_json_key( j, - "outputBufferOverutilized", - p.outputBufferOverutilized, - "TaskStatus", - "bool", - "outputBufferOverutilized"); + "clientInfo", + p.clientInfo, + "SessionRepresentation", + "String", + "clientInfo"); from_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "TaskStatus", - "int64_t", - "physicalWrittenDataSizeInBytes"); + "clientTags", + p.clientTags, + "SessionRepresentation", + "List", + "clientTags"); from_json_key( j, - "memoryReservationInBytes", - p.memoryReservationInBytes, - "TaskStatus", - "int64_t", - "memoryReservationInBytes"); + "resourceEstimates", + p.resourceEstimates, + "SessionRepresentation", + "ResourceEstimates", + "resourceEstimates"); from_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "TaskStatus", + "startTime", + p.startTime, + "SessionRepresentation", "int64_t", - "systemMemoryReservationInBytes"); + "startTime"); from_json_key( j, - "peakNodeTotalMemoryReservationInBytes", - p.peakNodeTotalMemoryReservationInBytes, - "TaskStatus", - "int64_t", - "peakNodeTotalMemoryReservationInBytes"); + "systemProperties", + p.systemProperties, + "SessionRepresentation", + "Map", + "systemProperties"); from_json_key( - j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); + j, + "catalogProperties", + p.catalogProperties, + "SessionRepresentation", + "Map>", + "catalogProperties"); + from_json_key( + j, + "unprocessedCatalogProperties", + p.unprocessedCatalogProperties, + "SessionRepresentation", + "Map>", + "unprocessedCatalogProperties"); + from_json_key( + j, + "roles", + p.roles, + "SessionRepresentation", + "Map", + "roles"); from_json_key( j, - "fullGcTimeInMillis", - p.fullGcTimeInMillis, - "TaskStatus", - "int64_t", - "fullGcTimeInMillis"); + "preparedStatements", + p.preparedStatements, + "SessionRepresentation", + "Map", + "preparedStatements"); from_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "TaskStatus", - "int64_t", - "totalCpuTimeInNanos"); - from_json_key( + "sessionFunctions", + p.sessionFunctions, + "SessionRepresentation", + "Map", + "sessionFunctions"); +} +} // 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 TypeCategory_enum_table[] = + { // NOLINT: cert-err58-cpp + {TypeCategory::PRIMITIVE, "PRIMITIVE"}, + {TypeCategory::STRUCT, "STRUCT"}, + {TypeCategory::ARRAY, "ARRAY"}, + {TypeCategory::MAP, "MAP"}}; +void to_json(json& j, const TypeCategory& e) { + static_assert( + std::is_enum::value, "TypeCategory must be an enum!"); + const auto* it = std::find_if( + std::begin(TypeCategory_enum_table), + std::end(TypeCategory_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(TypeCategory_enum_table)) + ? it + : std::begin(TypeCategory_enum_table)) + ->second; +} +void from_json(const json& j, TypeCategory& e) { + static_assert( + std::is_enum::value, "TypeCategory must be an enum!"); + const auto* it = std::find_if( + std::begin(TypeCategory_enum_table), + std::end(TypeCategory_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(TypeCategory_enum_table)) + ? it + : std::begin(TypeCategory_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const ColumnIdentity& p) { + j = json::object(); + to_json_key(j, "id", p.id, "ColumnIdentity", "int", "id"); + to_json_key(j, "name", p.name, "ColumnIdentity", "String", "name"); + to_json_key( j, - "taskAgeInMillis", - p.taskAgeInMillis, - "TaskStatus", - "int64_t", - "taskAgeInMillis"); + "typeCategory", + p.typeCategory, + "ColumnIdentity", + "TypeCategory", + "typeCategory"); + to_json_key( + j, + "children", + p.children, + "ColumnIdentity", + "List", + "children"); +} + +void from_json(const json& j, ColumnIdentity& p) { + from_json_key(j, "id", p.id, "ColumnIdentity", "int", "id"); + from_json_key(j, "name", p.name, "ColumnIdentity", "String", "name"); from_json_key( j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "TaskStatus", - "int64_t", - "queuedPartitionedSplitsWeight"); + "typeCategory", + p.typeCategory, + "ColumnIdentity", + "TypeCategory", + "typeCategory"); from_json_key( j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "TaskStatus", - "int64_t", - "runningPartitionedSplitsWeight"); + "children", + p.children, + "ColumnIdentity", + "List", + "children"); +} +} // 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 "); +} + +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; + } + + throw TypeError(type + " no abstract type ValueSet "); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const Domain& p) { + j = json::object(); + 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, Domain& p) { + from_json_key(j, "values", p.values, "Domain", "ValueSet", "values"); + from_json_key( + j, "nullAllowed", p.nullAllowed, "Domain", "bool", "nullAllowed"); +} +} // 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"); +} +} // 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; +} +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 { +LimitNode::LimitNode() noexcept { + _type = ".LimitNode"; +} + +void to_json(json& j, const LimitNode& 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"); +} + +void from_json(const json& j, LimitNode& 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"); +} +} // 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 TaskStats& p) { +void to_json(json& j, const InsertTableHandle& p) { j = json::object(); - to_json_key( - j, "createTime", p.createTime, "TaskStats", "DateTime", "createTime"); - to_json_key( - j, - "firstStartTime", - p.firstStartTime, - "TaskStats", - "DateTime", - "firstStartTime"); - to_json_key( - j, - "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"); + "connectorId", + p.connectorId, + "InsertTableHandle", + "ConnectorId", + "connectorId"); to_json_key( j, - "peakUserMemoryInBytes", - p.peakUserMemoryInBytes, - "TaskStats", - "int64_t", - "peakUserMemoryInBytes"); + "transactionHandle", + p.transactionHandle, + "InsertTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); to_json_key( j, - "peakNodeTotalMemoryInBytes", - p.peakNodeTotalMemoryInBytes, - "TaskStats", - "int64_t", - "peakNodeTotalMemoryInBytes"); - to_json_key( + "connectorHandle", + p.connectorHandle, + "InsertTableHandle", + "ConnectorInsertTableHandle", + "connectorHandle"); +} + +void from_json(const json& j, InsertTableHandle& p) { + from_json_key( j, - "totalScheduledTimeInNanos", - p.totalScheduledTimeInNanos, - "TaskStats", - "int64_t", - "totalScheduledTimeInNanos"); - to_json_key( + "connectorId", + p.connectorId, + "InsertTableHandle", + "ConnectorId", + "connectorId"); + from_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "TaskStats", - "int64_t", - "totalCpuTimeInNanos"); - to_json_key( + "transactionHandle", + p.transactionHandle, + "InsertTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); + from_json_key( j, - "totalBlockedTimeInNanos", - p.totalBlockedTimeInNanos, - "TaskStats", - "int64_t", - "totalBlockedTimeInNanos"); + "connectorHandle", + p.connectorHandle, + "InsertTableHandle", + "ConnectorInsertTableHandle", + "connectorHandle"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +ValuesNode::ValuesNode() noexcept { + _type = ".ValuesNode"; +} + +void to_json(json& j, const ValuesNode& p) { + j = json::object(); + j["@type"] = ".ValuesNode"; to_json_key( - j, "fullyBlocked", p.fullyBlocked, "TaskStats", "bool", "fullyBlocked"); + j, "location", p.location, "ValuesNode", "SourceLocation", "location"); + to_json_key(j, "id", p.id, "ValuesNode", "PlanNodeId", "id"); to_json_key( j, - "blockedReasons", - p.blockedReasons, - "TaskStats", - "List", - "blockedReasons"); + "outputVariables", + p.outputVariables, + "ValuesNode", + "List", + "outputVariables"); to_json_key( j, - "totalAllocationInBytes", - p.totalAllocationInBytes, - "TaskStats", - "int64_t", - "totalAllocationInBytes"); + "rows", + p.rows, + "ValuesNode", + "List>>", + "rows"); to_json_key( j, - "rawInputDataSizeInBytes", - p.rawInputDataSizeInBytes, - "TaskStats", - "int64_t", - "rawInputDataSizeInBytes"); - to_json_key( + "valuesNodeLabel", + p.valuesNodeLabel, + "ValuesNode", + "String", + "valuesNodeLabel"); +} + +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, - "rawInputPositions", - p.rawInputPositions, - "TaskStats", - "int64_t", - "rawInputPositions"); - to_json_key( + "outputVariables", + p.outputVariables, + "ValuesNode", + "List", + "outputVariables"); + from_json_key( j, - "processedInputDataSizeInBytes", - p.processedInputDataSizeInBytes, - "TaskStats", - "int64_t", - "processedInputDataSizeInBytes"); - to_json_key( + "rows", + p.rows, + "ValuesNode", + "List>>", + "rows"); + from_json_key( j, - "processedInputPositions", - p.processedInputPositions, - "TaskStats", - "int64_t", - "processedInputPositions"); + "valuesNodeLabel", + p.valuesNodeLabel, + "ValuesNode", + "String", + "valuesNodeLabel"); +} +} // 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)) + ->second; +} +void from_json(const json& j, 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), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(ColumnType_enum_table)) + ? it + : std::begin(ColumnType_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +HiveColumnHandle::HiveColumnHandle() noexcept { + _type = "hive"; +} + +void to_json(json& j, const HiveColumnHandle& p) { + j = json::object(); + j["@type"] = "hive"; + to_json_key(j, "name", p.name, "HiveColumnHandle", "String", "name"); to_json_key( - j, - "outputDataSizeInBytes", - p.outputDataSizeInBytes, - "TaskStats", - "int64_t", - "outputDataSizeInBytes"); + j, "hiveType", p.hiveType, "HiveColumnHandle", "HiveType", "hiveType"); to_json_key( j, - "outputPositions", - p.outputPositions, - "TaskStats", - "int64_t", - "outputPositions"); + "typeSignature", + p.typeSignature, + "HiveColumnHandle", + "TypeSignature", + "typeSignature"); to_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "TaskStats", - "int64_t", - "physicalWrittenDataSizeInBytes"); - to_json_key( - j, "fullGcCount", p.fullGcCount, "TaskStats", "int", "fullGcCount"); + "hiveColumnIndex", + p.hiveColumnIndex, + "HiveColumnHandle", + "int", + "hiveColumnIndex"); to_json_key( j, - "fullGcTimeInMillis", - p.fullGcTimeInMillis, - "TaskStats", - "int64_t", - "fullGcTimeInMillis"); + "columnType", + p.columnType, + "HiveColumnHandle", + "ColumnType", + "columnType"); + to_json_key(j, "comment", p.comment, "HiveColumnHandle", "String", "comment"); to_json_key( j, - "pipelines", - p.pipelines, - "TaskStats", - "List", - "pipelines"); + "requiredSubfields", + p.requiredSubfields, + "HiveColumnHandle", + "List", + "requiredSubfields"); to_json_key( j, - "runtimeStats", - p.runtimeStats, - "TaskStats", - "RuntimeStats", - "runtimeStats"); + "partialAggregation", + p.partialAggregation, + "HiveColumnHandle", + "Aggregation", + "partialAggregation"); } -void from_json(const json& j, TaskStats& p) { - from_json_key( - j, "createTime", p.createTime, "TaskStats", "DateTime", "createTime"); +void from_json(const json& j, HiveColumnHandle& p) { + p._type = j["@type"]; + from_json_key(j, "name", p.name, "HiveColumnHandle", "String", "name"); from_json_key( - j, - "firstStartTime", - p.firstStartTime, - "TaskStats", - "DateTime", - "firstStartTime"); + j, "hiveType", p.hiveType, "HiveColumnHandle", "HiveType", "hiveType"); from_json_key( j, - "lastStartTime", - p.lastStartTime, - "TaskStats", - "DateTime", - "lastStartTime"); - from_json_key( - j, "lastEndTime", p.lastEndTime, "TaskStats", "DateTime", "lastEndTime"); - from_json_key(j, "endTime", p.endTime, "TaskStats", "DateTime", "endTime"); + "typeSignature", + p.typeSignature, + "HiveColumnHandle", + "TypeSignature", + "typeSignature"); from_json_key( j, - "elapsedTimeInNanos", - p.elapsedTimeInNanos, - "TaskStats", - "int64_t", - "elapsedTimeInNanos"); + "hiveColumnIndex", + p.hiveColumnIndex, + "HiveColumnHandle", + "int", + "hiveColumnIndex"); from_json_key( j, - "queuedTimeInNanos", - p.queuedTimeInNanos, - "TaskStats", - "int64_t", - "queuedTimeInNanos"); + "columnType", + p.columnType, + "HiveColumnHandle", + "ColumnType", + "columnType"); from_json_key( - j, "totalDrivers", p.totalDrivers, "TaskStats", "int", "totalDrivers"); + j, "comment", p.comment, "HiveColumnHandle", "String", "comment"); from_json_key( - j, "queuedDrivers", p.queuedDrivers, "TaskStats", "int", "queuedDrivers"); + j, + "requiredSubfields", + p.requiredSubfields, + "HiveColumnHandle", + "List", + "requiredSubfields"); from_json_key( j, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "TaskStats", + "partialAggregation", + p.partialAggregation, + "HiveColumnHandle", + "Aggregation", + "partialAggregation"); +} +} // 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", - "queuedPartitionedDrivers"); + "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, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "TaskStats", - "int64_t", - "queuedPartitionedSplitsWeight"); + "tableBucketCount", + p.tableBucketCount, + "BucketConversion", + "int", + "tableBucketCount"); from_json_key( j, - "runningDrivers", - p.runningDrivers, - "TaskStats", + "partitionBucketCount", + p.partitionBucketCount, + "BucketConversion", "int", - "runningDrivers"); + "partitionBucketCount"); from_json_key( j, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "TaskStats", - "int", - "runningPartitionedDrivers"); - from_json_key( + "bucketColumnHandles", + p.bucketColumnHandles, + "BucketConversion", + "List", + "bucketColumnHandles"); +} +} // 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) { + 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, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "TaskStats", - "int64_t", - "runningPartitionedSplitsWeight"); - from_json_key( + "unnestVariables", + p.unnestVariables, + "UnnestNode", + "Map>", + "unnestVariables"); + to_json_key( j, - "blockedDrivers", - p.blockedDrivers, - "TaskStats", - "int", - "blockedDrivers"); + "ordinalityVariable", + p.ordinalityVariable, + "UnnestNode", + "VariableReferenceExpression", + "ordinalityVariable"); +} + +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, - "completedDrivers", - p.completedDrivers, - "TaskStats", - "int", - "completedDrivers"); + "replicateVariables", + p.replicateVariables, + "UnnestNode", + "List", + "replicateVariables"); from_json_key( j, - "cumulativeUserMemory", - p.cumulativeUserMemory, - "TaskStats", - "double", - "cumulativeUserMemory"); + "unnestVariables", + p.unnestVariables, + "UnnestNode", + "Map>", + "unnestVariables"); from_json_key( j, - "cumulativeTotalMemory", - p.cumulativeTotalMemory, - "TaskStats", - "double", - "cumulativeTotalMemory"); - from_json_key( + "ordinalityVariable", + p.ordinalityVariable, + "UnnestNode", + "VariableReferenceExpression", + "ordinalityVariable"); +} +} // 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 JoinType_enum_table[] = + { // NOLINT: cert-err58-cpp + {JoinType::INNER, "INNER"}, + {JoinType::LEFT, "LEFT"}, + {JoinType::RIGHT, "RIGHT"}, + {JoinType::FULL, "FULL"}}; +void to_json(json& j, const JoinType& e) { + static_assert(std::is_enum::value, "JoinType must be an enum!"); + const auto* it = std::find_if( + std::begin(JoinType_enum_table), + std::end(JoinType_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(JoinType_enum_table)) ? it + : std::begin(JoinType_enum_table)) + ->second; +} +void from_json(const json& j, JoinType& e) { + static_assert(std::is_enum::value, "JoinType must be an enum!"); + const auto* it = std::find_if( + std::begin(JoinType_enum_table), + std::end(JoinType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(JoinType_enum_table)) ? it + : std::begin(JoinType_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const EquiJoinClause& p) { + j = json::object(); + to_json_key( j, - "userMemoryReservationInBytes", - p.userMemoryReservationInBytes, - "TaskStats", - "int64_t", - "userMemoryReservationInBytes"); - from_json_key( + "left", + p.left, + "EquiJoinClause", + "VariableReferenceExpression", + "left"); + to_json_key( j, - "revocableMemoryReservationInBytes", - p.revocableMemoryReservationInBytes, - "TaskStats", - "int64_t", - "revocableMemoryReservationInBytes"); + "right", + p.right, + "EquiJoinClause", + "VariableReferenceExpression", + "right"); +} + +void from_json(const json& j, EquiJoinClause& p) { from_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "TaskStats", - "int64_t", - "systemMemoryReservationInBytes"); + "left", + p.left, + "EquiJoinClause", + "VariableReferenceExpression", + "left"); from_json_key( j, - "peakTotalMemoryInBytes", - p.peakTotalMemoryInBytes, - "TaskStats", - "int64_t", - "peakTotalMemoryInBytes"); - from_json_key( + "right", + p.right, + "EquiJoinClause", + "VariableReferenceExpression", + "right"); +} +} // 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", "JoinType", "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, - "peakUserMemoryInBytes", - p.peakUserMemoryInBytes, - "TaskStats", - "int64_t", - "peakUserMemoryInBytes"); - from_json_key( + "criteria", + p.criteria, + "MergeJoinNode", + "List", + "criteria"); + to_json_key( j, - "peakNodeTotalMemoryInBytes", - p.peakNodeTotalMemoryInBytes, - "TaskStats", - "int64_t", - "peakNodeTotalMemoryInBytes"); - from_json_key( + "outputVariables", + p.outputVariables, + "MergeJoinNode", + "List", + "outputVariables"); + to_json_key( j, - "totalScheduledTimeInNanos", - p.totalScheduledTimeInNanos, - "TaskStats", - "int64_t", - "totalScheduledTimeInNanos"); - from_json_key( + "filter", + p.filter, + "MergeJoinNode", + "std::shared_ptr", + "filter"); + to_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "TaskStats", - "int64_t", - "totalCpuTimeInNanos"); - from_json_key( + "leftHashVariable", + p.leftHashVariable, + "MergeJoinNode", + "VariableReferenceExpression", + "leftHashVariable"); + to_json_key( j, - "totalBlockedTimeInNanos", - p.totalBlockedTimeInNanos, - "TaskStats", - "int64_t", - "totalBlockedTimeInNanos"); - from_json_key( - j, "fullyBlocked", p.fullyBlocked, "TaskStats", "bool", "fullyBlocked"); + "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", "JoinType", "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, - "blockedReasons", - p.blockedReasons, - "TaskStats", - "List", - "blockedReasons"); + "criteria", + p.criteria, + "MergeJoinNode", + "List", + "criteria"); from_json_key( j, - "totalAllocationInBytes", - p.totalAllocationInBytes, - "TaskStats", - "int64_t", - "totalAllocationInBytes"); + "outputVariables", + p.outputVariables, + "MergeJoinNode", + "List", + "outputVariables"); from_json_key( j, - "rawInputDataSizeInBytes", - p.rawInputDataSizeInBytes, - "TaskStats", - "int64_t", - "rawInputDataSizeInBytes"); + "filter", + p.filter, + "MergeJoinNode", + "std::shared_ptr", + "filter"); from_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "TaskStats", - "int64_t", - "rawInputPositions"); + "leftHashVariable", + p.leftHashVariable, + "MergeJoinNode", + "VariableReferenceExpression", + "leftHashVariable"); from_json_key( j, - "processedInputDataSizeInBytes", - p.processedInputDataSizeInBytes, - "TaskStats", - "int64_t", - "processedInputDataSizeInBytes"); - from_json_key( + "rightHashVariable", + p.rightHashVariable, + "MergeJoinNode", + "VariableReferenceExpression", + "rightHashVariable"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +AssignUniqueId::AssignUniqueId() noexcept { + _type = "com.facebook.presto.sql.planner.plan.AssignUniqueId"; +} + +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, - "processedInputPositions", - p.processedInputPositions, - "TaskStats", - "int64_t", - "processedInputPositions"); + "idVariable", + p.idVariable, + "AssignUniqueId", + "VariableReferenceExpression", + "idVariable"); +} + +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, - "outputDataSizeInBytes", - p.outputDataSizeInBytes, - "TaskStats", - "int64_t", - "outputDataSizeInBytes"); + "idVariable", + p.idVariable, + "AssignUniqueId", + "VariableReferenceExpression", + "idVariable"); +} +} // 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, + "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, - "outputPositions", - p.outputPositions, - "TaskStats", - "int64_t", - "outputPositions"); + "cacheQuotaScope", + p.cacheQuotaScope, + "CacheQuotaRequirement", + "CacheQuotaScope", + "cacheQuotaScope"); from_json_key( + j, "quota", p.quota, "CacheQuotaRequirement", "DataSize", "quota"); +} +} // 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"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +EquatableValueSet::EquatableValueSet() noexcept { + _type = "equatable"; +} + +void to_json(json& j, const EquatableValueSet& 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"); + to_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "TaskStats", - "int64_t", - "physicalWrittenDataSizeInBytes"); + "entries", + p.entries, + "EquatableValueSet", + "List", + "entries"); +} + +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, "fullGcCount", p.fullGcCount, "TaskStats", "int", "fullGcCount"); + j, "whiteList", p.whiteList, "EquatableValueSet", "bool", "whiteList"); from_json_key( j, - "fullGcTimeInMillis", - p.fullGcTimeInMillis, - "TaskStats", - "int64_t", - "fullGcTimeInMillis"); - from_json_key( + "entries", + p.entries, + "EquatableValueSet", + "List", + "entries"); +} +} // 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 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 +namespace facebook::presto::protocol { + +void to_json(json& j, const HiveBucketFilter& p) { + j = json::object(); + to_json_key( j, - "pipelines", - p.pipelines, - "TaskStats", - "List", - "pipelines"); + "bucketsToKeep", + p.bucketsToKeep, + "HiveBucketFilter", + "List", + "bucketsToKeep"); +} + +void from_json(const json& j, HiveBucketFilter& p) { from_json_key( j, - "runtimeStats", - p.runtimeStats, - "TaskStats", - "RuntimeStats", - "runtimeStats"); + "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; -void to_json(json& j, const BufferInfo& p) { + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); + return; + } + + if (getConnectorKey(type) == "hive-iceberg") { + j = *std::static_pointer_cast(p); + return; + } + + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ColumnHandle "); +} + +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()) + " ColumnHandle ColumnHandle"); + } + + if (getConnectorKey(type) == "hive") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + + if (getConnectorKey(type) == "hive-iceberg") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + 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 +namespace facebook::presto::protocol { + +void to_json(json& j, const HiveBucketHandle& p) { j = json::object(); to_json_key( - j, "bufferId", p.bufferId, "BufferInfo", "OutputBufferId", "bufferId"); - to_json_key(j, "finished", p.finished, "BufferInfo", "bool", "finished"); + j, + "columns", + p.columns, + "HiveBucketHandle", + "List", + "columns"); to_json_key( j, - "bufferedPages", - p.bufferedPages, - "BufferInfo", + "tableBucketCount", + p.tableBucketCount, + "HiveBucketHandle", "int", - "bufferedPages"); - to_json_key( - j, "pagesSent", p.pagesSent, "BufferInfo", "int64_t", "pagesSent"); + "tableBucketCount"); to_json_key( j, - "pageBufferInfo", - p.pageBufferInfo, - "BufferInfo", - "PageBufferInfo", - "pageBufferInfo"); + "readBucketCount", + p.readBucketCount, + "HiveBucketHandle", + "int", + "readBucketCount"); } -void from_json(const json& j, BufferInfo& p) { +void from_json(const json& j, HiveBucketHandle& p) { from_json_key( - j, "bufferId", p.bufferId, "BufferInfo", "OutputBufferId", "bufferId"); - from_json_key(j, "finished", p.finished, "BufferInfo", "bool", "finished"); + j, + "columns", + p.columns, + "HiveBucketHandle", + "List", + "columns"); from_json_key( j, - "bufferedPages", - p.bufferedPages, - "BufferInfo", + "tableBucketCount", + p.tableBucketCount, + "HiveBucketHandle", "int", - "bufferedPages"); - from_json_key( - j, "pagesSent", p.pagesSent, "BufferInfo", "int64_t", "pagesSent"); + "tableBucketCount"); from_json_key( j, - "pageBufferInfo", - p.pageBufferInfo, - "BufferInfo", - "PageBufferInfo", - "pageBufferInfo"); + "readBucketCount", + p.readBucketCount, + "HiveBucketHandle", + "int", + "readBucketCount"); } } // 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[] = - { // 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!"); - const auto* it = std::find_if( - 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(BufferState_enum_table)) - ? it - : std::begin(BufferState_enum_table)) - ->second; -} -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(BufferState_enum_table), - std::end(BufferState_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(BufferState_enum_table)) - ? it - : std::begin(BufferState_enum_table)) - ->first; +HiveTableLayoutHandle::HiveTableLayoutHandle() noexcept { + _type = "hive"; } -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -void to_json(json& j, const OutputBufferInfo& p) { +void to_json(json& j, const HiveTableLayoutHandle& 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, - "canAddPages", - p.canAddPages, - "OutputBufferInfo", - "bool", - "canAddPages"); + j["@type"] = "hive"; to_json_key( j, - "totalBufferedBytes", - p.totalBufferedBytes, - "OutputBufferInfo", - "int64_t", - "totalBufferedBytes"); + "schemaTableName", + p.schemaTableName, + "HiveTableLayoutHandle", + "SchemaTableName", + "schemaTableName"); to_json_key( j, - "totalBufferedPages", - p.totalBufferedPages, - "OutputBufferInfo", - "int64_t", - "totalBufferedPages"); + "tablePath", + p.tablePath, + "HiveTableLayoutHandle", + "String", + "tablePath"); to_json_key( j, - "totalRowsSent", - p.totalRowsSent, - "OutputBufferInfo", - "int64_t", - "totalRowsSent"); + "partitionColumns", + p.partitionColumns, + "HiveTableLayoutHandle", + "List", + "partitionColumns"); to_json_key( j, - "totalPagesSent", - p.totalPagesSent, - "OutputBufferInfo", - "int64_t", - "totalPagesSent"); + "dataColumns", + p.dataColumns, + "HiveTableLayoutHandle", + "List", + "dataColumns"); to_json_key( j, - "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, - "canAddBuffers", - p.canAddBuffers, - "OutputBufferInfo", - "bool", - "canAddBuffers"); - from_json_key( + "tableParameters", + p.tableParameters, + "HiveTableLayoutHandle", + "Map", + "tableParameters"); + to_json_key( j, - "canAddPages", - p.canAddPages, - "OutputBufferInfo", - "bool", - "canAddPages"); - from_json_key( + "domainPredicate", + p.domainPredicate, + "HiveTableLayoutHandle", + "TupleDomain", + "domainPredicate"); + to_json_key( j, - "totalBufferedBytes", - p.totalBufferedBytes, - "OutputBufferInfo", - "int64_t", - "totalBufferedBytes"); - from_json_key( + "remainingPredicate", + p.remainingPredicate, + "HiveTableLayoutHandle", + "RowExpression", + "remainingPredicate"); + to_json_key( j, - "totalBufferedPages", - p.totalBufferedPages, - "OutputBufferInfo", - "int64_t", - "totalBufferedPages"); - from_json_key( + "predicateColumns", + p.predicateColumns, + "HiveTableLayoutHandle", + "Map", + "predicateColumns"); + to_json_key( j, - "totalRowsSent", - p.totalRowsSent, - "OutputBufferInfo", - "int64_t", - "totalRowsSent"); - from_json_key( + "partitionColumnPredicate", + p.partitionColumnPredicate, + "HiveTableLayoutHandle", + "TupleDomain>", + "partitionColumnPredicate"); + to_json_key( j, - "totalPagesSent", - p.totalPagesSent, - "OutputBufferInfo", - "int64_t", - "totalPagesSent"); - from_json_key( + "bucketHandle", + p.bucketHandle, + "HiveTableLayoutHandle", + "HiveBucketHandle", + "bucketHandle"); + to_json_key( j, - "buffers", - p.buffers, - "OutputBufferInfo", - "List", - "buffers"); -} -} // 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"); + "bucketFilter", + p.bucketFilter, + "HiveTableLayoutHandle", + "HiveBucketFilter", + "bucketFilter"); to_json_key( - j, "taskStatus", p.taskStatus, "TaskInfo", "TaskStatus", "taskStatus"); + j, + "pushdownFilterEnabled", + p.pushdownFilterEnabled, + "HiveTableLayoutHandle", + "bool", + "pushdownFilterEnabled"); to_json_key( j, - "lastHeartbeat", - p.lastHeartbeat, - "TaskInfo", - "DateTime", - "lastHeartbeat"); + "layoutString", + p.layoutString, + "HiveTableLayoutHandle", + "String", + "layoutString"); to_json_key( j, - "outputBuffers", - p.outputBuffers, - "TaskInfo", - "OutputBufferInfo", - "outputBuffers"); + "requestedColumns", + p.requestedColumns, + "HiveTableLayoutHandle", + "List", + "requestedColumns"); 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"); + "partialAggregationsPushedDown", + p.partialAggregationsPushedDown, + "HiveTableLayoutHandle", + "bool", + "partialAggregationsPushedDown"); to_json_key( j, - "metadataUpdates", - p.metadataUpdates, - "TaskInfo", - "MetadataUpdates", - "metadataUpdates"); - to_json_key(j, "nodeId", p.nodeId, "TaskInfo", "String", "nodeId"); + "appendRowNumber", + p.appendRowNumber, + "HiveTableLayoutHandle", + "bool", + "appendRowNumber"); + to_json_key( + j, + "footerStatsUnreliable", + p.footerStatsUnreliable, + "HiveTableLayoutHandle", + "bool", + "footerStatsUnreliable"); } -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( - j, - "lastHeartbeat", - p.lastHeartbeat, - "TaskInfo", - "DateTime", - "lastHeartbeat"); +void from_json(const json& j, HiveTableLayoutHandle& p) { + p._type = j["@type"]; from_json_key( j, - "outputBuffers", - p.outputBuffers, - "TaskInfo", - "OutputBufferInfo", - "outputBuffers"); + "schemaTableName", + p.schemaTableName, + "HiveTableLayoutHandle", + "SchemaTableName", + "schemaTableName"); from_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"); + "tablePath", + p.tablePath, + "HiveTableLayoutHandle", + "String", + "tablePath"); from_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 LongVariableConstraint& p) { - j = json::object(); - to_json_key(j, "name", p.name, "LongVariableConstraint", "String", "name"); - to_json_key( - j, - "expression", - p.expression, - "LongVariableConstraint", - "String", - "expression"); -} - -void from_json(const json& j, LongVariableConstraint& p) { - from_json_key(j, "name", p.name, "LongVariableConstraint", "String", "name"); + "partitionColumns", + p.partitionColumns, + "HiveTableLayoutHandle", + "List", + "partitionColumns"); from_json_key( j, - "expression", - p.expression, - "LongVariableConstraint", - "String", - "expression"); -} -} // 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 Signature& 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( + "dataColumns", + p.dataColumns, + "HiveTableLayoutHandle", + "List", + "dataColumns"); + from_json_key( j, - "typeVariableConstraints", - p.typeVariableConstraints, - "Signature", - "List", - "typeVariableConstraints"); - to_json_key( + "tableParameters", + p.tableParameters, + "HiveTableLayoutHandle", + "Map", + "tableParameters"); + from_json_key( j, - "longVariableConstraints", - p.longVariableConstraints, - "Signature", - "List", - "longVariableConstraints"); - to_json_key( + "domainPredicate", + p.domainPredicate, + "HiveTableLayoutHandle", + "TupleDomain", + "domainPredicate"); + from_json_key( j, - "returnType", - p.returnType, - "Signature", - "TypeSignature", - "returnType"); - to_json_key( + "remainingPredicate", + p.remainingPredicate, + "HiveTableLayoutHandle", + "RowExpression", + "remainingPredicate"); + from_json_key( j, - "argumentTypes", - p.argumentTypes, - "Signature", - "List", - "argumentTypes"); - to_json_key( + "predicateColumns", + p.predicateColumns, + "HiveTableLayoutHandle", + "Map", + "predicateColumns"); + from_json_key( j, - "variableArity", - p.variableArity, - "Signature", - "bool", - "variableArity"); -} - -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"); + "partitionColumnPredicate", + p.partitionColumnPredicate, + "HiveTableLayoutHandle", + "TupleDomain>", + "partitionColumnPredicate"); from_json_key( j, - "typeVariableConstraints", - p.typeVariableConstraints, - "Signature", - "List", - "typeVariableConstraints"); + "bucketHandle", + p.bucketHandle, + "HiveTableLayoutHandle", + "HiveBucketHandle", + "bucketHandle"); from_json_key( j, - "longVariableConstraints", - p.longVariableConstraints, - "Signature", - "List", - "longVariableConstraints"); + "bucketFilter", + p.bucketFilter, + "HiveTableLayoutHandle", + "HiveBucketFilter", + "bucketFilter"); from_json_key( j, - "returnType", - p.returnType, - "Signature", - "TypeSignature", - "returnType"); + "pushdownFilterEnabled", + p.pushdownFilterEnabled, + "HiveTableLayoutHandle", + "bool", + "pushdownFilterEnabled"); from_json_key( j, - "argumentTypes", - p.argumentTypes, - "Signature", - "List", - "argumentTypes"); + "layoutString", + p.layoutString, + "HiveTableLayoutHandle", + "String", + "layoutString"); from_json_key( j, - "variableArity", - p.variableArity, - "Signature", + "requestedColumns", + p.requestedColumns, + "HiveTableLayoutHandle", + "List", + "requestedColumns"); + from_json_key( + j, + "partialAggregationsPushedDown", + p.partialAggregationsPushedDown, + "HiveTableLayoutHandle", "bool", - "variableArity"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -BuiltInFunctionHandle::BuiltInFunctionHandle() noexcept { - _type = "$static"; -} - -void to_json(json& j, const BuiltInFunctionHandle& p) { - j = json::object(); - j["@type"] = "$static"; - to_json_key( + "partialAggregationsPushedDown"); + from_json_key( j, - "signature", - p.signature, - "BuiltInFunctionHandle", - "Signature", - "signature"); -} - -void from_json(const json& j, BuiltInFunctionHandle& p) { - p._type = j["@type"]; + "appendRowNumber", + p.appendRowNumber, + "HiveTableLayoutHandle", + "bool", + "appendRowNumber"); from_json_key( j, - "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 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; + "footerStatsUnreliable", + p.footerStatsUnreliable, + "HiveTableLayoutHandle", + "bool", + "footerStatsUnreliable"); } } // 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 HiveFileSplit& 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, "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, - "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( + "fileModifiedTime", + p.fileModifiedTime, + "HiveFileSplit", + "int64_t", + "fileModifiedTime"); + to_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 { -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"); + "extraFileInfo", + p.extraFileInfo, + "HiveFileSplit", + "String", + "extraFileInfo"); to_json_key( - j, "predicate", p.predicate, "FilterNode", "RowExpression", "predicate"); + j, + "customSplitInfo", + p.customSplitInfo, + "HiveFileSplit", + "Map", + "customSplitInfo"); } -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"); +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, "predicate", p.predicate, "FilterNode", "RowExpression", "predicate"); + 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 { -OutputNode::OutputNode() noexcept { - _type = ".OutputNode"; -} -void to_json(json& j, const OutputNode& p) { +void to_json(json& j, const RefreshMaterializedViewHandle& p) { j = json::object(); - j["@type"] = ".OutputNode"; - to_json_key(j, "id", p.id, "OutputNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "OutputNode", "PlanNode", "source"); to_json_key( j, - "columnNames", - p.columnNames, - "OutputNode", - "List", - "columnNames"); + "handle", + p.handle, + "RefreshMaterializedViewHandle", + "InsertTableHandle", + "handle"); to_json_key( j, - "outputVariables", - p.outputVariables, - "OutputNode", - "List", - "outputVariables"); + "schemaTableName", + p.schemaTableName, + "RefreshMaterializedViewHandle", + "SchemaTableName", + "schemaTableName"); } -void from_json(const json& j, OutputNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "OutputNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "OutputNode", "PlanNode", "source"); +void from_json(const json& j, RefreshMaterializedViewHandle& p) { from_json_key( j, - "columnNames", - p.columnNames, - "OutputNode", - "List", - "columnNames"); + "handle", + p.handle, + "RefreshMaterializedViewHandle", + "InsertTableHandle", + "handle"); from_json_key( j, - "outputVariables", - p.outputVariables, - "OutputNode", - "List", - "outputVariables"); + "schemaTableName", + p.schemaTableName, + "RefreshMaterializedViewHandle", + "SchemaTableName", + "schemaTableName"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const ResourceEstimates& p) { +void to_json(json& j, const DeleteScanInfo& p) { j = json::object(); + to_json_key(j, "id", p.id, "DeleteScanInfo", "PlanNodeId", "id"); to_json_key( j, - "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", - "peakMemory"); - to_json_key( - j, - "peakTaskMemory", - p.peakTaskMemory, - "ResourceEstimates", - "DataSize", - "peakTaskMemory"); + "tableHandle", + p.tableHandle, + "DeleteScanInfo", + "TableHandle", + "tableHandle"); } -void from_json(const json& j, ResourceEstimates& p) { - from_json_key( - j, - "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", - "peakMemory"); +void from_json(const json& j, DeleteScanInfo& p) { + from_json_key(j, "id", p.id, "DeleteScanInfo", "PlanNodeId", "id"); from_json_key( j, - "peakTaskMemory", - p.peakTaskMemory, - "ResourceEstimates", - "DataSize", - "peakTaskMemory"); + "tableHandle", + p.tableHandle, + "DeleteScanInfo", + "TableHandle", + "tableHandle"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; -// 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)) - ->second; + 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, 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), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(ColumnType_enum_table)) - ? it - : std::begin(ColumnType_enum_table)) - ->first; + +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 BaseHiveColumnHandle& p) { +void to_json(json& j, const TableWriteInfo& p) { j = json::object(); - to_json_key(j, "name", p.name, "BaseHiveColumnHandle", "String", "name"); to_json_key( - j, "comment", p.comment, "BaseHiveColumnHandle", "String", "comment"); + j, + "writerTarget", + p.writerTarget, + "TableWriteInfo", + "ExecutionWriterTarget", + "writerTarget"); to_json_key( j, - "columnType", - p.columnType, - "BaseHiveColumnHandle", - "ColumnType", - "columnType"); + "analyzeTableHandle", + p.analyzeTableHandle, + "TableWriteInfo", + "AnalyzeTableHandle", + "analyzeTableHandle"); to_json_key( j, - "requiredSubfields", - p.requiredSubfields, - "BaseHiveColumnHandle", - "List", - "requiredSubfields"); + "deleteScanInfo", + p.deleteScanInfo, + "TableWriteInfo", + "DeleteScanInfo", + "deleteScanInfo"); } -void from_json(const json& j, BaseHiveColumnHandle& p) { - from_json_key(j, "name", p.name, "BaseHiveColumnHandle", "String", "name"); +void from_json(const json& j, TableWriteInfo& p) { from_json_key( - j, "comment", p.comment, "BaseHiveColumnHandle", "String", "comment"); + j, + "writerTarget", + p.writerTarget, + "TableWriteInfo", + "ExecutionWriterTarget", + "writerTarget"); from_json_key( j, - "columnType", - p.columnType, - "BaseHiveColumnHandle", - "ColumnType", - "columnType"); + "analyzeTableHandle", + p.analyzeTableHandle, + "TableWriteInfo", + "AnalyzeTableHandle", + "analyzeTableHandle"); from_json_key( j, - "requiredSubfields", - p.requiredSubfields, - "BaseHiveColumnHandle", - "List", - "requiredSubfields"); + "deleteScanInfo", + p.deleteScanInfo, + "TableWriteInfo", + "DeleteScanInfo", + "deleteScanInfo"); } } // 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 + JoinDistributionType_enum_table[] = { // NOLINT: cert-err58-cpp + {JoinDistributionType::PARTITIONED, "PARTITIONED"}, + {JoinDistributionType::REPLICATED, "REPLICATED"}}; +void to_json(json& j, const JoinDistributionType& e) { static_assert( - std::is_enum::value, - "SelectedRoleType must be an enum!"); + std::is_enum::value, + "JoinDistributionType 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(JoinDistributionType_enum_table), + std::end(JoinDistributionType_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(JoinDistributionType_enum_table)) ? it - : std::begin(SelectedRoleType_enum_table)) + : std::begin(JoinDistributionType_enum_table)) ->second; } -void from_json(const json& j, SelectedRoleType& e) { +void from_json(const json& j, JoinDistributionType& e) { static_assert( - std::is_enum::value, - "SelectedRoleType must be an enum!"); + std::is_enum::value, + "JoinDistributionType 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(JoinDistributionType_enum_table), + std::end(JoinDistributionType_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(JoinDistributionType_enum_table)) ? it - : std::begin(SelectedRoleType_enum_table)) + : std::begin(JoinDistributionType_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"); +JoinNode::JoinNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.JoinNode"; } -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"); +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", "JoinType", "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( + j, + "outputVariables", + p.outputVariables, + "JoinNode", + "List", + "outputVariables"); + to_json_key( + j, + "filter", + p.filter, + "JoinNode", + "std::shared_ptr", + "filter"); + to_json_key( + j, + "leftHashVariable", + p.leftHashVariable, + "JoinNode", + "VariableReferenceExpression", + "leftHashVariable"); + to_json_key( + j, + "rightHashVariable", + p.rightHashVariable, + "JoinNode", + "VariableReferenceExpression", + "rightHashVariable"); + to_json_key( + j, + "distributionType", + p.distributionType, + "JoinNode", + "JoinDistributionType", + "distributionType"); + to_json_key( + j, + "dynamicFilters", + p.dynamicFilters, + "JoinNode", + "Map", + "dynamicFilters"); } -} // 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; +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", "JoinType", "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, + "criteria", + p.criteria, + "JoinNode", + "List", + "criteria"); + from_json_key( + j, + "outputVariables", + p.outputVariables, + "JoinNode", + "List", + "outputVariables"); + from_json_key( + j, + "filter", + p.filter, + "JoinNode", + "std::shared_ptr", + "filter"); + from_json_key( + j, + "leftHashVariable", + p.leftHashVariable, + "JoinNode", + "VariableReferenceExpression", + "leftHashVariable"); + from_json_key( + j, + "rightHashVariable", + p.rightHashVariable, + "JoinNode", + "VariableReferenceExpression", + "rightHashVariable"); + from_json_key( + j, + "distributionType", + p.distributionType, + "JoinNode", + "JoinDistributionType", + "distributionType"); + from_json_key( + j, + "dynamicFilters", + p.dynamicFilters, + "JoinNode", + "Map", + "dynamicFilters"); } } // 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"}, - {WindowType::GROUPS, "GROUPS"}, -}; -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; +/* + * 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 { + +std::string JsonEncodedSubclass::getSubclassKey(nlohmann::json j) { + return j["@type"]; } + } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Frame& p) { +void to_json(json& j, const DwrfEncryptionMetadata& 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, - "sortKeyCoercedForFrameStartComparison", - p.sortKeyCoercedForFrameStartComparison, - "Frame", - "VariableReferenceExpression", - "sortKeyCoercedForFrameStartComparison"); - to_json_key(j, "endType", p.endType, "Frame", "BoundType", "endType"); - to_json_key( - j, - "endValue", - p.endValue, - "Frame", - "VariableReferenceExpression", - "endValue"); + "fieldToKeyData", + p.fieldToKeyData, + "DwrfEncryptionMetadata", + "Map", + "fieldToKeyData"); to_json_key( j, - "sortKeyCoercedForFrameEndComparison", - p.sortKeyCoercedForFrameEndComparison, - "Frame", - "VariableReferenceExpression", - "sortKeyCoercedForFrameEndComparison"); + "extraMetadata", + p.extraMetadata, + "DwrfEncryptionMetadata", + "Map", + "extraMetadata"); to_json_key( j, - "originalStartValue", - p.originalStartValue, - "Frame", + "encryptionAlgorithm", + p.encryptionAlgorithm, + "DwrfEncryptionMetadata", "String", - "originalStartValue"); + "encryptionAlgorithm"); to_json_key( j, - "originalEndValue", - p.originalEndValue, - "Frame", + "encryptionProvider", + p.encryptionProvider, + "DwrfEncryptionMetadata", "String", - "originalEndValue"); + "encryptionProvider"); } -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, - "sortKeyCoercedForFrameStartComparison", - p.sortKeyCoercedForFrameStartComparison, - "Frame", - "VariableReferenceExpression", - "sortKeyCoercedForFrameStartComparison"); - from_json_key(j, "endType", p.endType, "Frame", "BoundType", "endType"); +void from_json(const json& j, DwrfEncryptionMetadata& p) { from_json_key( j, - "endValue", - p.endValue, - "Frame", - "VariableReferenceExpression", - "endValue"); + "fieldToKeyData", + p.fieldToKeyData, + "DwrfEncryptionMetadata", + "Map", + "fieldToKeyData"); from_json_key( j, - "sortKeyCoercedForFrameEndComparison", - p.sortKeyCoercedForFrameEndComparison, - "Frame", - "VariableReferenceExpression", - "sortKeyCoercedForFrameEndComparison"); + "extraMetadata", + p.extraMetadata, + "DwrfEncryptionMetadata", + "Map", + "extraMetadata"); from_json_key( j, - "originalStartValue", - p.originalStartValue, - "Frame", + "encryptionAlgorithm", + p.encryptionAlgorithm, + "DwrfEncryptionMetadata", "String", - "originalStartValue"); + "encryptionAlgorithm"); from_json_key( j, - "originalEndValue", - p.originalEndValue, - "Frame", + "encryptionProvider", + p.encryptionProvider, + "DwrfEncryptionMetadata", "String", - "originalEndValue"); + "encryptionProvider"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Function& p) { +void to_json(json& j, const JoinNodeStatsEstimate& p) { j = json::object(); to_json_key( j, - "functionCall", - p.functionCall, - "Function", - "CallExpression", - "functionCall"); - to_json_key(j, "frame", p.frame, "Function", "Frame", "frame"); + "nullJoinBuildKeyCount", + p.nullJoinBuildKeyCount, + "JoinNodeStatsEstimate", + "double", + "nullJoinBuildKeyCount"); to_json_key( - j, "ignoreNulls", p.ignoreNulls, "Function", "bool", "ignoreNulls"); + j, + "joinBuildKeyCount", + p.joinBuildKeyCount, + "JoinNodeStatsEstimate", + "double", + "joinBuildKeyCount"); + to_json_key( + j, + "nullJoinProbeKeyCount", + p.nullJoinProbeKeyCount, + "JoinNodeStatsEstimate", + "double", + "nullJoinProbeKeyCount"); + to_json_key( + j, + "joinProbeKeyCount", + p.joinProbeKeyCount, + "JoinNodeStatsEstimate", + "double", + "joinProbeKeyCount"); } -void from_json(const json& j, Function& p) { +void from_json(const json& j, JoinNodeStatsEstimate& p) { from_json_key( j, - "functionCall", - p.functionCall, - "Function", - "CallExpression", - "functionCall"); - from_json_key(j, "frame", p.frame, "Function", "Frame", "frame"); + "nullJoinBuildKeyCount", + p.nullJoinBuildKeyCount, + "JoinNodeStatsEstimate", + "double", + "nullJoinBuildKeyCount"); from_json_key( - j, "ignoreNulls", p.ignoreNulls, "Function", "bool", "ignoreNulls"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -ConstantExpression::ConstantExpression() noexcept { - _type = "constant"; -} - -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"); -} - -void from_json(const json& j, ConstantExpression& p) { - p._type = j["@type"]; + "joinBuildKeyCount", + p.joinBuildKeyCount, + "JoinNodeStatsEstimate", + "double", + "joinBuildKeyCount"); from_json_key( j, - "valueBlock", - p.valueBlock, - "ConstantExpression", - "Block", - "valueBlock"); - from_json_key(j, "type", p.type, "ConstantExpression", "Type", "type"); + "nullJoinProbeKeyCount", + p.nullJoinProbeKeyCount, + "JoinNodeStatsEstimate", + "double", + "nullJoinProbeKeyCount"); + from_json_key( + j, + "joinProbeKeyCount", + p.joinProbeKeyCount, + "JoinNodeStatsEstimate", + "double", + "joinProbeKeyCount"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -RemoteTransactionHandle::RemoteTransactionHandle() noexcept { - _type = "$remote"; -} - -void to_json(json& j, const RemoteTransactionHandle& p) { - j = json::object(); - j["@type"] = "$remote"; - to_json_key( - j, "dummy", p.dummy, "RemoteTransactionHandle", "String", "dummy"); -} +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void from_json(const json& j, RemoteTransactionHandle& p) { - p._type = j["@type"]; - from_json_key( - j, "dummy", p.dummy, "RemoteTransactionHandle", "String", "dummy"); +// 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, 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 namespace facebook::presto::protocol { -TableWriterMergeNode::TableWriterMergeNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.TableWriterMergeNode"; -} -void to_json(json& j, const TableWriterMergeNode& p) { +void to_json(json& j, const HiveBucketProperty& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.TableWriterMergeNode"; - to_json_key(j, "id", p.id, "TableWriterMergeNode", "PlanNodeId", "id"); - to_json_key( - j, "source", p.source, "TableWriterMergeNode", "PlanNode", "source"); to_json_key( j, - "rowCountVariable", - p.rowCountVariable, - "TableWriterMergeNode", - "VariableReferenceExpression", - "rowCountVariable"); + "bucketedBy", + p.bucketedBy, + "HiveBucketProperty", + "List", + "bucketedBy"); to_json_key( j, - "fragmentVariable", - p.fragmentVariable, - "TableWriterMergeNode", - "VariableReferenceExpression", - "fragmentVariable"); + "bucketCount", + p.bucketCount, + "HiveBucketProperty", + "int", + "bucketCount"); to_json_key( j, - "tableCommitContextVariable", - p.tableCommitContextVariable, - "TableWriterMergeNode", - "VariableReferenceExpression", - "tableCommitContextVariable"); + "sortedBy", + p.sortedBy, + "HiveBucketProperty", + "List", + "sortedBy"); to_json_key( j, - "statisticsAggregation", - p.statisticsAggregation, - "TableWriterMergeNode", - "StatisticAggregations", - "statisticsAggregation"); + "bucketFunctionType", + p.bucketFunctionType, + "HiveBucketProperty", + "BucketFunctionType", + "bucketFunctionType"); + to_json_key(j, "types", p.types, "HiveBucketProperty", "List", "types"); } -void from_json(const json& j, TableWriterMergeNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "TableWriterMergeNode", "PlanNodeId", "id"); - from_json_key( - j, "source", p.source, "TableWriterMergeNode", "PlanNode", "source"); +void from_json(const json& j, HiveBucketProperty& p) { from_json_key( j, - "rowCountVariable", - p.rowCountVariable, - "TableWriterMergeNode", - "VariableReferenceExpression", - "rowCountVariable"); + "bucketedBy", + p.bucketedBy, + "HiveBucketProperty", + "List", + "bucketedBy"); from_json_key( j, - "fragmentVariable", - p.fragmentVariable, - "TableWriterMergeNode", - "VariableReferenceExpression", - "fragmentVariable"); + "bucketCount", + p.bucketCount, + "HiveBucketProperty", + "int", + "bucketCount"); from_json_key( j, - "tableCommitContextVariable", - p.tableCommitContextVariable, - "TableWriterMergeNode", - "VariableReferenceExpression", - "tableCommitContextVariable"); + "sortedBy", + p.sortedBy, + "HiveBucketProperty", + "List", + "sortedBy"); from_json_key( j, - "statisticsAggregation", - p.statisticsAggregation, - "TableWriterMergeNode", - "StatisticAggregations", - "statisticsAggregation"); + "bucketFunctionType", + p.bucketFunctionType, + "HiveBucketProperty", + "BucketFunctionType", + "bucketFunctionType"); + from_json_key( + j, "types", p.types, "HiveBucketProperty", "List", "types"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveColumnHandle::HiveColumnHandle() noexcept { - _type = "hive"; +// 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 HiveColumnHandle& p) { +void to_json(json& j, const StorageFormat& p) { j = json::object(); - j["@type"] = "hive"; - to_json_key(j, "name", p.name, "HiveColumnHandle", "String", "name"); - to_json_key( - j, "hiveType", p.hiveType, "HiveColumnHandle", "HiveType", "hiveType"); - to_json_key( - j, - "typeSignature", - p.typeSignature, - "HiveColumnHandle", - "TypeSignature", - "typeSignature"); - to_json_key( - j, - "hiveColumnIndex", - p.hiveColumnIndex, - "HiveColumnHandle", - "int", - "hiveColumnIndex"); - to_json_key( - j, - "columnType", - p.columnType, - "HiveColumnHandle", - "ColumnType", - "columnType"); - to_json_key(j, "comment", p.comment, "HiveColumnHandle", "String", "comment"); + to_json_key(j, "serDe", p.serDe, "StorageFormat", "String", "serDe"); to_json_key( j, - "requiredSubfields", - p.requiredSubfields, - "HiveColumnHandle", - "List", - "requiredSubfields"); + "inputFormat", + p.inputFormat, + "StorageFormat", + "String", + "inputFormat"); to_json_key( j, - "partialAggregation", - p.partialAggregation, - "HiveColumnHandle", - "Aggregation", - "partialAggregation"); + "outputFormat", + p.outputFormat, + "StorageFormat", + "String", + "outputFormat"); } -void from_json(const json& j, HiveColumnHandle& p) { - p._type = j["@type"]; - from_json_key(j, "name", p.name, "HiveColumnHandle", "String", "name"); - from_json_key( - j, "hiveType", p.hiveType, "HiveColumnHandle", "HiveType", "hiveType"); - from_json_key( - j, - "typeSignature", - p.typeSignature, - "HiveColumnHandle", - "TypeSignature", - "typeSignature"); - from_json_key( - j, - "hiveColumnIndex", - p.hiveColumnIndex, - "HiveColumnHandle", - "int", - "hiveColumnIndex"); - from_json_key( - j, - "columnType", - p.columnType, - "HiveColumnHandle", - "ColumnType", - "columnType"); - from_json_key( - j, "comment", p.comment, "HiveColumnHandle", "String", "comment"); +void from_json(const json& j, StorageFormat& p) { + from_json_key(j, "serDe", p.serDe, "StorageFormat", "String", "serDe"); from_json_key( j, - "requiredSubfields", - p.requiredSubfields, - "HiveColumnHandle", - "List", - "requiredSubfields"); + "inputFormat", + p.inputFormat, + "StorageFormat", + "String", + "inputFormat"); from_json_key( j, - "partialAggregation", - p.partialAggregation, - "HiveColumnHandle", - "Aggregation", - "partialAggregation"); + "outputFormat", + p.outputFormat, + "StorageFormat", + "String", + "outputFormat"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const BucketConversion& p) { +void to_json(json& j, const Storage& p) { j = json::object(); to_json_key( j, - "tableBucketCount", - p.tableBucketCount, - "BucketConversion", - "int", - "tableBucketCount"); + "storageFormat", + p.storageFormat, + "Storage", + "StorageFormat", + "storageFormat"); + to_json_key(j, "location", p.location, "Storage", "String", "location"); to_json_key( j, - "partitionBucketCount", - p.partitionBucketCount, - "BucketConversion", - "int", - "partitionBucketCount"); + "bucketProperty", + p.bucketProperty, + "Storage", + "HiveBucketProperty", + "bucketProperty"); + to_json_key(j, "skewed", p.skewed, "Storage", "bool", "skewed"); to_json_key( j, - "bucketColumnHandles", - p.bucketColumnHandles, - "BucketConversion", - "List", - "bucketColumnHandles"); + "serdeParameters", + p.serdeParameters, + "Storage", + "Map", + "serdeParameters"); + to_json_key( + j, + "parameters", + p.parameters, + "Storage", + "Map", + "parameters"); } -void from_json(const json& j, BucketConversion& p) { +void from_json(const json& j, Storage& p) { from_json_key( j, - "tableBucketCount", - p.tableBucketCount, - "BucketConversion", - "int", - "tableBucketCount"); + "storageFormat", + p.storageFormat, + "Storage", + "StorageFormat", + "storageFormat"); + from_json_key(j, "location", p.location, "Storage", "String", "location"); from_json_key( j, - "partitionBucketCount", - p.partitionBucketCount, - "BucketConversion", - "int", - "partitionBucketCount"); + "bucketProperty", + p.bucketProperty, + "Storage", + "HiveBucketProperty", + "bucketProperty"); + from_json_key(j, "skewed", p.skewed, "Storage", "bool", "skewed"); from_json_key( j, - "bucketColumnHandles", - p.bucketColumnHandles, - "BucketConversion", - "List", - "bucketColumnHandles"); -} -} // 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 -/* - * 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"); + "serdeParameters", + p.serdeParameters, + "Storage", + "Map", + "serdeParameters"); + from_json_key( + j, + "parameters", + p.parameters, + "Storage", + "Map", + "parameters"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const TableHandle& p) { +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, - "connectorId", - p.connectorId, - "TableHandle", - "ConnectorId", - "connectorId"); + "partitionColumns", + p.partitionColumns, + "Table", + "List", + "partitionColumns"); to_json_key( j, - "connectorHandle", - p.connectorHandle, - "TableHandle", - "ConnectorTableHandle", - "connectorHandle"); + "parameters", + p.parameters, + "Table", + "Map", + "parameters"); to_json_key( j, - "transaction", - p.transaction, - "TableHandle", - "ConnectorTransactionHandle", - "transaction"); + "viewOriginalText", + p.viewOriginalText, + "Table", + "String", + "viewOriginalText"); to_json_key( j, - "connectorTableLayout", - p.connectorTableLayout, - "TableHandle", - "ConnectorTableLayoutHandle", - "connectorTableLayout"); + "viewExpandedText", + p.viewExpandedText, + "Table", + "String", + "viewExpandedText"); } -void from_json(const json& j, TableHandle& p) { +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, - "connectorId", - p.connectorId, - "TableHandle", - "ConnectorId", - "connectorId"); + "partitionColumns", + p.partitionColumns, + "Table", + "List", + "partitionColumns"); from_json_key( j, - "connectorHandle", - p.connectorHandle, - "TableHandle", - "ConnectorTableHandle", - "connectorHandle"); + "parameters", + p.parameters, + "Table", + "Map", + "parameters"); from_json_key( j, - "transaction", - p.transaction, - "TableHandle", - "ConnectorTransactionHandle", - "transaction"); + "viewOriginalText", + p.viewOriginalText, + "Table", + "String", + "viewOriginalText"); from_json_key( j, - "connectorTableLayout", - p.connectorTableLayout, - "TableHandle", - "ConnectorTableLayoutHandle", - "connectorTableLayout"); + "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 { -DeleteHandle::DeleteHandle() noexcept { - _type = "DeleteHandle"; -} -void to_json(json& j, const DeleteHandle& p) { +void to_json(json& j, const HivePageSinkMetadata& 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", + "HivePageSinkMetadata", "SchemaTableName", "schemaTableName"); + to_json_key(j, "table", p.table, "HivePageSinkMetadata", "Table", "table"); } -void from_json(const json& j, DeleteHandle& p) { - p._type = j["@type"]; - from_json_key(j, "handle", p.handle, "DeleteHandle", "TableHandle", "handle"); +void from_json(const json& j, HivePageSinkMetadata& p) { from_json_key( j, "schemaTableName", p.schemaTableName, - "DeleteHandle", + "HivePageSinkMetadata", "SchemaTableName", "schemaTableName"); + from_json_key(j, "table", p.table, "HivePageSinkMetadata", "Table", "table"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { @@ -7970,6 +7994,68 @@ void from_json(const json& j, HiveCompressionCodec& e) { ->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 { + +static const std::pair HiveStorageFormat_enum_table[] = + { // NOLINT: cert-err58-cpp + {HiveStorageFormat::ORC, "ORC"}, + {HiveStorageFormat::DWRF, "DWRF"}, + {HiveStorageFormat::ALPHA, "ALPHA"}, + {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 to_json(json& j, const HiveStorageFormat& p) { + 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), + [&p](const std::pair& ej_pair) -> bool { + return ej_pair.first == p; + }); + j = ((it != std::end(HiveStorageFormat_enum_table)) + ? it + : std::begin(HiveStorageFormat_enum_table)) + ->second; +} + +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() @@ -8074,776 +8160,704 @@ void from_json(const json& j, LocationHandle& p) { j, "writeMode", p.writeMode, "LocationHandle", "WriteMode", "writeMode"); } } // 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 { -static const std::pair HiveStorageFormat_enum_table[] = - { // NOLINT: cert-err58-cpp - {HiveStorageFormat::ORC, "ORC"}, - {HiveStorageFormat::DWRF, "DWRF"}, - {HiveStorageFormat::ALPHA, "ALPHA"}, - {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 to_json(json& j, const HiveStorageFormat& p) { - 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), - [&p](const std::pair& ej_pair) -> bool { - return ej_pair.first == p; - }); - j = ((it != std::end(HiveStorageFormat_enum_table)) - ? it - : std::begin(HiveStorageFormat_enum_table)) - ->second; +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, 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; +void from_json(const json& j, EncryptionInformation& p) { + 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 - 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, 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; +HiveOutputTableHandle::HiveOutputTableHandle() noexcept { + _type = "hive"; } -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -void to_json(json& j, const HiveBucketProperty& p) { +void to_json(json& j, const HiveOutputTableHandle& p) { j = json::object(); + j["@type"] = "hive"; + to_json_key( + j, + "schemaName", + p.schemaName, + "HiveOutputTableHandle", + "String", + "schemaName"); to_json_key( j, - "bucketedBy", - p.bucketedBy, - "HiveBucketProperty", + "tableName", + p.tableName, + "HiveOutputTableHandle", + "String", + "tableName"); + to_json_key( + j, + "inputColumns", + p.inputColumns, + "HiveOutputTableHandle", + "List", + "inputColumns"); + to_json_key( + j, + "pageSinkMetadata", + p.pageSinkMetadata, + "HiveOutputTableHandle", + "HivePageSinkMetadata", + "pageSinkMetadata"); + to_json_key( + j, + "locationHandle", + p.locationHandle, + "HiveOutputTableHandle", + "LocationHandle", + "locationHandle"); + to_json_key( + j, + "tableStorageFormat", + p.tableStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "tableStorageFormat"); + to_json_key( + j, + "partitionStorageFormat", + p.partitionStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "partitionStorageFormat"); + to_json_key( + j, + "actualStorageFormat", + p.actualStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "actualStorageFormat"); + to_json_key( + j, + "compressionCodec", + p.compressionCodec, + "HiveOutputTableHandle", + "HiveCompressionCodec", + "compressionCodec"); + to_json_key( + j, + "partitionedBy", + p.partitionedBy, + "HiveOutputTableHandle", "List", - "bucketedBy"); + "partitionedBy"); to_json_key( j, - "bucketCount", - p.bucketCount, + "bucketProperty", + p.bucketProperty, + "HiveOutputTableHandle", "HiveBucketProperty", - "int", - "bucketCount"); + "bucketProperty"); to_json_key( j, - "sortedBy", - p.sortedBy, - "HiveBucketProperty", + "preferredOrderingColumns", + p.preferredOrderingColumns, + "HiveOutputTableHandle", "List", - "sortedBy"); + "preferredOrderingColumns"); to_json_key( j, - "bucketFunctionType", - p.bucketFunctionType, - "HiveBucketProperty", - "BucketFunctionType", - "bucketFunctionType"); - to_json_key(j, "types", p.types, "HiveBucketProperty", "List", "types"); + "tableOwner", + p.tableOwner, + "HiveOutputTableHandle", + "String", + "tableOwner"); + to_json_key( + j, + "additionalTableParameters", + p.additionalTableParameters, + "HiveOutputTableHandle", + "Map", + "additionalTableParameters"); + to_json_key( + j, + "encryptionInformation", + p.encryptionInformation, + "HiveOutputTableHandle", + "EncryptionInformation", + "encryptionInformation"); } -void from_json(const json& j, HiveBucketProperty& p) { +void from_json(const json& j, HiveOutputTableHandle& p) { + p._type = j["@type"]; from_json_key( j, - "bucketedBy", - p.bucketedBy, - "HiveBucketProperty", + "schemaName", + p.schemaName, + "HiveOutputTableHandle", + "String", + "schemaName"); + from_json_key( + j, + "tableName", + p.tableName, + "HiveOutputTableHandle", + "String", + "tableName"); + from_json_key( + j, + "inputColumns", + p.inputColumns, + "HiveOutputTableHandle", + "List", + "inputColumns"); + from_json_key( + j, + "pageSinkMetadata", + p.pageSinkMetadata, + "HiveOutputTableHandle", + "HivePageSinkMetadata", + "pageSinkMetadata"); + from_json_key( + j, + "locationHandle", + p.locationHandle, + "HiveOutputTableHandle", + "LocationHandle", + "locationHandle"); + from_json_key( + j, + "tableStorageFormat", + p.tableStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "tableStorageFormat"); + from_json_key( + j, + "partitionStorageFormat", + p.partitionStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "partitionStorageFormat"); + from_json_key( + j, + "actualStorageFormat", + p.actualStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "actualStorageFormat"); + from_json_key( + j, + "compressionCodec", + p.compressionCodec, + "HiveOutputTableHandle", + "HiveCompressionCodec", + "compressionCodec"); + from_json_key( + j, + "partitionedBy", + p.partitionedBy, + "HiveOutputTableHandle", "List", - "bucketedBy"); + "partitionedBy"); from_json_key( j, - "bucketCount", - p.bucketCount, + "bucketProperty", + p.bucketProperty, + "HiveOutputTableHandle", "HiveBucketProperty", - "int", - "bucketCount"); + "bucketProperty"); from_json_key( j, - "sortedBy", - p.sortedBy, - "HiveBucketProperty", + "preferredOrderingColumns", + p.preferredOrderingColumns, + "HiveOutputTableHandle", "List", - "sortedBy"); + "preferredOrderingColumns"); from_json_key( j, - "bucketFunctionType", - p.bucketFunctionType, - "HiveBucketProperty", - "BucketFunctionType", - "bucketFunctionType"); + "tableOwner", + p.tableOwner, + "HiveOutputTableHandle", + "String", + "tableOwner"); from_json_key( - j, "types", p.types, "HiveBucketProperty", "List", "types"); + j, + "additionalTableParameters", + p.additionalTableParameters, + "HiveOutputTableHandle", + "Map", + "additionalTableParameters"); + from_json_key( + j, + "encryptionInformation", + p.encryptionInformation, + "HiveOutputTableHandle", + "EncryptionInformation", + "encryptionInformation"); +} +} // 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() +ConstantExpression::ConstantExpression() noexcept { + _type = "constant"; +} -// 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 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"); } -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; + +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"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +RemoteSplit::RemoteSplit() noexcept { + _type = "$remote"; +} -void to_json(json& j, const StorageFormat& p) { +void to_json(json& j, const RemoteSplit& p) { j = json::object(); - to_json_key(j, "serDe", p.serDe, "StorageFormat", "String", "serDe"); - to_json_key( - j, - "inputFormat", - p.inputFormat, - "StorageFormat", - "String", - "inputFormat"); + j["@type"] = "$remote"; + to_json_key(j, "location", p.location, "RemoteSplit", "Location", "location"); to_json_key( j, - "outputFormat", - p.outputFormat, - "StorageFormat", - "String", - "outputFormat"); + "remoteSourceTaskId", + p.remoteSourceTaskId, + "RemoteSplit", + "TaskId", + "remoteSourceTaskId"); } -void from_json(const json& j, StorageFormat& p) { - from_json_key(j, "serDe", p.serDe, "StorageFormat", "String", "serDe"); +void from_json(const json& j, RemoteSplit& p) { + p._type = j["@type"]; from_json_key( - j, - "inputFormat", - p.inputFormat, - "StorageFormat", - "String", - "inputFormat"); + j, "location", p.location, "RemoteSplit", "Location", "location"); from_json_key( j, - "outputFormat", - p.outputFormat, - "StorageFormat", - "String", - "outputFormat"); + "remoteSourceTaskId", + p.remoteSourceTaskId, + "RemoteSplit", + "TaskId", + "remoteSourceTaskId"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Storage& p) { +void to_json(json& j, const BaseHiveColumnHandle& p) { j = json::object(); + to_json_key(j, "name", p.name, "BaseHiveColumnHandle", "String", "name"); to_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"); + j, "comment", p.comment, "BaseHiveColumnHandle", "String", "comment"); to_json_key( j, - "serdeParameters", - p.serdeParameters, - "Storage", - "Map", - "serdeParameters"); + "columnType", + p.columnType, + "BaseHiveColumnHandle", + "ColumnType", + "columnType"); to_json_key( j, - "parameters", - p.parameters, - "Storage", - "Map", - "parameters"); + "requiredSubfields", + p.requiredSubfields, + "BaseHiveColumnHandle", + "List", + "requiredSubfields"); } -void from_json(const json& j, Storage& p) { - from_json_key( - j, - "storageFormat", - p.storageFormat, - "Storage", - "StorageFormat", - "storageFormat"); - from_json_key(j, "location", p.location, "Storage", "String", "location"); +void from_json(const json& j, BaseHiveColumnHandle& p) { + from_json_key(j, "name", p.name, "BaseHiveColumnHandle", "String", "name"); from_json_key( - j, - "bucketProperty", - p.bucketProperty, - "Storage", - "HiveBucketProperty", - "bucketProperty"); - from_json_key(j, "skewed", p.skewed, "Storage", "bool", "skewed"); + j, "comment", p.comment, "BaseHiveColumnHandle", "String", "comment"); from_json_key( j, - "serdeParameters", - p.serdeParameters, - "Storage", - "Map", - "serdeParameters"); + "columnType", + p.columnType, + "BaseHiveColumnHandle", + "ColumnType", + "columnType"); from_json_key( j, - "parameters", - p.parameters, - "Storage", - "Map", - "parameters"); + "requiredSubfields", + p.requiredSubfields, + "BaseHiveColumnHandle", + "List", + "requiredSubfields"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +SqlFunctionHandle::SqlFunctionHandle() noexcept { + _type = "json_file"; +} -void to_json(json& j, const Table& p) { +void to_json(json& j, const SqlFunctionHandle& 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"); + j["@type"] = "json_file"; to_json_key( j, - "viewOriginalText", - p.viewOriginalText, - "Table", - "String", - "viewOriginalText"); + "functionId", + p.functionId, + "SqlFunctionHandle", + "SqlFunctionId", + "functionId"); to_json_key( - j, - "viewExpandedText", - p.viewExpandedText, - "Table", - "String", - "viewExpandedText"); + j, "version", p.version, "SqlFunctionHandle", "String", "version"); } -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, - "partitionColumns", - p.partitionColumns, - "Table", - "List", - "partitionColumns"); - from_json_key( - j, - "parameters", - p.parameters, - "Table", - "Map", - "parameters"); +void from_json(const json& j, SqlFunctionHandle& p) { + p._type = j["@type"]; from_json_key( j, - "viewOriginalText", - p.viewOriginalText, - "Table", - "String", - "viewOriginalText"); + "functionId", + p.functionId, + "SqlFunctionHandle", + "SqlFunctionId", + "functionId"); from_json_key( - j, - "viewExpandedText", - p.viewExpandedText, - "Table", - "String", - "viewExpandedText"); + j, "version", p.version, "SqlFunctionHandle", "String", "version"); } } // 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 { +TpchTableHandle::TpchTableHandle() noexcept { + _type = "tpch"; +} -void to_json(json& j, const HivePageSinkMetadata& p) { +void to_json(json& j, const TpchTableHandle& p) { j = json::object(); + j["@type"] = "tpch"; + to_json_key( + j, "tableName", p.tableName, "TpchTableHandle", "String", "tableName"); to_json_key( j, - "schemaTableName", - p.schemaTableName, - "HivePageSinkMetadata", - "SchemaTableName", - "schemaTableName"); - to_json_key(j, "table", p.table, "HivePageSinkMetadata", "Table", "table"); + "scaleFactor", + p.scaleFactor, + "TpchTableHandle", + "double", + "scaleFactor"); } -void from_json(const json& j, HivePageSinkMetadata& p) { +void from_json(const json& j, TpchTableHandle& p) { + p._type = j["@type"]; + from_json_key( + j, "tableName", p.tableName, "TpchTableHandle", "String", "tableName"); from_json_key( j, - "schemaTableName", - p.schemaTableName, - "HivePageSinkMetadata", - "SchemaTableName", - "schemaTableName"); - from_json_key(j, "table", p.table, "HivePageSinkMetadata", "Table", "table"); + "scaleFactor", + p.scaleFactor, + "TpchTableHandle", + "double", + "scaleFactor"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveInsertTableHandle::HiveInsertTableHandle() noexcept { - _type = "hive"; +TpchTableLayoutHandle::TpchTableLayoutHandle() noexcept { + _type = "tpch"; } -void to_json(json& j, const HiveInsertTableHandle& p) { +void to_json(json& j, const TpchTableLayoutHandle& p) { j = json::object(); - j["@type"] = "hive"; - to_json_key( - j, - "schemaName", - p.schemaName, - "HiveInsertTableHandle", - "String", - "schemaName"); + j["@type"] = "tpch"; to_json_key( - j, - "tableName", - p.tableName, - "HiveInsertTableHandle", - "String", - "tableName"); + j, "table", p.table, "TpchTableLayoutHandle", "TpchTableHandle", "table"); to_json_key( j, - "inputColumns", - p.inputColumns, - "HiveInsertTableHandle", - "List", - "inputColumns"); - to_json_key( + "predicate", + p.predicate, + "TpchTableLayoutHandle", + "TupleDomain>", + "predicate"); +} + +void from_json(const json& j, TpchTableLayoutHandle& p) { + p._type = j["@type"]; + from_json_key( + j, "table", p.table, "TpchTableLayoutHandle", "TpchTableHandle", "table"); + from_json_key( j, - "pageSinkMetadata", - p.pageSinkMetadata, - "HiveInsertTableHandle", - "HivePageSinkMetadata", - "pageSinkMetadata"); + "predicate", + p.predicate, + "TpchTableLayoutHandle", + "TupleDomain>", + "predicate"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const MemoryAllocation& p) { + j = json::object(); + to_json_key(j, "tag", p.tag, "MemoryAllocation", "String", "tag"); to_json_key( j, - "locationHandle", - p.locationHandle, - "HiveInsertTableHandle", - "LocationHandle", - "locationHandle"); - 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, - "bucketProperty", - p.bucketProperty, - "HiveInsertTableHandle", - "HiveBucketProperty", - "bucketProperty"); + "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, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveInsertTableHandle", - "List", - "preferredOrderingColumns"); + j, "maxBytes", p.maxBytes, "MemoryPoolInfo", "int64_t", "maxBytes"); to_json_key( j, - "tableStorageFormat", - p.tableStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "tableStorageFormat"); + "reservedBytes", + p.reservedBytes, + "MemoryPoolInfo", + "int64_t", + "reservedBytes"); to_json_key( j, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); + "reservedRevocableBytes", + p.reservedRevocableBytes, + "MemoryPoolInfo", + "int64_t", + "reservedRevocableBytes"); to_json_key( j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); + "queryMemoryReservations", + p.queryMemoryReservations, + "MemoryPoolInfo", + "Map", + "queryMemoryReservations"); to_json_key( j, - "compressionCodec", - p.compressionCodec, - "HiveInsertTableHandle", - "HiveCompressionCodec", - "compressionCodec"); + "queryMemoryAllocations", + p.queryMemoryAllocations, + "MemoryPoolInfo", + "Map>", + "queryMemoryAllocations"); 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"); + "queryMemoryRevocableReservations", + p.queryMemoryRevocableReservations, + "MemoryPoolInfo", + "Map", + "queryMemoryRevocableReservations"); +} + +void from_json(const json& j, MemoryPoolInfo& p) { from_json_key( - j, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveInsertTableHandle", - "List", - "preferredOrderingColumns"); + j, "maxBytes", p.maxBytes, "MemoryPoolInfo", "int64_t", "maxBytes"); from_json_key( j, - "tableStorageFormat", - p.tableStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "tableStorageFormat"); + "reservedBytes", + p.reservedBytes, + "MemoryPoolInfo", + "int64_t", + "reservedBytes"); from_json_key( j, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); + "reservedRevocableBytes", + p.reservedRevocableBytes, + "MemoryPoolInfo", + "int64_t", + "reservedRevocableBytes"); from_json_key( j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); + "queryMemoryReservations", + p.queryMemoryReservations, + "MemoryPoolInfo", + "Map", + "queryMemoryReservations"); from_json_key( j, - "compressionCodec", - p.compressionCodec, - "HiveInsertTableHandle", - "HiveCompressionCodec", - "compressionCodec"); + "queryMemoryAllocations", + p.queryMemoryAllocations, + "MemoryPoolInfo", + "Map>", + "queryMemoryAllocations"); from_json_key( j, - "encryptionInformation", - p.encryptionInformation, - "HiveInsertTableHandle", - "EncryptionInformation", - "encryptionInformation"); -} -} // 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"); + "queryMemoryRevocableReservations", + p.queryMemoryRevocableReservations, + "MemoryPoolInfo", + "Map", + "queryMemoryRevocableReservations"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TpchPartitioningHandle::TpchPartitioningHandle() noexcept { - _type = "tpch"; +DeleteHandle::DeleteHandle() noexcept { + _type = "DeleteHandle"; } -void to_json(json& j, const TpchPartitioningHandle& p) { +void to_json(json& j, const DeleteHandle& p) { j = json::object(); - j["@type"] = "tpch"; - to_json_key(j, "table", p.table, "TpchPartitioningHandle", "String", "table"); + j["@type"] = "DeleteHandle"; + to_json_key(j, "handle", p.handle, "DeleteHandle", "TableHandle", "handle"); to_json_key( j, - "totalRows", - p.totalRows, - "TpchPartitioningHandle", - "int64_t", - "totalRows"); + "schemaTableName", + p.schemaTableName, + "DeleteHandle", + "SchemaTableName", + "schemaTableName"); } -void from_json(const json& j, TpchPartitioningHandle& p) { +void from_json(const json& j, DeleteHandle& p) { p._type = j["@type"]; - from_json_key( - j, "table", p.table, "TpchPartitioningHandle", "String", "table"); + from_json_key(j, "handle", p.handle, "DeleteHandle", "TableHandle", "handle"); from_json_key( j, - "totalRows", - p.totalRows, - "TpchPartitioningHandle", - "int64_t", - "totalRows"); + "schemaTableName", + p.schemaTableName, + "DeleteHandle", + "SchemaTableName", + "schemaTableName"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -MarkDistinctNode::MarkDistinctNode() noexcept { - _type = ".MarkDistinctNode"; +TableScanNode::TableScanNode() noexcept { + _type = ".TableScanNode"; } -void to_json(json& j, const MarkDistinctNode& p) { +void to_json(json& j, const TableScanNode& p) { j = json::object(); - j["@type"] = ".MarkDistinctNode"; - to_json_key(j, "id", p.id, "MarkDistinctNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "MarkDistinctNode", "PlanNode", "source"); - to_json_key( - j, - "markerVariable", - p.markerVariable, - "MarkDistinctNode", - "VariableReferenceExpression", - "markerVariable"); + 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, - "distinctVariables", - p.distinctVariables, - "MarkDistinctNode", + "outputVariables", + p.outputVariables, + "TableScanNode", "List", - "distinctVariables"); + "outputVariables"); to_json_key( j, - "hashVariable", - p.hashVariable, - "MarkDistinctNode", - "VariableReferenceExpression", - "hashVariable"); + "assignments", + p.assignments, + "TableScanNode", + "Map>", + "assignments"); } -void from_json(const json& j, MarkDistinctNode& p) { +void from_json(const json& j, TableScanNode& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "MarkDistinctNode", "PlanNodeId", "id"); - from_json_key( - j, "source", p.source, "MarkDistinctNode", "PlanNode", "source"); - from_json_key( - j, - "markerVariable", - p.markerVariable, - "MarkDistinctNode", - "VariableReferenceExpression", - "markerVariable"); + from_json_key(j, "id", p.id, "TableScanNode", "PlanNodeId", "id"); + from_json_key(j, "table", p.table, "TableScanNode", "TableHandle", "table"); from_json_key( j, - "distinctVariables", - p.distinctVariables, - "MarkDistinctNode", + "outputVariables", + p.outputVariables, + "TableScanNode", "List", - "distinctVariables"); + "outputVariables"); from_json_key( j, - "hashVariable", - p.hashVariable, - "MarkDistinctNode", - "VariableReferenceExpression", - "hashVariable"); + "assignments", + p.assignments, + "TableScanNode", + "Map>", + "assignments"); } } // 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!"); +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(ExchangeNodeType_enum_table), - std::end(ExchangeNodeType_enum_table), - [e](const std::pair& ej_pair) -> bool { + 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(ExchangeNodeType_enum_table)) + j = ((it != std::end(BoundType_enum_table)) ? it - : std::begin(ExchangeNodeType_enum_table)) + : std::begin(BoundType_enum_table)) ->second; } -void from_json(const json& j, ExchangeNodeType& e) { - static_assert( - std::is_enum::value, - "ExchangeNodeType must be an enum!"); +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(ExchangeNodeType_enum_table), - std::end(ExchangeNodeType_enum_table), - [&j](const std::pair& ej_pair) -> bool { + 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(ExchangeNodeType_enum_table)) + e = ((it != std::end(BoundType_enum_table)) ? it - : std::begin(ExchangeNodeType_enum_table)) + : std::begin(BoundType_enum_table)) ->first; } } // namespace facebook::presto::protocol @@ -8851,367 +8865,685 @@ 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!"); +static const std::pair WindowType_enum_table[] = { + // NOLINT: cert-err58-cpp + {WindowType::RANGE, "RANGE"}, + {WindowType::ROWS, "ROWS"}, + {WindowType::GROUPS, "GROUPS"}, +}; +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(ExchangeNodeScope_enum_table), - std::end(ExchangeNodeScope_enum_table), - [e](const std::pair& ej_pair) -> bool { + 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(ExchangeNodeScope_enum_table)) + j = ((it != std::end(WindowType_enum_table)) ? it - : std::begin(ExchangeNodeScope_enum_table)) + : std::begin(WindowType_enum_table)) ->second; } -void from_json(const json& j, ExchangeNodeScope& e) { - static_assert( - std::is_enum::value, - "ExchangeNodeScope must be an enum!"); +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(ExchangeNodeScope_enum_table), - std::end(ExchangeNodeScope_enum_table), - [&j](const std::pair& ej_pair) -> bool { + 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(ExchangeNodeScope_enum_table)) + e = ((it != std::end(WindowType_enum_table)) ? it - : std::begin(ExchangeNodeScope_enum_table)) + : std::begin(WindowType_enum_table)) ->first; } } // 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) { +void to_json(json& j, const Frame& 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, "type", p.type, "Frame", "WindowType", "type"); + to_json_key(j, "startType", p.startType, "Frame", "BoundType", "startType"); to_json_key( - j, "scope", p.scope, "ExchangeNode", "ExchangeNodeScope", "scope"); + j, + "startValue", + p.startValue, + "Frame", + "VariableReferenceExpression", + "startValue"); to_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "ExchangeNode", - "PartitioningScheme", - "partitioningScheme"); + "sortKeyCoercedForFrameStartComparison", + p.sortKeyCoercedForFrameStartComparison, + "Frame", + "VariableReferenceExpression", + "sortKeyCoercedForFrameStartComparison"); + to_json_key(j, "endType", p.endType, "Frame", "BoundType", "endType"); to_json_key( j, - "sources", - p.sources, - "ExchangeNode", - "List>", - "sources"); + "endValue", + p.endValue, + "Frame", + "VariableReferenceExpression", + "endValue"); to_json_key( j, - "inputs", - p.inputs, - "ExchangeNode", - "List>", - "inputs"); + "sortKeyCoercedForFrameEndComparison", + p.sortKeyCoercedForFrameEndComparison, + "Frame", + "VariableReferenceExpression", + "sortKeyCoercedForFrameEndComparison"); to_json_key( j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "ExchangeNode", - "bool", - "ensureSourceOrdering"); + "originalStartValue", + p.originalStartValue, + "Frame", + "String", + "originalStartValue"); to_json_key( j, - "orderingScheme", - p.orderingScheme, - "ExchangeNode", - "OrderingScheme", - "orderingScheme"); + "originalEndValue", + p.originalEndValue, + "Frame", + "String", + "originalEndValue"); } -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"); +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, "scope", p.scope, "ExchangeNode", "ExchangeNodeScope", "scope"); + j, + "startValue", + p.startValue, + "Frame", + "VariableReferenceExpression", + "startValue"); from_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "ExchangeNode", - "PartitioningScheme", - "partitioningScheme"); + "sortKeyCoercedForFrameStartComparison", + p.sortKeyCoercedForFrameStartComparison, + "Frame", + "VariableReferenceExpression", + "sortKeyCoercedForFrameStartComparison"); + from_json_key(j, "endType", p.endType, "Frame", "BoundType", "endType"); from_json_key( j, - "sources", - p.sources, - "ExchangeNode", - "List>", - "sources"); + "endValue", + p.endValue, + "Frame", + "VariableReferenceExpression", + "endValue"); from_json_key( j, - "inputs", - p.inputs, - "ExchangeNode", - "List>", - "inputs"); + "sortKeyCoercedForFrameEndComparison", + p.sortKeyCoercedForFrameEndComparison, + "Frame", + "VariableReferenceExpression", + "sortKeyCoercedForFrameEndComparison"); from_json_key( j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "ExchangeNode", - "bool", - "ensureSourceOrdering"); + "originalStartValue", + p.originalStartValue, + "Frame", + "String", + "originalStartValue"); from_json_key( j, - "orderingScheme", - p.orderingScheme, - "ExchangeNode", - "OrderingScheme", - "orderingScheme"); + "originalEndValue", + p.originalEndValue, + "Frame", + "String", + "originalEndValue"); } } // 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 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 RemoteSourceNode& 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 { +IcebergColumnHandle::IcebergColumnHandle() noexcept { + _type = "hive-iceberg"; +} + +void to_json(json& j, const IcebergColumnHandle& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.RemoteSourceNode"; - to_json_key(j, "id", p.id, "RemoteSourceNode", "PlanNodeId", "id"); + j["@type"] = "hive-iceberg"; to_json_key( j, - "sourceFragmentIds", - p.sourceFragmentIds, - "RemoteSourceNode", - "List", - "sourceFragmentIds"); + "columnIdentity", + p.columnIdentity, + "IcebergColumnHandle", + "ColumnIdentity", + "columnIdentity"); + to_json_key(j, "type", p.type, "IcebergColumnHandle", "Type", "type"); + to_json_key( + j, "comment", p.comment, "IcebergColumnHandle", "String", "comment"); to_json_key( j, - "outputVariables", - p.outputVariables, - "RemoteSourceNode", - "List", - "outputVariables"); + "columnType", + p.columnType, + "IcebergColumnHandle", + "ColumnType", + "columnType"); to_json_key( j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "RemoteSourceNode", - "bool", - "ensureSourceOrdering"); + "requiredSubfields", + p.requiredSubfields, + "IcebergColumnHandle", + "List", + "requiredSubfields"); +} + +void from_json(const json& j, IcebergColumnHandle& p) { + p._type = j["@type"]; + from_json_key( + j, + "columnIdentity", + p.columnIdentity, + "IcebergColumnHandle", + "ColumnIdentity", + "columnIdentity"); + from_json_key(j, "type", p.type, "IcebergColumnHandle", "Type", "type"); + from_json_key( + j, "comment", p.comment, "IcebergColumnHandle", "String", "comment"); + from_json_key( + j, + "columnType", + p.columnType, + "IcebergColumnHandle", + "ColumnType", + "columnType"); + from_json_key( + j, + "requiredSubfields", + p.requiredSubfields, + "IcebergColumnHandle", + "List", + "requiredSubfields"); +} +} // 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 + ChangelogOperation_enum_table[] = { // NOLINT: cert-err58-cpp + {ChangelogOperation::INSERT, "INSERT"}, + {ChangelogOperation::DELETE, "DELETE"}, + {ChangelogOperation::UPDATE_BEFORE, "UPDATE_BEFORE"}, + {ChangelogOperation::UPDATE_AFTER, "UPDATE_AFTER"}}; +void to_json(json& j, const ChangelogOperation& e) { + static_assert( + std::is_enum::value, + "ChangelogOperation must be an enum!"); + const auto* it = std::find_if( + std::begin(ChangelogOperation_enum_table), + std::end(ChangelogOperation_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(ChangelogOperation_enum_table)) + ? it + : std::begin(ChangelogOperation_enum_table)) + ->second; +} +void from_json(const json& j, ChangelogOperation& e) { + static_assert( + std::is_enum::value, + "ChangelogOperation must be an enum!"); + const auto* it = std::find_if( + std::begin(ChangelogOperation_enum_table), + std::end(ChangelogOperation_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(ChangelogOperation_enum_table)) + ? it + : std::begin(ChangelogOperation_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const ChangelogSplitInfo& p) { + j = json::object(); to_json_key( j, - "orderingScheme", - p.orderingScheme, - "RemoteSourceNode", - "OrderingScheme", - "orderingScheme"); + "operation", + p.operation, + "ChangelogSplitInfo", + "ChangelogOperation", + "operation"); + to_json_key( + j, "ordinal", p.ordinal, "ChangelogSplitInfo", "int64_t", "ordinal"); to_json_key( j, - "exchangeType", - p.exchangeType, - "RemoteSourceNode", - "ExchangeNodeType", - "exchangeType"); + "snapshotId", + p.snapshotId, + "ChangelogSplitInfo", + "int64_t", + "snapshotId"); + to_json_key( + j, + "icebergColumns", + p.icebergColumns, + "ChangelogSplitInfo", + "List", + "icebergColumns"); } -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, - "sourceFragmentIds", - p.sourceFragmentIds, - "RemoteSourceNode", - "List", - "sourceFragmentIds"); +void from_json(const json& j, ChangelogSplitInfo& p) { from_json_key( j, - "outputVariables", - p.outputVariables, - "RemoteSourceNode", - "List", - "outputVariables"); + "operation", + p.operation, + "ChangelogSplitInfo", + "ChangelogOperation", + "operation"); from_json_key( - j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "RemoteSourceNode", - "bool", - "ensureSourceOrdering"); + j, "ordinal", p.ordinal, "ChangelogSplitInfo", "int64_t", "ordinal"); from_json_key( j, - "orderingScheme", - p.orderingScheme, - "RemoteSourceNode", - "OrderingScheme", - "orderingScheme"); + "snapshotId", + p.snapshotId, + "ChangelogSplitInfo", + "int64_t", + "snapshotId"); from_json_key( j, - "exchangeType", - p.exchangeType, - "RemoteSourceNode", - "ExchangeNodeType", - "exchangeType"); + "icebergColumns", + p.icebergColumns, + "ChangelogSplitInfo", + "List", + "icebergColumns"); } } // 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 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 to_json(json& j, const RowNumberNode& p) { +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 { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair FileFormat_enum_table[] = + { // NOLINT: cert-err58-cpp + {FileFormat::ORC, "ORC"}, + {FileFormat::PARQUET, "PARQUET"}, + {FileFormat::AVRO, "AVRO"}, + {FileFormat::METADATA, "METADATA"}}; +void to_json(json& j, const FileFormat& e) { + static_assert(std::is_enum::value, "FileFormat must be an enum!"); + const auto* it = std::find_if( + std::begin(FileFormat_enum_table), + std::end(FileFormat_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(FileFormat_enum_table)) + ? it + : std::begin(FileFormat_enum_table)) + ->second; +} +void from_json(const json& j, FileFormat& e) { + static_assert(std::is_enum::value, "FileFormat must be an enum!"); + const auto* it = std::find_if( + std::begin(FileFormat_enum_table), + std::end(FileFormat_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(FileFormat_enum_table)) + ? it + : std::begin(FileFormat_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 FileContent_enum_table[] = + { // NOLINT: cert-err58-cpp + {FileContent::DATA, "DATA"}, + {FileContent::POSITION_DELETES, "POSITION_DELETES"}, + {FileContent::EQUALITY_DELETES, "EQUALITY_DELETES"}}; +void to_json(json& j, const FileContent& e) { + static_assert( + std::is_enum::value, "FileContent must be an enum!"); + const auto* it = std::find_if( + std::begin(FileContent_enum_table), + std::end(FileContent_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(FileContent_enum_table)) + ? it + : std::begin(FileContent_enum_table)) + ->second; +} +void from_json(const json& j, FileContent& e) { + static_assert( + std::is_enum::value, "FileContent must be an enum!"); + const auto* it = std::find_if( + std::begin(FileContent_enum_table), + std::end(FileContent_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(FileContent_enum_table)) + ? it + : std::begin(FileContent_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const DeleteFile& 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, "content", p.content, "DeleteFile", "FileContent", "content"); + to_json_key(j, "path", p.path, "DeleteFile", "String", "path"); + to_json_key(j, "format", p.format, "DeleteFile", "FileFormat", "format"); + to_json_key( + j, "recordCount", p.recordCount, "DeleteFile", "int64_t", "recordCount"); to_json_key( j, - "partitionBy", - p.partitionBy, - "RowNumberNode", - "List", - "partitionBy"); + "fileSizeInBytes", + p.fileSizeInBytes, + "DeleteFile", + "int64_t", + "fileSizeInBytes"); to_json_key( j, - "rowNumberVariable", - p.rowNumberVariable, - "RowNumberNode", - "VariableReferenceExpression", - "rowNumberVariable"); + "equalityFieldIds", + p.equalityFieldIds, + "DeleteFile", + "List", + "equalityFieldIds"); to_json_key( j, - "maxRowCountPerPartition", - p.maxRowCountPerPartition, - "RowNumberNode", - "Integer", - "maxRowCountPerPartition"); - to_json_key(j, "partial", p.partial, "RowNumberNode", "bool", "partial"); + "lowerBounds", + p.lowerBounds, + "DeleteFile", + "Map", + "lowerBounds"); to_json_key( j, - "hashVariable", - p.hashVariable, - "RowNumberNode", - "VariableReferenceExpression", - "hashVariable"); + "upperBounds", + p.upperBounds, + "DeleteFile", + "Map", + "upperBounds"); } -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"); +void from_json(const json& j, DeleteFile& p) { + from_json_key( + j, "content", p.content, "DeleteFile", "FileContent", "content"); + from_json_key(j, "path", p.path, "DeleteFile", "String", "path"); + from_json_key(j, "format", p.format, "DeleteFile", "FileFormat", "format"); + from_json_key( + j, "recordCount", p.recordCount, "DeleteFile", "int64_t", "recordCount"); from_json_key( j, - "partitionBy", - p.partitionBy, - "RowNumberNode", - "List", - "partitionBy"); + "fileSizeInBytes", + p.fileSizeInBytes, + "DeleteFile", + "int64_t", + "fileSizeInBytes"); from_json_key( j, - "rowNumberVariable", - p.rowNumberVariable, - "RowNumberNode", - "VariableReferenceExpression", - "rowNumberVariable"); + "equalityFieldIds", + p.equalityFieldIds, + "DeleteFile", + "List", + "equalityFieldIds"); from_json_key( j, - "maxRowCountPerPartition", - p.maxRowCountPerPartition, - "RowNumberNode", - "Integer", - "maxRowCountPerPartition"); - from_json_key(j, "partial", p.partial, "RowNumberNode", "bool", "partial"); + "lowerBounds", + p.lowerBounds, + "DeleteFile", + "Map", + "lowerBounds"); from_json_key( j, - "hashVariable", - p.hashVariable, - "RowNumberNode", - "VariableReferenceExpression", - "hashVariable"); + "upperBounds", + p.upperBounds, + "DeleteFile", + "Map", + "upperBounds"); } } // 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 SampleNodeType_enum_table[] = - { // NOLINT: cert-err58-cpp - {SampleNodeType::BERNOULLI, "BERNOULLI"}, - {SampleNodeType::SYSTEM, "SYSTEM"}}; -void to_json(json& j, const SampleNodeType& e) { +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, "SampleNodeType must be an enum!"); + std::is_enum::value, + "NodeSelectionStrategy must be an enum!"); const auto* it = std::find_if( - std::begin(SampleNodeType_enum_table), - std::end(SampleNodeType_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(SampleNodeType_enum_table)) + j = ((it != std::end(NodeSelectionStrategy_enum_table)) ? it - : std::begin(SampleNodeType_enum_table)) + : std::begin(NodeSelectionStrategy_enum_table)) ->second; } -void from_json(const json& j, SampleNodeType& e) { +void from_json(const json& j, NodeSelectionStrategy& e) { static_assert( - std::is_enum::value, "SampleNodeType must be an enum!"); + std::is_enum::value, + "NodeSelectionStrategy must be an enum!"); const auto* it = std::find_if( - std::begin(SampleNodeType_enum_table), - std::end(SampleNodeType_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(SampleNodeType_enum_table)) + e = ((it != std::end(NodeSelectionStrategy_enum_table)) ? it - : std::begin(SampleNodeType_enum_table)) + : std::begin(NodeSelectionStrategy_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -SampleNode::SampleNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.SampleNode"; +IcebergSplit::IcebergSplit() noexcept { + _type = "hive-iceberg"; +} + +void to_json(json& j, const IcebergSplit& p) { + j = json::object(); + j["@type"] = "hive-iceberg"; + to_json_key(j, "path", p.path, "IcebergSplit", "String", "path"); + to_json_key(j, "start", p.start, "IcebergSplit", "int64_t", "start"); + to_json_key(j, "length", p.length, "IcebergSplit", "int64_t", "length"); + to_json_key( + j, + "fileFormat", + p.fileFormat, + "IcebergSplit", + "FileFormat", + "fileFormat"); + to_json_key( + j, + "addresses", + p.addresses, + "IcebergSplit", + "List", + "addresses"); + to_json_key( + j, + "partitionKeys", + p.partitionKeys, + "IcebergSplit", + "Map", + "partitionKeys"); + to_json_key( + j, + "nodeSelectionStrategy", + p.nodeSelectionStrategy, + "IcebergSplit", + "NodeSelectionStrategy", + "nodeSelectionStrategy"); + to_json_key( + j, + "splitWeight", + p.splitWeight, + "IcebergSplit", + "SplitWeight", + "splitWeight"); + to_json_key( + j, "deletes", p.deletes, "IcebergSplit", "List", "deletes"); + to_json_key( + j, + "changelogSplitInfo", + p.changelogSplitInfo, + "IcebergSplit", + "ChangelogSplitInfo", + "changelogSplitInfo"); + to_json_key( + j, + "dataSequenceNumber", + p.dataSequenceNumber, + "IcebergSplit", + "int64_t", + "dataSequenceNumber"); +} + +void from_json(const json& j, IcebergSplit& p) { + p._type = j["@type"]; + from_json_key(j, "path", p.path, "IcebergSplit", "String", "path"); + from_json_key(j, "start", p.start, "IcebergSplit", "int64_t", "start"); + from_json_key(j, "length", p.length, "IcebergSplit", "int64_t", "length"); + from_json_key( + j, + "fileFormat", + p.fileFormat, + "IcebergSplit", + "FileFormat", + "fileFormat"); + from_json_key( + j, + "addresses", + p.addresses, + "IcebergSplit", + "List", + "addresses"); + from_json_key( + j, + "partitionKeys", + p.partitionKeys, + "IcebergSplit", + "Map", + "partitionKeys"); + from_json_key( + j, + "nodeSelectionStrategy", + p.nodeSelectionStrategy, + "IcebergSplit", + "NodeSelectionStrategy", + "nodeSelectionStrategy"); + from_json_key( + j, + "splitWeight", + p.splitWeight, + "IcebergSplit", + "SplitWeight", + "splitWeight"); + from_json_key( + j, "deletes", p.deletes, "IcebergSplit", "List", "deletes"); + from_json_key( + j, + "changelogSplitInfo", + p.changelogSplitInfo, + "IcebergSplit", + "ChangelogSplitInfo", + "changelogSplitInfo"); + from_json_key( + j, + "dataSequenceNumber", + p.dataSequenceNumber, + "IcebergSplit", + "int64_t", + "dataSequenceNumber"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +HiveTableHandle::HiveTableHandle() noexcept { + _type = "hive"; } -void to_json(json& j, const SampleNode& p) { +void to_json(json& j, const HiveTableHandle& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.SampleNode"; - to_json_key(j, "id", p.id, "SampleNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "SampleNode", "PlanNode", "source"); + j["@type"] = "hive"; to_json_key( - j, "sampleRatio", p.sampleRatio, "SampleNode", "double", "sampleRatio"); + j, "schemaName", p.schemaName, "HiveTableHandle", "String", "schemaName"); + to_json_key( + j, "tableName", p.tableName, "HiveTableHandle", "String", "tableName"); to_json_key( j, - "sampleType", - p.sampleType, - "SampleNode", - "SampleNodeType", - "sampleType"); + "analyzePartitionValues", + p.analyzePartitionValues, + "HiveTableHandle", + "List>", + "analyzePartitionValues"); } -void from_json(const json& j, SampleNode& p) { +void from_json(const json& j, HiveTableHandle& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "SampleNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "SampleNode", "PlanNode", "source"); from_json_key( - j, "sampleRatio", p.sampleRatio, "SampleNode", "double", "sampleRatio"); + j, "schemaName", p.schemaName, "HiveTableHandle", "String", "schemaName"); + from_json_key( + j, "tableName", p.tableName, "HiveTableHandle", "String", "tableName"); from_json_key( j, - "sampleType", - p.sampleType, - "SampleNode", - "SampleNodeType", - "sampleType"); + "analyzePartitionValues", + p.analyzePartitionValues, + "HiveTableHandle", + "List>", + "analyzePartitionValues"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { @@ -9294,2402 +9626,3130 @@ void from_json(const json& j, OutputBuffers& 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 + 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; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const StageExecutionDescriptor& p) { + j = json::object(); + to_json_key( + j, + "stageExecutionStrategy", + p.stageExecutionStrategy, + "StageExecutionDescriptor", + "StageExecutionStrategy", + "stageExecutionStrategy"); + to_json_key( + j, + "groupedExecutionScanNodes", + p.groupedExecutionScanNodes, + "StageExecutionDescriptor", + "List", + "groupedExecutionScanNodes"); + to_json_key( + j, + "totalLifespans", + p.totalLifespans, + "StageExecutionDescriptor", + "int", + "totalLifespans"); +} + +void from_json(const json& j, StageExecutionDescriptor& p) { + from_json_key( + j, + "stageExecutionStrategy", + p.stageExecutionStrategy, + "StageExecutionDescriptor", + "StageExecutionStrategy", + "stageExecutionStrategy"); + from_json_key( + j, + "groupedExecutionScanNodes", + p.groupedExecutionScanNodes, + "StageExecutionDescriptor", + "List", + "groupedExecutionScanNodes"); + from_json_key( + j, + "totalLifespans", + p.totalLifespans, + "StageExecutionDescriptor", + "int", + "totalLifespans"); +} +} // 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( + j, + "connectorId", + p.connectorId, + "PartitioningHandle", + "ConnectorId", + "connectorId"); + to_json_key( + j, + "transactionHandle", + p.transactionHandle, + "PartitioningHandle", + "ConnectorTransactionHandle", + "transactionHandle"); + to_json_key( + j, + "connectorHandle", + p.connectorHandle, + "PartitioningHandle", + "ConnectorPartitioningHandle", + "connectorHandle"); +} + +void from_json(const json& j, PartitioningHandle& p) { + from_json_key( + j, + "connectorId", + p.connectorId, + "PartitioningHandle", + "ConnectorId", + "connectorId"); + from_json_key( + j, + "transactionHandle", + p.transactionHandle, + "PartitioningHandle", + "ConnectorTransactionHandle", + "transactionHandle"); + from_json_key( + j, + "connectorHandle", + p.connectorHandle, + "PartitioningHandle", + "ConnectorPartitioningHandle", + "connectorHandle"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { -void to_json(json& j, const HiveBucketFilter& p) { +void to_json(json& j, const Partitioning& p) { j = json::object(); + to_json_key( + j, "handle", p.handle, "Partitioning", "PartitioningHandle", "handle"); to_json_key( j, - "bucketsToKeep", - p.bucketsToKeep, - "HiveBucketFilter", - "List", - "bucketsToKeep"); + "arguments", + p.arguments, + "Partitioning", + "List>", + "arguments"); } -void from_json(const json& j, HiveBucketFilter& p) { +void from_json(const json& j, Partitioning& p) { + from_json_key( + j, "handle", p.handle, "Partitioning", "PartitioningHandle", "handle"); from_json_key( j, - "bucketsToKeep", - p.bucketsToKeep, - "HiveBucketFilter", - "List", - "bucketsToKeep"); + "arguments", + p.arguments, + "Partitioning", + "List>", + "arguments"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const UpdateTarget& p) { +void to_json(json& j, const PartitioningScheme& p) { j = json::object(); - to_json_key(j, "handle", p.handle, "UpdateTarget", "TableHandle", "handle"); to_json_key( j, - "schemaTableName", - p.schemaTableName, - "UpdateTarget", - "SchemaTableName", - "schemaTableName"); + "partitioning", + p.partitioning, + "PartitioningScheme", + "Partitioning", + "partitioning"); to_json_key( j, - "updatedColumns", - p.updatedColumns, - "UpdateTarget", - "List", - "updatedColumns"); + "outputLayout", + p.outputLayout, + "PartitioningScheme", + "List", + "outputLayout"); to_json_key( j, - "updatedColumnHandles", - p.updatedColumnHandles, - "UpdateTarget", - "List>", - "updatedColumnHandles"); + "hashColumn", + p.hashColumn, + "PartitioningScheme", + "VariableReferenceExpression", + "hashColumn"); + to_json_key( + j, + "replicateNullsAndAny", + p.replicateNullsAndAny, + "PartitioningScheme", + "bool", + "replicateNullsAndAny"); + to_json_key( + j, + "bucketToPartition", + p.bucketToPartition, + "PartitioningScheme", + "List", + "bucketToPartition"); } -void from_json(const json& j, UpdateTarget& p) { - from_json_key(j, "handle", p.handle, "UpdateTarget", "TableHandle", "handle"); +void from_json(const json& j, PartitioningScheme& p) { from_json_key( j, - "schemaTableName", - p.schemaTableName, - "UpdateTarget", - "SchemaTableName", - "schemaTableName"); + "partitioning", + p.partitioning, + "PartitioningScheme", + "Partitioning", + "partitioning"); from_json_key( j, - "updatedColumns", - p.updatedColumns, - "UpdateTarget", - "List", - "updatedColumns"); + "outputLayout", + p.outputLayout, + "PartitioningScheme", + "List", + "outputLayout"); from_json_key( j, - "updatedColumnHandles", - p.updatedColumnHandles, - "UpdateTarget", - "List>", - "updatedColumnHandles"); + "hashColumn", + p.hashColumn, + "PartitioningScheme", + "VariableReferenceExpression", + "hashColumn"); + from_json_key( + j, + "replicateNullsAndAny", + p.replicateNullsAndAny, + "PartitioningScheme", + "bool", + "replicateNullsAndAny"); + from_json_key( + j, + "bucketToPartition", + p.bucketToPartition, + "PartitioningScheme", + "List", + "bucketToPartition"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HivePartitioningHandle::HivePartitioningHandle() noexcept { - _type = "hive"; -} -void to_json(json& j, const HivePartitioningHandle& p) { +void to_json(json& j, const PlanCostEstimate& p) { j = json::object(); - j["@type"] = "hive"; - to_json_key( - j, - "bucketCount", - p.bucketCount, - "HivePartitioningHandle", - "int", - "bucketCount"); + to_json_key(j, "cpuCost", p.cpuCost, "PlanCostEstimate", "double", "cpuCost"); to_json_key( - j, - "maxCompatibleBucketCount", - p.maxCompatibleBucketCount, - "HivePartitioningHandle", - "int", - "maxCompatibleBucketCount"); + j, "maxMemory", p.maxMemory, "PlanCostEstimate", "double", "maxMemory"); to_json_key( j, - "bucketFunctionType", - p.bucketFunctionType, - "HivePartitioningHandle", - "BucketFunctionType", - "bucketFunctionType"); + "maxMemoryWhenOutputting", + p.maxMemoryWhenOutputting, + "PlanCostEstimate", + "double", + "maxMemoryWhenOutputting"); to_json_key( j, - "hiveTypes", - p.hiveTypes, - "HivePartitioningHandle", - "List", - "hiveTypes"); - to_json_key( - j, "types", p.types, "HivePartitioningHandle", "List", "types"); + "networkCost", + p.networkCost, + "PlanCostEstimate", + "double", + "networkCost"); } -void from_json(const json& j, HivePartitioningHandle& p) { - p._type = j["@type"]; +void from_json(const json& j, PlanCostEstimate& p) { from_json_key( - j, - "bucketCount", - p.bucketCount, - "HivePartitioningHandle", - "int", - "bucketCount"); + j, "cpuCost", p.cpuCost, "PlanCostEstimate", "double", "cpuCost"); from_json_key( - j, - "maxCompatibleBucketCount", - p.maxCompatibleBucketCount, - "HivePartitioningHandle", - "int", - "maxCompatibleBucketCount"); + j, "maxMemory", p.maxMemory, "PlanCostEstimate", "double", "maxMemory"); from_json_key( j, - "bucketFunctionType", - p.bucketFunctionType, - "HivePartitioningHandle", - "BucketFunctionType", - "bucketFunctionType"); + "maxMemoryWhenOutputting", + p.maxMemoryWhenOutputting, + "PlanCostEstimate", + "double", + "maxMemoryWhenOutputting"); from_json_key( j, - "hiveTypes", - p.hiveTypes, - "HivePartitioningHandle", - "List", - "hiveTypes"); - from_json_key( - j, "types", p.types, "HivePartitioningHandle", "List", "types"); + "networkCost", + p.networkCost, + "PlanCostEstimate", + "double", + "networkCost"); } } // 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 PartialAggregationStatsEstimate& p) { j = json::object(); - j["@type"] = "hive"; to_json_key( - j, "schemaName", p.schemaName, "HiveTableHandle", "String", "schemaName"); + j, + "inputBytes", + p.inputBytes, + "PartialAggregationStatsEstimate", + "double", + "inputBytes"); to_json_key( - j, "tableName", p.tableName, "HiveTableHandle", "String", "tableName"); + j, + "outputBytes", + p.outputBytes, + "PartialAggregationStatsEstimate", + "double", + "outputBytes"); to_json_key( j, - "analyzePartitionValues", - p.analyzePartitionValues, - "HiveTableHandle", - "List>", - "analyzePartitionValues"); + "inputRowCount", + p.inputRowCount, + "PartialAggregationStatsEstimate", + "double", + "inputRowCount"); + to_json_key( + j, + "outputRowCount", + p.outputRowCount, + "PartialAggregationStatsEstimate", + "double", + "outputRowCount"); } -void from_json(const json& j, HiveTableHandle& p) { - p._type = j["@type"]; +void from_json(const json& j, PartialAggregationStatsEstimate& p) { from_json_key( - j, "schemaName", p.schemaName, "HiveTableHandle", "String", "schemaName"); + j, + "inputBytes", + p.inputBytes, + "PartialAggregationStatsEstimate", + "double", + "inputBytes"); + from_json_key( + j, + "outputBytes", + p.outputBytes, + "PartialAggregationStatsEstimate", + "double", + "outputBytes"); from_json_key( - j, "tableName", p.tableName, "HiveTableHandle", "String", "tableName"); + j, + "inputRowCount", + p.inputRowCount, + "PartialAggregationStatsEstimate", + "double", + "inputRowCount"); from_json_key( j, - "analyzePartitionValues", - p.analyzePartitionValues, - "HiveTableHandle", - "List>", - "analyzePartitionValues"); + "outputRowCount", + p.outputRowCount, + "PartialAggregationStatsEstimate", + "double", + "outputRowCount"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TpchColumnHandle::TpchColumnHandle() noexcept { - _type = "tpch"; -} -void to_json(json& j, const TpchColumnHandle& p) { +void to_json(json& j, const TableWriterNodeStatsEstimate& 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", "Type", "type"); + "taskCountIfScaledWriter", + p.taskCountIfScaledWriter, + "TableWriterNodeStatsEstimate", + "double", + "taskCountIfScaledWriter"); } -void from_json(const json& j, TpchColumnHandle& p) { - p._type = j["@type"]; +void from_json(const json& j, TableWriterNodeStatsEstimate& p) { from_json_key( j, - "columnName", - p.columnName, - "TpchColumnHandle", - "String", - "columnName"); - from_json_key(j, "type", p.type, "TpchColumnHandle", "Type", "type"); -} -} // 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"); -} -} // 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 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; + "taskCountIfScaledWriter", + p.taskCountIfScaledWriter, + "TableWriterNodeStatsEstimate", + "double", + "taskCountIfScaledWriter"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const RoutineCharacteristics& p) { +void to_json(json& j, const PlanNodeStatsEstimate& p) { j = json::object(); to_json_key( j, - "language", - p.language, - "RoutineCharacteristics", - "Language", - "language"); + "outputRowCount", + p.outputRowCount, + "PlanNodeStatsEstimate", + "double", + "outputRowCount"); to_json_key( j, - "determinism", - p.determinism, - "RoutineCharacteristics", - "Determinism", - "determinism"); + "totalSize", + p.totalSize, + "PlanNodeStatsEstimate", + "double", + "totalSize"); to_json_key( j, - "nullCallClause", - p.nullCallClause, - "RoutineCharacteristics", - "NullCallClause", - "nullCallClause"); + "confident", + p.confident, + "PlanNodeStatsEstimate", + "bool", + "confident"); + to_json_key( + j, + "variableStatistics", + p.variableStatistics, + "PlanNodeStatsEstimate", + "Map", + "variableStatistics"); + to_json_key( + j, + "joinNodeStatsEstimate", + p.joinNodeStatsEstimate, + "PlanNodeStatsEstimate", + "JoinNodeStatsEstimate", + "joinNodeStatsEstimate"); + to_json_key( + j, + "tableWriterNodeStatsEstimate", + p.tableWriterNodeStatsEstimate, + "PlanNodeStatsEstimate", + "TableWriterNodeStatsEstimate", + "tableWriterNodeStatsEstimate"); + to_json_key( + j, + "partialAggregationStatsEstimate", + p.partialAggregationStatsEstimate, + "PlanNodeStatsEstimate", + "PartialAggregationStatsEstimate", + "partialAggregationStatsEstimate"); } -void from_json(const json& j, RoutineCharacteristics& p) { +void from_json(const json& j, PlanNodeStatsEstimate& p) { from_json_key( j, - "language", - p.language, - "RoutineCharacteristics", - "Language", - "language"); + "outputRowCount", + p.outputRowCount, + "PlanNodeStatsEstimate", + "double", + "outputRowCount"); from_json_key( j, - "determinism", - p.determinism, - "RoutineCharacteristics", - "Determinism", - "determinism"); + "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"); + from_json_key( + j, + "joinNodeStatsEstimate", + p.joinNodeStatsEstimate, + "PlanNodeStatsEstimate", + "JoinNodeStatsEstimate", + "joinNodeStatsEstimate"); + from_json_key( + j, + "tableWriterNodeStatsEstimate", + p.tableWriterNodeStatsEstimate, + "PlanNodeStatsEstimate", + "TableWriterNodeStatsEstimate", + "tableWriterNodeStatsEstimate"); from_json_key( j, - "nullCallClause", - p.nullCallClause, - "RoutineCharacteristics", - "NullCallClause", - "nullCallClause"); + "partialAggregationStatsEstimate", + p.partialAggregationStatsEstimate, + "PlanNodeStatsEstimate", + "PartialAggregationStatsEstimate", + "partialAggregationStatsEstimate"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const UpdateHandle& p) { +void to_json(json& j, const StatsAndCosts& p) { j = json::object(); - to_json_key(j, "handle", p.handle, "UpdateHandle", "TableHandle", "handle"); to_json_key( j, - "schemaTableName", - p.schemaTableName, - "UpdateHandle", - "SchemaTableName", - "schemaTableName"); + "stats", + p.stats, + "StatsAndCosts", + "Map", + "stats"); + to_json_key( + j, + "costs", + p.costs, + "StatsAndCosts", + "Map", + "costs"); } -void from_json(const json& j, UpdateHandle& p) { - from_json_key(j, "handle", p.handle, "UpdateHandle", "TableHandle", "handle"); +void from_json(const json& j, StatsAndCosts& p) { from_json_key( j, - "schemaTableName", - p.schemaTableName, - "UpdateHandle", - "SchemaTableName", - "schemaTableName"); + "stats", + p.stats, + "StatsAndCosts", + "Map", + "stats"); + from_json_key( + j, + "costs", + p.costs, + "StatsAndCosts", + "Map", + "costs"); } } // 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 AnalyzeTableHandle& p) { +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, - "connectorId", - p.connectorId, - "AnalyzeTableHandle", - "ConnectorId", - "connectorId"); + "variables", + p.variables, + "PlanFragment", + "List", + "variables"); to_json_key( j, - "transactionHandle", - p.transactionHandle, - "AnalyzeTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "partitioning", + p.partitioning, + "PlanFragment", + "PartitioningHandle", + "partitioning"); to_json_key( j, - "connectorHandle", - p.connectorHandle, - "AnalyzeTableHandle", - "ConnectorTableHandle", - "connectorHandle"); + "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, AnalyzeTableHandle& p) { +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, - "connectorId", - p.connectorId, - "AnalyzeTableHandle", - "ConnectorId", - "connectorId"); + "variables", + p.variables, + "PlanFragment", + "List", + "variables"); from_json_key( j, - "transactionHandle", - p.transactionHandle, - "AnalyzeTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "partitioning", + p.partitioning, + "PlanFragment", + "PartitioningHandle", + "partitioning"); from_json_key( j, - "connectorHandle", - p.connectorHandle, - "AnalyzeTableHandle", - "ConnectorTableHandle", - "connectorHandle"); + "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 { -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; - } +void to_json(json& j, const UpdateTarget& p) { + j = json::object(); + to_json_key(j, "handle", p.handle, "UpdateTarget", "TableHandle", "handle"); + to_json_key( + j, + "schemaTableName", + p.schemaTableName, + "UpdateTarget", + "SchemaTableName", + "schemaTableName"); + to_json_key( + j, + "updatedColumns", + p.updatedColumns, + "UpdateTarget", + "List", + "updatedColumns"); + to_json_key( + j, + "updatedColumnHandles", + p.updatedColumnHandles, + "UpdateTarget", + "List>", + "updatedColumnHandles"); +} - throw TypeError(type + " no abstract type ExecutionWriterTarget "); +void from_json(const json& j, UpdateTarget& p) { + from_json_key(j, "handle", p.handle, "UpdateTarget", "TableHandle", "handle"); + from_json_key( + j, + "schemaTableName", + p.schemaTableName, + "UpdateTarget", + "SchemaTableName", + "schemaTableName"); + from_json_key( + j, + "updatedColumns", + p.updatedColumns, + "UpdateTarget", + "List", + "updatedColumns"); + from_json_key( + j, + "updatedColumnHandles", + p.updatedColumnHandles, + "UpdateTarget", + "List>", + "updatedColumnHandles"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const DeleteScanInfo& p) { +void to_json(json& j, const MemoryInfo& p) { j = json::object(); - to_json_key(j, "id", p.id, "DeleteScanInfo", "PlanNodeId", "id"); to_json_key( j, - "tableHandle", - p.tableHandle, - "DeleteScanInfo", - "TableHandle", - "tableHandle"); + "totalNodeMemory", + p.totalNodeMemory, + "MemoryInfo", + "DataSize", + "totalNodeMemory"); + to_json_key( + j, + "pools", + p.pools, + "MemoryInfo", + "Map", + "pools"); } -void from_json(const json& j, DeleteScanInfo& p) { - from_json_key(j, "id", p.id, "DeleteScanInfo", "PlanNodeId", "id"); +void from_json(const json& j, MemoryInfo& p) { from_json_key( j, - "tableHandle", - p.tableHandle, - "DeleteScanInfo", - "TableHandle", - "tableHandle"); + "totalNodeMemory", + p.totalNodeMemory, + "MemoryInfo", + "DataSize", + "totalNodeMemory"); + from_json_key( + j, + "pools", + p.pools, + "MemoryInfo", + "Map", + "pools"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +HiveInsertTableHandle::HiveInsertTableHandle() noexcept { + _type = "hive"; +} -void to_json(json& j, const TableWriteInfo& p) { +void to_json(json& j, const HiveInsertTableHandle& p) { j = json::object(); + j["@type"] = "hive"; to_json_key( j, - "writerTarget", - p.writerTarget, - "TableWriteInfo", - "ExecutionWriterTarget", - "writerTarget"); + "schemaName", + p.schemaName, + "HiveInsertTableHandle", + "String", + "schemaName"); to_json_key( j, - "analyzeTableHandle", - p.analyzeTableHandle, - "TableWriteInfo", - "AnalyzeTableHandle", - "analyzeTableHandle"); + "tableName", + p.tableName, + "HiveInsertTableHandle", + "String", + "tableName"); to_json_key( j, - "deleteScanInfo", - p.deleteScanInfo, - "TableWriteInfo", - "DeleteScanInfo", - "deleteScanInfo"); + "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, TableWriteInfo& p) { +void from_json(const json& j, HiveInsertTableHandle& p) { + p._type = j["@type"]; from_json_key( j, - "writerTarget", - p.writerTarget, - "TableWriteInfo", - "ExecutionWriterTarget", - "writerTarget"); + "schemaName", + p.schemaName, + "HiveInsertTableHandle", + "String", + "schemaName"); from_json_key( j, - "analyzeTableHandle", - p.analyzeTableHandle, - "TableWriteInfo", - "AnalyzeTableHandle", - "analyzeTableHandle"); + "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, - "deleteScanInfo", - p.deleteScanInfo, - "TableWriteInfo", - "DeleteScanInfo", - "deleteScanInfo"); + "encryptionInformation", + p.encryptionInformation, + "HiveInsertTableHandle", + "EncryptionInformation", + "encryptionInformation"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +HiveTransactionHandle::HiveTransactionHandle() noexcept { + _type = "hive"; +} + +void to_json(json& j, const HiveTransactionHandle& p) { + j = json::object(); + 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 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 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, "LimitNodeStep must be an enum!"); + std::is_enum::value, + "ExchangeNodeType 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 { + 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(LimitNodeStep_enum_table)) + j = ((it != std::end(ExchangeNodeType_enum_table)) ? it - : std::begin(LimitNodeStep_enum_table)) + : std::begin(ExchangeNodeType_enum_table)) ->second; } -void from_json(const json& j, LimitNodeStep& e) { +void from_json(const json& j, ExchangeNodeType& e) { static_assert( - std::is_enum::value, "LimitNodeStep must be an enum!"); + std::is_enum::value, + "ExchangeNodeType 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 { + 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(LimitNodeStep_enum_table)) + e = ((it != std::end(ExchangeNodeType_enum_table)) ? it - : std::begin(LimitNodeStep_enum_table)) + : std::begin(ExchangeNodeType_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -LimitNode::LimitNode() noexcept { - _type = ".LimitNode"; -} +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const LimitNode& 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"); +// 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, LimitNode& 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"); +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 { - -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"); +ExchangeNode::ExchangeNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.ExchangeNode"; } -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -void to_json(json& j, const SqlInvokedFunction& p) { +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, - "parameters", - p.parameters, - "SqlInvokedFunction", - "List", - "parameters"); + "partitioningScheme", + p.partitioningScheme, + "ExchangeNode", + "PartitioningScheme", + "partitioningScheme"); to_json_key( j, - "description", - p.description, - "SqlInvokedFunction", - "String", - "description"); + "sources", + p.sources, + "ExchangeNode", + "List>", + "sources"); to_json_key( j, - "routineCharacteristics", - p.routineCharacteristics, - "SqlInvokedFunction", - "RoutineCharacteristics", - "routineCharacteristics"); - to_json_key(j, "body", p.body, "SqlInvokedFunction", "String", "body"); + "inputs", + p.inputs, + "ExchangeNode", + "List>", + "inputs"); to_json_key( j, - "signature", - p.signature, - "SqlInvokedFunction", - "Signature", - "signature"); + "ensureSourceOrdering", + p.ensureSourceOrdering, + "ExchangeNode", + "bool", + "ensureSourceOrdering"); to_json_key( j, - "functionId", - p.functionId, - "SqlInvokedFunction", - "SqlFunctionId", - "functionId"); + "orderingScheme", + p.orderingScheme, + "ExchangeNode", + "OrderingScheme", + "orderingScheme"); } -void from_json(const json& j, SqlInvokedFunction& p) { +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, "scope", p.scope, "ExchangeNode", "ExchangeNodeScope", "scope"); from_json_key( j, - "parameters", - p.parameters, - "SqlInvokedFunction", - "List", - "parameters"); + "partitioningScheme", + p.partitioningScheme, + "ExchangeNode", + "PartitioningScheme", + "partitioningScheme"); from_json_key( j, - "description", - p.description, - "SqlInvokedFunction", - "String", - "description"); + "sources", + p.sources, + "ExchangeNode", + "List>", + "sources"); from_json_key( j, - "routineCharacteristics", - p.routineCharacteristics, - "SqlInvokedFunction", - "RoutineCharacteristics", - "routineCharacteristics"); - from_json_key(j, "body", p.body, "SqlInvokedFunction", "String", "body"); + "inputs", + p.inputs, + "ExchangeNode", + "List>", + "inputs"); from_json_key( j, - "signature", - p.signature, - "SqlInvokedFunction", - "Signature", - "signature"); + "ensureSourceOrdering", + p.ensureSourceOrdering, + "ExchangeNode", + "bool", + "ensureSourceOrdering"); from_json_key( j, - "functionId", - p.functionId, - "SqlInvokedFunction", - "SqlFunctionId", - "functionId"); + "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 SessionRepresentation& p) { +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, "queryId", p.queryId, "SessionRepresentation", "String", "queryId"); + j, + "sourceFragmentIds", + p.sourceFragmentIds, + "RemoteSourceNode", + "List", + "sourceFragmentIds"); to_json_key( j, - "transactionId", - p.transactionId, - "SessionRepresentation", - "TransactionId", - "transactionId"); + "outputVariables", + p.outputVariables, + "RemoteSourceNode", + "List", + "outputVariables"); to_json_key( j, - "clientTransactionSupport", - p.clientTransactionSupport, - "SessionRepresentation", + "ensureSourceOrdering", + p.ensureSourceOrdering, + "RemoteSourceNode", "bool", - "clientTransactionSupport"); - to_json_key(j, "user", p.user, "SessionRepresentation", "String", "user"); + "ensureSourceOrdering"); 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"); + "orderingScheme", + p.orderingScheme, + "RemoteSourceNode", + "OrderingScheme", + "orderingScheme"); to_json_key( j, - "traceToken", - p.traceToken, - "SessionRepresentation", - "String", - "traceToken"); - to_json_key( + "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, - "timeZoneKey", - p.timeZoneKey, - "SessionRepresentation", - "TimeZoneKey", - "timeZoneKey"); - to_json_key( - j, "locale", p.locale, "SessionRepresentation", "Locale", "locale"); - to_json_key( + "sourceFragmentIds", + p.sourceFragmentIds, + "RemoteSourceNode", + "List", + "sourceFragmentIds"); + from_json_key( j, - "remoteUserAddress", - p.remoteUserAddress, - "SessionRepresentation", - "String", - "remoteUserAddress"); - to_json_key( + "outputVariables", + p.outputVariables, + "RemoteSourceNode", + "List", + "outputVariables"); + from_json_key( j, - "userAgent", - p.userAgent, - "SessionRepresentation", - "String", - "userAgent"); - to_json_key( + "ensureSourceOrdering", + p.ensureSourceOrdering, + "RemoteSourceNode", + "bool", + "ensureSourceOrdering"); + from_json_key( j, - "clientInfo", - p.clientInfo, - "SessionRepresentation", - "String", - "clientInfo"); - to_json_key( + "orderingScheme", + p.orderingScheme, + "RemoteSourceNode", + "OrderingScheme", + "orderingScheme"); + from_json_key( j, - "clientTags", - p.clientTags, - "SessionRepresentation", - "List", - "clientTags"); + "exchangeType", + p.exchangeType, + "RemoteSourceNode", + "ExchangeNodeType", + "exchangeType"); +} +} // 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) { + 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, - "resourceEstimates", - p.resourceEstimates, - "SessionRepresentation", - "ResourceEstimates", - "resourceEstimates"); + "rowCountVariable", + p.rowCountVariable, + "TableWriterNode", + "VariableReferenceExpression", + "rowCountVariable"); to_json_key( j, - "startTime", - p.startTime, - "SessionRepresentation", - "int64_t", - "startTime"); + "fragmentVariable", + p.fragmentVariable, + "TableWriterNode", + "VariableReferenceExpression", + "fragmentVariable"); to_json_key( j, - "systemProperties", - p.systemProperties, - "SessionRepresentation", - "Map", - "systemProperties"); + "tableCommitContextVariable", + p.tableCommitContextVariable, + "TableWriterNode", + "VariableReferenceExpression", + "tableCommitContextVariable"); to_json_key( j, - "catalogProperties", - p.catalogProperties, - "SessionRepresentation", - "Map>", - "catalogProperties"); + "columns", + p.columns, + "TableWriterNode", + "List", + "columns"); to_json_key( j, - "unprocessedCatalogProperties", - p.unprocessedCatalogProperties, - "SessionRepresentation", - "Map>", - "unprocessedCatalogProperties"); + "columnNames", + p.columnNames, + "TableWriterNode", + "List", + "columnNames"); to_json_key( j, - "roles", - p.roles, - "SessionRepresentation", - "Map", - "roles"); + "notNullColumnVariables", + p.notNullColumnVariables, + "TableWriterNode", + "List", + "notNullColumnVariables"); to_json_key( j, - "preparedStatements", - p.preparedStatements, - "SessionRepresentation", - "Map", - "preparedStatements"); + "partitioningScheme", + p.partitioningScheme, + "TableWriterNode", + "PartitioningScheme", + "partitioningScheme"); to_json_key( j, - "sessionFunctions", - p.sessionFunctions, - "SessionRepresentation", - "Map", - "sessionFunctions"); -} - -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( + "preferredShufflePartitioningScheme", + p.preferredShufflePartitioningScheme, + "TableWriterNode", + "PartitioningScheme", + "preferredShufflePartitioningScheme"); + to_json_key( j, - "traceToken", - p.traceToken, - "SessionRepresentation", - "String", - "traceToken"); + "statisticsAggregation", + p.statisticsAggregation, + "TableWriterNode", + "StatisticAggregations", + "statisticsAggregation"); +} + +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, - "timeZoneKey", - p.timeZoneKey, - "SessionRepresentation", - "TimeZoneKey", - "timeZoneKey"); - from_json_key( - j, "locale", p.locale, "SessionRepresentation", "Locale", "locale"); + "rowCountVariable", + p.rowCountVariable, + "TableWriterNode", + "VariableReferenceExpression", + "rowCountVariable"); from_json_key( j, - "remoteUserAddress", - p.remoteUserAddress, - "SessionRepresentation", - "String", - "remoteUserAddress"); + "fragmentVariable", + p.fragmentVariable, + "TableWriterNode", + "VariableReferenceExpression", + "fragmentVariable"); from_json_key( j, - "userAgent", - p.userAgent, - "SessionRepresentation", - "String", - "userAgent"); + "tableCommitContextVariable", + p.tableCommitContextVariable, + "TableWriterNode", + "VariableReferenceExpression", + "tableCommitContextVariable"); from_json_key( j, - "clientInfo", - p.clientInfo, - "SessionRepresentation", - "String", - "clientInfo"); + "columns", + p.columns, + "TableWriterNode", + "List", + "columns"); from_json_key( j, - "clientTags", - p.clientTags, - "SessionRepresentation", + "columnNames", + p.columnNames, + "TableWriterNode", "List", - "clientTags"); + "columnNames"); from_json_key( j, - "resourceEstimates", - p.resourceEstimates, - "SessionRepresentation", - "ResourceEstimates", - "resourceEstimates"); + "notNullColumnVariables", + p.notNullColumnVariables, + "TableWriterNode", + "List", + "notNullColumnVariables"); from_json_key( j, - "startTime", - p.startTime, - "SessionRepresentation", - "int64_t", - "startTime"); + "partitioningScheme", + p.partitioningScheme, + "TableWriterNode", + "PartitioningScheme", + "partitioningScheme"); from_json_key( j, - "systemProperties", - p.systemProperties, - "SessionRepresentation", - "Map", - "systemProperties"); + "preferredShufflePartitioningScheme", + p.preferredShufflePartitioningScheme, + "TableWriterNode", + "PartitioningScheme", + "preferredShufflePartitioningScheme"); from_json_key( j, - "catalogProperties", - p.catalogProperties, - "SessionRepresentation", - "Map>", - "catalogProperties"); - from_json_key( + "statisticsAggregation", + p.statisticsAggregation, + "TableWriterNode", + "StatisticAggregations", + "statisticsAggregation"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const Specification& p) { + j = json::object(); + to_json_key( j, - "unprocessedCatalogProperties", - p.unprocessedCatalogProperties, - "SessionRepresentation", - "Map>", - "unprocessedCatalogProperties"); - from_json_key( + "partitionBy", + p.partitionBy, + "Specification", + "List", + "partitionBy"); + to_json_key( j, - "roles", - p.roles, - "SessionRepresentation", - "Map", - "roles"); + "orderingScheme", + p.orderingScheme, + "Specification", + "OrderingScheme", + "orderingScheme"); +} + +void from_json(const json& j, Specification& p) { from_json_key( j, - "preparedStatements", - p.preparedStatements, - "SessionRepresentation", - "Map", - "preparedStatements"); + "partitionBy", + p.partitionBy, + "Specification", + "List", + "partitionBy"); from_json_key( j, - "sessionFunctions", - p.sessionFunctions, - "SessionRepresentation", - "Map", - "sessionFunctions"); + "orderingScheme", + p.orderingScheme, + "Specification", + "OrderingScheme", + "orderingScheme"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +TopNRowNumberNode::TopNRowNumberNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.TopNRowNumberNode"; +} -void to_json(json& j, const TaskUpdateRequest& p) { +void to_json(json& j, const TopNRowNumberNode& p) { j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.TopNRowNumberNode"; + to_json_key(j, "id", p.id, "TopNRowNumberNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "TopNRowNumberNode", "PlanNode", "source"); to_json_key( j, - "session", - p.session, - "TaskUpdateRequest", - "SessionRepresentation", - "session"); - to_json_key( - j, - "extraCredentials", - p.extraCredentials, - "TaskUpdateRequest", - "Map", - "extraCredentials"); - to_json_key( - j, "fragment", p.fragment, "TaskUpdateRequest", "String", "fragment"); + "specification", + p.specification, + "TopNRowNumberNode", + "Specification", + "specification"); to_json_key( j, - "sources", - p.sources, - "TaskUpdateRequest", - "List", - "sources"); + "rowNumberVariable", + p.rowNumberVariable, + "TopNRowNumberNode", + "VariableReferenceExpression", + "rowNumberVariable"); to_json_key( j, - "outputIds", - p.outputIds, - "TaskUpdateRequest", - "OutputBuffers", - "outputIds"); + "maxRowCountPerPartition", + p.maxRowCountPerPartition, + "TopNRowNumberNode", + "int", + "maxRowCountPerPartition"); + to_json_key(j, "partial", p.partial, "TopNRowNumberNode", "bool", "partial"); to_json_key( j, - "tableWriteInfo", - p.tableWriteInfo, - "TaskUpdateRequest", - "TableWriteInfo", - "tableWriteInfo"); + "hashVariable", + p.hashVariable, + "TopNRowNumberNode", + "VariableReferenceExpression", + "hashVariable"); } -void from_json(const json& j, TaskUpdateRequest& p) { +void from_json(const json& j, TopNRowNumberNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "TopNRowNumberNode", "PlanNodeId", "id"); from_json_key( - j, - "session", - p.session, - "TaskUpdateRequest", - "SessionRepresentation", - "session"); + j, "source", p.source, "TopNRowNumberNode", "PlanNode", "source"); from_json_key( j, - "extraCredentials", - p.extraCredentials, - "TaskUpdateRequest", - "Map", - "extraCredentials"); - from_json_key( - j, "fragment", p.fragment, "TaskUpdateRequest", "String", "fragment"); + "specification", + p.specification, + "TopNRowNumberNode", + "Specification", + "specification"); from_json_key( j, - "sources", - p.sources, - "TaskUpdateRequest", - "List", - "sources"); + "rowNumberVariable", + p.rowNumberVariable, + "TopNRowNumberNode", + "VariableReferenceExpression", + "rowNumberVariable"); from_json_key( j, - "outputIds", - p.outputIds, - "TaskUpdateRequest", - "OutputBuffers", - "outputIds"); + "maxRowCountPerPartition", + p.maxRowCountPerPartition, + "TopNRowNumberNode", + "int", + "maxRowCountPerPartition"); + from_json_key( + j, "partial", p.partial, "TopNRowNumberNode", "bool", "partial"); from_json_key( j, - "tableWriteInfo", - p.tableWriteInfo, - "TaskUpdateRequest", - "TableWriteInfo", - "tableWriteInfo"); + "hashVariable", + p.hashVariable, + "TopNRowNumberNode", + "VariableReferenceExpression", + "hashVariable"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const BatchTaskUpdateRequest& p) { +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair IcebergTableType_enum_table[] = + { // NOLINT: cert-err58-cpp + {IcebergTableType::DATA, "DATA"}, + {IcebergTableType::HISTORY, "HISTORY"}, + {IcebergTableType::SNAPSHOTS, "SNAPSHOTS"}, + {IcebergTableType::MANIFESTS, "MANIFESTS"}, + {IcebergTableType::PARTITIONS, "PARTITIONS"}, + {IcebergTableType::FILES, "FILES"}, + {IcebergTableType::PROPERTIES, "PROPERTIES"}, + {IcebergTableType::CHANGELOG, "CHANGELOG"}, + {IcebergTableType::EQUALITY_DELETES, "EQUALITY_DELETES"}, + {IcebergTableType::DATA_WITHOUT_EQUALITY_DELETES, + "DATA_WITHOUT_EQUALITY_DELETES"}}; +void to_json(json& j, const IcebergTableType& e) { + static_assert( + std::is_enum::value, + "IcebergTableType must be an enum!"); + const auto* it = std::find_if( + std::begin(IcebergTableType_enum_table), + std::end(IcebergTableType_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(IcebergTableType_enum_table)) + ? it + : std::begin(IcebergTableType_enum_table)) + ->second; +} +void from_json(const json& j, IcebergTableType& e) { + static_assert( + std::is_enum::value, + "IcebergTableType must be an enum!"); + const auto* it = std::find_if( + std::begin(IcebergTableType_enum_table), + std::end(IcebergTableType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(IcebergTableType_enum_table)) + ? it + : std::begin(IcebergTableType_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const IcebergTableName& p) { j = json::object(); to_json_key( - j, - "taskUpdateRequest", - p.taskUpdateRequest, - "BatchTaskUpdateRequest", - "TaskUpdateRequest", - "taskUpdateRequest"); + j, "tableName", p.tableName, "IcebergTableName", "String", "tableName"); to_json_key( j, - "shuffleWriteInfo", - p.shuffleWriteInfo, - "BatchTaskUpdateRequest", - "String", - "shuffleWriteInfo"); + "tableType", + p.tableType, + "IcebergTableName", + "IcebergTableType", + "tableType"); + to_json_key( + j, "snapshotId", p.snapshotId, "IcebergTableName", "Long", "snapshotId"); to_json_key( j, - "broadcastBasePath", - p.broadcastBasePath, - "BatchTaskUpdateRequest", - "String", - "broadcastBasePath"); + "changelogEndSnapshot", + p.changelogEndSnapshot, + "IcebergTableName", + "Long", + "changelogEndSnapshot"); } -void from_json(const json& j, BatchTaskUpdateRequest& p) { +void from_json(const json& j, IcebergTableName& p) { from_json_key( - j, - "taskUpdateRequest", - p.taskUpdateRequest, - "BatchTaskUpdateRequest", - "TaskUpdateRequest", - "taskUpdateRequest"); + j, "tableName", p.tableName, "IcebergTableName", "String", "tableName"); from_json_key( j, - "shuffleWriteInfo", - p.shuffleWriteInfo, - "BatchTaskUpdateRequest", - "String", - "shuffleWriteInfo"); + "tableType", + p.tableType, + "IcebergTableName", + "IcebergTableType", + "tableType"); + from_json_key( + j, "snapshotId", p.snapshotId, "IcebergTableName", "Long", "snapshotId"); from_json_key( j, - "broadcastBasePath", - p.broadcastBasePath, - "BatchTaskUpdateRequest", - "String", - "broadcastBasePath"); + "changelogEndSnapshot", + p.changelogEndSnapshot, + "IcebergTableName", + "Long", + "changelogEndSnapshot"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveOutputTableHandle::HiveOutputTableHandle() noexcept { - _type = "hive"; +IcebergTableHandle::IcebergTableHandle() noexcept { + _type = "hive-iceberg"; } -void to_json(json& j, const HiveOutputTableHandle& p) { +void to_json(json& j, const IcebergTableHandle& p) { j = json::object(); - j["@type"] = "hive"; + j["@type"] = "hive-iceberg"; to_json_key( j, "schemaName", p.schemaName, - "HiveOutputTableHandle", + "IcebergTableHandle", "String", "schemaName"); to_json_key( j, - "tableName", - p.tableName, - "HiveOutputTableHandle", - "String", - "tableName"); - to_json_key( - j, - "inputColumns", - p.inputColumns, - "HiveOutputTableHandle", - "List", - "inputColumns"); - to_json_key( - j, - "pageSinkMetadata", - p.pageSinkMetadata, - "HiveOutputTableHandle", - "HivePageSinkMetadata", - "pageSinkMetadata"); - to_json_key( - j, - "locationHandle", - p.locationHandle, - "HiveOutputTableHandle", - "LocationHandle", - "locationHandle"); - to_json_key( - j, - "tableStorageFormat", - p.tableStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "tableStorageFormat"); - to_json_key( - j, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); - to_json_key( - j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); - to_json_key( - j, - "compressionCodec", - p.compressionCodec, - "HiveOutputTableHandle", - "HiveCompressionCodec", - "compressionCodec"); - to_json_key( - j, - "partitionedBy", - p.partitionedBy, - "HiveOutputTableHandle", - "List", - "partitionedBy"); + "icebergTableName", + p.icebergTableName, + "IcebergTableHandle", + "IcebergTableName", + "icebergTableName"); to_json_key( j, - "bucketProperty", - p.bucketProperty, - "HiveOutputTableHandle", - "HiveBucketProperty", - "bucketProperty"); + "snapshotSpecified", + p.snapshotSpecified, + "IcebergTableHandle", + "bool", + "snapshotSpecified"); to_json_key( j, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveOutputTableHandle", - "List", - "preferredOrderingColumns"); + "predicate", + p.predicate, + "IcebergTableHandle", + "TupleDomain", + "predicate"); to_json_key( j, - "tableOwner", - p.tableOwner, - "HiveOutputTableHandle", + "tableSchemaJson", + p.tableSchemaJson, + "IcebergTableHandle", "String", - "tableOwner"); + "tableSchemaJson"); to_json_key( j, - "additionalTableParameters", - p.additionalTableParameters, - "HiveOutputTableHandle", - "Map", - "additionalTableParameters"); + "partitionFieldIds", + p.partitionFieldIds, + "IcebergTableHandle", + "List", + "partitionFieldIds"); to_json_key( j, - "encryptionInformation", - p.encryptionInformation, - "HiveOutputTableHandle", - "EncryptionInformation", - "encryptionInformation"); + "equalityFieldIds", + p.equalityFieldIds, + "IcebergTableHandle", + "List", + "equalityFieldIds"); } -void from_json(const json& j, HiveOutputTableHandle& p) { +void from_json(const json& j, IcebergTableHandle& p) { p._type = j["@type"]; from_json_key( j, "schemaName", p.schemaName, - "HiveOutputTableHandle", + "IcebergTableHandle", "String", "schemaName"); from_json_key( j, - "tableName", - p.tableName, - "HiveOutputTableHandle", - "String", - "tableName"); + "icebergTableName", + p.icebergTableName, + "IcebergTableHandle", + "IcebergTableName", + "icebergTableName"); from_json_key( j, - "inputColumns", - p.inputColumns, - "HiveOutputTableHandle", - "List", - "inputColumns"); + "snapshotSpecified", + p.snapshotSpecified, + "IcebergTableHandle", + "bool", + "snapshotSpecified"); from_json_key( j, - "pageSinkMetadata", - p.pageSinkMetadata, - "HiveOutputTableHandle", - "HivePageSinkMetadata", - "pageSinkMetadata"); + "predicate", + p.predicate, + "IcebergTableHandle", + "TupleDomain", + "predicate"); from_json_key( j, - "locationHandle", - p.locationHandle, - "HiveOutputTableHandle", - "LocationHandle", - "locationHandle"); + "tableSchemaJson", + p.tableSchemaJson, + "IcebergTableHandle", + "String", + "tableSchemaJson"); + from_json_key( + j, + "partitionFieldIds", + p.partitionFieldIds, + "IcebergTableHandle", + "List", + "partitionFieldIds"); from_json_key( j, - "tableStorageFormat", - p.tableStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "tableStorageFormat"); + "equalityFieldIds", + p.equalityFieldIds, + "IcebergTableHandle", + "List", + "equalityFieldIds"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +IcebergTableLayoutHandle::IcebergTableLayoutHandle() noexcept { + _type = "hive-iceberg"; +} + +void to_json(json& j, const IcebergTableLayoutHandle& p) { + j = json::object(); + j["@type"] = "hive-iceberg"; + to_json_key( + j, + "partitionColumns", + p.partitionColumns, + "IcebergTableLayoutHandle", + "List", + "partitionColumns"); + to_json_key( + j, + "dataColumns", + p.dataColumns, + "IcebergTableLayoutHandle", + "List", + "dataColumns"); + to_json_key( + j, + "domainPredicate", + p.domainPredicate, + "IcebergTableLayoutHandle", + "TupleDomain", + "domainPredicate"); + to_json_key( + j, + "remainingPredicate", + p.remainingPredicate, + "IcebergTableLayoutHandle", + "RowExpression", + "remainingPredicate"); + to_json_key( + j, + "predicateColumns", + p.predicateColumns, + "IcebergTableLayoutHandle", + "Map", + "predicateColumns"); + to_json_key( + j, + "requestedColumns", + p.requestedColumns, + "IcebergTableLayoutHandle", + "List", + "requestedColumns"); + to_json_key( + j, + "pushdownFilterEnabled", + p.pushdownFilterEnabled, + "IcebergTableLayoutHandle", + "bool", + "pushdownFilterEnabled"); + to_json_key( + j, + "partitionColumnPredicate", + p.partitionColumnPredicate, + "IcebergTableLayoutHandle", + "TupleDomain>", + "partitionColumnPredicate"); + to_json_key( + j, + "table", + p.table, + "IcebergTableLayoutHandle", + "IcebergTableHandle", + "table"); +} + +void from_json(const json& j, IcebergTableLayoutHandle& p) { + p._type = j["@type"]; from_json_key( j, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); + "partitionColumns", + p.partitionColumns, + "IcebergTableLayoutHandle", + "List", + "partitionColumns"); from_json_key( j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); + "dataColumns", + p.dataColumns, + "IcebergTableLayoutHandle", + "List", + "dataColumns"); from_json_key( j, - "compressionCodec", - p.compressionCodec, - "HiveOutputTableHandle", - "HiveCompressionCodec", - "compressionCodec"); + "domainPredicate", + p.domainPredicate, + "IcebergTableLayoutHandle", + "TupleDomain", + "domainPredicate"); from_json_key( j, - "partitionedBy", - p.partitionedBy, - "HiveOutputTableHandle", - "List", - "partitionedBy"); + "remainingPredicate", + p.remainingPredicate, + "IcebergTableLayoutHandle", + "RowExpression", + "remainingPredicate"); from_json_key( j, - "bucketProperty", - p.bucketProperty, - "HiveOutputTableHandle", - "HiveBucketProperty", - "bucketProperty"); + "predicateColumns", + p.predicateColumns, + "IcebergTableLayoutHandle", + "Map", + "predicateColumns"); from_json_key( j, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveOutputTableHandle", - "List", - "preferredOrderingColumns"); + "requestedColumns", + p.requestedColumns, + "IcebergTableLayoutHandle", + "List", + "requestedColumns"); from_json_key( j, - "tableOwner", - p.tableOwner, - "HiveOutputTableHandle", - "String", - "tableOwner"); + "pushdownFilterEnabled", + p.pushdownFilterEnabled, + "IcebergTableLayoutHandle", + "bool", + "pushdownFilterEnabled"); from_json_key( j, - "additionalTableParameters", - p.additionalTableParameters, - "HiveOutputTableHandle", - "Map", - "additionalTableParameters"); + "partitionColumnPredicate", + p.partitionColumnPredicate, + "IcebergTableLayoutHandle", + "TupleDomain>", + "partitionColumnPredicate"); from_json_key( j, - "encryptionInformation", - p.encryptionInformation, - "HiveOutputTableHandle", - "EncryptionInformation", - "encryptionInformation"); + "table", + p.table, + "IcebergTableLayoutHandle", + "IcebergTableHandle", + "table"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TableScanNode::TableScanNode() noexcept { - _type = ".TableScanNode"; +OutputNode::OutputNode() noexcept { + _type = ".OutputNode"; } -void to_json(json& j, const TableScanNode& p) { +void to_json(json& j, const OutputNode& 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"); + j["@type"] = ".OutputNode"; + to_json_key(j, "id", p.id, "OutputNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "OutputNode", "PlanNode", "source"); + to_json_key( + j, + "columnNames", + p.columnNames, + "OutputNode", + "List", + "columnNames"); to_json_key( j, "outputVariables", p.outputVariables, - "TableScanNode", + "OutputNode", "List", "outputVariables"); - to_json_key( - j, - "assignments", - p.assignments, - "TableScanNode", - "Map>", - "assignments"); } -void from_json(const json& j, TableScanNode& p) { +void from_json(const json& j, OutputNode& 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, "id", p.id, "OutputNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "OutputNode", "PlanNode", "source"); + from_json_key( + j, + "columnNames", + p.columnNames, + "OutputNode", + "List", + "columnNames"); from_json_key( j, "outputVariables", p.outputVariables, - "TableScanNode", + "OutputNode", "List", "outputVariables"); - from_json_key( - j, - "assignments", - p.assignments, - "TableScanNode", - "Map>", - "assignments"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -EquatableValueSet::EquatableValueSet() noexcept { - _type = "equatable"; +HivePartitioningHandle::HivePartitioningHandle() noexcept { + _type = "hive"; } -void to_json(json& j, const EquatableValueSet& p) { +void to_json(json& j, const HivePartitioningHandle& p) { j = json::object(); - j["@type"] = "equatable"; - to_json_key(j, "type", p.type, "EquatableValueSet", "Type", "type"); + j["@type"] = "hive"; to_json_key( - j, "whiteList", p.whiteList, "EquatableValueSet", "bool", "whiteList"); + j, + "bucketCount", + p.bucketCount, + "HivePartitioningHandle", + "int", + "bucketCount"); to_json_key( j, - "entries", - p.entries, - "EquatableValueSet", - "List", - "entries"); + "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, EquatableValueSet& p) { +void from_json(const json& j, HivePartitioningHandle& 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"); + j, + "bucketCount", + p.bucketCount, + "HivePartitioningHandle", + "int", + "bucketCount"); from_json_key( j, - "entries", - p.entries, - "EquatableValueSet", - "List", - "entries"); + "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 +namespace facebook::presto::protocol { +EnforceSingleRowNode::EnforceSingleRowNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode"; +} + +void to_json(json& j, const EnforceSingleRowNode& p) { + j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode"; + to_json_key(j, "id", p.id, "EnforceSingleRowNode", "PlanNodeId", "id"); + to_json_key( + 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 UpdateHandle& p) { + j = json::object(); + to_json_key(j, "handle", p.handle, "UpdateHandle", "TableHandle", "handle"); + to_json_key( + j, + "schemaTableName", + p.schemaTableName, + "UpdateHandle", + "SchemaTableName", + "schemaTableName"); +} + +void from_json(const json& j, UpdateHandle& p) { + from_json_key(j, "handle", p.handle, "UpdateHandle", "TableHandle", "handle"); + from_json_key( + j, + "schemaTableName", + p.schemaTableName, + "UpdateHandle", + "SchemaTableName", + "schemaTableName"); } } // 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 HiveBucketHandle& 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, - "columns", - p.columns, - "HiveBucketHandle", - "List", - "columns"); + "groupingSets", + p.groupingSets, + "GroupIdNode", + "List>", + "groupingSets"); to_json_key( j, - "tableBucketCount", - p.tableBucketCount, - "HiveBucketHandle", - "int", - "tableBucketCount"); + "groupingColumns", + p.groupingColumns, + "GroupIdNode", + "Map", + "groupingColumns"); to_json_key( j, - "readBucketCount", - p.readBucketCount, - "HiveBucketHandle", - "int", - "readBucketCount"); + "aggregationArguments", + p.aggregationArguments, + "GroupIdNode", + "List", + "aggregationArguments"); + to_json_key( + j, + "groupIdVariable", + p.groupIdVariable, + "GroupIdNode", + "VariableReferenceExpression", + "groupIdVariable"); } -void from_json(const json& j, HiveBucketHandle& 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, - "columns", - p.columns, - "HiveBucketHandle", - "List", - "columns"); + "groupingSets", + p.groupingSets, + "GroupIdNode", + "List>", + "groupingSets"); from_json_key( j, - "tableBucketCount", - p.tableBucketCount, - "HiveBucketHandle", - "int", - "tableBucketCount"); + "groupingColumns", + p.groupingColumns, + "GroupIdNode", + "Map", + "groupingColumns"); from_json_key( j, - "readBucketCount", - p.readBucketCount, - "HiveBucketHandle", - "int", - "readBucketCount"); + "aggregationArguments", + p.aggregationArguments, + "GroupIdNode", + "List", + "aggregationArguments"); + from_json_key( + j, + "groupIdVariable", + p.groupIdVariable, + "GroupIdNode", + "VariableReferenceExpression", + "groupIdVariable"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveTableLayoutHandle::HiveTableLayoutHandle() noexcept { - _type = "hive"; -} -void to_json(json& j, const HiveTableLayoutHandle& p) { +void to_json(json& j, const NodeStatus& p) { j = json::object(); - j["@type"] = "hive"; - to_json_key( - j, - "schemaTableName", - p.schemaTableName, - "HiveTableLayoutHandle", - "SchemaTableName", - "schemaTableName"); + to_json_key(j, "nodeId", p.nodeId, "NodeStatus", "String", "nodeId"); to_json_key( j, - "tablePath", - p.tablePath, - "HiveTableLayoutHandle", - "String", - "tablePath"); + "nodeVersion", + p.nodeVersion, + "NodeStatus", + "NodeVersion", + "nodeVersion"); to_json_key( - j, - "partitionColumns", - p.partitionColumns, - "HiveTableLayoutHandle", - "List", - "partitionColumns"); + j, "environment", p.environment, "NodeStatus", "String", "environment"); to_json_key( - j, - "dataColumns", - p.dataColumns, - "HiveTableLayoutHandle", - "List", - "dataColumns"); + j, "coordinator", p.coordinator, "NodeStatus", "bool", "coordinator"); + to_json_key(j, "uptime", p.uptime, "NodeStatus", "Duration", "uptime"); to_json_key( j, - "tableParameters", - p.tableParameters, - "HiveTableLayoutHandle", - "Map", - "tableParameters"); + "externalAddress", + p.externalAddress, + "NodeStatus", + "String", + "externalAddress"); to_json_key( j, - "domainPredicate", - p.domainPredicate, - "HiveTableLayoutHandle", - "TupleDomain", - "domainPredicate"); + "internalAddress", + p.internalAddress, + "NodeStatus", + "String", + "internalAddress"); to_json_key( - j, - "remainingPredicate", - p.remainingPredicate, - "HiveTableLayoutHandle", - "RowExpression", - "remainingPredicate"); + j, "memoryInfo", p.memoryInfo, "NodeStatus", "MemoryInfo", "memoryInfo"); + to_json_key(j, "processors", p.processors, "NodeStatus", "int", "processors"); to_json_key( j, - "predicateColumns", - p.predicateColumns, - "HiveTableLayoutHandle", - "Map", - "predicateColumns"); + "processCpuLoad", + p.processCpuLoad, + "NodeStatus", + "double", + "processCpuLoad"); to_json_key( j, - "partitionColumnPredicate", - p.partitionColumnPredicate, - "HiveTableLayoutHandle", - "TupleDomain>", - "partitionColumnPredicate"); + "systemCpuLoad", + p.systemCpuLoad, + "NodeStatus", + "double", + "systemCpuLoad"); + to_json_key(j, "heapUsed", p.heapUsed, "NodeStatus", "int64_t", "heapUsed"); to_json_key( j, - "bucketHandle", - p.bucketHandle, - "HiveTableLayoutHandle", - "HiveBucketHandle", - "bucketHandle"); + "heapAvailable", + p.heapAvailable, + "NodeStatus", + "int64_t", + "heapAvailable"); to_json_key( + j, "nonHeapUsed", p.nonHeapUsed, "NodeStatus", "int64_t", "nonHeapUsed"); +} + +void from_json(const json& j, NodeStatus& p) { + from_json_key(j, "nodeId", p.nodeId, "NodeStatus", "String", "nodeId"); + from_json_key( j, - "bucketFilter", - p.bucketFilter, - "HiveTableLayoutHandle", - "HiveBucketFilter", - "bucketFilter"); - to_json_key( + "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"); + from_json_key( j, - "pushdownFilterEnabled", - p.pushdownFilterEnabled, - "HiveTableLayoutHandle", - "bool", - "pushdownFilterEnabled"); - to_json_key( + "externalAddress", + p.externalAddress, + "NodeStatus", + "String", + "externalAddress"); + from_json_key( j, - "layoutString", - p.layoutString, - "HiveTableLayoutHandle", + "internalAddress", + p.internalAddress, + "NodeStatus", "String", - "layoutString"); - to_json_key( + "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, - "requestedColumns", - p.requestedColumns, - "HiveTableLayoutHandle", - "List", - "requestedColumns"); - to_json_key( + "processCpuLoad", + p.processCpuLoad, + "NodeStatus", + "double", + "processCpuLoad"); + from_json_key( j, - "partialAggregationsPushedDown", - p.partialAggregationsPushedDown, - "HiveTableLayoutHandle", - "bool", - "partialAggregationsPushedDown"); + "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 { + +void to_json(json& j, const TableToPartitionMapping& p) { + j = json::object(); to_json_key( j, - "appendRowNumber", - p.appendRowNumber, - "HiveTableLayoutHandle", - "bool", - "appendRowNumber"); + "tableToPartitionColumns", + p.tableToPartitionColumns, + "TableToPartitionMapping", + "Map", + "tableToPartitionColumns"); to_json_key( j, - "footerStatsUnreliable", - p.footerStatsUnreliable, - "HiveTableLayoutHandle", - "bool", - "footerStatsUnreliable"); + "partitionSchemaDifference", + p.partitionSchemaDifference, + "TableToPartitionMapping", + "Map", + "partitionSchemaDifference"); } -void from_json(const json& j, HiveTableLayoutHandle& p) { - p._type = j["@type"]; +void from_json(const json& j, TableToPartitionMapping& p) { from_json_key( j, - "schemaTableName", - p.schemaTableName, - "HiveTableLayoutHandle", - "SchemaTableName", - "schemaTableName"); + "tableToPartitionColumns", + p.tableToPartitionColumns, + "TableToPartitionMapping", + "Map", + "tableToPartitionColumns"); from_json_key( j, - "tablePath", - p.tablePath, - "HiveTableLayoutHandle", - "String", - "tablePath"); - from_json_key( + "partitionSchemaDifference", + p.partitionSchemaDifference, + "TableToPartitionMapping", + "Map", + "partitionSchemaDifference"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +EmptySplit::EmptySplit() noexcept { + _type = "$empty"; +} + +void to_json(json& j, const EmptySplit& p) { + j = json::object(); + j["@type"] = "$empty"; + to_json_key( j, - "partitionColumns", - p.partitionColumns, - "HiveTableLayoutHandle", - "List", - "partitionColumns"); + "connectorId", + p.connectorId, + "EmptySplit", + "ConnectorId", + "connectorId"); +} + +void from_json(const json& j, EmptySplit& p) { + p._type = j["@type"]; from_json_key( j, - "dataColumns", - p.dataColumns, - "HiveTableLayoutHandle", - "List", - "dataColumns"); - from_json_key( + "connectorId", + p.connectorId, + "EmptySplit", + "ConnectorId", + "connectorId"); +} +} // 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( j, - "tableParameters", - p.tableParameters, - "HiveTableLayoutHandle", - "Map", - "tableParameters"); - from_json_key( + "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( j, - "domainPredicate", - p.domainPredicate, - "HiveTableLayoutHandle", - "TupleDomain", - "domainPredicate"); - from_json_key( + "specification", + p.specification, + "WindowNode", + "Specification", + "specification"); + to_json_key( j, - "remainingPredicate", - p.remainingPredicate, - "HiveTableLayoutHandle", - "RowExpression", - "remainingPredicate"); - from_json_key( + "windowFunctions", + p.windowFunctions, + "WindowNode", + "Map", + "windowFunctions"); + to_json_key( j, - "predicateColumns", - p.predicateColumns, - "HiveTableLayoutHandle", - "Map", - "predicateColumns"); - from_json_key( + "hashVariable", + p.hashVariable, + "WindowNode", + "VariableReferenceExpression", + "hashVariable"); + to_json_key( j, - "partitionColumnPredicate", - p.partitionColumnPredicate, - "HiveTableLayoutHandle", - "TupleDomain>", - "partitionColumnPredicate"); - from_json_key( + "prePartitionedInputs", + p.prePartitionedInputs, + "WindowNode", + "List", + "prePartitionedInputs"); + to_json_key( j, - "bucketHandle", - p.bucketHandle, - "HiveTableLayoutHandle", - "HiveBucketHandle", - "bucketHandle"); + "preSortedOrderPrefix", + p.preSortedOrderPrefix, + "WindowNode", + "int", + "preSortedOrderPrefix"); +} + +void from_json(const json& j, WindowNode& p) { + p._type = j["@type"]; from_json_key( j, - "bucketFilter", - p.bucketFilter, - "HiveTableLayoutHandle", - "HiveBucketFilter", - "bucketFilter"); + "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"); from_json_key( j, - "pushdownFilterEnabled", - p.pushdownFilterEnabled, - "HiveTableLayoutHandle", - "bool", - "pushdownFilterEnabled"); + "specification", + p.specification, + "WindowNode", + "Specification", + "specification"); from_json_key( j, - "layoutString", - p.layoutString, - "HiveTableLayoutHandle", - "String", - "layoutString"); + "windowFunctions", + p.windowFunctions, + "WindowNode", + "Map", + "windowFunctions"); from_json_key( j, - "requestedColumns", - p.requestedColumns, - "HiveTableLayoutHandle", - "List", - "requestedColumns"); + "hashVariable", + p.hashVariable, + "WindowNode", + "VariableReferenceExpression", + "hashVariable"); from_json_key( j, - "partialAggregationsPushedDown", - p.partialAggregationsPushedDown, - "HiveTableLayoutHandle", - "bool", - "partialAggregationsPushedDown"); + "prePartitionedInputs", + p.prePartitionedInputs, + "WindowNode", + "List", + "prePartitionedInputs"); from_json_key( j, - "appendRowNumber", - p.appendRowNumber, - "HiveTableLayoutHandle", - "bool", - "appendRowNumber"); + "preSortedOrderPrefix", + p.preSortedOrderPrefix, + "WindowNode", + "int", + "preSortedOrderPrefix"); +} +} // 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", "Type", "type"); +} + +void from_json(const json& j, TpchColumnHandle& p) { + p._type = j["@type"]; from_json_key( j, - "footerStatsUnreliable", - p.footerStatsUnreliable, - "HiveTableLayoutHandle", - "bool", - "footerStatsUnreliable"); + "columnName", + p.columnName, + "TpchColumnHandle", + "String", + "columnName"); + from_json_key(j, "type", p.type, "TpchColumnHandle", "Type", "type"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -UnnestNode::UnnestNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.UnnestNode"; +DistinctLimitNode::DistinctLimitNode() noexcept { + _type = ".DistinctLimitNode"; } -void to_json(json& j, const UnnestNode& p) { +void to_json(json& j, const DistinctLimitNode& 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"); + 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, - "replicateVariables", - p.replicateVariables, - "UnnestNode", + "distinctVariables", + p.distinctVariables, + "DistinctLimitNode", "List", - "replicateVariables"); + "distinctVariables"); to_json_key( j, - "unnestVariables", - p.unnestVariables, - "UnnestNode", - "Map>", - "unnestVariables"); + "hashVariable", + p.hashVariable, + "DistinctLimitNode", + "VariableReferenceExpression", + "hashVariable"); to_json_key( j, - "ordinalityVariable", - p.ordinalityVariable, - "UnnestNode", - "VariableReferenceExpression", - "ordinalityVariable"); + "timeoutMillis", + p.timeoutMillis, + "DistinctLimitNode", + "int", + "timeoutMillis"); } -void from_json(const json& j, UnnestNode& p) { +void from_json(const json& j, DistinctLimitNode& 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, "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, - "replicateVariables", - p.replicateVariables, - "UnnestNode", + "distinctVariables", + p.distinctVariables, + "DistinctLimitNode", "List", - "replicateVariables"); + "distinctVariables"); from_json_key( j, - "unnestVariables", - p.unnestVariables, - "UnnestNode", - "Map>", - "unnestVariables"); + "hashVariable", + p.hashVariable, + "DistinctLimitNode", + "VariableReferenceExpression", + "hashVariable"); from_json_key( j, - "ordinalityVariable", - p.ordinalityVariable, - "UnnestNode", - "VariableReferenceExpression", - "ordinalityVariable"); -} -} // 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 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; + "timeoutMillis", + p.timeoutMillis, + "DistinctLimitNode", + "int", + "timeoutMillis"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -SpecialFormExpression::SpecialFormExpression() noexcept { - _type = "special"; +InsertHandle::InsertHandle() noexcept { + _type = "InsertHandle"; } -void to_json(json& j, const SpecialFormExpression& p) { +void to_json(json& j, const InsertHandle& p) { j = json::object(); - j["@type"] = "special"; - to_json_key( - j, - "sourceLocation", - p.sourceLocation, - "SpecialFormExpression", - "SourceLocation", - "sourceLocation"); - to_json_key(j, "form", p.form, "SpecialFormExpression", "Form", "form"); + j["@type"] = "InsertHandle"; to_json_key( - j, - "returnType", - p.returnType, - "SpecialFormExpression", - "Type", - "returnType"); + j, "handle", p.handle, "InsertHandle", "InsertTableHandle", "handle"); to_json_key( j, - "arguments", - p.arguments, - "SpecialFormExpression", - "List>", - "arguments"); + "schemaTableName", + p.schemaTableName, + "InsertHandle", + "SchemaTableName", + "schemaTableName"); } -void from_json(const json& j, SpecialFormExpression& p) { +void from_json(const json& j, InsertHandle& p) { p._type = j["@type"]; from_json_key( - j, - "sourceLocation", - p.sourceLocation, - "SpecialFormExpression", - "SourceLocation", - "sourceLocation"); - from_json_key(j, "form", p.form, "SpecialFormExpression", "Form", "form"); - from_json_key( - j, - "returnType", - p.returnType, - "SpecialFormExpression", - "Type", - "returnType"); + j, "handle", p.handle, "InsertHandle", "InsertTableHandle", "handle"); from_json_key( j, - "arguments", - p.arguments, - "SpecialFormExpression", - "List>", - "arguments"); + "schemaTableName", + p.schemaTableName, + "InsertHandle", + "SchemaTableName", + "schemaTableName"); } } // 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 ConnectorOutputTableHandle "); +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, + "planNodeId", + p.planNodeId, + "ScheduledSplit", + "PlanNodeId", + "planNodeId"); + to_json_key(j, "split", p.split, "ScheduledSplit", "Split", "split"); } -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"); - } +void from_json(const json& j, ScheduledSplit& p) { + from_json_key( + j, "sequenceId", p.sequenceId, "ScheduledSplit", "int64_t", "sequenceId"); + from_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 { - if (getConnectorKey(type) == "hive") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } +void to_json(json& j, const TaskSource& p) { + j = json::object(); + to_json_key( + j, "planNodeId", p.planNodeId, "TaskSource", "PlanNodeId", "planNodeId"); + to_json_key( + j, "splits", p.splits, "TaskSource", "List", "splits"); + to_json_key( + j, + "noMoreSplitsForLifespan", + p.noMoreSplitsForLifespan, + "TaskSource", + "List", + "noMoreSplitsForLifespan"); + to_json_key( + j, "noMoreSplits", p.noMoreSplits, "TaskSource", "bool", "noMoreSplits"); +} - throw TypeError(type + " no abstract type ConnectorOutputTableHandle "); +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 OutputTableHandle& p) { +void to_json(json& j, const TaskUpdateRequest& p) { j = json::object(); to_json_key( j, - "connectorId", - p.connectorId, - "OutputTableHandle", - "ConnectorId", - "connectorId"); + "session", + p.session, + "TaskUpdateRequest", + "SessionRepresentation", + "session"); to_json_key( j, - "transactionHandle", - p.transactionHandle, - "OutputTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "extraCredentials", + p.extraCredentials, + "TaskUpdateRequest", + "Map", + "extraCredentials"); + to_json_key( + j, "fragment", p.fragment, "TaskUpdateRequest", "String", "fragment"); to_json_key( j, - "connectorHandle", - p.connectorHandle, - "OutputTableHandle", - "ConnectorOutputTableHandle", - "connectorHandle"); + "sources", + p.sources, + "TaskUpdateRequest", + "List", + "sources"); + to_json_key( + j, + "outputIds", + p.outputIds, + "TaskUpdateRequest", + "OutputBuffers", + "outputIds"); + to_json_key( + j, + "tableWriteInfo", + p.tableWriteInfo, + "TaskUpdateRequest", + "TableWriteInfo", + "tableWriteInfo"); } -void from_json(const json& j, OutputTableHandle& p) { +void from_json(const json& j, TaskUpdateRequest& p) { from_json_key( j, - "connectorId", - p.connectorId, - "OutputTableHandle", - "ConnectorId", - "connectorId"); + "session", + p.session, + "TaskUpdateRequest", + "SessionRepresentation", + "session"); from_json_key( j, - "transactionHandle", - p.transactionHandle, - "OutputTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "extraCredentials", + p.extraCredentials, + "TaskUpdateRequest", + "Map", + "extraCredentials"); + from_json_key( + j, "fragment", p.fragment, "TaskUpdateRequest", "String", "fragment"); from_json_key( j, - "connectorHandle", - p.connectorHandle, - "OutputTableHandle", - "ConnectorOutputTableHandle", - "connectorHandle"); + "sources", + p.sources, + "TaskUpdateRequest", + "List", + "sources"); + from_json_key( + j, + "outputIds", + p.outputIds, + "TaskUpdateRequest", + "OutputBuffers", + "outputIds"); + from_json_key( + j, + "tableWriteInfo", + p.tableWriteInfo, + "TaskUpdateRequest", + "TableWriteInfo", + "tableWriteInfo"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -CreateHandle::CreateHandle() noexcept { - _type = "CreateHandle"; +BuiltInFunctionHandle::BuiltInFunctionHandle() noexcept { + _type = "$static"; } -void to_json(json& j, const CreateHandle& p) { +void to_json(json& j, const BuiltInFunctionHandle& p) { j = json::object(); - j["@type"] = "CreateHandle"; - to_json_key( - j, "handle", p.handle, "CreateHandle", "OutputTableHandle", "handle"); + j["@type"] = "$static"; to_json_key( j, - "schemaTableName", - p.schemaTableName, - "CreateHandle", - "SchemaTableName", - "schemaTableName"); + "signature", + p.signature, + "BuiltInFunctionHandle", + "Signature", + "signature"); } -void from_json(const json& j, CreateHandle& p) { +void from_json(const json& j, BuiltInFunctionHandle& 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"); + "signature", + p.signature, + "BuiltInFunctionHandle", + "Signature", + "signature"); } } // 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 GroupingSetDescriptor& 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"); + "groupingKeys", + p.groupingKeys, + "GroupingSetDescriptor", + "List", + "groupingKeys"); to_json_key( j, - "fragmentVariable", - p.fragmentVariable, - "TableWriterNode", - "VariableReferenceExpression", - "fragmentVariable"); + "groupingSetCount", + p.groupingSetCount, + "GroupingSetDescriptor", + "int", + "groupingSetCount"); to_json_key( j, - "tableCommitContextVariable", - p.tableCommitContextVariable, - "TableWriterNode", - "VariableReferenceExpression", - "tableCommitContextVariable"); - to_json_key( + "globalGroupingSets", + p.globalGroupingSets, + "GroupingSetDescriptor", + "List", + "globalGroupingSets"); +} + +void from_json(const json& j, GroupingSetDescriptor& p) { + from_json_key( j, - "columns", - p.columns, - "TableWriterNode", + "groupingKeys", + p.groupingKeys, + "GroupingSetDescriptor", "List", - "columns"); - to_json_key( + "groupingKeys"); + from_json_key( j, - "columnNames", - p.columnNames, - "TableWriterNode", - "List", - "columnNames"); + "groupingSetCount", + p.groupingSetCount, + "GroupingSetDescriptor", + "int", + "groupingSetCount"); + from_json_key( + j, + "globalGroupingSets", + p.globalGroupingSets, + "GroupingSetDescriptor", + "List", + "globalGroupingSets"); +} +} // 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, - "notNullColumnVariables", - p.notNullColumnVariables, - "TableWriterNode", - "List", - "notNullColumnVariables"); + 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 { +TpchSplit::TpchSplit() noexcept { + _type = "tpch"; +} + +void to_json(json& j, const TpchSplit& p) { + j = json::object(); + j["@type"] = "tpch"; to_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "partitioningScheme"); + "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, - "preferredShufflePartitioningScheme", - p.preferredShufflePartitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "preferredShufflePartitioningScheme"); + "addresses", + p.addresses, + "TpchSplit", + "List", + "addresses"); to_json_key( j, - "statisticsAggregation", - p.statisticsAggregation, - "TableWriterNode", - "StatisticAggregations", - "statisticsAggregation"); + "predicate", + p.predicate, + "TpchSplit", + "TupleDomain>", + "predicate"); } -void from_json(const json& j, TableWriterNode& p) { +void from_json(const json& j, TpchSplit& 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"); + "tableHandle", + p.tableHandle, + "TpchSplit", + "TpchTableHandle", + "tableHandle"); from_json_key( - j, - "columns", - p.columns, - "TableWriterNode", - "List", - "columns"); + j, "partNumber", p.partNumber, "TpchSplit", "int", "partNumber"); from_json_key( - j, - "columnNames", - p.columnNames, - "TableWriterNode", - "List", - "columnNames"); + j, "totalParts", p.totalParts, "TpchSplit", "int", "totalParts"); from_json_key( j, - "notNullColumnVariables", - p.notNullColumnVariables, - "TableWriterNode", - "List", - "notNullColumnVariables"); + "addresses", + p.addresses, + "TpchSplit", + "List", + "addresses"); from_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "partitioningScheme"); - from_json_key( + "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 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, - "preferredShufflePartitioningScheme", - p.preferredShufflePartitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "preferredShufflePartitioningScheme"); + "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, - "statisticsAggregation", - p.statisticsAggregation, - "TableWriterNode", - "StatisticAggregations", - "statisticsAggregation"); + "orderingScheme", + p.orderingScheme, + "TopNNode", + "OrderingScheme", + "orderingScheme"); + from_json_key(j, "step", p.step, "TopNNode", "Step", "step"); } } // 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 ServerInfo& p) { +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, - "nodeVersion", - p.nodeVersion, - "ServerInfo", - "NodeVersion", - "nodeVersion"); + "partitionBy", + p.partitionBy, + "RowNumberNode", + "List", + "partitionBy"); to_json_key( - j, "environment", p.environment, "ServerInfo", "String", "environment"); + j, + "rowNumberVariable", + p.rowNumberVariable, + "RowNumberNode", + "VariableReferenceExpression", + "rowNumberVariable"); 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"); + j, + "maxRowCountPerPartition", + p.maxRowCountPerPartition, + "RowNumberNode", + "Integer", + "maxRowCountPerPartition"); + to_json_key(j, "partial", p.partial, "RowNumberNode", "bool", "partial"); + to_json_key( + j, + "hashVariable", + p.hashVariable, + "RowNumberNode", + "VariableReferenceExpression", + "hashVariable"); } -void from_json(const json& j, ServerInfo& p) { +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, - "nodeVersion", - p.nodeVersion, - "ServerInfo", - "NodeVersion", - "nodeVersion"); + "partitionBy", + p.partitionBy, + "RowNumberNode", + "List", + "partitionBy"); from_json_key( - j, "environment", p.environment, "ServerInfo", "String", "environment"); + j, + "rowNumberVariable", + p.rowNumberVariable, + "RowNumberNode", + "VariableReferenceExpression", + "rowNumberVariable"); 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"); + j, + "maxRowCountPerPartition", + p.maxRowCountPerPartition, + "RowNumberNode", + "Integer", + "maxRowCountPerPartition"); + from_json_key(j, "partial", p.partial, "RowNumberNode", "bool", "partial"); + from_json_key( + j, + "hashVariable", + p.hashVariable, + "RowNumberNode", + "VariableReferenceExpression", + "hashVariable"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -SortNode::SortNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.SortNode"; -} +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -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"); +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair SampleNodeType_enum_table[] = + { // NOLINT: cert-err58-cpp + {SampleNodeType::BERNOULLI, "BERNOULLI"}, + {SampleNodeType::SYSTEM, "SYSTEM"}}; +void to_json(json& j, const SampleNodeType& e) { + static_assert( + std::is_enum::value, "SampleNodeType must be an enum!"); + const auto* it = std::find_if( + std::begin(SampleNodeType_enum_table), + std::end(SampleNodeType_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(SampleNodeType_enum_table)) + ? it + : std::begin(SampleNodeType_enum_table)) + ->second; } - -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"); +void from_json(const json& j, SampleNodeType& e) { + static_assert( + std::is_enum::value, "SampleNodeType must be an enum!"); + const auto* it = std::find_if( + std::begin(SampleNodeType_enum_table), + std::end(SampleNodeType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(SampleNodeType_enum_table)) + ? it + : std::begin(SampleNodeType_enum_table)) + ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -EnforceSingleRowNode::EnforceSingleRowNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode"; +SampleNode::SampleNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.SampleNode"; } -void to_json(json& j, const EnforceSingleRowNode& p) { +void to_json(json& j, const SampleNode& 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"] = "com.facebook.presto.sql.planner.plan.SampleNode"; + to_json_key(j, "id", p.id, "SampleNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "SampleNode", "PlanNode", "source"); to_json_key( - j, "source", p.source, "EnforceSingleRowNode", "PlanNode", "source"); + j, "sampleRatio", p.sampleRatio, "SampleNode", "double", "sampleRatio"); + to_json_key( + j, + "sampleType", + p.sampleType, + "SampleNode", + "SampleNodeType", + "sampleType"); } -void from_json(const json& j, EnforceSingleRowNode& p) { +void from_json(const json& j, SampleNode& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "EnforceSingleRowNode", "PlanNodeId", "id"); + from_json_key(j, "id", p.id, "SampleNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "SampleNode", "PlanNode", "source"); from_json_key( - j, "source", p.source, "EnforceSingleRowNode", "PlanNode", "source"); + j, "sampleRatio", p.sampleRatio, "SampleNode", "double", "sampleRatio"); + from_json_key( + j, + "sampleType", + p.sampleType, + "SampleNode", + "SampleNodeType", + "sampleType"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TpchTableLayoutHandle::TpchTableLayoutHandle() noexcept { +TpchPartitioningHandle::TpchPartitioningHandle() noexcept { _type = "tpch"; } -void to_json(json& j, const TpchTableLayoutHandle& p) { +void to_json(json& j, const TpchPartitioningHandle& p) { j = json::object(); j["@type"] = "tpch"; - to_json_key( - j, "table", p.table, "TpchTableLayoutHandle", "TpchTableHandle", "table"); + to_json_key(j, "table", p.table, "TpchPartitioningHandle", "String", "table"); to_json_key( j, - "predicate", - p.predicate, - "TpchTableLayoutHandle", - "TupleDomain>", - "predicate"); + "totalRows", + p.totalRows, + "TpchPartitioningHandle", + "int64_t", + "totalRows"); } -void from_json(const json& j, TpchTableLayoutHandle& p) { +void from_json(const json& j, TpchPartitioningHandle& p) { p._type = j["@type"]; from_json_key( - j, "table", p.table, "TpchTableLayoutHandle", "TpchTableHandle", "table"); + j, "table", p.table, "TpchPartitioningHandle", "String", "table"); from_json_key( j, - "predicate", - p.predicate, - "TpchTableLayoutHandle", - "TupleDomain>", - "predicate"); -} -} // 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 { - -std::string JsonEncodedSubclass::getSubclassKey(nlohmann::json j) { - return j["@type"]; + "totalRows", + p.totalRows, + "TpchPartitioningHandle", + "int64_t", + "totalRows"); } - } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -EmptySplit::EmptySplit() noexcept { - _type = "$empty"; +SortNode::SortNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.SortNode"; } -void to_json(json& j, const EmptySplit& p) { +void to_json(json& j, const SortNode& p) { j = json::object(); - j["@type"] = "$empty"; + 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, - "connectorId", - p.connectorId, - "EmptySplit", - "ConnectorId", - "connectorId"); -} - -void from_json(const json& j, EmptySplit& p) { - p._type = j["@type"]; - from_json_key( - j, - "connectorId", - p.connectorId, - "EmptySplit", - "ConnectorId", - "connectorId"); + "orderingScheme", + p.orderingScheme, + "SortNode", + "OrderingScheme", + "orderingScheme"); + to_json_key(j, "isPartial", p.isPartial, "SortNode", "bool", "isPartial"); } -} // 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; +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 { @@ -11736,6 +12796,48 @@ void from_json(const json& j, SystemPartitioning& e) { } } // 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; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { SystemPartitioningHandle::SystemPartitioningHandle() noexcept { _type = "$remote"; } @@ -11778,1083 +12880,898 @@ void from_json(const json& j, SystemPartitioningHandle& p) { } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -AssignUniqueId::AssignUniqueId() noexcept { - _type = "com.facebook.presto.sql.planner.plan.AssignUniqueId"; -} +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -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"); +// 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, 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"); +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 { -HiveMetadataUpdateHandle::HiveMetadataUpdateHandle() noexcept { - _type = "hive"; -} -void to_json(json& j, const HiveMetadataUpdateHandle& p) { +void to_json(json& j, const RuntimeMetric& p) { j = json::object(); - j["@type"] = "hive"; - to_json_key( - j, - "requestId", - p.requestId, - "HiveMetadataUpdateHandle", - "UUID", - "requestId"); - to_json_key( - j, - "schemaTableName", - p.schemaTableName, - "HiveMetadataUpdateHandle", - "SchemaTableName", - "schemaTableName"); - to_json_key( - j, - "partitionName", - p.partitionName, - "HiveMetadataUpdateHandle", - "String", - "partitionName"); - to_json_key( - j, - "fileName", - p.fileName, - "HiveMetadataUpdateHandle", - "String", - "fileName"); + 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, 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"); - from_json_key( - j, - "partitionName", - p.partitionName, - "HiveMetadataUpdateHandle", - "String", - "partitionName"); - from_json_key( - j, - "fileName", - p.fileName, - "HiveMetadataUpdateHandle", - "String", - "fileName"); +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 { -GroupIdNode::GroupIdNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.GroupIdNode"; +CreateHandle::CreateHandle() noexcept { + _type = "CreateHandle"; } -void to_json(json& j, const GroupIdNode& p) { +void to_json(json& j, const CreateHandle& 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, - "groupingSets", - p.groupingSets, - "GroupIdNode", - "List>", - "groupingSets"); - to_json_key( - j, - "groupingColumns", - p.groupingColumns, - "GroupIdNode", - "Map", - "groupingColumns"); + j["@type"] = "CreateHandle"; to_json_key( - j, - "aggregationArguments", - p.aggregationArguments, - "GroupIdNode", - "List", - "aggregationArguments"); + j, "handle", p.handle, "CreateHandle", "OutputTableHandle", "handle"); to_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "GroupIdNode", - "VariableReferenceExpression", - "groupIdVariable"); + "schemaTableName", + p.schemaTableName, + "CreateHandle", + "SchemaTableName", + "schemaTableName"); } -void from_json(const json& j, GroupIdNode& p) { +void from_json(const json& j, CreateHandle& 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, - "groupingSets", - p.groupingSets, - "GroupIdNode", - "List>", - "groupingSets"); - from_json_key( - j, - "groupingColumns", - p.groupingColumns, - "GroupIdNode", - "Map", - "groupingColumns"); - from_json_key( - j, - "aggregationArguments", - p.aggregationArguments, - "GroupIdNode", - "List", - "aggregationArguments"); + from_json_key( + j, "handle", p.handle, "CreateHandle", "OutputTableHandle", "handle"); from_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "GroupIdNode", - "VariableReferenceExpression", - "groupIdVariable"); + "schemaTableName", + p.schemaTableName, + "CreateHandle", + "SchemaTableName", + "schemaTableName"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -SqlFunctionHandle::SqlFunctionHandle() noexcept { - _type = "json_file"; -} -void to_json(json& j, const SqlFunctionHandle& p) { +void to_json(json& j, const BatchTaskUpdateRequest& p) { j = json::object(); - j["@type"] = "json_file"; to_json_key( j, - "functionId", - p.functionId, - "SqlFunctionHandle", - "SqlFunctionId", - "functionId"); + "taskUpdateRequest", + p.taskUpdateRequest, + "BatchTaskUpdateRequest", + "TaskUpdateRequest", + "taskUpdateRequest"); to_json_key( - j, "version", p.version, "SqlFunctionHandle", "String", "version"); + j, + "shuffleWriteInfo", + p.shuffleWriteInfo, + "BatchTaskUpdateRequest", + "String", + "shuffleWriteInfo"); + to_json_key( + j, + "broadcastBasePath", + p.broadcastBasePath, + "BatchTaskUpdateRequest", + "String", + "broadcastBasePath"); } -void from_json(const json& j, SqlFunctionHandle& p) { - p._type = j["@type"]; +void from_json(const json& j, BatchTaskUpdateRequest& p) { from_json_key( j, - "functionId", - p.functionId, - "SqlFunctionHandle", - "SqlFunctionId", - "functionId"); + "taskUpdateRequest", + p.taskUpdateRequest, + "BatchTaskUpdateRequest", + "TaskUpdateRequest", + "taskUpdateRequest"); from_json_key( - j, "version", p.version, "SqlFunctionHandle", "String", "version"); + j, + "shuffleWriteInfo", + p.shuffleWriteInfo, + "BatchTaskUpdateRequest", + "String", + "shuffleWriteInfo"); + from_json_key( + j, + "broadcastBasePath", + p.broadcastBasePath, + "BatchTaskUpdateRequest", + "String", + "broadcastBasePath"); } } // 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 - JoinDistributionType_enum_table[] = { // NOLINT: cert-err58-cpp - {JoinDistributionType::PARTITIONED, "PARTITIONED"}, - {JoinDistributionType::REPLICATED, "REPLICATED"}}; -void to_json(json& j, const JoinDistributionType& e) { - static_assert( - std::is_enum::value, - "JoinDistributionType must be an enum!"); +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(JoinDistributionType_enum_table), - std::end(JoinDistributionType_enum_table), - [e](const std::pair& ej_pair) -> bool { + 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(JoinDistributionType_enum_table)) - ? it - : std::begin(JoinDistributionType_enum_table)) + j = ((it != std::end(Locality_enum_table)) ? it + : std::begin(Locality_enum_table)) ->second; } -void from_json(const json& j, JoinDistributionType& e) { - static_assert( - std::is_enum::value, - "JoinDistributionType must be an enum!"); +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(JoinDistributionType_enum_table), - std::end(JoinDistributionType_enum_table), - [&j](const std::pair& ej_pair) -> bool { + 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(JoinDistributionType_enum_table)) - ? it - : std::begin(JoinDistributionType_enum_table)) + e = ((it != std::end(Locality_enum_table)) ? it + : std::begin(Locality_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -JoinNode::JoinNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.JoinNode"; +ProjectNode::ProjectNode() noexcept { + _type = ".ProjectNode"; } -void to_json(json& j, const JoinNode& p) { +void to_json(json& j, const ProjectNode& 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", "JoinType", "type"); - to_json_key(j, "left", p.left, "JoinNode", "PlanNode", "left"); - to_json_key(j, "right", p.right, "JoinNode", "PlanNode", "right"); + 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, - "criteria", - p.criteria, - "JoinNode", - "List", - "criteria"); + "assignments", + p.assignments, + "ProjectNode", + "Assignments", + "assignments"); + to_json_key(j, "locality", p.locality, "ProjectNode", "Locality", "locality"); +} + +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, + "assignments", + p.assignments, + "ProjectNode", + "Assignments", + "assignments"); + from_json_key( + j, "locality", p.locality, "ProjectNode", "Locality", "locality"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +HiveSplit::HiveSplit() noexcept { + _type = "hive"; +} + +void to_json(json& j, const HiveSplit& 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, - "outputVariables", - p.outputVariables, - "JoinNode", - "List", - "outputVariables"); + "partitionName", + p.partitionName, + "HiveSplit", + "String", + "partitionName"); + to_json_key(j, "storage", p.storage, "HiveSplit", "Storage", "storage"); to_json_key( j, - "filter", - p.filter, - "JoinNode", - "std::shared_ptr", - "filter"); + "partitionKeys", + p.partitionKeys, + "HiveSplit", + "List", + "partitionKeys"); to_json_key( j, - "leftHashVariable", - p.leftHashVariable, - "JoinNode", - "VariableReferenceExpression", - "leftHashVariable"); + "addresses", + p.addresses, + "HiveSplit", + "List", + "addresses"); to_json_key( j, - "rightHashVariable", - p.rightHashVariable, - "JoinNode", - "VariableReferenceExpression", - "rightHashVariable"); + "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, - "distributionType", - p.distributionType, - "JoinNode", - "JoinDistributionType", - "distributionType"); + "encryptionMetadata", + p.encryptionMetadata, + "HiveSplit", + "EncryptionInformation", + "encryptionMetadata"); to_json_key( j, - "dynamicFilters", - p.dynamicFilters, - "JoinNode", - "Map", - "dynamicFilters"); + "redundantColumnDomains", + p.redundantColumnDomains, + "HiveSplit", + "List>", + "redundantColumnDomains"); + to_json_key( + j, + "splitWeight", + p.splitWeight, + "HiveSplit", + "SplitWeight", + "splitWeight"); } -void from_json(const json& j, JoinNode& p) { +void from_json(const json& j, HiveSplit& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "JoinNode", "PlanNodeId", "id"); - from_json_key(j, "type", p.type, "JoinNode", "JoinType", "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, "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, - "criteria", - p.criteria, - "JoinNode", - "List", - "criteria"); + "partitionName", + p.partitionName, + "HiveSplit", + "String", + "partitionName"); + from_json_key(j, "storage", p.storage, "HiveSplit", "Storage", "storage"); from_json_key( j, - "outputVariables", - p.outputVariables, - "JoinNode", - "List", - "outputVariables"); + "partitionKeys", + p.partitionKeys, + "HiveSplit", + "List", + "partitionKeys"); from_json_key( j, - "filter", - p.filter, - "JoinNode", - "std::shared_ptr", - "filter"); + "addresses", + p.addresses, + "HiveSplit", + "List", + "addresses"); from_json_key( j, - "leftHashVariable", - p.leftHashVariable, - "JoinNode", - "VariableReferenceExpression", - "leftHashVariable"); + "readBucketNumber", + p.readBucketNumber, + "HiveSplit", + "int", + "readBucketNumber"); from_json_key( j, - "rightHashVariable", - p.rightHashVariable, - "JoinNode", - "VariableReferenceExpression", - "rightHashVariable"); + "tableBucketNumber", + p.tableBucketNumber, + "HiveSplit", + "int", + "tableBucketNumber"); from_json_key( j, - "distributionType", - p.distributionType, - "JoinNode", - "JoinDistributionType", - "distributionType"); + "nodeSelectionStrategy", + p.nodeSelectionStrategy, + "HiveSplit", + "NodeSelectionStrategy", + "nodeSelectionStrategy"); from_json_key( j, - "dynamicFilters", - p.dynamicFilters, - "JoinNode", - "Map", - "dynamicFilters"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Specification& p) { - j = json::object(); - to_json_key( + "partitionDataColumnCount", + p.partitionDataColumnCount, + "HiveSplit", + "int", + "partitionDataColumnCount"); + from_json_key( j, - "partitionBy", - p.partitionBy, - "Specification", - "List", - "partitionBy"); - to_json_key( + "tableToPartitionMapping", + p.tableToPartitionMapping, + "HiveSplit", + "TableToPartitionMapping", + "tableToPartitionMapping"); + from_json_key( j, - "orderingScheme", - p.orderingScheme, - "Specification", - "OrderingScheme", - "orderingScheme"); -} - -void from_json(const json& j, Specification& p) { + "bucketConversion", + p.bucketConversion, + "HiveSplit", + "BucketConversion", + "bucketConversion"); from_json_key( j, - "partitionBy", - p.partitionBy, - "Specification", - "List", - "partitionBy"); + "s3SelectPushdownEnabled", + p.s3SelectPushdownEnabled, + "HiveSplit", + "bool", + "s3SelectPushdownEnabled"); from_json_key( j, - "orderingScheme", - p.orderingScheme, - "Specification", - "OrderingScheme", - "orderingScheme"); + "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 { -TopNRowNumberNode::TopNRowNumberNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.TopNRowNumberNode"; +RemoteTransactionHandle::RemoteTransactionHandle() noexcept { + _type = "$remote"; } -void to_json(json& j, const TopNRowNumberNode& p) { +void to_json(json& j, const RemoteTransactionHandle& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.TopNRowNumberNode"; - to_json_key(j, "id", p.id, "TopNRowNumberNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "TopNRowNumberNode", "PlanNode", "source"); - to_json_key( - j, - "specification", - p.specification, - "TopNRowNumberNode", - "Specification", - "specification"); - to_json_key( - j, - "rowNumberVariable", - p.rowNumberVariable, - "TopNRowNumberNode", - "VariableReferenceExpression", - "rowNumberVariable"); - to_json_key( - j, - "maxRowCountPerPartition", - p.maxRowCountPerPartition, - "TopNRowNumberNode", - "int", - "maxRowCountPerPartition"); - to_json_key(j, "partial", p.partial, "TopNRowNumberNode", "bool", "partial"); + j["@type"] = "$remote"; to_json_key( - j, - "hashVariable", - p.hashVariable, - "TopNRowNumberNode", - "VariableReferenceExpression", - "hashVariable"); + j, "dummy", p.dummy, "RemoteTransactionHandle", "String", "dummy"); } -void from_json(const json& j, TopNRowNumberNode& p) { +void from_json(const json& j, RemoteTransactionHandle& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "TopNRowNumberNode", "PlanNodeId", "id"); - from_json_key( - j, "source", p.source, "TopNRowNumberNode", "PlanNode", "source"); - from_json_key( - j, - "specification", - p.specification, - "TopNRowNumberNode", - "Specification", - "specification"); - from_json_key( - j, - "rowNumberVariable", - p.rowNumberVariable, - "TopNRowNumberNode", - "VariableReferenceExpression", - "rowNumberVariable"); - from_json_key( - j, - "maxRowCountPerPartition", - p.maxRowCountPerPartition, - "TopNRowNumberNode", - "int", - "maxRowCountPerPartition"); - from_json_key( - j, "partial", p.partial, "TopNRowNumberNode", "bool", "partial"); - from_json_key( - j, - "hashVariable", - p.hashVariable, - "TopNRowNumberNode", - "VariableReferenceExpression", - "hashVariable"); + from_json_key( + j, "dummy", p.dummy, "RemoteTransactionHandle", "String", "dummy"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +HiveMetadataUpdateHandle::HiveMetadataUpdateHandle() noexcept { + _type = "hive"; +} -void to_json(json& j, const RefreshMaterializedViewHandle& p) { +void to_json(json& j, const HiveMetadataUpdateHandle& p) { j = json::object(); + j["@type"] = "hive"; to_json_key( j, - "handle", - p.handle, - "RefreshMaterializedViewHandle", - "InsertTableHandle", - "handle"); + "requestId", + p.requestId, + "HiveMetadataUpdateHandle", + "UUID", + "requestId"); to_json_key( j, "schemaTableName", p.schemaTableName, - "RefreshMaterializedViewHandle", + "HiveMetadataUpdateHandle", "SchemaTableName", "schemaTableName"); + to_json_key( + j, + "partitionName", + p.partitionName, + "HiveMetadataUpdateHandle", + "String", + "partitionName"); + to_json_key( + j, + "fileName", + p.fileName, + "HiveMetadataUpdateHandle", + "String", + "fileName"); } -void from_json(const json& j, RefreshMaterializedViewHandle& p) { +void from_json(const json& j, HiveMetadataUpdateHandle& p) { + p._type = j["@type"]; from_json_key( j, - "handle", - p.handle, - "RefreshMaterializedViewHandle", - "InsertTableHandle", - "handle"); + "requestId", + p.requestId, + "HiveMetadataUpdateHandle", + "UUID", + "requestId"); from_json_key( j, "schemaTableName", p.schemaTableName, - "RefreshMaterializedViewHandle", + "HiveMetadataUpdateHandle", "SchemaTableName", "schemaTableName"); + from_json_key( + j, + "partitionName", + p.partitionName, + "HiveMetadataUpdateHandle", + "String", + "partitionName"); + from_json_key( + j, + "fileName", + p.fileName, + "HiveMetadataUpdateHandle", + "String", + "fileName"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -AllOrNoneValueSet::AllOrNoneValueSet() noexcept { - _type = "allOrNone"; -} - -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 { -HiveTransactionHandle::HiveTransactionHandle() noexcept { - _type = "hive"; -} - -void to_json(json& j, const HiveTransactionHandle& p) { - j = json::object(); - 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 -namespace facebook::presto::protocol { -ValuesNode::ValuesNode() noexcept { - _type = ".ValuesNode"; +LambdaDefinitionExpression::LambdaDefinitionExpression() noexcept { + _type = "lambda"; } -void to_json(json& j, const ValuesNode& p) { +void to_json(json& j, const LambdaDefinitionExpression& 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"); + j["@type"] = "lambda"; to_json_key( j, - "outputVariables", - p.outputVariables, - "ValuesNode", - "List", - "outputVariables"); + "sourceLocation", + p.sourceLocation, + "LambdaDefinitionExpression", + "SourceLocation", + "sourceLocation"); to_json_key( j, - "rows", - p.rows, - "ValuesNode", - "List>>", - "rows"); + "argumentTypes", + p.argumentTypes, + "LambdaDefinitionExpression", + "List", + "argumentTypes"); to_json_key( j, - "valuesNodeLabel", - p.valuesNodeLabel, - "ValuesNode", - "String", - "valuesNodeLabel"); + "arguments", + p.arguments, + "LambdaDefinitionExpression", + "List", + "arguments"); + to_json_key( + j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); } -void from_json(const json& j, ValuesNode& p) { +void from_json(const json& j, LambdaDefinitionExpression& 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"); + "sourceLocation", + p.sourceLocation, + "LambdaDefinitionExpression", + "SourceLocation", + "sourceLocation"); from_json_key( j, - "rows", - p.rows, - "ValuesNode", - "List>>", - "rows"); + "argumentTypes", + p.argumentTypes, + "LambdaDefinitionExpression", + "List", + "argumentTypes"); from_json_key( j, - "valuesNodeLabel", - p.valuesNodeLabel, - "ValuesNode", - "String", - "valuesNodeLabel"); + "arguments", + p.arguments, + "LambdaDefinitionExpression", + "List", + "arguments"); + from_json_key( + j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -WindowNode::WindowNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.WindowNode"; +// 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 { +SemiJoinNode::SemiJoinNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.SemiJoinNode"; } -void to_json(json& j, const WindowNode& p) { +void to_json(json& j, const SemiJoinNode& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.WindowNode"; + 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, - "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"); + "filteringSource", + p.filteringSource, + "SemiJoinNode", + "PlanNode", + "filteringSource"); to_json_key( j, - "specification", - p.specification, - "WindowNode", - "Specification", - "specification"); + "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, - "windowFunctions", - p.windowFunctions, - "WindowNode", - "Map", - "windowFunctions"); + "sourceHashVariable", + p.sourceHashVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "sourceHashVariable"); to_json_key( j, - "hashVariable", - p.hashVariable, - "WindowNode", + "filteringSourceHashVariable", + p.filteringSourceHashVariable, + "SemiJoinNode", "VariableReferenceExpression", - "hashVariable"); + "filteringSourceHashVariable"); to_json_key( j, - "prePartitionedInputs", - p.prePartitionedInputs, - "WindowNode", - "List", - "prePartitionedInputs"); + "distributionType", + p.distributionType, + "SemiJoinNode", + "DistributionType", + "distributionType"); to_json_key( j, - "preSortedOrderPrefix", - p.preSortedOrderPrefix, - "WindowNode", - "int", - "preSortedOrderPrefix"); + "dynamicFilters", + p.dynamicFilters, + "SemiJoinNode", + "Map", + "dynamicFilters"); } -void from_json(const json& j, WindowNode& p) { +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, - "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"); - from_json_key( - j, - "specification", - p.specification, - "WindowNode", - "Specification", - "specification"); - from_json_key( - j, - "windowFunctions", - p.windowFunctions, - "WindowNode", - "Map", - "windowFunctions"); + "filteringSource", + p.filteringSource, + "SemiJoinNode", + "PlanNode", + "filteringSource"); from_json_key( j, - "hashVariable", - p.hashVariable, - "WindowNode", + "sourceJoinVariable", + p.sourceJoinVariable, + "SemiJoinNode", "VariableReferenceExpression", - "hashVariable"); - from_json_key( - j, - "prePartitionedInputs", - p.prePartitionedInputs, - "WindowNode", - "List", - "prePartitionedInputs"); + "sourceJoinVariable"); from_json_key( j, - "preSortedOrderPrefix", - p.preSortedOrderPrefix, - "WindowNode", - "int", - "preSortedOrderPrefix"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -DistinctLimitNode::DistinctLimitNode() noexcept { - _type = ".DistinctLimitNode"; -} - -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, - "distinctVariables", - p.distinctVariables, - "DistinctLimitNode", - "List", - "distinctVariables"); - to_json_key( - j, - "hashVariable", - p.hashVariable, - "DistinctLimitNode", + "filteringSourceJoinVariable", + p.filteringSourceJoinVariable, + "SemiJoinNode", "VariableReferenceExpression", - "hashVariable"); - to_json_key( - j, - "timeoutMillis", - p.timeoutMillis, - "DistinctLimitNode", - "int", - "timeoutMillis"); -} - -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"); + "filteringSourceJoinVariable"); from_json_key( j, - "distinctVariables", - p.distinctVariables, - "DistinctLimitNode", - "List", - "distinctVariables"); + "semiJoinOutput", + p.semiJoinOutput, + "SemiJoinNode", + "VariableReferenceExpression", + "semiJoinOutput"); from_json_key( j, - "hashVariable", - p.hashVariable, - "DistinctLimitNode", + "sourceHashVariable", + p.sourceHashVariable, + "SemiJoinNode", "VariableReferenceExpression", - "hashVariable"); + "sourceHashVariable"); from_json_key( j, - "timeoutMillis", - p.timeoutMillis, - "DistinctLimitNode", - "int", - "timeoutMillis"); -} -} // 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) { + "filteringSourceHashVariable", + p.filteringSourceHashVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "filteringSourceHashVariable"); from_json_key( j, - "tableToPartitionColumns", - p.tableToPartitionColumns, - "TableToPartitionMapping", - "Map", - "tableToPartitionColumns"); + "distributionType", + p.distributionType, + "SemiJoinNode", + "DistributionType", + "distributionType"); from_json_key( j, - "partitionSchemaDifference", - p.partitionSchemaDifference, - "TableToPartitionMapping", - "Map", - "partitionSchemaDifference"); + "dynamicFilters", + p.dynamicFilters, + "SemiJoinNode", + "Map", + "dynamicFilters"); } } // 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 + 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, - "NodeSelectionStrategy must be an enum!"); + std::is_enum::value, + "AggregationNodeStep 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(AggregationNodeStep_enum_table), + std::end(AggregationNodeStep_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(AggregationNodeStep_enum_table)) ? it - : std::begin(NodeSelectionStrategy_enum_table)) + : std::begin(AggregationNodeStep_enum_table)) ->second; } -void from_json(const json& j, NodeSelectionStrategy& e) { +void from_json(const json& j, AggregationNodeStep& e) { static_assert( - std::is_enum::value, - "NodeSelectionStrategy must be an enum!"); + std::is_enum::value, + "AggregationNodeStep 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(AggregationNodeStep_enum_table), + std::end(AggregationNodeStep_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(AggregationNodeStep_enum_table)) ? it - : std::begin(NodeSelectionStrategy_enum_table)) + : std::begin(AggregationNodeStep_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveSplit::HiveSplit() noexcept { - _type = "hive"; +AggregationNode::AggregationNode() noexcept { + _type = ".AggregationNode"; } -void to_json(json& j, const HiveSplit& p) { +void to_json(json& j, const AggregationNode& 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"); - 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"); + 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, - "tableToPartitionMapping", - p.tableToPartitionMapping, - "HiveSplit", - "TableToPartitionMapping", - "tableToPartitionMapping"); + "aggregations", + p.aggregations, + "AggregationNode", + "Map", + "aggregations"); to_json_key( j, - "bucketConversion", - p.bucketConversion, - "HiveSplit", - "BucketConversion", - "bucketConversion"); + "groupingSets", + p.groupingSets, + "AggregationNode", + "GroupingSetDescriptor", + "groupingSets"); to_json_key( j, - "s3SelectPushdownEnabled", - p.s3SelectPushdownEnabled, - "HiveSplit", - "bool", - "s3SelectPushdownEnabled"); + "preGroupedVariables", + p.preGroupedVariables, + "AggregationNode", + "List", + "preGroupedVariables"); to_json_key( - j, - "cacheQuota", - p.cacheQuota, - "HiveSplit", - "CacheQuotaRequirement", - "cacheQuota"); + j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); to_json_key( j, - "encryptionMetadata", - p.encryptionMetadata, - "HiveSplit", - "EncryptionInformation", - "encryptionMetadata"); + "hashVariable", + p.hashVariable, + "AggregationNode", + "VariableReferenceExpression", + "hashVariable"); to_json_key( j, - "redundantColumnDomains", - p.redundantColumnDomains, - "HiveSplit", - "List>", - "redundantColumnDomains"); + "groupIdVariable", + p.groupIdVariable, + "AggregationNode", + "VariableReferenceExpression", + "groupIdVariable"); to_json_key( j, - "splitWeight", - p.splitWeight, - "HiveSplit", - "SplitWeight", - "splitWeight"); + "aggregationId", + p.aggregationId, + "AggregationNode", + "Integer", + "aggregationId"); } -void from_json(const json& j, HiveSplit& p) { +void from_json(const json& j, AggregationNode& 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, "id", p.id, "AggregationNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "AggregationNode", "PlanNode", "source"); from_json_key( j, - "tableToPartitionMapping", - p.tableToPartitionMapping, - "HiveSplit", - "TableToPartitionMapping", - "tableToPartitionMapping"); + "aggregations", + p.aggregations, + "AggregationNode", + "Map", + "aggregations"); from_json_key( j, - "bucketConversion", - p.bucketConversion, - "HiveSplit", - "BucketConversion", - "bucketConversion"); + "groupingSets", + p.groupingSets, + "AggregationNode", + "GroupingSetDescriptor", + "groupingSets"); from_json_key( j, - "s3SelectPushdownEnabled", - p.s3SelectPushdownEnabled, - "HiveSplit", - "bool", - "s3SelectPushdownEnabled"); + "preGroupedVariables", + p.preGroupedVariables, + "AggregationNode", + "List", + "preGroupedVariables"); from_json_key( - j, - "cacheQuota", - p.cacheQuota, - "HiveSplit", - "CacheQuotaRequirement", - "cacheQuota"); + j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); from_json_key( j, - "encryptionMetadata", - p.encryptionMetadata, - "HiveSplit", - "EncryptionInformation", - "encryptionMetadata"); + "hashVariable", + p.hashVariable, + "AggregationNode", + "VariableReferenceExpression", + "hashVariable"); from_json_key( j, - "redundantColumnDomains", - p.redundantColumnDomains, - "HiveSplit", - "List>", - "redundantColumnDomains"); + "groupIdVariable", + p.groupIdVariable, + "AggregationNode", + "VariableReferenceExpression", + "groupIdVariable"); from_json_key( j, - "splitWeight", - p.splitWeight, - "HiveSplit", - "SplitWeight", - "splitWeight"); + "aggregationId", + p.aggregationId, + "AggregationNode", + "Integer", + "aggregationId"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { 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 5f8cc5a6e6e7..00f7b197345d 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.h +++ b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.h @@ -257,9 +257,31 @@ 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); +struct ConnectorTableHandle : 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 {}; +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 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 ConnectorSplit : 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 { @@ -269,11 +291,6 @@ 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 PlanNode : public JsonEncodedSubclass { PlanNodeId id = {}; }; @@ -281,16 +298,19 @@ 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 FunctionHandle : 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 ConnectorMetadataUpdateHandle : 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 { @@ -302,91 +322,16 @@ 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 FunctionHandle : 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 ConnectorSplit : 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 ConnectorTableHandle : 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 {}; -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 { -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 RuntimeMetric { - String name = {}; - RuntimeUnit unit = {}; - int64_t sum = {}; - int64_t count = {}; - int64_t max = {}; - int64_t min = {}; -}; -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 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 HiveFileSplit& p); -void from_json(const json& j, HiveFileSplit& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct HivePartitionKey { - String name = {}; - std::shared_ptr value = {}; -}; -void to_json(json& j, const HivePartitionKey& p); -void from_json(const json& j, HivePartitionKey& 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); +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 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 /* * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -416,100 +361,23 @@ void from_json(const json& j, TpchTransactionHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct InsertTableHandle { +struct TableHandle { ConnectorId connectorId = {}; - std::shared_ptr transactionHandle = {}; - std::shared_ptr connectorHandle = {}; -}; -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 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 -namespace facebook::presto::protocol { -struct InsertHandle : public ExecutionWriterTarget { - InsertTableHandle handle = {}; - SchemaTableName schemaTableName = {}; - - InsertHandle() noexcept; -}; -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 SourceLocation { - int line = {}; - int column = {}; + std::shared_ptr connectorHandle = {}; + std::shared_ptr transaction = {}; + std::shared_ptr connectorTableLayout = {}; }; -void to_json(json& j, const SourceLocation& p); -void from_json(const json& j, SourceLocation& 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 PartitioningHandle { - std::shared_ptr connectorId = {}; +struct AnalyzeTableHandle { + ConnectorId connectorId = {}; std::shared_ptr transactionHandle = {}; - std::shared_ptr connectorHandle = {}; -}; -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 Partitioning { - PartitioningHandle handle = {}; - List> arguments = {}; -}; -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 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 { -struct PageBufferInfo { - int partition = {}; - int64_t bufferedPages = {}; - int64_t bufferedBytes = {}; - int64_t rowsAdded = {}; - int64_t pagesAdded = {}; -}; -void to_json(json& j, const PageBufferInfo& p); -void from_json(const json& j, PageBufferInfo& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct LambdaDefinitionExpression : public RowExpression { - List argumentTypes = {}; - List arguments = {}; - std::shared_ptr body = {}; - - LambdaDefinitionExpression() noexcept; + std::shared_ptr connectorHandle = {}; }; -void to_json(json& j, const LambdaDefinitionExpression& p); -void from_json(const json& j, LambdaDefinitionExpression& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class Locality { UNKNOWN, LOCAL, REMOTE }; -extern void to_json(json& j, const Locality& e); -extern void from_json(const json& j, Locality& e); +void to_json(json& j, const AnalyzeTableHandle& p); +void from_json(const json& j, AnalyzeTableHandle& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -526,84 +394,34 @@ extern void from_json(const json& j, Locality& e); */ 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); - } - - bool operator<(const VariableReferenceExpression& o) const { - if (name == o.name) { - return type < o.type; - } - - return name < o.name; - } -}; +std::ostream& operator<<(std::ostream& os, const DataSize& d); -void to_json(json& j, const VariableReferenceExpression& p); -void from_json(const json& j, VariableReferenceExpression& p); +void to_json(nlohmann::json& j, const DataSize& p); +void from_json(const nlohmann::json& j, DataSize& p); -std::string json_map_key( - const facebook::presto::protocol::VariableReferenceExpression& p); - -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct Assignments { - Map> assignments = - {}; -}; -void to_json(json& j, const Assignments& p); -void from_json(const json& j, Assignments& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct ProjectNode : public PlanNode { - std::shared_ptr source = {}; - Assignments assignments = {}; - Locality locality = {}; - - ProjectNode() noexcept; -}; -void to_json(json& j, const ProjectNode& p); -void from_json(const json& j, ProjectNode& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -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 -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 +/* + * 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 SortingColumn { - String columnName = {}; - Order order = {}; -}; -void to_json(json& j, const SortingColumn& p); -void from_json(const json& j, SortingColumn& p); +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 MetadataUpdates { - ConnectorId connectorId = {}; - List> metadataUpdates = {}; -}; -void to_json(json& j, const MetadataUpdates& p); -void from_json(const json& j, MetadataUpdates& 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"); @@ -618,151 +436,63 @@ void from_json(const json& j, MetadataUpdates& p); * See the License for the specific language governing permissions and * limitations under the License. */ -namespace facebook::presto::protocol { - -template -struct pointerDerefCompare { - bool operator()(const std::shared_ptr& a, const std::shared_ptr& b) - const { - return *a < *b; - } -}; -template -struct TupleDomain { - std::shared_ptr> domains; -}; +namespace facebook::presto::protocol { -template -struct TupleDomain> { - std::shared_ptr, Domain, pointerDerefCompare>> - domains; -}; +std::ostream& operator<<(std::ostream& os, const Duration& d); -template -struct ColumnDomain { - T column; - Domain domain; // dependency -}; +void to_json(json& j, const Duration& p); +void from_json(const json& j, Duration& 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; - } - } -}; - -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; - } - } -}; - -} // namespace nlohmann namespace facebook::presto::protocol { -struct TpchTableHandle : public ConnectorTableHandle { - String tableName = {}; - double scaleFactor = {}; - - TpchTableHandle() 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 = {}; + int64_t nullJoinBuildKeyCount = {}; + int64_t joinBuildKeyCount = {}; + int64_t nullJoinProbeKeyCount = {}; + int64_t joinProbeKeyCount = {}; }; -void to_json(json& j, const TpchTableHandle& p); -void from_json(const json& j, TpchTableHandle& p); +void to_json(json& j, const OperatorStats& p); +void from_json(const json& j, OperatorStats& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -777,124 +507,201 @@ void from_json(const json& j, TpchTableHandle& p); * See the License for the specific language governing permissions and * limitations under the License. */ - namespace facebook::presto::protocol { -using HostAddress = std::string; - -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct TpchSplit : public ConnectorSplit { - TpchTableHandle tableHandle = {}; - int partNumber = {}; - int totalParts = {}; - List addresses = {}; - TupleDomain> predicate = {}; +struct Lifespan { + bool isgroup = false; + long groupid = 0; - TpchSplit() noexcept; -}; -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 PlanCostEstimate { - double cpuCost = {}; - double maxMemory = {}; - double maxMemoryWhenOutputting = {}; - double networkCost = {}; + bool operator<(const Lifespan& o) const { + return groupid < o.groupid; + } }; -void to_json(json& j, const PlanCostEstimate& p); -void from_json(const json& j, PlanCostEstimate& p); + +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 JoinNodeStatsEstimate { - double nullJoinBuildKeyCount = {}; - double joinBuildKeyCount = {}; - double nullJoinProbeKeyCount = {}; - double joinProbeKeyCount = {}; +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 JoinNodeStatsEstimate& p); -void from_json(const json& j, JoinNodeStatsEstimate& 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 VariableStatsEstimate { - double lowValue = {}; - double highValue = {}; - double nullsFraction = {}; - double averageRowSize = {}; - double distinctValuesCount = {}; +struct DistributionSnapshot { + double maxError = {}; + double count = {}; + double total = {}; + int64_t p01 = {}; + int64_t p05 = {}; + int64_t p10 = {}; + int64_t p25 = {}; + int64_t p50 = {}; + int64_t p75 = {}; + int64_t p90 = {}; + int64_t p95 = {}; + int64_t p99 = {}; + int64_t min = {}; + int64_t max = {}; + double avg = {}; }; -void to_json(json& j, const VariableStatsEstimate& p); -void from_json(const json& j, VariableStatsEstimate& p); +void to_json(json& j, const DistributionSnapshot& p); +void from_json(const json& j, DistributionSnapshot& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct PartialAggregationStatsEstimate { - double inputBytes = {}; - double outputBytes = {}; - double inputRowCount = {}; - double outputRowCount = {}; +struct PipelineStats { + int pipelineId = {}; + DateTime firstStartTime = {}; + DateTime lastStartTime = {}; + DateTime lastEndTime = {}; + bool inputPipeline = {}; + bool outputPipeline = {}; + int totalDrivers = {}; + int queuedDrivers = {}; + int queuedPartitionedDrivers = {}; + int64_t queuedPartitionedSplitsWeight = {}; + int runningDrivers = {}; + int runningPartitionedDrivers = {}; + int64_t runningPartitionedSplitsWeight = {}; + int blockedDrivers = {}; + int completedDrivers = {}; + int64_t userMemoryReservationInBytes = {}; + int64_t revocableMemoryReservationInBytes = {}; + int64_t systemMemoryReservationInBytes = {}; + DistributionSnapshot queuedTime = {}; + DistributionSnapshot elapsedTime = {}; + int64_t totalScheduledTimeInNanos = {}; + int64_t totalCpuTimeInNanos = {}; + int64_t totalBlockedTimeInNanos = {}; + bool fullyBlocked = {}; + List blockedReasons = {}; + int64_t totalAllocationInBytes = {}; + int64_t rawInputDataSizeInBytes = {}; + int64_t rawInputPositions = {}; + int64_t processedInputDataSizeInBytes = {}; + int64_t processedInputPositions = {}; + int64_t outputDataSizeInBytes = {}; + int64_t outputPositions = {}; + int64_t physicalWrittenDataSizeInBytes = {}; + List operatorSummaries = {}; + List drivers = {}; }; -void to_json(json& j, const PartialAggregationStatsEstimate& p); -void from_json(const json& j, PartialAggregationStatsEstimate& p); +void to_json(json& j, const PipelineStats& p); +void from_json(const json& j, PipelineStats& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TableWriterNodeStatsEstimate { - double taskCountIfScaledWriter = {}; +struct TaskStats { + DateTime createTime = {}; + DateTime firstStartTime = {}; + DateTime lastStartTime = {}; + DateTime lastEndTime = {}; + DateTime endTime = {}; + int64_t elapsedTimeInNanos = {}; + int64_t queuedTimeInNanos = {}; + int totalDrivers = {}; + int queuedDrivers = {}; + int queuedPartitionedDrivers = {}; + int64_t queuedPartitionedSplitsWeight = {}; + int runningDrivers = {}; + int runningPartitionedDrivers = {}; + int64_t runningPartitionedSplitsWeight = {}; + int blockedDrivers = {}; + int completedDrivers = {}; + double cumulativeUserMemory = {}; + double cumulativeTotalMemory = {}; + int64_t userMemoryReservationInBytes = {}; + int64_t revocableMemoryReservationInBytes = {}; + int64_t systemMemoryReservationInBytes = {}; + int64_t peakTotalMemoryInBytes = {}; + int64_t peakUserMemoryInBytes = {}; + int64_t peakNodeTotalMemoryInBytes = {}; + int64_t totalScheduledTimeInNanos = {}; + int64_t totalCpuTimeInNanos = {}; + int64_t totalBlockedTimeInNanos = {}; + bool fullyBlocked = {}; + List blockedReasons = {}; + int64_t totalAllocationInBytes = {}; + int64_t rawInputDataSizeInBytes = {}; + int64_t rawInputPositions = {}; + int64_t processedInputDataSizeInBytes = {}; + int64_t processedInputPositions = {}; + int64_t outputDataSizeInBytes = {}; + int64_t outputPositions = {}; + int64_t physicalWrittenDataSizeInBytes = {}; + int fullGcCount = {}; + int64_t fullGcTimeInMillis = {}; + List pipelines = {}; + RuntimeStats runtimeStats = {}; }; -void to_json(json& j, const TableWriterNodeStatsEstimate& p); -void from_json(const json& j, TableWriterNodeStatsEstimate& p); +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 PlanNodeStatsEstimate { - double outputRowCount = {}; - double totalSize = {}; - bool confident = {}; - Map variableStatistics = - {}; - JoinNodeStatsEstimate joinNodeStatsEstimate = {}; - TableWriterNodeStatsEstimate tableWriterNodeStatsEstimate = {}; - PartialAggregationStatsEstimate partialAggregationStatsEstimate = {}; -}; -void to_json(json& j, const PlanNodeStatsEstimate& p); -void from_json(const json& j, PlanNodeStatsEstimate& p); +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 StatsAndCosts { - Map stats = {}; - Map costs = {}; +enum class ErrorCause { + UNKNOWN, + LOW_PARTITION_COUNT, + EXCEEDS_BROADCAST_MEMORY_LIMIT }; -void to_json(json& j, const StatsAndCosts& p); -void from_json(const json& j, StatsAndCosts& p); +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 StageExecutionStrategy { - UNGROUPED_EXECUTION, - FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, - DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, - RECOVERABLE_GROUPED_EXECUTION +struct ErrorLocation { + int lineNumber = {}; + int columnNumber = {}; }; -extern void to_json(json& j, const StageExecutionStrategy& e); -extern void from_json(const json& j, StageExecutionStrategy& 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 StageExecutionDescriptor { - StageExecutionStrategy stageExecutionStrategy = {}; - List groupedExecutionScanNodes = {}; - int totalLifespans = {}; +enum class ErrorType { + USER_ERROR, + INTERNAL_ERROR, + INSUFFICIENT_RESOURCES, + EXTERNAL }; -void to_json(json& j, const StageExecutionDescriptor& p); -void from_json(const json& j, StageExecutionDescriptor& p); +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 PartitioningScheme { - Partitioning partitioning = {}; - List outputLayout = {}; - std::shared_ptr hashColumn = {}; - bool replicateNullsAndAny = {}; - std::shared_ptr> bucketToPartition = {}; +struct ErrorCode { + int code = {}; + String name = {}; + ErrorType type = {}; + bool retriable = {}; }; -void to_json(json& j, const PartitioningScheme& p); -void from_json(const json& j, PartitioningScheme& p); +void to_json(json& j, const ErrorCode& p); +void from_json(const json& j, ErrorCode& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -911,125 +718,205 @@ void from_json(const json& j, PartitioningScheme& p); */ 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 = {}; + +using HostAddress = std::string; + +} // 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 PlanFragment& p); -void from_json(const json& j, PlanFragment& p); +void to_json(json& j, const ExecutionFailureInfo& p); +void from_json(const json& j, ExecutionFailureInfo& 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 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 = {}; }; -extern void to_json(json& j, const SortOrder& e); -extern void from_json(const json& j, SortOrder& e); +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 Ordering { - VariableReferenceExpression variable = {}; - SortOrder sortOrder = {}; +struct MetadataUpdates { + ConnectorId connectorId = {}; + List> metadataUpdates = {}; }; -void to_json(json& j, const Ordering& p); -void from_json(const json& j, Ordering& 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 OrderingScheme { - List orderBy = {}; +struct PageBufferInfo { + int partition = {}; + int64_t bufferedPages = {}; + int64_t bufferedBytes = {}; + int64_t rowsAdded = {}; + int64_t pagesAdded = {}; }; -void to_json(json& j, const OrderingScheme& p); -void from_json(const json& j, OrderingScheme& 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 { -struct CallExpression : public RowExpression { - String displayName = {}; - std::shared_ptr functionHandle = {}; - Type returnType = {}; - List> arguments = {}; - - CallExpression() noexcept; +struct BufferInfo { + OutputBufferId bufferId = {}; + bool finished = {}; + int bufferedPages = {}; + int64_t pagesSent = {}; + PageBufferInfo pageBufferInfo = {}; }; -void to_json(json& j, const CallExpression& p); -void from_json(const json& j, CallExpression& p); +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 Aggregation { - CallExpression call = {}; - std::shared_ptr> filter = {}; - std::shared_ptr orderBy = {}; - bool distinct = {}; - std::shared_ptr mask = {}; - std::shared_ptr functionHandle = {}; - List> arguments = {}; +enum class BufferState { + OPEN, + NO_MORE_BUFFERS, + NO_MORE_PAGES, + FLUSHING, + FINISHED, + FAILED }; -void to_json(json& j, const Aggregation& p); -void from_json(const json& j, Aggregation& 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 GroupingSetDescriptor { - List groupingKeys = {}; - int groupingSetCount = {}; - List globalGroupingSets = {}; +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 GroupingSetDescriptor& p); -void from_json(const json& j, GroupingSetDescriptor& 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 { -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); +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 TaskInfo& p); +void from_json(const json& j, TaskInfo& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct AggregationNode : public PlanNode { - std::shared_ptr source = {}; - Map aggregations = {}; - GroupingSetDescriptor groupingSets = {}; - List preGroupedVariables = {}; - AggregationNodeStep step = {}; - std::shared_ptr hashVariable = {}; - std::shared_ptr groupIdVariable = {}; - std::shared_ptr aggregationId = {}; +struct OutputTableHandle { + ConnectorId connectorId = {}; + std::shared_ptr transactionHandle = {}; + std::shared_ptr connectorHandle = {}; +}; +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 AllOrNoneValueSet : public ValueSet { + Type type = {}; + bool all = {}; - AggregationNode() noexcept; + AllOrNoneValueSet() noexcept; }; -void to_json(json& j, const AggregationNode& p); -void from_json(const json& j, AggregationNode& 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 { -enum class BlockedReason { WAITING_FOR_MEMORY }; -extern void to_json(json& j, const BlockedReason& e); -extern void from_json(const json& j, BlockedReason& e); +struct VariableStatsEstimate { + double lowValue = {}; + double highValue = {}; + double nullsFraction = {}; + double averageRowSize = {}; + double distinctValuesCount = {}; +}; +void to_json(json& j, const VariableStatsEstimate& p); +void from_json(const json& j, VariableStatsEstimate& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct DistributionSnapshot { - double maxError = {}; - double count = {}; - double total = {}; - int64_t p01 = {}; - int64_t p05 = {}; - int64_t p10 = {}; - int64_t p25 = {}; - int64_t p50 = {}; - int64_t p75 = {}; - int64_t p90 = {}; - int64_t p95 = {}; - int64_t p99 = {}; - int64_t min = {}; - int64_t max = {}; - double avg = {}; +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 DistributionSnapshot& p); -void from_json(const json& j, DistributionSnapshot& 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 { +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 SelectedRole { + SelectedRoleType type = {}; + std::shared_ptr role = {}; +}; +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 SplitContext { + bool cacheable = {}; +}; +void to_json(json& j, const SplitContext& p); +void from_json(const json& j, SplitContext& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct Split { + ConnectorId connectorId = {}; + std::shared_ptr transactionHandle = {}; + std::shared_ptr connectorSplit = {}; + Lifespan lifespan = {}; + SplitContext splitContext = {}; +}; +void to_json(json& j, const Split& p); +void from_json(const json& j, Split& 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"); @@ -1046,40 +933,180 @@ void from_json(const json& j, DistributionSnapshot& p); */ namespace facebook::presto::protocol { -struct Lifespan { - bool isgroup = false; - long groupid = 0; - - bool operator<(const Lifespan& o) const { - return groupid < o.groupid; +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); + } + + bool operator<(const VariableReferenceExpression& o) const { + if (name == o.name) { + return type < o.type; + } + + return name < o.name; } }; -void to_json(json& j, const Lifespan& p); -void from_json(const json& j, Lifespan& p); +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 -/* - * 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 Assignments { + Map> assignments = + {}; +}; +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 SortOrder { + ASC_NULLS_FIRST, + ASC_NULLS_LAST, + DESC_NULLS_FIRST, + DESC_NULLS_LAST +}; +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 Ordering { + VariableReferenceExpression variable = {}; + SortOrder sortOrder = {}; +}; +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 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 { +struct CallExpression : public RowExpression { + String displayName = {}; + std::shared_ptr functionHandle = {}; + Type returnType = {}; + List> arguments = {}; + CallExpression() noexcept; +}; +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 Aggregation { + CallExpression call = {}; + std::shared_ptr> filter = {}; + std::shared_ptr orderBy = {}; + bool distinct = {}; + std::shared_ptr mask = {}; + std::shared_ptr functionHandle = {}; + List> arguments = {}; +}; +void to_json(json& j, const Aggregation& p); +void from_json(const json& j, Aggregation& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct StatisticAggregations { + List outputVariables = {}; + Map aggregations = {}; + List groupingVariables = {}; +}; +void to_json(json& j, const StatisticAggregations& p); +void from_json(const json& j, StatisticAggregations& p); +} // namespace facebook::presto::protocol namespace facebook::presto::protocol { +struct TableWriterMergeNode : public PlanNode { + std::shared_ptr source = {}; + VariableReferenceExpression rowCountVariable = {}; + VariableReferenceExpression fragmentVariable = {}; + VariableReferenceExpression tableCommitContextVariable = {}; + std::shared_ptr statisticsAggregation = {}; -std::ostream& operator<<(std::ostream& os, const Duration& d); + TableWriterMergeNode() noexcept; +}; +void to_json(json& j, const TableWriterMergeNode& p); +void from_json(const json& j, TableWriterMergeNode& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct NodeVersion { + String version = {}; +}; +void to_json(json& j, const NodeVersion& p); +void from_json(const json& j, NodeVersion& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +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 { +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 Form& e); +extern void from_json(const json& j, Form& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct SpecialFormExpression : public RowExpression { + Form form = {}; + Type returnType = {}; + List> arguments = {}; -void to_json(json& j, const Duration& p); -void from_json(const json& j, Duration& p); + SpecialFormExpression() noexcept; +}; +void to_json(json& j, const SpecialFormExpression& p); +void from_json(const json& j, SpecialFormExpression& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct MarkDistinctNode : public PlanNode { + std::shared_ptr source = {}; + VariableReferenceExpression markerVariable = {}; + List distinctVariables = {}; + std::shared_ptr hashVariable = {}; + MarkDistinctNode() noexcept; +}; +void to_json(json& j, const MarkDistinctNode& p); +void from_json(const json& j, MarkDistinctNode& 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"); @@ -1096,11 +1123,28 @@ void from_json(const json& j, Duration& p); */ namespace facebook::presto::protocol { -std::ostream& operator<<(std::ostream& os, const DataSize& d); +struct Block { + std::string data; +}; -void to_json(nlohmann::json& j, const DataSize& p); -void from_json(const nlohmann::json& j, DataSize& p); +void to_json(json& j, const Block& p); + +void from_json(const json& j, Block& p); +} // 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 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); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -1116,229 +1160,150 @@ void from_json(const nlohmann::json& j, DataSize& 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 { -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 = {}; - int64_t nullJoinBuildKeyCount = {}; - int64_t joinBuildKeyCount = {}; - int64_t nullJoinProbeKeyCount = {}; - int64_t joinProbeKeyCount = {}; +struct LongVariableConstraint { + String name = {}; + String expression = {}; }; -void to_json(json& j, const OperatorStats& p); -void from_json(const json& j, OperatorStats& 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 { -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 = {}; +struct Signature { + QualifiedObjectName name = {}; + FunctionKind kind = {}; + List typeVariableConstraints = {}; + List longVariableConstraints = {}; + TypeSignature returnType = {}; + List argumentTypes = {}; + bool variableArity = {}; }; -void to_json(json& j, const DriverStats& p); -void from_json(const json& j, DriverStats& p); +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 PipelineStats { - int pipelineId = {}; - DateTime firstStartTime = {}; - DateTime lastStartTime = {}; - DateTime lastEndTime = {}; - bool inputPipeline = {}; - bool outputPipeline = {}; - int totalDrivers = {}; - int queuedDrivers = {}; - int queuedPartitionedDrivers = {}; - int64_t queuedPartitionedSplitsWeight = {}; - int runningDrivers = {}; - int runningPartitionedDrivers = {}; - int64_t runningPartitionedSplitsWeight = {}; - int blockedDrivers = {}; - int completedDrivers = {}; - int64_t userMemoryReservationInBytes = {}; - int64_t revocableMemoryReservationInBytes = {}; - int64_t systemMemoryReservationInBytes = {}; - DistributionSnapshot queuedTime = {}; - DistributionSnapshot elapsedTime = {}; - int64_t totalScheduledTimeInNanos = {}; - int64_t totalCpuTimeInNanos = {}; - int64_t totalBlockedTimeInNanos = {}; - bool fullyBlocked = {}; - List blockedReasons = {}; - int64_t totalAllocationInBytes = {}; - int64_t rawInputDataSizeInBytes = {}; - int64_t rawInputPositions = {}; - int64_t processedInputDataSizeInBytes = {}; - int64_t processedInputPositions = {}; - int64_t outputDataSizeInBytes = {}; - int64_t outputPositions = {}; - int64_t physicalWrittenDataSizeInBytes = {}; - List operatorSummaries = {}; - List drivers = {}; +struct Parameter { + String name = {}; + TypeSignature type = {}; }; -void to_json(json& j, const PipelineStats& p); -void from_json(const json& j, PipelineStats& p); +void to_json(json& j, const Parameter& p); +void from_json(const json& j, Parameter& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct StatisticAggregations { - List outputVariables = {}; - Map aggregations = {}; - List groupingVariables = {}; +enum class Determinism { + DETERMINISTIC, + NOT_DETERMINISTIC, }; -void to_json(json& j, const StatisticAggregations& p); -void from_json(const json& j, StatisticAggregations& 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 EquiJoinClause { - VariableReferenceExpression left = {}; - VariableReferenceExpression right = {}; +struct Language { + String language = {}; }; -void to_json(json& j, const EquiJoinClause& p); -void from_json(const json& j, EquiJoinClause& 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 { -enum class DistributionType { PARTITIONED, REPLICATED }; -extern void to_json(json& j, const DistributionType& e); -extern void from_json(const json& j, DistributionType& e); +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 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; +struct RoutineCharacteristics { + std::shared_ptr language = {}; + std::shared_ptr determinism = {}; + std::shared_ptr nullCallClause = {}; }; -void to_json(json& j, const SemiJoinNode& p); -void from_json(const json& j, SemiJoinNode& p); +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 CacheQuotaScope { GLOBAL, SCHEMA, TABLE, PARTITION }; -extern void to_json(json& j, const CacheQuotaScope& e); -extern void from_json(const json& j, CacheQuotaScope& e); +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 CacheQuotaRequirement { - CacheQuotaScope cacheQuotaScope = {}; - std::shared_ptr quota = {}; +struct ResourceEstimates { + std::shared_ptr executionTime = {}; + std::shared_ptr cpuTime = {}; + std::shared_ptr peakMemory = {}; + std::shared_ptr peakTaskMemory = {}; }; -void to_json(json& j, const CacheQuotaRequirement& p); -void from_json(const json& j, CacheQuotaRequirement& p); +void to_json(json& j, const ResourceEstimates& p); +void from_json(const json& j, ResourceEstimates& 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 Column { - String name; - String type; - - Column() = default; - explicit Column(const String& str) { - name = str; - } +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 Column& p); -void from_json(const json& j, Column& 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 SplitContext { - bool cacheable = {}; +enum class TypeCategory { PRIMITIVE, STRUCT, ARRAY, MAP }; +extern void to_json(json& j, const TypeCategory& e); +extern void from_json(const json& j, TypeCategory& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct ColumnIdentity { + int id = {}; + String name = {}; + TypeCategory typeCategory = {}; + List children = {}; }; -void to_json(json& j, const SplitContext& p); -void from_json(const json& j, SplitContext& p); +void to_json(json& j, const ColumnIdentity& p); +void from_json(const json& j, ColumnIdentity& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Split { - ConnectorId connectorId = {}; - std::shared_ptr transactionHandle = {}; - std::shared_ptr connectorSplit = {}; - Lifespan lifespan = {}; - SplitContext splitContext = {}; +struct Domain { + std::shared_ptr values = {}; + bool nullAllowed = {}; }; -void to_json(json& j, const Split& p); -void from_json(const json& j, Split& p); +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"); @@ -1355,34 +1320,189 @@ void from_json(const json& j, Split& p); */ namespace facebook::presto::protocol { -struct ScheduledSplit { - long sequenceId = {}; - PlanNodeId planNodeId = {}; // dependency - Split split = {}; - - bool operator<(const ScheduledSplit& o) const { - return sequenceId < o.sequenceId; +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 ScheduledSplit& p); -void from_json(const json& j, ScheduledSplit& p); - -} // namespace facebook::presto::protocol +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 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; + } + } +}; + +} // namespace nlohmann namespace facebook::presto::protocol { -struct TaskSource { - PlanNodeId planNodeId = {}; - List splits = {}; - List noMoreSplitsForLifespan = {}; - bool noMoreSplits = {}; +struct Location { + String location = {}; }; -void to_json(json& j, const TaskSource& p); -void from_json(const json& j, TaskSource& p); +void to_json(json& j, const Location& p); +void from_json(const json& j, Location& 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); +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 = {}; + + 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 InsertTableHandle { + ConnectorId connectorId = {}; + std::shared_ptr transactionHandle = {}; + std::shared_ptr connectorHandle = {}; +}; +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 ValuesNode : public PlanNode { + std::shared_ptr location = {}; + + List outputVariables = {}; + List>> rows = {}; + std::shared_ptr valuesNodeLabel = {}; + + ValuesNode() noexcept; +}; +void to_json(json& j, const ValuesNode& p); +void from_json(const json& j, ValuesNode& 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 /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -1397,39 +1517,68 @@ extern void from_json(const json& j, Bound& e); * See the License for the specific language governing permissions and * limitations under the License. */ + +// HiveColumnHandle is special since we require an implementation of +// operator<(). + namespace facebook::presto::protocol { -struct Block { - std::string data; -}; +struct HiveColumnHandle : public ColumnHandle { + String name = {}; + HiveType hiveType = {}; + TypeSignature typeSignature = {}; + int hiveColumnIndex = {}; + ColumnType columnType = {}; + std::shared_ptr comment = {}; + List requiredSubfields = {}; + std::shared_ptr partialAggregation = {}; -void to_json(json& j, const Block& p); + HiveColumnHandle() noexcept; -void from_json(const json& j, Block& p); + bool operator<(const ColumnHandle& o) const override { + return name < dynamic_cast(o).name; + } +}; + +void to_json(json& j, const HiveColumnHandle& p); +void from_json(const json& j, HiveColumnHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Marker { - Type type = {}; - std::shared_ptr valueBlock = {}; - Bound bound = {}; +struct BucketConversion { + int tableBucketCount = {}; + int partitionBucketCount = {}; + List bucketColumnHandles = {}; }; -void to_json(json& j, const Marker& p); -void from_json(const json& j, Marker& 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 Range { - Marker low = {}; - Marker high = {}; +struct UnnestNode : public PlanNode { + std::shared_ptr source = {}; + List replicateVariables = {}; + Map> + unnestVariables = {}; + std::shared_ptr ordinalityVariable = {}; + + UnnestNode() noexcept; }; -void to_json(json& j, const Range& p); -void from_json(const json& j, Range& 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 { enum class JoinType { INNER, LEFT, RIGHT, FULL }; extern void to_json(json& j, const JoinType& e); extern void from_json(const json& j, JoinType& e); } // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct EquiJoinClause { + VariableReferenceExpression left = {}; + VariableReferenceExpression right = {}; +}; +void to_json(json& j, const EquiJoinClause& p); +void from_json(const json& j, EquiJoinClause& 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. @@ -1459,382 +1608,532 @@ struct MergeJoinNode : public PlanNode { void to_json(json& j, const MergeJoinNode& p); void from_json(const json& j, MergeJoinNode& 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 AssignUniqueId : public PlanNode { + std::shared_ptr source = {}; + VariableReferenceExpression idVariable = {}; -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); - + AssignUniqueId() noexcept; +}; +void to_json(json& j, const AssignUniqueId& p); +void from_json(const json& j, AssignUniqueId& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct NodeVersion { - String version = {}; +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 NodeVersion& p); -void from_json(const json& j, NodeVersion& p); +void to_json(json& j, const CacheQuotaRequirement& p); +void from_json(const json& j, CacheQuotaRequirement& 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 MemoryAllocation { - String tag = {}; - int64_t allocation = {}; + +class ValueEntry { + public: + Type type; + std::shared_ptr block; }; -void to_json(json& j, const MemoryAllocation& p); -void from_json(const json& j, MemoryAllocation& p); + +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 MemoryPoolInfo { - int64_t maxBytes = {}; - int64_t reservedBytes = {}; - int64_t reservedRevocableBytes = {}; - Map queryMemoryReservations = {}; - Map> queryMemoryAllocations = {}; - Map queryMemoryRevocableReservations = {}; +struct EquatableValueSet : public ValueSet { + Type type = {}; + bool whiteList = {}; + List entries = {}; + + EquatableValueSet() noexcept; }; -void to_json(json& j, const MemoryPoolInfo& p); -void from_json(const json& j, MemoryPoolInfo& p); +void to_json(json& j, const EquatableValueSet& p); +void from_json(const json& j, EquatableValueSet& 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 MemoryInfo { - DataSize totalNodeMemory = {}; - Map pools = {}; + +struct Column { + String name; + String type; + + Column() = default; + explicit Column(const String& str) { + name = str; + } }; -void to_json(json& j, const MemoryInfo& p); -void from_json(const json& j, MemoryInfo& 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 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 SchemaTableName { + String schema = {}; + String table = {}; }; -void to_json(json& j, const NodeStatus& p); -void from_json(const json& j, NodeStatus& 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 { -enum class ErrorType { - USER_ERROR, - INTERNAL_ERROR, - INSUFFICIENT_RESOURCES, - EXTERNAL +struct HiveBucketFilter { + List bucketsToKeep = {}; }; -extern void to_json(json& j, const ErrorType& e); -extern void from_json(const json& j, ErrorType& e); +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 ErrorCode { - int code = {}; - String name = {}; - ErrorType type = {}; - bool retriable = {}; +struct HiveBucketHandle { + List columns = {}; + int tableBucketCount = {}; + int readBucketCount = {}; }; -void to_json(json& j, const ErrorCode& p); -void from_json(const json& j, ErrorCode& 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 { -enum class ErrorCause { - UNKNOWN, - LOW_PARTITION_COUNT, - EXCEEDS_BROADCAST_MEMORY_LIMIT +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 = {}; + bool footerStatsUnreliable = {}; + + HiveTableLayoutHandle() noexcept; }; -extern void to_json(json& j, const ErrorCause& e); -extern void from_json(const json& j, ErrorCause& e); +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 ErrorLocation { - int lineNumber = {}; - int columnNumber = {}; +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 ErrorLocation& p); -void from_json(const json& j, ErrorLocation& 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 { -struct ExecutionFailureInfo { - String type = {}; - String message = {}; - std::shared_ptr cause = {}; - List suppressed = {}; - List stack = {}; - ErrorLocation errorLocation = {}; - ErrorCode errorCode = {}; - HostAddress remoteHost = {}; - ErrorCause errorCause = {}; +struct RefreshMaterializedViewHandle { + InsertTableHandle handle = {}; + SchemaTableName schemaTableName = {}; }; -void to_json(json& j, const ExecutionFailureInfo& p); -void from_json(const json& j, ExecutionFailureInfo& 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 { -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); +struct DeleteScanInfo { + PlanNodeId id = {}; + TableHandle tableHandle = {}; +}; +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 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 TableWriteInfo { + std::shared_ptr writerTarget = {}; + std::shared_ptr analyzeTableHandle = {}; + std::shared_ptr deleteScanInfo = {}; }; -void to_json(json& j, const TaskStatus& p); -void from_json(const json& j, TaskStatus& p); +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 TaskStats { - DateTime createTime = {}; - DateTime firstStartTime = {}; - DateTime lastStartTime = {}; - DateTime lastEndTime = {}; - DateTime endTime = {}; - int64_t elapsedTimeInNanos = {}; - int64_t queuedTimeInNanos = {}; - int totalDrivers = {}; - int queuedDrivers = {}; - int queuedPartitionedDrivers = {}; - int64_t queuedPartitionedSplitsWeight = {}; - int runningDrivers = {}; - int runningPartitionedDrivers = {}; - int64_t runningPartitionedSplitsWeight = {}; - int blockedDrivers = {}; - int completedDrivers = {}; - double cumulativeUserMemory = {}; - double cumulativeTotalMemory = {}; - int64_t userMemoryReservationInBytes = {}; - int64_t revocableMemoryReservationInBytes = {}; - int64_t systemMemoryReservationInBytes = {}; - int64_t peakTotalMemoryInBytes = {}; - int64_t peakUserMemoryInBytes = {}; - int64_t peakNodeTotalMemoryInBytes = {}; - int64_t totalScheduledTimeInNanos = {}; - int64_t totalCpuTimeInNanos = {}; - int64_t totalBlockedTimeInNanos = {}; - bool fullyBlocked = {}; - List blockedReasons = {}; - int64_t totalAllocationInBytes = {}; - int64_t rawInputDataSizeInBytes = {}; - int64_t rawInputPositions = {}; - int64_t processedInputDataSizeInBytes = {}; - int64_t processedInputPositions = {}; - int64_t outputDataSizeInBytes = {}; - int64_t outputPositions = {}; - int64_t physicalWrittenDataSizeInBytes = {}; - int fullGcCount = {}; - int64_t fullGcTimeInMillis = {}; - List pipelines = {}; - RuntimeStats runtimeStats = {}; +enum class JoinDistributionType { PARTITIONED, REPLICATED }; +extern void to_json(json& j, const JoinDistributionType& e); +extern void from_json(const json& j, JoinDistributionType& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct JoinNode : public PlanNode { + JoinType 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; }; -void to_json(json& j, const TaskStats& p); -void from_json(const json& j, TaskStats& p); +void to_json(json& j, const JoinNode& p); +void from_json(const json& j, JoinNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct BufferInfo { - OutputBufferId bufferId = {}; - bool finished = {}; - int bufferedPages = {}; - int64_t pagesSent = {}; - PageBufferInfo pageBufferInfo = {}; +struct DwrfEncryptionMetadata { + Map fieldToKeyData = {}; + Map extraMetadata = {}; + String encryptionAlgorithm = {}; + String encryptionProvider = {}; }; -void to_json(json& j, const BufferInfo& p); -void from_json(const json& j, BufferInfo& 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 { -enum class BufferState { - OPEN, - NO_MORE_BUFFERS, - NO_MORE_PAGES, - FLUSHING, - FINISHED, - FAILED +struct JoinNodeStatsEstimate { + double nullJoinBuildKeyCount = {}; + double joinBuildKeyCount = {}; + double nullJoinProbeKeyCount = {}; + double joinProbeKeyCount = {}; }; -extern void to_json(json& j, const BufferState& e); -extern void from_json(const json& j, BufferState& e); +void to_json(json& j, const JoinNodeStatsEstimate& p); +void from_json(const json& j, JoinNodeStatsEstimate& 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 = {}; +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 HiveBucketProperty { + List bucketedBy = {}; + int bucketCount = {}; + List sortedBy = {}; + BucketFunctionType bucketFunctionType = {}; + std::shared_ptr> types = {}; }; -void to_json(json& j, const OutputBufferInfo& p); -void from_json(const json& j, OutputBufferInfo& 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 TaskInfo { - TaskId taskId = {}; - TaskStatus taskStatus = {}; - DateTime lastHeartbeat = {}; - OutputBufferInfo outputBuffers = {}; - List noMoreSplits = {}; - TaskStats stats = {}; - bool needsPlan = {}; - MetadataUpdates metadataUpdates = {}; - String nodeId = {}; +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 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 { +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 +/* + * 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 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); + +} // 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, + ALPHA, + 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 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 { +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 +namespace facebook::presto::protocol { +struct LocationHandle { + String targetPath = {}; + String writePath = {}; + std::shared_ptr tempPath = {}; + TableType tableType = {}; + WriteMode writeMode = {}; }; -void to_json(json& j, const TaskInfo& p); -void from_json(const json& j, TaskInfo& 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 LongVariableConstraint { - String name = {}; - String expression = {}; +struct EncryptionInformation { + std::shared_ptr dwrfEncryptionMetadata = {}; }; -void to_json(json& j, const LongVariableConstraint& p); -void from_json(const json& j, LongVariableConstraint& p); +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 FunctionKind { SCALAR, AGGREGATE, WINDOW }; -extern void to_json(json& j, const FunctionKind& e); -extern void from_json(const json& j, FunctionKind& e); +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; +}; +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 Signature { - QualifiedObjectName name = {}; - FunctionKind kind = {}; - List typeVariableConstraints = {}; - List longVariableConstraints = {}; - TypeSignature returnType = {}; - List argumentTypes = {}; - bool variableArity = {}; +struct Range { + Marker low = {}; + Marker high = {}; }; -void to_json(json& j, const Signature& p); -void from_json(const json& j, Signature& p); +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 BuiltInFunctionHandle : public FunctionHandle { - Signature signature = {}; +struct SortedRangeSet : public ValueSet { + Type type = {}; + List ranges = {}; - BuiltInFunctionHandle() noexcept; + SortedRangeSet() noexcept; }; -void to_json(json& j, const BuiltInFunctionHandle& p); -void from_json(const json& j, BuiltInFunctionHandle& p); +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 Step { SINGLE, PARTIAL, FINAL }; -extern void to_json(json& j, const Step& e); -extern void from_json(const json& j, Step& e); +struct ConstantExpression : public RowExpression { + Block valueBlock = {}; + Type type = {}; + + ConstantExpression() noexcept; +}; +void to_json(json& j, const ConstantExpression& p); +void from_json(const json& j, ConstantExpression& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TopNNode : public PlanNode { - std::shared_ptr source = {}; - int64_t count = {}; - OrderingScheme orderingScheme = {}; - Step step = {}; +struct RemoteSplit : public ConnectorSplit { + Location location = {}; + TaskId remoteSourceTaskId = {}; - TopNNode() noexcept; + RemoteSplit() noexcept; }; -void to_json(json& j, const TopNNode& p); -void from_json(const json& j, TopNNode& p); +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 FilterNode : public PlanNode { - std::shared_ptr source = {}; - std::shared_ptr predicate = {}; +struct BaseHiveColumnHandle { + String name = {}; + std::shared_ptr comment = {}; + ColumnType columnType = {}; + List requiredSubfields = {}; +}; +void to_json(json& j, const BaseHiveColumnHandle& p); +void from_json(const json& j, BaseHiveColumnHandle& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct SqlFunctionHandle : public FunctionHandle { + SqlFunctionId functionId = {}; + String version = {}; - FilterNode() noexcept; + SqlFunctionHandle() noexcept; }; -void to_json(json& j, const FilterNode& p); -void from_json(const json& j, FilterNode& p); +void to_json(json& j, const SqlFunctionHandle& p); +void from_json(const json& j, SqlFunctionHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct OutputNode : public PlanNode { - std::shared_ptr source = {}; - List columnNames = {}; - List outputVariables = {}; +struct TpchTableHandle : public ConnectorTableHandle { + String tableName = {}; + double scaleFactor = {}; - OutputNode() noexcept; + TpchTableHandle() noexcept; }; -void to_json(json& j, const OutputNode& p); -void from_json(const json& j, OutputNode& 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 ResourceEstimates { - std::shared_ptr executionTime = {}; - std::shared_ptr cpuTime = {}; - std::shared_ptr peakMemory = {}; - std::shared_ptr peakTaskMemory = {}; +struct TpchTableLayoutHandle : public ConnectorTableLayoutHandle { + TpchTableHandle table = {}; + TupleDomain> predicate = {}; + + TpchTableLayoutHandle() noexcept; }; -void to_json(json& j, const ResourceEstimates& p); -void from_json(const json& j, ResourceEstimates& p); +void to_json(json& j, const TpchTableLayoutHandle& p); +void from_json(const json& j, TpchTableLayoutHandle& 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); +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 BaseHiveColumnHandle { - String name = {}; - std::shared_ptr comment = {}; - ColumnType columnType = {}; - List requiredSubfields = {}; +struct MemoryPoolInfo { + int64_t maxBytes = {}; + int64_t reservedBytes = {}; + int64_t reservedRevocableBytes = {}; + Map queryMemoryReservations = {}; + Map> queryMemoryAllocations = {}; + Map queryMemoryRevocableReservations = {}; }; -void to_json(json& j, const BaseHiveColumnHandle& p); -void from_json(const json& j, BaseHiveColumnHandle& p); +void to_json(json& j, const MemoryPoolInfo& p); +void from_json(const json& j, MemoryPoolInfo& 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); +struct DeleteHandle : public ExecutionWriterTarget { + TableHandle handle = {}; + SchemaTableName schemaTableName = {}; + + DeleteHandle() noexcept; +}; +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 SelectedRole { - SelectedRoleType type = {}; - std::shared_ptr role = {}; +struct TableScanNode : public PlanNode { + TableHandle table = {}; + List outputVariables = {}; + Map> assignments = + {}; + + TableScanNode() noexcept; }; -void to_json(json& j, const SelectedRole& p); -void from_json(const json& j, SelectedRole& 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 { enum class BoundType { @@ -1882,38 +2181,6 @@ struct Function { 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 ConstantExpression : public RowExpression { - Block valueBlock = {}; - Type type = {}; - - ConstantExpression() noexcept; -}; -void to_json(json& j, const ConstantExpression& p); -void from_json(const json& j, ConstantExpression& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct RemoteTransactionHandle : public ConnectorTransactionHandle { - std::shared_ptr dummy = {}; - - RemoteTransactionHandle() noexcept; -}; -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 TableWriterMergeNode : public PlanNode { - std::shared_ptr source = {}; - VariableReferenceExpression rowCountVariable = {}; - VariableReferenceExpression fragmentVariable = {}; - VariableReferenceExpression tableCommitContextVariable = {}; - std::shared_ptr statisticsAggregation = {}; - - TableWriterMergeNode() noexcept; -}; -void to_json(json& j, const TableWriterMergeNode& p); -void from_json(const json& j, TableWriterMergeNode& 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. @@ -1928,187 +2195,232 @@ void from_json(const json& j, TableWriterMergeNode& p); * limitations under the License. */ +// IcebergColumnHandle is special since we require an implementation of +// operator<(). + namespace facebook::presto::protocol { -struct HiveColumnHandle : public ColumnHandle { - String name = {}; - HiveType hiveType = {}; - TypeSignature typeSignature = {}; - int hiveColumnIndex = {}; - ColumnType columnType = {}; +struct IcebergColumnHandle : public ColumnHandle { + ColumnIdentity columnIdentity = {}; + Type type = {}; std::shared_ptr comment = {}; + ColumnType columnType = {}; List requiredSubfields = {}; - std::shared_ptr partialAggregation = {}; - HiveColumnHandle() noexcept; + IcebergColumnHandle() noexcept; bool operator<(const ColumnHandle& o) const override { - return name < dynamic_cast(o).name; + return columnIdentity.name < + dynamic_cast(o).columnIdentity.name; } }; -void to_json(json& j, const HiveColumnHandle& p); -void from_json(const json& j, HiveColumnHandle& p); +void to_json(json& j, const IcebergColumnHandle& p); +void from_json(const json& j, IcebergColumnHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct BucketConversion { - int tableBucketCount = {}; - int partitionBucketCount = {}; - List bucketColumnHandles = {}; +enum class ChangelogOperation { INSERT, DELETE, UPDATE_BEFORE, UPDATE_AFTER }; +extern void to_json(json& j, const ChangelogOperation& e); +extern void from_json(const json& j, ChangelogOperation& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct ChangelogSplitInfo { + ChangelogOperation operation = {}; + int64_t ordinal = {}; + int64_t snapshotId = {}; + List icebergColumns = {}; +}; +void to_json(json& j, const ChangelogSplitInfo& p); +void from_json(const json& j, ChangelogSplitInfo& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct HivePartitionKey { + String name = {}; + std::shared_ptr value = {}; +}; +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 FileFormat { ORC, PARQUET, AVRO, METADATA }; +extern void to_json(json& j, const FileFormat& e); +extern void from_json(const json& j, FileFormat& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +enum class FileContent { DATA, POSITION_DELETES, EQUALITY_DELETES }; +extern void to_json(json& j, const FileContent& e); +extern void from_json(const json& j, FileContent& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct DeleteFile { + FileContent content = {}; + String path = {}; + FileFormat format = {}; + int64_t recordCount = {}; + int64_t fileSizeInBytes = {}; + List equalityFieldIds = {}; + Map lowerBounds = {}; + Map upperBounds = {}; +}; +void to_json(json& j, const DeleteFile& p); +void from_json(const json& j, DeleteFile& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +enum class NodeSelectionStrategy { + HARD_AFFINITY, + SOFT_AFFINITY, + NO_PREFERENCE }; -void to_json(json& j, const BucketConversion& p); -void from_json(const json& j, BucketConversion& p); +extern void to_json(json& j, const NodeSelectionStrategy& e); +extern void from_json(const json& j, NodeSelectionStrategy& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TableHandle { - ConnectorId connectorId = {}; - std::shared_ptr connectorHandle = {}; - std::shared_ptr transaction = {}; - std::shared_ptr connectorTableLayout = {}; +struct IcebergSplit : public ConnectorSplit { + String path = {}; + int64_t start = {}; + int64_t length = {}; + FileFormat fileFormat = {}; + List addresses = {}; + Map partitionKeys = {}; + NodeSelectionStrategy nodeSelectionStrategy = {}; + SplitWeight splitWeight = {}; + List deletes = {}; + std::shared_ptr changelogSplitInfo = {}; + int64_t dataSequenceNumber = {}; + + IcebergSplit() noexcept; }; -void to_json(json& j, const TableHandle& p); -void from_json(const json& j, TableHandle& p); +void to_json(json& j, const IcebergSplit& p); +void from_json(const json& j, IcebergSplit& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct DeleteHandle : public ExecutionWriterTarget { - TableHandle handle = {}; - SchemaTableName schemaTableName = {}; +struct HiveTableHandle : public ConnectorTableHandle { + String schemaName = {}; + String tableName = {}; + std::shared_ptr>> analyzePartitionValues = {}; - DeleteHandle() noexcept; + HiveTableHandle() noexcept; }; -void to_json(json& j, const DeleteHandle& p); -void from_json(const json& j, DeleteHandle& 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 { -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); +enum class BufferType { + PARTITIONED, + BROADCAST, + ARBITRARY, + DISCARDING, + SPOOLING +}; +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 { -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 OutputBuffers { + BufferType type = {}; + int64_t version = {}; + bool noMoreBufferIds = {}; + Map buffers = {}; +}; +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 WriteMode { - STAGE_AND_MOVE_TO_TARGET_DIRECTORY, - DIRECT_TO_TARGET_NEW_DIRECTORY, - DIRECT_TO_TARGET_EXISTING_DIRECTORY +enum class StageExecutionStrategy { + UNGROUPED_EXECUTION, + FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, + DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, + RECOVERABLE_GROUPED_EXECUTION }; -extern void to_json(json& j, const WriteMode& e); -extern void from_json(const json& j, WriteMode& e); +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 LocationHandle { - String targetPath = {}; - String writePath = {}; - std::shared_ptr tempPath = {}; - TableType tableType = {}; - WriteMode writeMode = {}; +struct StageExecutionDescriptor { + StageExecutionStrategy stageExecutionStrategy = {}; + List groupedExecutionScanNodes = {}; + int totalLifespans = {}; }; -void to_json(json& j, const LocationHandle& p); -void from_json(const json& j, LocationHandle& p); +void to_json(json& j, const StageExecutionDescriptor& p); +void from_json(const json& j, StageExecutionDescriptor& 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 { - -enum class HiveStorageFormat { - ORC, - DWRF, - ALPHA, - PARQUET, - AVRO, - RCBINARY, - RCTEXT, - SEQUENCEFILE, - JSON, - TEXTFILE, - CSV, - PAGEFILE +struct PartitioningHandle { + std::shared_ptr connectorId = {}; + std::shared_ptr transactionHandle = {}; + std::shared_ptr connectorHandle = {}; }; - -void to_json(json& j, const HiveStorageFormat& p); -void from_json(const json& j, HiveStorageFormat& 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 { -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); +struct Partitioning { + PartitioningHandle handle = {}; + List> arguments = {}; +}; +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 HiveBucketProperty { - List bucketedBy = {}; - int bucketCount = {}; - List sortedBy = {}; - BucketFunctionType bucketFunctionType = {}; - std::shared_ptr> types = {}; +struct PartitioningScheme { + Partitioning partitioning = {}; + List outputLayout = {}; + std::shared_ptr hashColumn = {}; + bool replicateNullsAndAny = {}; + std::shared_ptr> bucketToPartition = {}; }; -void to_json(json& j, const HiveBucketProperty& p); -void from_json(const json& j, HiveBucketProperty& 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 { -enum class PrestoTableType { - MANAGED_TABLE, - EXTERNAL_TABLE, - VIRTUAL_VIEW, - MATERIALIZED_VIEW, - TEMPORARY_TABLE, - OTHER +struct PlanCostEstimate { + double cpuCost = {}; + double maxMemory = {}; + double maxMemoryWhenOutputting = {}; + double networkCost = {}; }; -extern void to_json(json& j, const PrestoTableType& e); -extern void from_json(const json& j, PrestoTableType& 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 { -struct StorageFormat { - String serDe = {}; - String inputFormat = {}; - String outputFormat = {}; +struct PartialAggregationStatsEstimate { + double inputBytes = {}; + double outputBytes = {}; + double inputRowCount = {}; + double outputRowCount = {}; }; -void to_json(json& j, const StorageFormat& p); -void from_json(const json& j, StorageFormat& p); +void to_json(json& j, const PartialAggregationStatsEstimate& p); +void from_json(const json& j, PartialAggregationStatsEstimate& 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 = {}; +struct TableWriterNodeStatsEstimate { + double taskCountIfScaledWriter = {}; }; -void to_json(json& j, const Storage& p); -void from_json(const json& j, Storage& p); +void to_json(json& j, const TableWriterNodeStatsEstimate& p); +void from_json(const json& j, TableWriterNodeStatsEstimate& p); } // 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 = {}; +struct PlanNodeStatsEstimate { + double outputRowCount = {}; + double totalSize = {}; + bool confident = {}; + Map variableStatistics = + {}; + JoinNodeStatsEstimate joinNodeStatsEstimate = {}; + TableWriterNodeStatsEstimate tableWriterNodeStatsEstimate = {}; + PartialAggregationStatsEstimate partialAggregationStatsEstimate = {}; }; -void to_json(json& j, const Table& p); -void from_json(const json& j, Table& p); +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 StatsAndCosts { + Map stats = {}; + Map costs = {}; +}; +void to_json(json& j, const StatsAndCosts& p); +void from_json(const json& j, StatsAndCosts& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -2124,19 +2436,38 @@ void from_json(const json& j, Table& p); * limitations under the License. */ -// dependency Table -// dependency SchemaTableName - namespace facebook::presto::protocol { - -struct HivePageSinkMetadata { +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 { +struct UpdateTarget { + TableHandle handle = {}; SchemaTableName schemaTableName = {}; - std::shared_ptr
table = {}; - // TODO Add modifiedPartitions + List updatedColumns = {}; + List> updatedColumnHandles = {}; +}; +void to_json(json& j, const UpdateTarget& p); +void from_json(const json& j, UpdateTarget& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct MemoryInfo { + DataSize totalNodeMemory = {}; + Map pools = {}; }; -void to_json(json& j, const HivePageSinkMetadata& p); -void from_json(const json& j, HivePageSinkMetadata& 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 { struct HiveInsertTableHandle : public ConnectorInsertTableHandle { @@ -2159,36 +2490,13 @@ 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 SortedRangeSet : public ValueSet { - Type type = {}; - List ranges = {}; - - SortedRangeSet() noexcept; -}; -void to_json(json& j, const SortedRangeSet& p); -void from_json(const json& j, SortedRangeSet& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct TpchPartitioningHandle : public ConnectorPartitioningHandle { - String table = {}; - 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 MarkDistinctNode : public PlanNode { - std::shared_ptr source = {}; - VariableReferenceExpression markerVariable = {}; - List distinctVariables = {}; - std::shared_ptr hashVariable = {}; +struct HiveTransactionHandle : public ConnectorTransactionHandle { + UUID uuid = {}; - MarkDistinctNode() noexcept; + HiveTransactionHandle() noexcept; }; -void to_json(json& j, const MarkDistinctNode& p); -void from_json(const json& j, MarkDistinctNode& p); +void to_json(json& j, const HiveTransactionHandle& p); +void from_json(const json& j, HiveTransactionHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { enum class ExchangeNodeType { @@ -2226,449 +2534,232 @@ struct RemoteSourceNode : public PlanNode { bool ensureSourceOrdering = {}; std::shared_ptr orderingScheme = {}; ExchangeNodeType exchangeType = {}; - - RemoteSourceNode() noexcept; -}; -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 RowNumberNode : public PlanNode { - std::shared_ptr source = {}; - List partitionBy = {}; - VariableReferenceExpression rowNumberVariable = {}; - std::shared_ptr maxRowCountPerPartition = {}; - bool partial = {}; - 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 -namespace facebook::presto::protocol { -enum class SampleNodeType { BERNOULLI, SYSTEM }; -extern void to_json(json& j, const SampleNodeType& e); -extern void from_json(const json& j, SampleNodeType& e); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct SampleNode : public PlanNode { - std::shared_ptr source = {}; - double sampleRatio = {}; - SampleNodeType sampleType = {}; - - SampleNode() noexcept; -}; -void to_json(json& j, const SampleNode& p); -void from_json(const json& j, SampleNode& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class BufferType { - PARTITIONED, - BROADCAST, - ARBITRARY, - DISCARDING, - SPOOLING -}; -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 OutputBuffers { - BufferType type = {}; - int64_t version = {}; - bool noMoreBufferIds = {}; - Map buffers = {}; -}; -void to_json(json& j, const OutputBuffers& p); -void from_json(const json& j, OutputBuffers& 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 UpdateTarget { - TableHandle handle = {}; - SchemaTableName schemaTableName = {}; - List updatedColumns = {}; - List> updatedColumnHandles = {}; -}; -void to_json(json& j, const UpdateTarget& p); -void from_json(const json& j, UpdateTarget& 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; -}; -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 HiveTableHandle : public ConnectorTableHandle { - String schemaName = {}; - String tableName = {}; - std::shared_ptr>> analyzePartitionValues = {}; - - HiveTableHandle() noexcept; -}; -void to_json(json& j, const HiveTableHandle& p); -void from_json(const json& j, HiveTableHandle& 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 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 -/* - * 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 { - -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 Language { - String language = {}; -}; -void to_json(json& j, const Language& p); -void from_json(const json& j, Language& 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); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class Determinism { - DETERMINISTIC, - NOT_DETERMINISTIC, -}; -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 RoutineCharacteristics { - std::shared_ptr language = {}; - std::shared_ptr determinism = {}; - std::shared_ptr nullCallClause = {}; -}; -void to_json(json& j, const RoutineCharacteristics& p); -void from_json(const json& j, RoutineCharacteristics& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct UpdateHandle { - TableHandle handle = {}; - SchemaTableName schemaTableName = {}; -}; -void to_json(json& j, const UpdateHandle& p); -void from_json(const json& j, UpdateHandle& p); -} // 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); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct DeleteScanInfo { - PlanNodeId id = {}; - TableHandle tableHandle = {}; -}; -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 TableWriteInfo { - std::shared_ptr writerTarget = {}; - std::shared_ptr analyzeTableHandle = {}; - std::shared_ptr deleteScanInfo = {}; + + RemoteSourceNode() noexcept; }; -void to_json(json& j, const TableWriteInfo& p); -void from_json(const json& j, TableWriteInfo& 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); +void to_json(json& j, const RemoteSourceNode& p); +void from_json(const json& j, RemoteSourceNode& 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 LimitNode : public PlanNode { +struct TableWriterNode : public PlanNode { std::shared_ptr source = {}; - int64_t count = {}; - LimitNodeStep step = {}; + // TODO Add target + VariableReferenceExpression rowCountVariable = {}; + VariableReferenceExpression fragmentVariable = {}; + VariableReferenceExpression tableCommitContextVariable = {}; + List columns = {}; + List columnNames = {}; + List notNullColumnVariables = {}; + std::shared_ptr partitioningScheme = {}; + std::shared_ptr preferredShufflePartitioningScheme = {}; + std::shared_ptr statisticsAggregation = {}; - LimitNode() noexcept; + TableWriterNode() noexcept; }; -void to_json(json& j, const LimitNode& p); -void from_json(const json& j, LimitNode& 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 Parameter { - String name = {}; - TypeSignature type = {}; +struct Specification { + List partitionBy = {}; + std::shared_ptr orderingScheme = {}; }; -void to_json(json& j, const Parameter& p); -void from_json(const json& j, Parameter& p); +void to_json(json& j, const Specification& p); +void from_json(const json& j, Specification& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct SqlInvokedFunction { - List parameters = {}; - String description = {}; - RoutineCharacteristics routineCharacteristics = {}; - String body = {}; - Signature signature = {}; - SqlFunctionId functionId = {}; +struct TopNRowNumberNode : public PlanNode { + std::shared_ptr source = {}; + Specification specification = {}; + VariableReferenceExpression rowNumberVariable = {}; + int maxRowCountPerPartition = {}; + bool partial = {}; + std::shared_ptr hashVariable = {}; + + TopNRowNumberNode() noexcept; }; -void to_json(json& j, const SqlInvokedFunction& p); -void from_json(const json& j, SqlInvokedFunction& p); +void to_json(json& j, const TopNRowNumberNode& p); +void from_json(const json& j, TopNRowNumberNode& 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 = {}; +enum class IcebergTableType { + DATA, + HISTORY, + SNAPSHOTS, + MANIFESTS, + PARTITIONS, + FILES, + PROPERTIES, + CHANGELOG, + EQUALITY_DELETES, + DATA_WITHOUT_EQUALITY_DELETES }; -void to_json(json& j, const SessionRepresentation& p); -void from_json(const json& j, SessionRepresentation& p); +extern void to_json(json& j, const IcebergTableType& e); +extern void from_json(const json& j, IcebergTableType& e); } // 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 IcebergTableName { + String tableName = {}; + IcebergTableType tableType = {}; + std::shared_ptr snapshotId = {}; + std::shared_ptr changelogEndSnapshot = {}; }; -void to_json(json& j, const TaskUpdateRequest& p); -void from_json(const json& j, TaskUpdateRequest& p); +void to_json(json& j, const IcebergTableName& p); +void from_json(const json& j, IcebergTableName& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct BatchTaskUpdateRequest { - TaskUpdateRequest taskUpdateRequest = {}; - std::shared_ptr shuffleWriteInfo = {}; - std::shared_ptr broadcastBasePath = {}; +struct IcebergTableHandle : public ConnectorTableHandle { + String schemaName = {}; + IcebergTableName icebergTableName = {}; + bool snapshotSpecified = {}; + TupleDomain predicate = {}; + std::shared_ptr tableSchemaJson = {}; + std::shared_ptr> partitionFieldIds = {}; + std::shared_ptr> equalityFieldIds = {}; + + IcebergTableHandle() noexcept; }; -void to_json(json& j, const BatchTaskUpdateRequest& p); -void from_json(const json& j, BatchTaskUpdateRequest& p); +void to_json(json& j, const IcebergTableHandle& p); +void from_json(const json& j, IcebergTableHandle& p); } // 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 = {}; +struct IcebergTableLayoutHandle : public ConnectorTableLayoutHandle { + List partitionColumns = {}; + List dataColumns = {}; + TupleDomain domainPredicate = {}; + std::shared_ptr remainingPredicate = {}; + Map predicateColumns = {}; + std::shared_ptr> requestedColumns = {}; + bool pushdownFilterEnabled = {}; + TupleDomain> partitionColumnPredicate = {}; + IcebergTableHandle table = {}; - HiveOutputTableHandle() noexcept; + IcebergTableLayoutHandle() noexcept; }; -void to_json(json& j, const HiveOutputTableHandle& p); -void from_json(const json& j, HiveOutputTableHandle& p); +void to_json(json& j, const IcebergTableLayoutHandle& p); +void from_json(const json& j, IcebergTableLayoutHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TableScanNode : public PlanNode { - TableHandle table = {}; +struct OutputNode : public PlanNode { + std::shared_ptr source = {}; + List columnNames = {}; List outputVariables = {}; - Map> assignments = - {}; - TableScanNode() noexcept; + OutputNode() noexcept; }; -void to_json(json& j, const TableScanNode& p); -void from_json(const json& j, TableScanNode& p); +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 EquatableValueSet : public ValueSet { - Type type = {}; - bool whiteList = {}; - List entries = {}; +struct HivePartitioningHandle : public ConnectorPartitioningHandle { + int bucketCount = {}; + std::shared_ptr maxCompatibleBucketCount = {}; + BucketFunctionType bucketFunctionType = {}; + std::shared_ptr> hiveTypes = {}; + std::shared_ptr> types = {}; - EquatableValueSet() noexcept; + HivePartitioningHandle() noexcept; }; -void to_json(json& j, const EquatableValueSet& p); -void from_json(const json& j, EquatableValueSet& 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 HiveBucketHandle { - List columns = {}; - int tableBucketCount = {}; - int readBucketCount = {}; +struct EnforceSingleRowNode : public PlanNode { + std::shared_ptr source = {}; + + EnforceSingleRowNode() noexcept; }; -void to_json(json& j, const HiveBucketHandle& p); -void from_json(const json& j, HiveBucketHandle& 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 HiveTableLayoutHandle : public ConnectorTableLayoutHandle { +struct UpdateHandle { + TableHandle handle = {}; 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 = {}; - bool footerStatsUnreliable = {}; - - HiveTableLayoutHandle() noexcept; }; -void to_json(json& j, const HiveTableLayoutHandle& p); -void from_json(const json& j, HiveTableLayoutHandle& p); +void to_json(json& j, const UpdateHandle& p); +void from_json(const json& j, UpdateHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct UnnestNode : public PlanNode { +struct GroupIdNode : public PlanNode { std::shared_ptr source = {}; - List replicateVariables = {}; - Map> - unnestVariables = {}; - std::shared_ptr ordinalityVariable = {}; + List> groupingSets = {}; + Map + groupingColumns = {}; + List aggregationArguments = {}; + VariableReferenceExpression groupIdVariable = {}; - UnnestNode() noexcept; + GroupIdNode() noexcept; }; -void to_json(json& j, const UnnestNode& p); -void from_json(const json& j, UnnestNode& 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 { -enum class Form { - IF, - NULL_IF, - SWITCH, - WHEN, - IS_NULL, - COALESCE, - IN, - AND, - OR, - DEREFERENCE, - ROW_CONSTRUCTOR, - BIND +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 = {}; }; -extern void to_json(json& j, const Form& e); -extern void from_json(const json& j, Form& e); +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 SpecialFormExpression : public RowExpression { - Form form = {}; - Type returnType = {}; - List> arguments = {}; - - SpecialFormExpression() noexcept; +struct TableToPartitionMapping { + std::shared_ptr> tableToPartitionColumns = {}; + Map partitionSchemaDifference = {}; }; -void to_json(json& j, const SpecialFormExpression& p); -void from_json(const json& j, SpecialFormExpression& 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 OutputTableHandle { +struct EmptySplit : public ConnectorSplit { ConnectorId connectorId = {}; - std::shared_ptr transactionHandle = {}; - std::shared_ptr connectorHandle = {}; + + EmptySplit() noexcept; }; -void to_json(json& j, const OutputTableHandle& p); -void from_json(const json& j, OutputTableHandle& p); +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 CreateHandle : public ExecutionWriterTarget { - OutputTableHandle handle = {}; - SchemaTableName schemaTableName = {}; +struct WindowNode : public PlanNode { + std::shared_ptr sourceLocation = {}; - CreateHandle() noexcept; + std::shared_ptr source = {}; + Specification specification = {}; + Map windowFunctions = {}; + std::shared_ptr hashVariable = {}; + List prePartitionedInputs = {}; + int preSortedOrderPrefix = {}; + + WindowNode() noexcept; }; -void to_json(json& j, const CreateHandle& p); -void from_json(const json& j, CreateHandle& p); +void to_json(json& j, const WindowNode& p); +void from_json(const json& j, WindowNode& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -2684,285 +2775,291 @@ void from_json(const json& j, CreateHandle& p); * 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 = {}; - std::shared_ptr statisticsAggregation = {}; +// TpchColumnHandle is special since we require an implementation of +// operator<(). - TableWriterNode() noexcept; -}; -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 ServerInfo { - NodeVersion nodeVersion = {}; - String environment = {}; - bool coordinator = {}; - bool starting = {}; - std::shared_ptr uptime = {}; +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 ServerInfo& p); -void from_json(const json& j, ServerInfo& p); +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 SortNode : public PlanNode { +struct DistinctLimitNode : public PlanNode { std::shared_ptr source = {}; - OrderingScheme orderingScheme = {}; - bool isPartial = {}; + int64_t limit = {}; + bool partial = {}; + List distinctVariables = {}; + std::shared_ptr hashVariable = {}; + int timeoutMillis = {}; - SortNode() noexcept; + DistinctLimitNode() noexcept; }; -void to_json(json& j, const SortNode& p); -void from_json(const json& j, SortNode& 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 { -struct EnforceSingleRowNode : public PlanNode { - std::shared_ptr source = {}; +struct InsertHandle : public ExecutionWriterTarget { + InsertTableHandle handle = {}; + SchemaTableName schemaTableName = {}; - EnforceSingleRowNode() noexcept; + InsertHandle() noexcept; }; -void to_json(json& j, const EnforceSingleRowNode& p); -void from_json(const json& j, EnforceSingleRowNode& p); +void to_json(json& j, const InsertHandle& p); +void from_json(const json& j, InsertHandle& 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 TpchTableLayoutHandle : public ConnectorTableLayoutHandle { - TpchTableHandle table = {}; - TupleDomain> predicate = {}; - TpchTableLayoutHandle() noexcept; +struct ScheduledSplit { + long sequenceId = {}; + PlanNodeId planNodeId = {}; // dependency + Split split = {}; + + bool operator<(const ScheduledSplit& o) const { + return sequenceId < o.sequenceId; + } }; -void to_json(json& j, const TpchTableLayoutHandle& p); -void from_json(const json& j, TpchTableLayoutHandle& 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 { -struct EmptySplit : public ConnectorSplit { - ConnectorId connectorId = {}; - - EmptySplit() noexcept; +struct TaskSource { + PlanNodeId planNodeId = {}; + List splits = {}; + List noMoreSplitsForLifespan = {}; + bool noMoreSplits = {}; }; -void to_json(json& j, const EmptySplit& p); -void from_json(const json& j, EmptySplit& p); +void to_json(json& j, const TaskSource& p); +void from_json(const json& j, TaskSource& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class SystemPartitionFunction { - SINGLE, - HASH, - ROUND_ROBIN, - BROADCAST, - UNKNOWN +struct TaskUpdateRequest { + SessionRepresentation session = {}; + Map extraCredentials = {}; + std::shared_ptr fragment = {}; + List sources = {}; + OutputBuffers outputIds = {}; + std::shared_ptr tableWriteInfo = {}; }; -extern void to_json(json& j, const SystemPartitionFunction& e); -extern void from_json(const json& j, SystemPartitionFunction& e); +void to_json(json& j, const TaskUpdateRequest& p); +void from_json(const json& j, TaskUpdateRequest& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class SystemPartitioning { - SINGLE, - FIXED, - SOURCE, - SCALED, - COORDINATOR_ONLY, - ARBITRARY +struct BuiltInFunctionHandle : public FunctionHandle { + Signature signature = {}; + + BuiltInFunctionHandle() noexcept; }; -extern void to_json(json& j, const SystemPartitioning& e); -extern void from_json(const json& j, SystemPartitioning& e); +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 SystemPartitioningHandle : public ConnectorPartitioningHandle { - SystemPartitioning partitioning = {}; - SystemPartitionFunction function = {}; - - SystemPartitioningHandle() noexcept; +struct GroupingSetDescriptor { + List groupingKeys = {}; + int groupingSetCount = {}; + List globalGroupingSets = {}; }; -void to_json(json& j, const SystemPartitioningHandle& p); -void from_json(const json& j, SystemPartitioningHandle& 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 AssignUniqueId : public PlanNode { +struct FilterNode : public PlanNode { std::shared_ptr source = {}; - VariableReferenceExpression idVariable = {}; + std::shared_ptr predicate = {}; - AssignUniqueId() noexcept; + FilterNode() noexcept; }; -void to_json(json& j, const AssignUniqueId& p); -void from_json(const json& j, AssignUniqueId& 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 HiveMetadataUpdateHandle : public ConnectorMetadataUpdateHandle { - UUID requestId = {}; - SchemaTableName schemaTableName = {}; - std::shared_ptr partitionName = {}; - std::shared_ptr fileName = {}; +struct TpchSplit : public ConnectorSplit { + TpchTableHandle tableHandle = {}; + int partNumber = {}; + int totalParts = {}; + List addresses = {}; + TupleDomain> predicate = {}; - HiveMetadataUpdateHandle() noexcept; + TpchSplit() noexcept; }; -void to_json(json& j, const HiveMetadataUpdateHandle& p); -void from_json(const json& j, HiveMetadataUpdateHandle& 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 GroupIdNode : public PlanNode { +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 TopNNode : public PlanNode { std::shared_ptr source = {}; - List> groupingSets = {}; - Map - groupingColumns = {}; - List aggregationArguments = {}; - VariableReferenceExpression groupIdVariable = {}; + int64_t count = {}; + OrderingScheme orderingScheme = {}; + Step step = {}; - GroupIdNode() noexcept; + TopNNode() noexcept; }; -void to_json(json& j, const GroupIdNode& p); -void from_json(const json& j, GroupIdNode& 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 SqlFunctionHandle : public FunctionHandle { - SqlFunctionId functionId = {}; - String version = {}; +struct RowNumberNode : public PlanNode { + std::shared_ptr source = {}; + List partitionBy = {}; + VariableReferenceExpression rowNumberVariable = {}; + std::shared_ptr maxRowCountPerPartition = {}; + bool partial = {}; + std::shared_ptr hashVariable = {}; - SqlFunctionHandle() noexcept; + RowNumberNode() noexcept; }; -void to_json(json& j, const SqlFunctionHandle& p); -void from_json(const json& j, SqlFunctionHandle& p); +void to_json(json& j, const RowNumberNode& p); +void from_json(const json& j, RowNumberNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class JoinDistributionType { PARTITIONED, REPLICATED }; -extern void to_json(json& j, const JoinDistributionType& e); -extern void from_json(const json& j, JoinDistributionType& e); +enum class SampleNodeType { BERNOULLI, SYSTEM }; +extern void to_json(json& j, const SampleNodeType& e); +extern void from_json(const json& j, SampleNodeType& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct JoinNode : public PlanNode { - JoinType 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 SampleNode : public PlanNode { + std::shared_ptr source = {}; + double sampleRatio = {}; + SampleNodeType sampleType = {}; - JoinNode() noexcept; + SampleNode() noexcept; }; -void to_json(json& j, const JoinNode& p); -void from_json(const json& j, JoinNode& p); +void to_json(json& j, const SampleNode& p); +void from_json(const json& j, SampleNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Specification { - List partitionBy = {}; - std::shared_ptr orderingScheme = {}; +struct TpchPartitioningHandle : public ConnectorPartitioningHandle { + String table = {}; + int64_t totalRows = {}; + + TpchPartitioningHandle() noexcept; }; -void to_json(json& j, const Specification& p); -void from_json(const json& j, Specification& p); +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 TopNRowNumberNode : public PlanNode { +struct SortNode : public PlanNode { std::shared_ptr source = {}; - Specification specification = {}; - VariableReferenceExpression rowNumberVariable = {}; - int maxRowCountPerPartition = {}; - bool partial = {}; - std::shared_ptr hashVariable = {}; + OrderingScheme orderingScheme = {}; + bool isPartial = {}; - TopNRowNumberNode() noexcept; + SortNode() noexcept; }; -void to_json(json& j, const TopNRowNumberNode& p); -void from_json(const json& j, TopNRowNumberNode& p); +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 RefreshMaterializedViewHandle { - InsertTableHandle handle = {}; - SchemaTableName schemaTableName = {}; +enum class SystemPartitioning { + SINGLE, + FIXED, + SOURCE, + SCALED, + COORDINATOR_ONLY, + ARBITRARY }; -void to_json(json& j, const RefreshMaterializedViewHandle& p); -void from_json(const json& j, RefreshMaterializedViewHandle& p); +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 AllOrNoneValueSet : public ValueSet { - Type type = {}; - bool all = {}; - - AllOrNoneValueSet() noexcept; +enum class SystemPartitionFunction { + SINGLE, + HASH, + ROUND_ROBIN, + BROADCAST, + UNKNOWN }; -void to_json(json& j, const AllOrNoneValueSet& p); -void from_json(const json& j, AllOrNoneValueSet& 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 { -struct HiveTransactionHandle : public ConnectorTransactionHandle { - UUID uuid = {}; +struct SystemPartitioningHandle : public ConnectorPartitioningHandle { + SystemPartitioning partitioning = {}; + SystemPartitionFunction function = {}; - HiveTransactionHandle() noexcept; + SystemPartitioningHandle() noexcept; }; -void to_json(json& j, const HiveTransactionHandle& p); -void from_json(const json& j, HiveTransactionHandle& 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 ValuesNode : public PlanNode { - std::shared_ptr location = {}; - - List outputVariables = {}; - List>> rows = {}; - std::shared_ptr valuesNodeLabel = {}; - - ValuesNode() noexcept; -}; -void to_json(json& j, const ValuesNode& p); -void from_json(const json& j, ValuesNode& p); +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 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; +struct RuntimeMetric { + String name = {}; + RuntimeUnit unit = {}; + int64_t sum = {}; + int64_t count = {}; + int64_t max = {}; + int64_t min = {}; }; -void to_json(json& j, const WindowNode& p); -void from_json(const json& j, WindowNode& 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 DistinctLimitNode : public PlanNode { - std::shared_ptr source = {}; - int64_t limit = {}; - bool partial = {}; - List distinctVariables = {}; - std::shared_ptr hashVariable = {}; - int timeoutMillis = {}; +struct CreateHandle : public ExecutionWriterTarget { + OutputTableHandle handle = {}; + SchemaTableName schemaTableName = {}; - DistinctLimitNode() noexcept; + CreateHandle() noexcept; }; -void to_json(json& j, const DistinctLimitNode& p); -void from_json(const json& j, DistinctLimitNode& 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 TableToPartitionMapping { - std::shared_ptr> tableToPartitionColumns = {}; - Map partitionSchemaDifference = {}; +struct BatchTaskUpdateRequest { + TaskUpdateRequest taskUpdateRequest = {}; + std::shared_ptr shuffleWriteInfo = {}; + std::shared_ptr broadcastBasePath = {}; }; -void to_json(json& j, const TableToPartitionMapping& p); -void from_json(const json& j, TableToPartitionMapping& p); +void to_json(json& j, const BatchTaskUpdateRequest& p); +void from_json(const json& j, BatchTaskUpdateRequest& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class NodeSelectionStrategy { - HARD_AFFINITY, - SOFT_AFFINITY, - NO_PREFERENCE +enum class Locality { UNKNOWN, LOCAL, REMOTE }; +extern void to_json(json& j, const Locality& e); +extern void from_json(const json& j, Locality& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct ProjectNode : public PlanNode { + std::shared_ptr source = {}; + Assignments assignments = {}; + Locality locality = {}; + + ProjectNode() noexcept; }; -extern void to_json(json& j, const NodeSelectionStrategy& e); -extern void from_json(const json& j, NodeSelectionStrategy& e); +void to_json(json& j, const ProjectNode& p); +void from_json(const json& j, ProjectNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct HiveSplit : public ConnectorSplit { @@ -2991,6 +3088,81 @@ 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 RemoteTransactionHandle : public ConnectorTransactionHandle { + std::shared_ptr dummy = {}; + + RemoteTransactionHandle() noexcept; +}; +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 HiveMetadataUpdateHandle : public ConnectorMetadataUpdateHandle { + UUID requestId = {}; + SchemaTableName schemaTableName = {}; + std::shared_ptr partitionName = {}; + std::shared_ptr fileName = {}; + + HiveMetadataUpdateHandle() noexcept; +}; +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 LambdaDefinitionExpression : public RowExpression { + List argumentTypes = {}; + List arguments = {}; + std::shared_ptr body = {}; + + LambdaDefinitionExpression() noexcept; +}; +void to_json(json& j, const LambdaDefinitionExpression& p); +void from_json(const json& j, LambdaDefinitionExpression& 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); +} // 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; +}; +void to_json(json& j, const SemiJoinNode& p); +void from_json(const json& j, SemiJoinNode& 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); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct AggregationNode : public PlanNode { + std::shared_ptr source = {}; + Map aggregations = {}; + GroupingSetDescriptor groupingSets = {}; + List preGroupedVariables = {}; + AggregationNodeStep step = {}; + std::shared_ptr hashVariable = {}; + std::shared_ptr groupIdVariable = {}; + std::shared_ptr aggregationId = {}; + + AggregationNode() noexcept; +}; +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 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.yml b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.yml index afeb7d4f91c3..200df402d6c9 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.yml +++ b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.yml @@ -50,6 +50,7 @@ AbstractClasses: comparable: true subclasses: - { name: HiveColumnHandle, key: hive } + - { name: IcebergColumnHandle, key: hive-iceberg } - { name: TpchColumnHandle, key: tpch } ConnectorPartitioningHandle: @@ -63,6 +64,7 @@ AbstractClasses: super: JsonEncodedSubclass subclasses: - { name: HiveTableHandle, key: hive } + - { name: IcebergTableHandle, key: hive-iceberg } - { name: TpchTableHandle, key: tpch } ConnectorOutputTableHandle: @@ -85,6 +87,7 @@ AbstractClasses: super: JsonEncodedSubclass subclasses: - { name: HiveTableLayoutHandle, key: hive } + - { name: IcebergTableLayoutHandle, key: hive-iceberg } - { name: TpchTableLayoutHandle, key: tpch } ConnectorMetadataUpdateHandle: @@ -96,6 +99,7 @@ AbstractClasses: super: JsonEncodedSubclass subclasses: - { name: HiveSplit, key: hive } + - { name: IcebergSplit, key: hive-iceberg } - { name: TpchSplit, key: tpch } - { name: RemoteSplit, key: $remote } - { name: EmptySplit, key: $empty } @@ -177,6 +181,7 @@ JavaClasses: - presto-hive-metastore/src/main/java/com/facebook/presto/hive/BucketFunctionType.java - presto-hive-common/src/main/java/com/facebook/presto/hive/CacheQuotaRequirement.java - presto-hive-common/src/main/java/com/facebook/presto/hive/CacheQuotaScope.java + - presto-hive-common/src/main/java/com/facebook/presto/hive/BaseHiveColumnHandle.java - presto-spi/src/main/java/com/facebook/presto/spi/relation/CallExpression.java - presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/Column.java - presto-verifier/src/main/java/com/facebook/presto/verifier/framework/Column.java @@ -233,6 +238,18 @@ JavaClasses: - presto-hive-metastore/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-iceberg/src/main/java/com/facebook/presto/iceberg/FileContent.java + - presto-iceberg/src/main/java/com/facebook/presto/iceberg/FileFormat.java + - presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSplit.java + - presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableHandle.java + - presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableName.java + - presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableType.java + - presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableLayoutHandle.java + - presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergColumnHandle.java + - presto-iceberg/src/main/java/com/facebook/presto/iceberg/ColumnIdentity.java + - presto-iceberg/src/main/java/com/facebook/presto/iceberg/delete/DeleteFile.java + - presto-iceberg/src/main/java/com/facebook/presto/iceberg/changelog/ChangelogOperation.java + - presto-iceberg/src/main/java/com/facebook/presto/iceberg/changelog/ChangelogSplitInfo.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 @@ -321,5 +338,4 @@ JavaClasses: - presto-spark-base/src/main/java/com/facebook/presto/spark/execution/http/BatchTaskUpdateRequest.java - presto-spi/src/main/java/com/facebook/presto/spi/plan/JoinType.java - presto-spi/src/main/java/com/facebook/presto/spi/plan/JoinDistributionType.java - - presto-spi/src/main/java/com/facebook/presto/spi/plan/EquiJoinClause.java - - presto-hive-common/src/main/java/com/facebook/presto/hive/BaseHiveColumnHandle.java \ No newline at end of file + - presto-spi/src/main/java/com/facebook/presto/spi/plan/EquiJoinClause.java \ No newline at end of file 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 2d8c6b0957c5..a5b9d6fc6925 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) == "hive-iceberg") { + j = *std::static_pointer_cast(p); + return; + } + if (getConnectorKey(type) == "tpch") { j = *std::static_pointer_cast(p); return; @@ -46,6 +51,14 @@ void from_json(const json& j, std::shared_ptr& p) { return; } + if (getConnectorKey(type) == "hive-iceberg") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (getConnectorKey(type) == "tpch") { std::shared_ptr k = std::make_shared(); j.get_to(*k); 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 e793613cdfd9..9088d8510b81 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) == "hive-iceberg") { + j = *std::static_pointer_cast(p); + return; + } + if (getConnectorKey(type) == "tpch") { j = *std::static_pointer_cast(p); return; @@ -66,6 +71,13 @@ void from_json(const json& j, std::shared_ptr& p) { return; } + if (getConnectorKey(type) == "hive-iceberg") { + 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); 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 c45faac1a029..3ec5d8249229 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) == "hive-iceberg") { + j = *std::static_pointer_cast(p); + return; + } + if (getConnectorKey(type) == "tpch") { j = *std::static_pointer_cast(p); return; @@ -47,6 +52,13 @@ void from_json(const json& j, std::shared_ptr& p) { return; } + if (getConnectorKey(type) == "hive-iceberg") { + 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); 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 1ba7ad7746f9..f265f1094b39 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) == "hive-iceberg") { + j = *std::static_pointer_cast(p); + return; + } + if (getConnectorKey(type) == "tpch") { j = *std::static_pointer_cast(p); return; @@ -48,6 +53,13 @@ void from_json(const json& j, std::shared_ptr& p) { return; } + if (getConnectorKey(type) == "hive-iceberg") { + 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); 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 f8109a7a29b5..50164b051fbb 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 @@ -29,6 +29,12 @@ void to_json(json& j, const std::shared_ptr& p) { j = *std::static_pointer_cast(p); return; } + + if (getConnectorKey(type) == "hive-iceberg") { + j = *std::static_pointer_cast(p); + return; + } + if (getConnectorKey(type) == "tpch") { j = *std::static_pointer_cast(p); return; @@ -64,6 +70,12 @@ void from_json(const json& j, std::shared_ptr& p) { p = k; return; } + if (getConnectorKey(type) == "hive-iceberg") { + 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); diff --git a/presto-native-execution/presto_cpp/presto_protocol/special/HiveColumnHandle.hpp.inc b/presto-native-execution/presto_cpp/presto_protocol/special/HiveColumnHandle.hpp.inc index ffd25bea34c0..2eccf1e1450a 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/special/HiveColumnHandle.hpp.inc +++ b/presto-native-execution/presto_cpp/presto_protocol/special/HiveColumnHandle.hpp.inc @@ -12,6 +12,9 @@ * limitations under the License. */ +// HiveColumnHandle is special since it needs an implementation of +// operator<(). + namespace facebook::presto::protocol { struct HiveColumnHandle : public ColumnHandle { diff --git a/presto-native-execution/presto_cpp/presto_protocol/special/IcebergColumnHandle.hpp.inc b/presto-native-execution/presto_cpp/presto_protocol/special/IcebergColumnHandle.hpp.inc new file mode 100644 index 000000000000..7debe9331c9f --- /dev/null +++ b/presto-native-execution/presto_cpp/presto_protocol/special/IcebergColumnHandle.hpp.inc @@ -0,0 +1,38 @@ +/* + * 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. + */ + +// IcebergColumnHandle is special since it needs an implementation of +// operator<(). + +namespace facebook::presto::protocol { + +struct IcebergColumnHandle : public ColumnHandle { + ColumnIdentity columnIdentity = {}; + Type type = {}; + std::shared_ptr comment = {}; + ColumnType columnType = {}; + List requiredSubfields = {}; + + IcebergColumnHandle() noexcept; + + bool operator<(const ColumnHandle& o) const override { + return columnIdentity.name < + dynamic_cast(o).columnIdentity.name; + } +}; + +void to_json(json& j, const IcebergColumnHandle& p); +void from_json(const json& j, IcebergColumnHandle& p); + +} // namespace facebook::presto::protocol \ No newline at end of file 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 index b5d7ef49be3f..6cc66239ea38 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/special/TpchColumnHandle.hpp.inc +++ b/presto-native-execution/presto_cpp/presto_protocol/special/TpchColumnHandle.hpp.inc @@ -12,7 +12,7 @@ * limitations under the License. */ -// TpchColumnHandle is special since we require an implementation of +// TpchColumnHandle is special since it needs an implementation of // operator<(). namespace facebook::presto::protocol {