From 38bcdce9ee56a1bb010c1d3198bfb9fbc1856a30 Mon Sep 17 00:00:00 2001 From: aditi-pandit Date: Thu, 21 Mar 2024 16:00:09 -0700 Subject: [PATCH] [native] SystemConnector to query system.runtime.tasks table --- .../presto_cpp/main/CMakeLists.txt | 1 + .../presto_cpp/main/PrestoServer.cpp | 23 + .../presto_cpp/main/PrestoServer.h | 2 + .../presto_cpp/main/PrestoTask.cpp | 6 + .../presto_cpp/main/PrestoTask.h | 3 + .../presto_cpp/main/SystemConnector.cpp | 396 + .../presto_cpp/main/SystemConnector.h | 206 + .../presto_cpp/main/SystemSplit.h | 42 + .../presto_protocol/ConnectorProtocol.h | 11 + .../presto_protocol/presto_protocol.cpp | 20144 ++++++++-------- .../presto_protocol/presto_protocol.h | 3733 +-- .../presto_protocol/presto_protocol.yml | 12 +- .../AbstractTestNativeSystemQueries.java | 40 +- 13 files changed, 12840 insertions(+), 11779 deletions(-) create mode 100644 presto-native-execution/presto_cpp/main/SystemConnector.cpp create mode 100644 presto-native-execution/presto_cpp/main/SystemConnector.h create mode 100644 presto-native-execution/presto_cpp/main/SystemSplit.h diff --git a/presto-native-execution/presto_cpp/main/CMakeLists.txt b/presto-native-execution/presto_cpp/main/CMakeLists.txt index 6b354bb02d1e..1e912d20266a 100644 --- a/presto-native-execution/presto_cpp/main/CMakeLists.txt +++ b/presto-native-execution/presto_cpp/main/CMakeLists.txt @@ -29,6 +29,7 @@ add_library( QueryContextManager.cpp ServerOperation.cpp SignalHandler.cpp + SystemConnector.cpp TaskManager.cpp TaskResource.cpp PeriodicHeartbeatManager.cpp diff --git a/presto-native-execution/presto_cpp/main/PrestoServer.cpp b/presto-native-execution/presto_cpp/main/PrestoServer.cpp index 3b3064a4cea8..a833a55ac671 100644 --- a/presto-native-execution/presto_cpp/main/PrestoServer.cpp +++ b/presto-native-execution/presto_cpp/main/PrestoServer.cpp @@ -21,6 +21,7 @@ #include "presto_cpp/main/Announcer.h" #include "presto_cpp/main/PeriodicTaskManager.h" #include "presto_cpp/main/SignalHandler.h" +#include "presto_cpp/main/SystemConnector.h" #include "presto_cpp/main/TaskResource.h" #include "presto_cpp/main/common/ConfigReader.h" #include "presto_cpp/main/common/Counters.h" @@ -232,6 +233,17 @@ void PrestoServer::run() { std::make_unique("iceberg")); registerPrestoToVeloxConnector( std::make_unique("tpch")); + // Presto server uses system catalog or system schema in other catalogs + // in different places in the code. All these resolve to the SystemConnector. + // Depending on where the operator or column is used, different prefixes can + // be used in the naming. So the protocol class is mapped + // to all the different prefixes for System tables/columns. + registerPrestoToVeloxConnector( + std::make_unique("$system")); + registerPrestoToVeloxConnector( + std::make_unique("system")); + registerPrestoToVeloxConnector( + std::make_unique("$system@system")); initializeVeloxMemory(); initializeThreadPools(); @@ -438,6 +450,7 @@ void PrestoServer::run() { } prestoServerOperations_ = std::make_unique(taskManager_.get(), this); + registerSystemConnector(); // The endpoint used by operation in production. httpServer_->registerGet( @@ -939,6 +952,15 @@ std::vector PrestoServer::registerConnectors( return catalogNames; } +void PrestoServer::registerSystemConnector() { + PRESTO_STARTUP_LOG(INFO) << "Registering system catalog " + << " using connector SystemConnector"; + VELOX_CHECK(taskManager_); + auto systemConnector = + std::make_shared("$system@system", taskManager_.get()); + velox::connector::registerConnector(systemConnector); +} + void PrestoServer::unregisterConnectors() { PRESTO_SHUTDOWN_LOG(INFO) << "Unregistering connectors"; auto connectors = facebook::velox::connector::getAllConnectors(); @@ -959,6 +981,7 @@ void PrestoServer::unregisterConnectors() { } } + facebook::velox::connector::unregisterConnector("$system@system"); PRESTO_SHUTDOWN_LOG(INFO) << "Unregistered " << connectors.size() << " connectors"; } diff --git a/presto-native-execution/presto_cpp/main/PrestoServer.h b/presto-native-execution/presto_cpp/main/PrestoServer.h index 55e654d7e64f..a413c3ead311 100644 --- a/presto-native-execution/presto_cpp/main/PrestoServer.h +++ b/presto-native-execution/presto_cpp/main/PrestoServer.h @@ -200,6 +200,8 @@ class PrestoServer { // Periodically yield tasks if there are tasks queued. void yieldTasks(); + void registerSystemConnector(); + const std::string configDirectoryPath_; std::shared_ptr coordinatorDiscoverer_; diff --git a/presto-native-execution/presto_cpp/main/PrestoTask.cpp b/presto-native-execution/presto_cpp/main/PrestoTask.cpp index 58e1845fc72d..9ad8c227eed0 100644 --- a/presto-native-execution/presto_cpp/main/PrestoTask.cpp +++ b/presto-native-execution/presto_cpp/main/PrestoTask.cpp @@ -497,12 +497,15 @@ void PrestoTask::updateTimeInfoLocked( util::toISOTimestamp(veloxTaskStats.executionStartTimeMs); prestoTaskStats.firstStartTime = util::toISOTimestamp(veloxTaskStats.firstSplitStartTimeMs); + createTimeMs = veloxTaskStats.executionStartTimeMs; + firstSplitStartTimeMs = veloxTaskStats.firstSplitStartTimeMs; prestoTaskStats.lastStartTime = util::toISOTimestamp(veloxTaskStats.lastSplitStartTimeMs); prestoTaskStats.lastEndTime = util::toISOTimestamp(veloxTaskStats.executionEndTimeMs); prestoTaskStats.endTime = util::toISOTimestamp(veloxTaskStats.executionEndTimeMs); + lastEndTimeMs = veloxTaskStats.executionEndTimeMs; if (veloxTaskStats.executionEndTimeMs > veloxTaskStats.executionStartTimeMs) { prestoTaskStats.elapsedTimeInNanos = (veloxTaskStats.executionEndTimeMs - @@ -804,6 +807,9 @@ folly::dynamic PrestoTask::toJson() const { obj["lastHeartbeatMs"] = lastHeartbeatMs; obj["lastTaskStatsUpdateMs"] = lastTaskStatsUpdateMs; obj["lastMemoryReservation"] = lastMemoryReservation; + obj["createTimeMs"] = createTimeMs; + obj["firstSplitStartTimeMs"] = firstSplitStartTimeMs; + obj["lastEndTimeMs"] = lastEndTimeMs; json j; to_json(j, info); diff --git a/presto-native-execution/presto_cpp/main/PrestoTask.h b/presto-native-execution/presto_cpp/main/PrestoTask.h index 07d7c0abbbe2..b0bfd123ac96 100644 --- a/presto-native-execution/presto_cpp/main/PrestoTask.h +++ b/presto-native-execution/presto_cpp/main/PrestoTask.h @@ -98,6 +98,9 @@ struct PrestoTask { uint64_t lastHeartbeatMs{0}; uint64_t lastTaskStatsUpdateMs = {0}; uint64_t lastMemoryReservation = {0}; + uint64_t createTimeMs{0}; + uint64_t firstSplitStartTimeMs{0}; + uint64_t lastEndTimeMs{0}; mutable std::mutex mutex; /// Error before task is created or when task is being created. diff --git a/presto-native-execution/presto_cpp/main/SystemConnector.cpp b/presto-native-execution/presto_cpp/main/SystemConnector.cpp new file mode 100644 index 000000000000..9de215e014ca --- /dev/null +++ b/presto-native-execution/presto_cpp/main/SystemConnector.cpp @@ -0,0 +1,396 @@ +/* + * 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. + */ +#include "presto_cpp/main/SystemConnector.h" +#include "presto_cpp/main/PrestoTask.h" +#include "presto_cpp/main/TaskManager.h" + +#include "velox/type/Timestamp.h" + +namespace facebook::presto { + +using namespace velox; + +namespace { + +static const std::string kRuntimeSchema = "runtime"; +static const std::string kTasksTable = "tasks"; + +} // namespace + +const velox::RowTypePtr SystemTableHandle::taskSchema() { + static std::vector kTaskColumnNames = { + "node_id", + "task_id", + "stage_execution_id", + "stage_id", + "query_id", + "state", + "splits", + "queued_splits", + "running_splits", + "completed_splits", + "split_scheduled_time_ms", + "split_cpu_time_ms", + "split_blocked_time_ms", + "raw_input_bytes", + "raw_input_rows", + "processed_input_bytes", + "processed_input_rows", + "output_bytes", + "output_rows", + "physical_written_bytes", + "created", + "start", + "last_heartbeat", + "end"}; + + static std::vector kTaskColumnTypes = { + velox::VARCHAR(), velox::VARCHAR(), velox::VARCHAR(), + velox::VARCHAR(), velox::VARCHAR(), velox::VARCHAR(), + velox::BIGINT(), velox::BIGINT(), velox::BIGINT(), + velox::BIGINT(), velox::BIGINT(), velox::BIGINT(), + velox::BIGINT(), velox::BIGINT(), velox::BIGINT(), + velox::BIGINT(), velox::BIGINT(), velox::BIGINT(), + velox::BIGINT(), velox::BIGINT(), velox::TIMESTAMP(), + velox::TIMESTAMP(), velox::TIMESTAMP(), velox::TIMESTAMP()}; + static const RowTypePtr kTaskSchema = + ROW(std::move(kTaskColumnNames), std::move(kTaskColumnTypes)); + return kTaskSchema; +} + +SystemTableHandle::SystemTableHandle( + std::string connectorId, + std::string schemaName, + std::string tableName) + : ConnectorTableHandle(std::move(connectorId)), + schemaName_(std::move(schemaName)), + tableName_(std::move(tableName)) { + VELOX_USER_CHECK_EQ( + schemaName_, + kRuntimeSchema, + "SystemConnector supports only runtime schema"); + VELOX_USER_CHECK_EQ( + tableName_, kTasksTable, "SystemConnector supports only tasks table"); +} + +std::string SystemTableHandle::toString() const { + return fmt::format("schema: {} table: {}", schemaName_, tableName_); +} + +SystemDataSource::SystemDataSource( + const std::shared_ptr& outputType, + const std::shared_ptr& tableHandle, + const std::unordered_map< + std::string, + std::shared_ptr>& columnHandles, + const TaskManager* taskManager, + velox::memory::MemoryPool* FOLLY_NONNULL pool) + : taskManager_(taskManager), pool_(pool) { + auto systemTableHandle = + std::dynamic_pointer_cast(tableHandle); + VELOX_CHECK_NOT_NULL( + systemTableHandle, + "TableHandle must be an instance of SystemTableHandle"); + + outputColumnMappings_.reserve(outputType->names().size()); + auto taskSchema = systemTableHandle->taskSchema(); + for (const auto& outputName : outputType->names()) { + auto it = columnHandles.find(outputName); + VELOX_CHECK( + it != columnHandles.end(), + "ColumnHandle is missing for output column '{}'", + outputName); + + auto handle = std::dynamic_pointer_cast(it->second); + VELOX_CHECK_NOT_NULL( + handle, + "ColumnHandle must be an instance of SystemColumnHandle " + "for '{}' on table '{}'", + handle->name(), + systemTableHandle->tableName()); + + auto columnIndex = taskSchema->getChildIdxIfExists(handle->name()); + VELOX_CHECK( + columnIndex.has_value(), + "Column {} not found in SystemTable task schema", + handle->name()); + outputColumnMappings_.push_back(columnIndex.value()); + } + + outputType_ = outputType; +} + +void SystemDataSource::addSplit( + std::shared_ptr split) { + VELOX_CHECK_NULL( + currentSplit_, + "Previous split has not been processed yet. Call next() to process the split."); + currentSplit_ = std::dynamic_pointer_cast(split); + VELOX_CHECK(currentSplit_, "Wrong type of split for SystemDataSource."); +} + +#define SET_TASK_COLUMN(value) \ + int j = 0; \ + for (const auto& taskEntry : taskMap) { \ + auto task = taskEntry.second; \ + auto taskInfo = taskInfos[j]; \ + flat->set(j, value); \ + j++; \ + } + +#define SET_TASK_FMT_COLUMN(value) \ + int j = 0; \ + std::string temp; \ + for (const auto& taskEntry : taskMap) { \ + auto task = taskEntry.second; \ + auto taskInfo = taskInfos[j]; \ + temp = fmt::format("{}", value); \ + flat->set(j, StringView(temp)); \ + j++; \ + } + +RowVectorPtr SystemDataSource::getTaskResults() { + auto taskMap = taskManager_->tasks(); + auto numRows = taskMap.size(); + + std::vector taskInfos; + taskInfos.reserve(numRows); + for (const auto& taskEntry : taskMap) { + taskInfos.push_back(taskEntry.second->updateInfo()); + } + + auto result = std::dynamic_pointer_cast( + BaseVector::create(outputType_, numRows, pool_)); + + static constexpr int64_t kNanosecondsInMillisecond = 1'000'000; + auto toMillis = [](int64_t nanos) -> int64_t { + return nanos / kNanosecondsInMillisecond; + }; + for (auto i = 0; i < outputColumnMappings_.size(); i++) { + result->childAt(i)->resize(numRows); + auto taskColumn = outputColumnMappings_.at(i); + auto taskEnum = TaskColumnEnum(taskColumn); + switch (taskEnum) { + case TaskColumnEnum::kNodeId: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(StringView(taskInfo.nodeId)); + break; + } + + case TaskColumnEnum::kTaskId: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(StringView(taskInfo.taskId)); + break; + } + + case TaskColumnEnum::kStageExecutionId: { + auto flat = result->childAt(i)->as>(); + SET_TASK_FMT_COLUMN(task->id.stageExecutionId()); + break; + } + + case TaskColumnEnum::kStageId: { + auto flat = result->childAt(i)->as>(); + SET_TASK_FMT_COLUMN(task->id.stageId()); + break; + } + + case TaskColumnEnum::kQueryId: { + auto flat = result->childAt(i)->as>(); + SET_TASK_FMT_COLUMN(task->id.queryId()); + break; + } + + case TaskColumnEnum::kState: { + auto flat = result->childAt(i)->as>(); + SET_TASK_FMT_COLUMN(json(taskInfo.taskStatus.state).dump()); + break; + } + + case TaskColumnEnum::kSplits: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(taskInfo.stats.totalDrivers); + break; + } + + case TaskColumnEnum::kQueuedSplits: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(taskInfo.stats.queuedDrivers); + break; + } + + case TaskColumnEnum::kRunningSplits: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(taskInfo.stats.runningDrivers); + break; + } + + case TaskColumnEnum::kCompletedSplits: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(taskInfo.stats.completedDrivers); + break; + } + + case TaskColumnEnum::kSplitScheduledTimeMs: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(toMillis(taskInfo.stats.totalScheduledTimeInNanos)); + break; + } + + case TaskColumnEnum::kSplitCpuTimeMs: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(toMillis(taskInfo.stats.totalCpuTimeInNanos)); + break; + } + + case TaskColumnEnum::kSplitBlockedTimeMs: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(toMillis(taskInfo.stats.totalBlockedTimeInNanos)); + break; + } + + case TaskColumnEnum::kRawInputBytes: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(taskInfo.stats.rawInputDataSizeInBytes); + break; + } + + case TaskColumnEnum::kRawInputRows: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(taskInfo.stats.rawInputPositions); + break; + } + + case TaskColumnEnum::kProcessedInputBytes: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(taskInfo.stats.processedInputDataSizeInBytes); + break; + } + + case TaskColumnEnum::kProcessedInputRows: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(taskInfo.stats.processedInputPositions); + break; + } + + case TaskColumnEnum::kOutputBytes: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(taskInfo.stats.outputDataSizeInBytes); + break; + } + + case TaskColumnEnum::kOutputRows: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(taskInfo.stats.outputPositions); + break; + } + + case TaskColumnEnum::kPhysicalWrittenBytes: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(taskInfo.stats.physicalWrittenDataSizeInBytes); + break; + } + + case TaskColumnEnum::kCreated: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(velox::Timestamp::fromMillis(task->createTimeMs)); + break; + } + + case TaskColumnEnum::kStart: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN( + velox::Timestamp::fromMillis(task->firstSplitStartTimeMs)); + break; + } + + case TaskColumnEnum::kLastHeartBeat: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(velox::Timestamp::fromMillis(task->lastHeartbeatMs)); + break; + } + + case TaskColumnEnum::kEnd: { + auto flat = result->childAt(i)->as>(); + SET_TASK_COLUMN(velox::Timestamp::fromMillis(task->lastEndTimeMs)); + break; + } + } + } + return result; +} + +std::optional SystemDataSource::next( + uint64_t size, + velox::ContinueFuture& /*future*/) { + if (!currentSplit_) { + return nullptr; + } + + auto result = getTaskResults(); + completedRows_ += result->size(); + completedBytes_ += result->estimateFlatSize(); + + currentSplit_ = nullptr; + + return result; +} + +std::unique_ptr +SystemPrestoToVeloxConnector::toVeloxSplit( + const protocol::ConnectorId& catalogId, + const protocol::ConnectorSplit* const connectorSplit) const { + auto systemSplit = dynamic_cast(connectorSplit); + VELOX_CHECK_NOT_NULL( + systemSplit, "Unexpected split type {}", connectorSplit->_type); + return std::make_unique( + catalogId, + systemSplit->tableHandle.schemaName, + systemSplit->tableHandle.tableName); +} + +std::unique_ptr +SystemPrestoToVeloxConnector::toVeloxColumnHandle( + const protocol::ColumnHandle* column, + const TypeParser& typeParser) const { + auto systemColumn = dynamic_cast(column); + VELOX_CHECK_NOT_NULL( + systemColumn, "Unexpected column handle type {}", column->_type); + return std::make_unique(systemColumn->columnName); +} + +std::unique_ptr +SystemPrestoToVeloxConnector::toVeloxTableHandle( + const protocol::TableHandle& tableHandle, + const VeloxExprConverter& exprConverter, + const TypeParser& typeParser, + std::unordered_map< + std::string, + std::shared_ptr>& assignments) const { + auto systemLayout = + std::dynamic_pointer_cast( + tableHandle.connectorTableLayout); + VELOX_CHECK_NOT_NULL( + systemLayout, "Unexpected table handle type {}", tableHandle.connectorId); + return std::make_unique( + tableHandle.connectorId, + systemLayout->table.schemaName, + systemLayout->table.tableName); +} + +std::unique_ptr +SystemPrestoToVeloxConnector::createConnectorProtocol() const { + return std::make_unique(); +} +} // namespace facebook::presto \ No newline at end of file diff --git a/presto-native-execution/presto_cpp/main/SystemConnector.h b/presto-native-execution/presto_cpp/main/SystemConnector.h new file mode 100644 index 000000000000..b467cf25676b --- /dev/null +++ b/presto-native-execution/presto_cpp/main/SystemConnector.h @@ -0,0 +1,206 @@ +/* + * 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. + */ +#pragma once + +#include "presto_cpp/main/SystemSplit.h" +#include "presto_cpp/main/types/PrestoToVeloxConnector.h" + +#include "velox/connectors/Connector.h" + +namespace facebook::presto { + +class TaskManager; + +class SystemColumnHandle : public velox::connector::ColumnHandle { + public: + explicit SystemColumnHandle(const std::string& name) : name_(name) {} + + const std::string& name() const { + return name_; + } + + private: + const std::string name_; +}; + +class SystemTableHandle : public velox::connector::ConnectorTableHandle { + public: + explicit SystemTableHandle( + std::string connectorId, + std::string schemaName, + std::string tableName); + + std::string toString() const override; + + const std::string& schemaName() { + return schemaName_; + } + + const std::string& tableName() { + return tableName_; + } + + const velox::RowTypePtr taskSchema(); + + private: + const std::string schemaName_; + const std::string tableName_; +}; + +class SystemDataSource : public velox::connector::DataSource { + public: + SystemDataSource( + const velox::RowTypePtr& outputType, + const std::shared_ptr& + tableHandle, + const std::unordered_map< + std::string, + std::shared_ptr>& columnHandles, + const TaskManager* taskManager, + velox::memory::MemoryPool* pool); + + void addSplit( + std::shared_ptr split) override; + + void addDynamicFilter( + velox::column_index_t /*outputChannel*/, + const std::shared_ptr& /*filter*/) override { + VELOX_NYI("Dynamic filters not supported by SystemConnector."); + } + + std::optional next( + uint64_t size, + velox::ContinueFuture& future) override; + + uint64_t getCompletedRows() override { + return completedRows_; + } + + uint64_t getCompletedBytes() override { + return completedBytes_; + } + + std::unordered_map runtimeStats() + override { + return {}; + } + + private: + enum class TaskColumnEnum { + // Note: These values are in the same order as SystemTableHandle schema. + kNodeId = 0, + kTaskId, + kStageExecutionId, + kStageId, + kQueryId, + kState, + kSplits, + kQueuedSplits, + kRunningSplits, + kCompletedSplits, + kSplitScheduledTimeMs, + kSplitCpuTimeMs, + kSplitBlockedTimeMs, + kRawInputBytes, + kRawInputRows, + kProcessedInputBytes, + kProcessedInputRows, + kOutputBytes, + kOutputRows, + kPhysicalWrittenBytes, + kCreated, + kStart, + kLastHeartBeat, + kEnd, + }; + + velox::RowVectorPtr getTaskResults(); + + // Mapping between output columns and their indices (column_index_t) + // corresponding to the taskInfo fields for them. + std::vector outputColumnMappings_; + velox::RowTypePtr outputType_; + + const TaskManager* taskManager_; + velox::memory::MemoryPool* pool_; + + std::shared_ptr currentSplit_; + + size_t completedRows_{0}; + size_t completedBytes_{0}; +}; + +class SystemConnector : public velox::connector::Connector { + public: + SystemConnector(const std::string& id, const TaskManager* taskManager) + : Connector(id), taskManager_(taskManager) {} + + std::unique_ptr createDataSource( + const velox::RowTypePtr& outputType, + const std::shared_ptr& + tableHandle, + const std::unordered_map< + std::string, + std::shared_ptr>& columnHandles, + velox::connector::ConnectorQueryCtx* connectorQueryCtx) override final { + VELOX_CHECK(taskManager_); + return std::make_unique( + outputType, + tableHandle, + columnHandles, + taskManager_, + connectorQueryCtx->memoryPool()); + } + + std::unique_ptr createDataSink( + velox::RowTypePtr /*inputType*/, + std::shared_ptr< + velox::connector:: + ConnectorInsertTableHandle> /*connectorInsertTableHandle*/, + velox::connector::ConnectorQueryCtx* /*connectorQueryCtx*/, + velox::connector::CommitStrategy /*commitStrategy*/) override final { + VELOX_NYI("SystemConnector does not support data sink."); + } + + private: + const TaskManager* taskManager_; +}; + +class SystemPrestoToVeloxConnector final : public PrestoToVeloxConnector { + public: + explicit SystemPrestoToVeloxConnector(std::string connectorId) + : PrestoToVeloxConnector(std::move(connectorId)) {} + + std::unique_ptr toVeloxSplit( + const protocol::ConnectorId& catalogId, + const protocol::ConnectorSplit* connectorSplit) const final; + + std::unique_ptr toVeloxColumnHandle( + const protocol::ColumnHandle* column, + const TypeParser& typeParser) const final; + + std::unique_ptr toVeloxTableHandle( + const protocol::TableHandle& tableHandle, + const VeloxExprConverter& exprConverter, + const TypeParser& typeParser, + std::unordered_map< + std::string, + std::shared_ptr>& assignments) + const final; + + std::unique_ptr createConnectorProtocol() + const final; +}; + +} // namespace facebook::presto diff --git a/presto-native-execution/presto_cpp/main/SystemSplit.h b/presto-native-execution/presto_cpp/main/SystemSplit.h new file mode 100644 index 000000000000..f1af48d6dfc4 --- /dev/null +++ b/presto-native-execution/presto_cpp/main/SystemSplit.h @@ -0,0 +1,42 @@ +/* + * 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. + */ +#pragma once + +#include "velox/connectors/Connector.h" + +namespace facebook::presto { + +struct SystemSplit : public velox::connector::ConnectorSplit { + explicit SystemSplit( + const std::string& connectorId, + const std::string& schemaName, + const std::string& tableName) + : ConnectorSplit(connectorId), + schemaName_(schemaName), + tableName_(tableName) {} + + const std::string& schemaName() { + return schemaName_; + } + + const std::string& tableName() { + return tableName_; + } + + private: + const std::string schemaName_; + const std::string tableName_; +}; + +} // namespace facebook::presto diff --git a/presto-native-execution/presto_cpp/presto_protocol/ConnectorProtocol.h b/presto-native-execution/presto_cpp/presto_protocol/ConnectorProtocol.h index b43e31d8803d..b91131791bb0 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/ConnectorProtocol.h +++ b/presto-native-execution/presto_cpp/presto_protocol/ConnectorProtocol.h @@ -261,4 +261,15 @@ using TpchConnectorProtocol = ConnectorProtocolTemplate< TpchTransactionHandle, NotImplemented>; +using SystemConnectorProtocol = ConnectorProtocolTemplate< + SystemTableHandle, + SystemTableLayoutHandle, + SystemColumnHandle, + NotImplemented, + NotImplemented, + SystemSplit, + SystemPartitioningHandle, + SystemTransactionHandle, + NotImplemented>; + } // namespace facebook::presto::protocol 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 777c577a8822..11e91b5f9359 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.cpp +++ b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.cpp @@ -58,501 +58,476 @@ 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 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 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 to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_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() + 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 == ".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; + } -// 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; + 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 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 { +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"); + } -void to_json(json& j, const Language& p) { + 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 == ".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 { + +void to_json(json& j, const SourceLocation& p) { j = json::object(); - to_json_key(j, "language", p.language, "Language", "String", "language"); + 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, Language& p) { - from_json_key(j, "language", p.language, "Language", "String", "language"); +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 { +VariableReferenceExpression::VariableReferenceExpression() noexcept { + _type = "variable"; +} -void to_json(json& j, const RoutineCharacteristics& p) { +void to_json(json& j, const VariableReferenceExpression& p) { j = json::object(); + j["@type"] = "variable"; to_json_key( j, - "language", - p.language, - "RoutineCharacteristics", - "Language", - "language"); - to_json_key( - j, - "determinism", - p.determinism, - "RoutineCharacteristics", - "Determinism", - "determinism"); + "sourceLocation", + p.sourceLocation, + "VariableReferenceExpression", + "SourceLocation", + "sourceLocation"); to_json_key( - j, - "nullCallClause", - p.nullCallClause, - "RoutineCharacteristics", - "NullCallClause", - "nullCallClause"); + j, "name", p.name, "VariableReferenceExpression", "String", "name"); + to_json_key(j, "type", p.type, "VariableReferenceExpression", "Type", "type"); } -void from_json(const json& j, RoutineCharacteristics& p) { +void from_json(const json& j, VariableReferenceExpression& p) { + p._type = j["@type"]; from_json_key( j, - "language", - p.language, - "RoutineCharacteristics", - "Language", - "language"); + "sourceLocation", + p.sourceLocation, + "VariableReferenceExpression", + "SourceLocation", + "sourceLocation"); from_json_key( - j, - "determinism", - p.determinism, - "RoutineCharacteristics", - "Determinism", - "determinism"); + j, "name", p.name, "VariableReferenceExpression", "String", "name"); from_json_key( - j, - "nullCallClause", - p.nullCallClause, - "RoutineCharacteristics", - "NullCallClause", - "nullCallClause"); + j, "type", p.type, "VariableReferenceExpression", "Type", "type"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +MarkDistinctNode::MarkDistinctNode() noexcept { + _type = ".MarkDistinctNode"; +} -void to_json(json& j, const TypeVariableConstraint& p) { +void to_json(json& j, const MarkDistinctNode& p) { j = json::object(); - to_json_key(j, "name", p.name, "TypeVariableConstraint", "String", "name"); + 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, - "comparableRequired", - p.comparableRequired, - "TypeVariableConstraint", - "bool", - "comparableRequired"); + "markerVariable", + p.markerVariable, + "MarkDistinctNode", + "VariableReferenceExpression", + "markerVariable"); to_json_key( j, - "orderableRequired", - p.orderableRequired, - "TypeVariableConstraint", - "bool", - "orderableRequired"); - to_json_key( - j, - "variadicBound", - p.variadicBound, - "TypeVariableConstraint", - "String", - "variadicBound"); + "distinctVariables", + p.distinctVariables, + "MarkDistinctNode", + "List", + "distinctVariables"); to_json_key( j, - "nonDecimalNumericRequired", - p.nonDecimalNumericRequired, - "TypeVariableConstraint", - "bool", - "nonDecimalNumericRequired"); + "hashVariable", + p.hashVariable, + "MarkDistinctNode", + "VariableReferenceExpression", + "hashVariable"); } -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"); +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, - "orderableRequired", - p.orderableRequired, - "TypeVariableConstraint", - "bool", - "orderableRequired"); + j, "source", p.source, "MarkDistinctNode", "PlanNode", "source"); from_json_key( j, - "variadicBound", - p.variadicBound, - "TypeVariableConstraint", - "String", - "variadicBound"); + "markerVariable", + p.markerVariable, + "MarkDistinctNode", + "VariableReferenceExpression", + "markerVariable"); from_json_key( j, - "nonDecimalNumericRequired", - p.nonDecimalNumericRequired, - "TypeVariableConstraint", - "bool", - "nonDecimalNumericRequired"); -} -} // 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"); + "distinctVariables", + p.distinctVariables, + "MarkDistinctNode", + "List", + "distinctVariables"); from_json_key( j, - "expression", - p.expression, - "LongVariableConstraint", - "String", - "expression"); + "hashVariable", + p.hashVariable, + "MarkDistinctNode", + "VariableReferenceExpression", + "hashVariable"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair FunctionKind_enum_table[] = +static const std::pair SortOrder_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!"); + {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(FunctionKind_enum_table), - std::end(FunctionKind_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(FunctionKind_enum_table)) + j = ((it != std::end(SortOrder_enum_table)) ? it - : std::begin(FunctionKind_enum_table)) + : std::begin(SortOrder_enum_table)) ->second; } -void from_json(const json& j, FunctionKind& e) { - static_assert( - std::is_enum::value, "FunctionKind 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(FunctionKind_enum_table), - std::end(FunctionKind_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(FunctionKind_enum_table)) + e = ((it != std::end(SortOrder_enum_table)) ? it - : std::begin(FunctionKind_enum_table)) + : std::begin(SortOrder_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Signature& p) { +void to_json(json& j, const Ordering& p) { j = json::object(); - to_json_key(j, "name", p.name, "Signature", "QualifiedObjectName", "name"); - to_json_key(j, "kind", p.kind, "Signature", "FunctionKind", "kind"); - to_json_key( - j, - "typeVariableConstraints", - p.typeVariableConstraints, - "Signature", - "List", - "typeVariableConstraints"); - to_json_key( - j, - "longVariableConstraints", - p.longVariableConstraints, - "Signature", - "List", - "longVariableConstraints"); - to_json_key( - j, - "returnType", - p.returnType, - "Signature", - "TypeSignature", - "returnType"); to_json_key( j, - "argumentTypes", - p.argumentTypes, - "Signature", - "List", - "argumentTypes"); + "variable", + p.variable, + "Ordering", + "VariableReferenceExpression", + "variable"); to_json_key( - j, - "variableArity", - p.variableArity, - "Signature", - "bool", - "variableArity"); + j, "sortOrder", p.sortOrder, "Ordering", "SortOrder", "sortOrder"); } -void from_json(const json& j, Signature& p) { - from_json_key(j, "name", p.name, "Signature", "QualifiedObjectName", "name"); - from_json_key(j, "kind", p.kind, "Signature", "FunctionKind", "kind"); - from_json_key( - j, - "typeVariableConstraints", - p.typeVariableConstraints, - "Signature", - "List", - "typeVariableConstraints"); - from_json_key( - j, - "longVariableConstraints", - p.longVariableConstraints, - "Signature", - "List", - "longVariableConstraints"); - from_json_key( - j, - "returnType", - p.returnType, - "Signature", - "TypeSignature", - "returnType"); +void from_json(const json& j, Ordering& p) { from_json_key( j, - "argumentTypes", - p.argumentTypes, - "Signature", - "List", - "argumentTypes"); + "variable", + p.variable, + "Ordering", + "VariableReferenceExpression", + "variable"); from_json_key( - j, - "variableArity", - p.variableArity, - "Signature", - "bool", - "variableArity"); + j, "sortOrder", p.sortOrder, "Ordering", "SortOrder", "sortOrder"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const SqlInvokedFunction& p) { +void to_json(json& j, const OrderingScheme& 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"); + j, "orderBy", p.orderBy, "OrderingScheme", "List", "orderBy"); } -void from_json(const json& j, SqlInvokedFunction& p) { +void from_json(const json& j, OrderingScheme& 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, - "signature", - p.signature, - "SqlInvokedFunction", - "Signature", - "signature"); - from_json_key( - j, - "functionId", - p.functionId, - "SqlInvokedFunction", - "SqlFunctionId", - "functionId"); + j, "orderBy", p.orderBy, "OrderingScheme", "List", "orderBy"); } } // namespace facebook::presto::protocol /* @@ -569,20 +544,21 @@ void from_json(const json& j, SqlInvokedFunction& p) { * limitations under the License. */ -namespace facebook::presto::protocol { +// TpchTransactionHandle is special since +// the corresponding class in Java is an enum. -void to_json(json& j, const Duration& p) { - j = p.toString(); -} +namespace facebook::presto::protocol { -void from_json(const json& j, Duration& p) { - p = Duration(std::string(j)); +void to_json(json& j, const TpchTransactionHandle& p) { + j = json::array(); + j.push_back(p._type); + j.push_back(p.instance); } -std::ostream& operator<<(std::ostream& os, const Duration& d) { - return os << d.toString(); +void from_json(const json& j, TpchTransactionHandle& p) { + j[0].get_to(p._type); + j[1].get_to(p.instance); } - } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -598,343 +574,132 @@ std::ostream& operator<<(std::ostream& os, const Duration& d) { * limitations under the License. */ +// dependency TpchTransactionHandle + namespace facebook::presto::protocol { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; -void to_json(nlohmann::json& j, const DataSize& p) { - j = p.toString(); + if (type == "$remote") { + j = *std::static_pointer_cast(p); + return; + } + getConnectorProtocol(type).to_json(j, p); } -void from_json(const nlohmann::json& j, DataSize& p) { - p = DataSize(std::string(j)); -} +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError( + std::string(e.what()) + + " ConnectorTransactionHandle ConnectorTransactionHandle"); + } -std::ostream& operator<<(std::ostream& os, const DataSize& d) { - return os << d.toString(); + if (type == "$remote") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + getConnectorProtocol(type).from_json(j, p); } - } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +SystemTransactionHandle::SystemTransactionHandle() noexcept { + _type = "$system@system"; +} -void to_json(json& j, const ResourceEstimates& p) { +void to_json(json& j, const SystemTransactionHandle& p) { j = json::object(); + j["@type"] = "$system@system"; to_json_key( j, - "executionTime", - p.executionTime, - "ResourceEstimates", - "Duration", - "executionTime"); - to_json_key( - j, "cpuTime", p.cpuTime, "ResourceEstimates", "Duration", "cpuTime"); + "connectorId", + p.connectorId, + "SystemTransactionHandle", + "ConnectorId", + "connectorId"); to_json_key( j, - "peakMemory", - p.peakMemory, - "ResourceEstimates", - "DataSize", - "peakMemory"); + "transactionId", + p.transactionId, + "SystemTransactionHandle", + "TransactionId", + "transactionId"); to_json_key( j, - "peakTaskMemory", - p.peakTaskMemory, - "ResourceEstimates", - "DataSize", - "peakTaskMemory"); + "connectorTransactionHandle", + p.connectorTransactionHandle, + "SystemTransactionHandle", + "ConnectorTransactionHandle", + "connectorTransactionHandle"); } -void from_json(const json& j, ResourceEstimates& p) { +void from_json(const json& j, SystemTransactionHandle& p) { + p._type = j["@type"]; from_json_key( j, - "executionTime", - p.executionTime, - "ResourceEstimates", - "Duration", - "executionTime"); - from_json_key( - j, "cpuTime", p.cpuTime, "ResourceEstimates", "Duration", "cpuTime"); + "connectorId", + p.connectorId, + "SystemTransactionHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "peakMemory", - p.peakMemory, - "ResourceEstimates", - "DataSize", - "peakMemory"); + "transactionId", + p.transactionId, + "SystemTransactionHandle", + "TransactionId", + "transactionId"); from_json_key( j, - "peakTaskMemory", - p.peakTaskMemory, - "ResourceEstimates", - "DataSize", - "peakTaskMemory"); + "connectorTransactionHandle", + p.connectorTransactionHandle, + "SystemTransactionHandle", + "ConnectorTransactionHandle", + "connectorTransactionHandle"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const SessionRepresentation& p) { +void to_json(json& j, const Specification& p) { j = json::object(); - to_json_key( - j, "queryId", p.queryId, "SessionRepresentation", "String", "queryId"); - to_json_key( - j, - "transactionId", - p.transactionId, - "SessionRepresentation", - "TransactionId", - "transactionId"); - to_json_key( - j, - "clientTransactionSupport", - p.clientTransactionSupport, - "SessionRepresentation", - "bool", - "clientTransactionSupport"); - to_json_key(j, "user", p.user, "SessionRepresentation", "String", "user"); - to_json_key( - j, - "principal", - p.principal, - "SessionRepresentation", - "String", - "principal"); - to_json_key( - j, "source", p.source, "SessionRepresentation", "String", "source"); - to_json_key( - j, "catalog", p.catalog, "SessionRepresentation", "String", "catalog"); - to_json_key( - j, "schema", p.schema, "SessionRepresentation", "String", "schema"); - to_json_key( - j, - "traceToken", - p.traceToken, - "SessionRepresentation", - "String", - "traceToken"); - to_json_key( - j, - "timeZoneKey", - p.timeZoneKey, - "SessionRepresentation", - "TimeZoneKey", - "timeZoneKey"); - to_json_key( - j, "locale", p.locale, "SessionRepresentation", "Locale", "locale"); to_json_key( j, - "remoteUserAddress", - p.remoteUserAddress, - "SessionRepresentation", - "String", - "remoteUserAddress"); - to_json_key( - j, - "userAgent", - p.userAgent, - "SessionRepresentation", - "String", - "userAgent"); - to_json_key( - j, - "clientInfo", - p.clientInfo, - "SessionRepresentation", - "String", - "clientInfo"); - to_json_key( - j, - "clientTags", - p.clientTags, - "SessionRepresentation", - "List", - "clientTags"); + "partitionBy", + p.partitionBy, + "Specification", + "List", + "partitionBy"); to_json_key( j, - "resourceEstimates", - p.resourceEstimates, - "SessionRepresentation", - "ResourceEstimates", - "resourceEstimates"); - to_json_key( + "orderingScheme", + p.orderingScheme, + "Specification", + "OrderingScheme", + "orderingScheme"); +} + +void from_json(const json& j, Specification& p) { + from_json_key( j, - "startTime", - p.startTime, - "SessionRepresentation", - "int64_t", - "startTime"); - to_json_key( + "partitionBy", + p.partitionBy, + "Specification", + "List", + "partitionBy"); + from_json_key( j, - "systemProperties", - p.systemProperties, - "SessionRepresentation", - "Map", - "systemProperties"); - to_json_key( - j, - "catalogProperties", - p.catalogProperties, - "SessionRepresentation", - "Map>", - "catalogProperties"); - to_json_key( - j, - "unprocessedCatalogProperties", - p.unprocessedCatalogProperties, - "SessionRepresentation", - "Map>", - "unprocessedCatalogProperties"); - to_json_key( - j, - "roles", - p.roles, - "SessionRepresentation", - "Map", - "roles"); - to_json_key( - j, - "preparedStatements", - p.preparedStatements, - "SessionRepresentation", - "Map", - "preparedStatements"); - to_json_key( - j, - "sessionFunctions", - p.sessionFunctions, - "SessionRepresentation", - "Map", - "sessionFunctions"); -} - -void from_json(const json& j, SessionRepresentation& p) { - from_json_key( - j, "queryId", p.queryId, "SessionRepresentation", "String", "queryId"); - from_json_key( - j, - "transactionId", - p.transactionId, - "SessionRepresentation", - "TransactionId", - "transactionId"); - from_json_key( - j, - "clientTransactionSupport", - p.clientTransactionSupport, - "SessionRepresentation", - "bool", - "clientTransactionSupport"); - from_json_key(j, "user", p.user, "SessionRepresentation", "String", "user"); - from_json_key( - j, - "principal", - p.principal, - "SessionRepresentation", - "String", - "principal"); - from_json_key( - j, "source", p.source, "SessionRepresentation", "String", "source"); - from_json_key( - j, "catalog", p.catalog, "SessionRepresentation", "String", "catalog"); - from_json_key( - j, "schema", p.schema, "SessionRepresentation", "String", "schema"); - from_json_key( - j, - "traceToken", - p.traceToken, - "SessionRepresentation", - "String", - "traceToken"); - from_json_key( - j, - "timeZoneKey", - p.timeZoneKey, - "SessionRepresentation", - "TimeZoneKey", - "timeZoneKey"); - from_json_key( - j, "locale", p.locale, "SessionRepresentation", "Locale", "locale"); - from_json_key( - j, - "remoteUserAddress", - p.remoteUserAddress, - "SessionRepresentation", - "String", - "remoteUserAddress"); - from_json_key( - j, - "userAgent", - p.userAgent, - "SessionRepresentation", - "String", - "userAgent"); - from_json_key( - j, - "clientInfo", - p.clientInfo, - "SessionRepresentation", - "String", - "clientInfo"); - from_json_key( - j, - "clientTags", - p.clientTags, - "SessionRepresentation", - "List", - "clientTags"); - from_json_key( - j, - "resourceEstimates", - p.resourceEstimates, - "SessionRepresentation", - "ResourceEstimates", - "resourceEstimates"); - from_json_key( - j, - "startTime", - p.startTime, - "SessionRepresentation", - "int64_t", - "startTime"); - from_json_key( - j, - "systemProperties", - p.systemProperties, - "SessionRepresentation", - "Map", - "systemProperties"); - from_json_key( - 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, - "preparedStatements", - p.preparedStatements, - "SessionRepresentation", - "Map", - "preparedStatements"); - from_json_key( - j, - "sessionFunctions", - p.sessionFunctions, - "SessionRepresentation", - "Map", - "sessionFunctions"); + "orderingScheme", + p.orderingScheme, + "Specification", + "OrderingScheme", + "orderingScheme"); } } // namespace facebook::presto::protocol /* @@ -976,83 +741,16 @@ void from_json(const json& j, Lifespan& 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. - */ - -// TpchTransactionHandle is special since -// the corresponding class in Java is an enum. - -namespace facebook::presto::protocol { - -void to_json(json& j, const TpchTransactionHandle& p) { - j = json::array(); - j.push_back(p._type); - j.push_back(p.instance); -} - -void from_json(const json& j, TpchTransactionHandle& p) { - j[0].get_to(p._type); - j[1].get_to(p.instance); -} -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -// dependency TpchTransactionHandle - namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - if (type == "$remote") { - j = *std::static_pointer_cast(p); - return; - } - getConnectorProtocol(type).to_json(j, p); +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, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError( - std::string(e.what()) + - " ConnectorTransactionHandle ConnectorTransactionHandle"); - } - - if (type == "$remote") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - getConnectorProtocol(type).from_json(j, p); +void from_json(const json& j, SplitContext& p) { + from_json_key( + j, "cacheable", p.cacheable, "SplitContext", "bool", "cacheable"); } } // namespace facebook::presto::protocol /* @@ -1111,19 +809,7 @@ void from_json(const json& j, std::shared_ptr& p) { } // 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 -namespace facebook::presto::protocol { - -void to_json(json& j, const Split& p) { +void to_json(json& j, const Split& p) { j = json::object(); to_json_key( j, "connectorId", p.connectorId, "Split", "ConnectorId", "connectorId"); @@ -1242,37 +928,157 @@ void from_json(const json& j, TaskSource& p) { j, "noMoreSplits", p.noMoreSplits, "TaskSource", "bool", "noMoreSplits"); } } // 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; - getConnectorProtocol(type).to_json(j, p); +// 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 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"); - } - getConnectorProtocol(type).from_json(j, p); +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, "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 + 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 { + +void to_json(json& j, const HiveBucketProperty& p) { + j = json::object(); + to_json_key( + j, + "bucketedBy", + p.bucketedBy, + "HiveBucketProperty", + "List", + "bucketedBy"); + to_json_key( + j, + "bucketCount", + p.bucketCount, + "HiveBucketProperty", + "int", + "bucketCount"); + to_json_key( + j, + "sortedBy", + p.sortedBy, + "HiveBucketProperty", + "List", + "sortedBy"); + to_json_key( + j, + "bucketFunctionType", + p.bucketFunctionType, + "HiveBucketProperty", + "BucketFunctionType", + "bucketFunctionType"); + to_json_key(j, "types", p.types, "HiveBucketProperty", "List", "types"); +} + +void from_json(const json& j, HiveBucketProperty& p) { + from_json_key( + j, + "bucketedBy", + p.bucketedBy, + "HiveBucketProperty", + "List", + "bucketedBy"); + from_json_key( + j, + "bucketCount", + p.bucketCount, + "HiveBucketProperty", + "int", + "bucketCount"); + from_json_key( + j, + "sortedBy", + p.sortedBy, + "HiveBucketProperty", + "List", + "sortedBy"); + from_json_key( + j, + "bucketFunctionType", + p.bucketFunctionType, + "HiveBucketProperty", + "BucketFunctionType", + "bucketFunctionType"); + from_json_key( + j, "types", p.types, "HiveBucketProperty", "List", "types"); } } // namespace facebook::presto::protocol /* @@ -1289,7 +1095,7 @@ 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; } @@ -1297,578 +1103,515 @@ void to_json(json& j, const std::shared_ptr& p) { getConnectorProtocol(type).to_json(j, p); } -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()) + - " ConnectorTableLayoutHandle ConnectorTableLayoutHandle"); + throw ParseError(std::string(e.what()) + " ColumnHandle ColumnHandle"); } getConnectorProtocol(type).from_json(j, p); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const TableHandle& p) { +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, - "connectorId", - p.connectorId, - "TableHandle", - "ConnectorId", - "connectorId"); + j, "fileSize", p.fileSize, "HiveFileSplit", "int64_t", "fileSize"); to_json_key( j, - "connectorHandle", - p.connectorHandle, - "TableHandle", - "ConnectorTableHandle", - "connectorHandle"); + "fileModifiedTime", + p.fileModifiedTime, + "HiveFileSplit", + "int64_t", + "fileModifiedTime"); to_json_key( j, - "transaction", - p.transaction, - "TableHandle", - "ConnectorTransactionHandle", - "transaction"); + "extraFileInfo", + p.extraFileInfo, + "HiveFileSplit", + "String", + "extraFileInfo"); to_json_key( j, - "connectorTableLayout", - p.connectorTableLayout, - "TableHandle", - "ConnectorTableLayoutHandle", - "connectorTableLayout"); + "customSplitInfo", + p.customSplitInfo, + "HiveFileSplit", + "Map", + "customSplitInfo"); } -void from_json(const json& j, TableHandle& p) { +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, - "connectorId", - p.connectorId, - "TableHandle", - "ConnectorId", - "connectorId"); + "fileModifiedTime", + p.fileModifiedTime, + "HiveFileSplit", + "int64_t", + "fileModifiedTime"); from_json_key( j, - "connectorHandle", - p.connectorHandle, - "TableHandle", - "ConnectorTableHandle", - "connectorHandle"); - from_json_key( - j, - "transaction", - p.transaction, - "TableHandle", - "ConnectorTransactionHandle", - "transaction"); + "extraFileInfo", + p.extraFileInfo, + "HiveFileSplit", + "String", + "extraFileInfo"); from_json_key( j, - "connectorTableLayout", - p.connectorTableLayout, - "TableHandle", - "ConnectorTableLayoutHandle", - "connectorTableLayout"); + "customSplitInfo", + p.customSplitInfo, + "HiveFileSplit", + "Map", + "customSplitInfo"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -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, - "tableHandle", - p.tableHandle, - "DeleteScanInfo", - "TableHandle", - "tableHandle"); +// 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 { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; -void from_json(const json& j, DeleteScanInfo& p) { - from_json_key(j, "id", p.id, "DeleteScanInfo", "PlanNodeId", "id"); - from_json_key( - j, - "tableHandle", - p.tableHandle, - "DeleteScanInfo", - "TableHandle", - "tableHandle"); + 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 { -void to_json(json& j, const std::shared_ptr& p) { +void to_json(json& j, const std::shared_ptr& p) { if (p == nullptr) { return; } String type = p->_type; - if (type == "CreateHandle") { - j = *std::static_pointer_cast(p); + if (type == "call") { + j = *std::static_pointer_cast(p); return; } - if (type == "InsertHandle") { - j = *std::static_pointer_cast(p); + if (type == "constant") { + j = *std::static_pointer_cast(p); return; } - if (type == "DeleteHandle") { - j = *std::static_pointer_cast(p); + 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 ExecutionWriterTarget "); + throw TypeError(type + " no abstract type RowExpression "); } -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()) + - " ExecutionWriterTarget ExecutionWriterTarget"); + throw ParseError(std::string(e.what()) + " RowExpression RowExpression"); } - if (type == "CreateHandle") { - std::shared_ptr k = std::make_shared(); + if (type == "call") { + std::shared_ptr k = std::make_shared(); j.get_to(*k); - p = std::static_pointer_cast(k); + p = std::static_pointer_cast(k); return; } - if (type == "InsertHandle") { - std::shared_ptr k = std::make_shared(); + if (type == "constant") { + std::shared_ptr k = + std::make_shared(); j.get_to(*k); - p = std::static_pointer_cast(k); + p = std::static_pointer_cast(k); return; } - if (type == "DeleteHandle") { - std::shared_ptr k = std::make_shared(); + if (type == "special") { + std::shared_ptr k = + std::make_shared(); j.get_to(*k); - p = std::static_pointer_cast(k); + p = std::static_pointer_cast(k); + return; + } + if (type == "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 ExecutionWriterTarget "); + throw TypeError(type + " no abstract type RowExpression "); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +CallExpression::CallExpression() noexcept { + _type = "call"; +} -void to_json(json& j, const AnalyzeTableHandle& p) { +void to_json(json& j, const CallExpression& p) { j = json::object(); + j["@type"] = "call"; to_json_key( j, - "connectorId", - p.connectorId, - "AnalyzeTableHandle", - "ConnectorId", - "connectorId"); + "sourceLocation", + p.sourceLocation, + "CallExpression", + "SourceLocation", + "sourceLocation"); to_json_key( j, - "transactionHandle", - p.transactionHandle, - "AnalyzeTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "displayName", + p.displayName, + "CallExpression", + "String", + "displayName"); to_json_key( j, - "connectorHandle", - p.connectorHandle, - "AnalyzeTableHandle", - "ConnectorTableHandle", - "connectorHandle"); + "functionHandle", + p.functionHandle, + "CallExpression", + "FunctionHandle", + "functionHandle"); + to_json_key( + j, "returnType", p.returnType, "CallExpression", "Type", "returnType"); + to_json_key( + j, + "arguments", + p.arguments, + "CallExpression", + "List>", + "arguments"); } -void from_json(const json& j, AnalyzeTableHandle& p) { +void from_json(const json& j, CallExpression& p) { + p._type = j["@type"]; from_json_key( j, - "connectorId", - p.connectorId, - "AnalyzeTableHandle", - "ConnectorId", - "connectorId"); + "sourceLocation", + p.sourceLocation, + "CallExpression", + "SourceLocation", + "sourceLocation"); from_json_key( j, - "transactionHandle", - p.transactionHandle, - "AnalyzeTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "displayName", + p.displayName, + "CallExpression", + "String", + "displayName"); from_json_key( j, - "connectorHandle", - p.connectorHandle, - "AnalyzeTableHandle", - "ConnectorTableHandle", - "connectorHandle"); + "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 TableWriteInfo& 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, - "writerTarget", - p.writerTarget, - "TableWriteInfo", - "ExecutionWriterTarget", - "writerTarget"); + "filter", + p.filter, + "Aggregation", + "std::shared_ptr", + "filter"); + to_json_key( + 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, - "analyzeTableHandle", - p.analyzeTableHandle, - "TableWriteInfo", - "AnalyzeTableHandle", - "analyzeTableHandle"); + "functionHandle", + p.functionHandle, + "Aggregation", + "FunctionHandle", + "functionHandle"); to_json_key( j, - "deleteScanInfo", - p.deleteScanInfo, - "TableWriteInfo", - "DeleteScanInfo", - "deleteScanInfo"); + "arguments", + p.arguments, + "Aggregation", + "List>", + "arguments"); } -void from_json(const json& j, TableWriteInfo& p) { +void from_json(const json& j, Aggregation& p) { + from_json_key(j, "call", p.call, "Aggregation", "CallExpression", "call"); from_json_key( j, - "writerTarget", - p.writerTarget, - "TableWriteInfo", - "ExecutionWriterTarget", - "writerTarget"); + "filter", + p.filter, + "Aggregation", + "std::shared_ptr", + "filter"); + from_json_key( + 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, - "analyzeTableHandle", - p.analyzeTableHandle, - "TableWriteInfo", - "AnalyzeTableHandle", - "analyzeTableHandle"); + "functionHandle", + p.functionHandle, + "Aggregation", + "FunctionHandle", + "functionHandle"); from_json_key( j, - "deleteScanInfo", - p.deleteScanInfo, - "TableWriteInfo", - "DeleteScanInfo", - "deleteScanInfo"); + "arguments", + p.arguments, + "Aggregation", + "List>", + "arguments"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair BufferType_enum_table[] = - { // NOLINT: cert-err58-cpp - {BufferType::PARTITIONED, "PARTITIONED"}, - {BufferType::BROADCAST, "BROADCAST"}, - {BufferType::ARBITRARY, "ARBITRARY"}, - {BufferType::DISCARDING, "DISCARDING"}, - {BufferType::SPOOLING, "SPOOLING"}}; -void to_json(json& j, const BufferType& e) { - static_assert(std::is_enum::value, "BufferType must be an enum!"); - const auto* it = std::find_if( - std::begin(BufferType_enum_table), - std::end(BufferType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(BufferType_enum_table)) - ? it - : std::begin(BufferType_enum_table)) - ->second; -} -void from_json(const json& j, BufferType& e) { - static_assert(std::is_enum::value, "BufferType must be an enum!"); - const auto* it = std::find_if( - std::begin(BufferType_enum_table), - std::end(BufferType_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(BufferType_enum_table)) - ? it - : std::begin(BufferType_enum_table)) - ->first; +HiveColumnHandle::HiveColumnHandle() noexcept { + _type = "hive"; } -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -void to_json(json& j, const OutputBuffers& p) { +void to_json(json& j, const HiveColumnHandle& p) { j = json::object(); - to_json_key(j, "type", p.type, "OutputBuffers", "BufferType", "type"); - to_json_key(j, "version", p.version, "OutputBuffers", "int64_t", "version"); + 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, - "noMoreBufferIds", - p.noMoreBufferIds, - "OutputBuffers", - "bool", - "noMoreBufferIds"); + "typeSignature", + p.typeSignature, + "HiveColumnHandle", + "TypeSignature", + "typeSignature"); to_json_key( j, - "buffers", - p.buffers, - "OutputBuffers", - "Map", - "buffers"); + "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, + "requiredSubfields", + p.requiredSubfields, + "HiveColumnHandle", + "List", + "requiredSubfields"); + to_json_key( + j, + "partialAggregation", + p.partialAggregation, + "HiveColumnHandle", + "Aggregation", + "partialAggregation"); } -void from_json(const json& j, OutputBuffers& p) { - from_json_key(j, "type", p.type, "OutputBuffers", "BufferType", "type"); - from_json_key(j, "version", p.version, "OutputBuffers", "int64_t", "version"); +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, - "noMoreBufferIds", - p.noMoreBufferIds, - "OutputBuffers", - "bool", - "noMoreBufferIds"); + "typeSignature", + p.typeSignature, + "HiveColumnHandle", + "TypeSignature", + "typeSignature"); from_json_key( j, - "buffers", - p.buffers, - "OutputBuffers", - "Map", - "buffers"); + "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"); + from_json_key( + j, + "requiredSubfields", + p.requiredSubfields, + "HiveColumnHandle", + "List", + "requiredSubfields"); + from_json_key( + j, + "partialAggregation", + p.partialAggregation, + "HiveColumnHandle", + "Aggregation", + "partialAggregation"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const TaskUpdateRequest& p) { +void to_json(json& j, const BucketConversion& p) { j = json::object(); to_json_key( j, - "session", - p.session, - "TaskUpdateRequest", - "SessionRepresentation", - "session"); + "tableBucketCount", + p.tableBucketCount, + "BucketConversion", + "int", + "tableBucketCount"); to_json_key( j, - "extraCredentials", - p.extraCredentials, - "TaskUpdateRequest", - "Map", - "extraCredentials"); - to_json_key( - j, "fragment", p.fragment, "TaskUpdateRequest", "String", "fragment"); + "partitionBucketCount", + p.partitionBucketCount, + "BucketConversion", + "int", + "partitionBucketCount"); to_json_key( j, - "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, TaskUpdateRequest& p) { - from_json_key( - j, - "session", - p.session, - "TaskUpdateRequest", - "SessionRepresentation", - "session"); - from_json_key( - j, - "extraCredentials", - p.extraCredentials, - "TaskUpdateRequest", - "Map", - "extraCredentials"); - from_json_key( - j, "fragment", p.fragment, "TaskUpdateRequest", "String", "fragment"); - from_json_key( - j, - "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 { -// 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"); + "bucketColumnHandles", + p.bucketColumnHandles, + "BucketConversion", + "List", + "bucketColumnHandles"); } -void from_json(const json& j, SortingColumn& p) { +void from_json(const json& j, BucketConversion& p) { from_json_key( - 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 - 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 { - -void to_json(json& j, const HiveBucketProperty& p) { - j = json::object(); - to_json_key( - j, - "bucketedBy", - p.bucketedBy, - "HiveBucketProperty", - "List", - "bucketedBy"); - to_json_key( j, - "bucketCount", - p.bucketCount, - "HiveBucketProperty", + "tableBucketCount", + p.tableBucketCount, + "BucketConversion", "int", - "bucketCount"); - to_json_key( - j, - "sortedBy", - p.sortedBy, - "HiveBucketProperty", - "List", - "sortedBy"); - to_json_key( - j, - "bucketFunctionType", - p.bucketFunctionType, - "HiveBucketProperty", - "BucketFunctionType", - "bucketFunctionType"); - to_json_key(j, "types", p.types, "HiveBucketProperty", "List", "types"); -} - -void from_json(const json& j, HiveBucketProperty& p) { - from_json_key( - j, - "bucketedBy", - p.bucketedBy, - "HiveBucketProperty", - "List", - "bucketedBy"); + "tableBucketCount"); from_json_key( j, - "bucketCount", - p.bucketCount, - "HiveBucketProperty", + "partitionBucketCount", + p.partitionBucketCount, + "BucketConversion", "int", - "bucketCount"); - from_json_key( - j, - "sortedBy", - p.sortedBy, - "HiveBucketProperty", - "List", - "sortedBy"); + "partitionBucketCount"); from_json_key( j, - "bucketFunctionType", - p.bucketFunctionType, - "HiveBucketProperty", - "BucketFunctionType", - "bucketFunctionType"); - from_json_key( - j, "types", p.types, "HiveBucketProperty", "List", "types"); + "bucketColumnHandles", + p.bucketColumnHandles, + "BucketConversion", + "List", + "bucketColumnHandles"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { @@ -1980,5138 +1723,6097 @@ void from_json(const json& j, Storage& p) { } } // 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 { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair RuntimeUnit_enum_table[] = - { // NOLINT: cert-err58-cpp - {RuntimeUnit::NONE, "NONE"}, - {RuntimeUnit::NANO, "NANO"}, - {RuntimeUnit::BYTE, "BYTE"}}; -void to_json(json& j, const RuntimeUnit& e) { +static const std::pair + 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, "RuntimeUnit must be an enum!"); + std::is_enum::value, + "NodeSelectionStrategy must be an enum!"); const auto* it = std::find_if( - std::begin(RuntimeUnit_enum_table), - std::end(RuntimeUnit_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(NodeSelectionStrategy_enum_table), + std::end(NodeSelectionStrategy_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(RuntimeUnit_enum_table)) + j = ((it != std::end(NodeSelectionStrategy_enum_table)) ? it - : std::begin(RuntimeUnit_enum_table)) + : std::begin(NodeSelectionStrategy_enum_table)) ->second; } -void from_json(const json& j, RuntimeUnit& e) { +void from_json(const json& j, NodeSelectionStrategy& e) { static_assert( - std::is_enum::value, "RuntimeUnit must be an enum!"); + std::is_enum::value, + "NodeSelectionStrategy must be an enum!"); const auto* it = std::find_if( - std::begin(RuntimeUnit_enum_table), - std::end(RuntimeUnit_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(NodeSelectionStrategy_enum_table), + std::end(NodeSelectionStrategy_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(RuntimeUnit_enum_table)) + e = ((it != std::end(NodeSelectionStrategy_enum_table)) ? it - : std::begin(RuntimeUnit_enum_table)) + : std::begin(NodeSelectionStrategy_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const RuntimeMetric& p) { +void to_json(json& j, const Column& p) { j = json::object(); - to_json_key(j, "name", p.name, "RuntimeMetric", "String", "name"); - to_json_key(j, "unit", p.unit, "RuntimeMetric", "RuntimeUnit", "unit"); - to_json_key(j, "sum", p.sum, "RuntimeMetric", "int64_t", "sum"); - to_json_key(j, "count", p.count, "RuntimeMetric", "int64_t", "count"); - to_json_key(j, "max", p.max, "RuntimeMetric", "int64_t", "max"); - to_json_key(j, "min", p.min, "RuntimeMetric", "int64_t", "min"); + to_json_key(j, "name", p.name, "Column", "String", "name"); + to_json_key(j, "type", p.type, "Column", "String", "type"); } -void from_json(const json& j, RuntimeMetric& p) { - from_json_key(j, "name", p.name, "RuntimeMetric", "String", "name"); - from_json_key(j, "unit", p.unit, "RuntimeMetric", "RuntimeUnit", "unit"); - from_json_key(j, "sum", p.sum, "RuntimeMetric", "int64_t", "sum"); - from_json_key(j, "count", p.count, "RuntimeMetric", "int64_t", "count"); - from_json_key(j, "max", p.max, "RuntimeMetric", "int64_t", "max"); - from_json_key(j, "min", p.min, "RuntimeMetric", "int64_t", "min"); +void from_json(const json& j, 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 { -// 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 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, 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; + +void from_json(const json& j, TableToPartitionMapping& p) { + from_json_key( + j, + "tableToPartitionColumns", + p.tableToPartitionColumns, + "TableToPartitionMapping", + "Map", + "tableToPartitionColumns"); + from_json_key( + j, + "partitionSchemaDifference", + p.partitionSchemaDifference, + "TableToPartitionMapping", + "Map", + "partitionSchemaDifference"); +} +} // namespace facebook::presto::protocol +/* + * 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 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[] = +static const std::pair CacheQuotaScope_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) { + {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, "TypeCategory must be an enum!"); + std::is_enum::value, "CacheQuotaScope 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 { + 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(TypeCategory_enum_table)) + j = ((it != std::end(CacheQuotaScope_enum_table)) ? it - : std::begin(TypeCategory_enum_table)) + : std::begin(CacheQuotaScope_enum_table)) ->second; } -void from_json(const json& j, TypeCategory& e) { +void from_json(const json& j, CacheQuotaScope& e) { static_assert( - std::is_enum::value, "TypeCategory must be an enum!"); + std::is_enum::value, "CacheQuotaScope 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 { + 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(TypeCategory_enum_table)) + e = ((it != std::end(CacheQuotaScope_enum_table)) ? it - : std::begin(TypeCategory_enum_table)) + : std::begin(CacheQuotaScope_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const ColumnIdentity& p) { +void to_json(json& j, const CacheQuotaRequirement& 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, - "typeCategory", - p.typeCategory, - "ColumnIdentity", - "TypeCategory", - "typeCategory"); + "cacheQuotaScope", + p.cacheQuotaScope, + "CacheQuotaRequirement", + "CacheQuotaScope", + "cacheQuotaScope"); to_json_key( - j, - "children", - p.children, - "ColumnIdentity", - "List", - "children"); + j, "quota", p.quota, "CacheQuotaRequirement", "DataSize", "quota"); } -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"); +void from_json(const json& j, CacheQuotaRequirement& p) { from_json_key( j, - "typeCategory", - p.typeCategory, - "ColumnIdentity", - "TypeCategory", - "typeCategory"); + "cacheQuotaScope", + p.cacheQuotaScope, + "CacheQuotaRequirement", + "CacheQuotaScope", + "cacheQuotaScope"); from_json_key( - j, - "children", - p.children, - "ColumnIdentity", - "List", - "children"); -} -} // 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; + j, "quota", p.quota, "CacheQuotaRequirement", "DataSize", "quota"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -IcebergColumnHandle::IcebergColumnHandle() noexcept { - _type = "hive-iceberg"; -} -void to_json(json& j, const IcebergColumnHandle& p) { +void to_json(json& j, const DwrfEncryptionMetadata& p) { j = json::object(); - j["@type"] = "hive-iceberg"; to_json_key( j, - "columnIdentity", - p.columnIdentity, - "IcebergColumnHandle", - "ColumnIdentity", - "columnIdentity"); - to_json_key(j, "type", p.type, "IcebergColumnHandle", "Type", "type"); + "fieldToKeyData", + p.fieldToKeyData, + "DwrfEncryptionMetadata", + "Map", + "fieldToKeyData"); to_json_key( - j, "comment", p.comment, "IcebergColumnHandle", "String", "comment"); + j, + "extraMetadata", + p.extraMetadata, + "DwrfEncryptionMetadata", + "Map", + "extraMetadata"); to_json_key( j, - "columnType", - p.columnType, - "IcebergColumnHandle", - "ColumnType", - "columnType"); + "encryptionAlgorithm", + p.encryptionAlgorithm, + "DwrfEncryptionMetadata", + "String", + "encryptionAlgorithm"); to_json_key( j, - "requiredSubfields", - p.requiredSubfields, - "IcebergColumnHandle", - "List", - "requiredSubfields"); + "encryptionProvider", + p.encryptionProvider, + "DwrfEncryptionMetadata", + "String", + "encryptionProvider"); } -void from_json(const json& j, IcebergColumnHandle& p) { - p._type = j["@type"]; +void from_json(const json& j, DwrfEncryptionMetadata& p) { from_json_key( j, - "columnIdentity", - p.columnIdentity, - "IcebergColumnHandle", - "ColumnIdentity", - "columnIdentity"); - from_json_key(j, "type", p.type, "IcebergColumnHandle", "Type", "type"); + "fieldToKeyData", + p.fieldToKeyData, + "DwrfEncryptionMetadata", + "Map", + "fieldToKeyData"); from_json_key( - j, "comment", p.comment, "IcebergColumnHandle", "String", "comment"); + j, + "extraMetadata", + p.extraMetadata, + "DwrfEncryptionMetadata", + "Map", + "extraMetadata"); from_json_key( j, - "columnType", - p.columnType, - "IcebergColumnHandle", - "ColumnType", - "columnType"); + "encryptionAlgorithm", + p.encryptionAlgorithm, + "DwrfEncryptionMetadata", + "String", + "encryptionAlgorithm"); from_json_key( j, - "requiredSubfields", - p.requiredSubfields, - "IcebergColumnHandle", - "List", - "requiredSubfields"); + "encryptionProvider", + p.encryptionProvider, + "DwrfEncryptionMetadata", + "String", + "encryptionProvider"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const ChangelogSplitInfo& p) { +void to_json(json& j, const EncryptionInformation& p) { j = json::object(); to_json_key( j, - "operation", - p.operation, - "ChangelogSplitInfo", - "ChangelogOperation", - "operation"); - to_json_key( - j, "ordinal", p.ordinal, "ChangelogSplitInfo", "int64_t", "ordinal"); - to_json_key( - j, - "snapshotId", - p.snapshotId, - "ChangelogSplitInfo", - "int64_t", - "snapshotId"); - to_json_key( - j, - "icebergColumns", - p.icebergColumns, - "ChangelogSplitInfo", - "List", - "icebergColumns"); + "dwrfEncryptionMetadata", + p.dwrfEncryptionMetadata, + "EncryptionInformation", + "DwrfEncryptionMetadata", + "dwrfEncryptionMetadata"); } -void from_json(const json& j, ChangelogSplitInfo& p) { - from_json_key( - j, - "operation", - p.operation, - "ChangelogSplitInfo", - "ChangelogOperation", - "operation"); - from_json_key( - j, "ordinal", p.ordinal, "ChangelogSplitInfo", "int64_t", "ordinal"); - from_json_key( - j, - "snapshotId", - p.snapshotId, - "ChangelogSplitInfo", - "int64_t", - "snapshotId"); +void from_json(const json& j, EncryptionInformation& p) { from_json_key( j, - "icebergColumns", - p.icebergColumns, - "ChangelogSplitInfo", - "List", - "icebergColumns"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const DistributionSnapshot& p) { - j = json::object(); - to_json_key( - j, "maxError", p.maxError, "DistributionSnapshot", "double", "maxError"); - to_json_key(j, "count", p.count, "DistributionSnapshot", "double", "count"); - to_json_key(j, "total", p.total, "DistributionSnapshot", "double", "total"); - to_json_key(j, "p01", p.p01, "DistributionSnapshot", "int64_t", "p01"); - to_json_key(j, "p05", p.p05, "DistributionSnapshot", "int64_t", "p05"); - to_json_key(j, "p10", p.p10, "DistributionSnapshot", "int64_t", "p10"); - to_json_key(j, "p25", p.p25, "DistributionSnapshot", "int64_t", "p25"); - to_json_key(j, "p50", p.p50, "DistributionSnapshot", "int64_t", "p50"); - to_json_key(j, "p75", p.p75, "DistributionSnapshot", "int64_t", "p75"); - to_json_key(j, "p90", p.p90, "DistributionSnapshot", "int64_t", "p90"); - to_json_key(j, "p95", p.p95, "DistributionSnapshot", "int64_t", "p95"); - to_json_key(j, "p99", p.p99, "DistributionSnapshot", "int64_t", "p99"); - to_json_key(j, "min", p.min, "DistributionSnapshot", "int64_t", "min"); - to_json_key(j, "max", p.max, "DistributionSnapshot", "int64_t", "max"); - to_json_key(j, "avg", p.avg, "DistributionSnapshot", "double", "avg"); -} - -void from_json(const json& j, DistributionSnapshot& p) { - from_json_key( - j, "maxError", p.maxError, "DistributionSnapshot", "double", "maxError"); - from_json_key(j, "count", p.count, "DistributionSnapshot", "double", "count"); - from_json_key(j, "total", p.total, "DistributionSnapshot", "double", "total"); - from_json_key(j, "p01", p.p01, "DistributionSnapshot", "int64_t", "p01"); - from_json_key(j, "p05", p.p05, "DistributionSnapshot", "int64_t", "p05"); - from_json_key(j, "p10", p.p10, "DistributionSnapshot", "int64_t", "p10"); - from_json_key(j, "p25", p.p25, "DistributionSnapshot", "int64_t", "p25"); - from_json_key(j, "p50", p.p50, "DistributionSnapshot", "int64_t", "p50"); - from_json_key(j, "p75", p.p75, "DistributionSnapshot", "int64_t", "p75"); - from_json_key(j, "p90", p.p90, "DistributionSnapshot", "int64_t", "p90"); - from_json_key(j, "p95", p.p95, "DistributionSnapshot", "int64_t", "p95"); - from_json_key(j, "p99", p.p99, "DistributionSnapshot", "int64_t", "p99"); - from_json_key(j, "min", p.min, "DistributionSnapshot", "int64_t", "min"); - from_json_key(j, "max", p.max, "DistributionSnapshot", "int64_t", "max"); - from_json_key(j, "avg", p.avg, "DistributionSnapshot", "double", "avg"); + "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 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; +HiveSplit::HiveSplit() noexcept { + _type = "hive"; } -} // 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 HiveSplit& p) { j = json::object(); - to_json_key(j, "stageId", p.stageId, "OperatorStats", "int", "stageId"); - to_json_key( - j, - "stageExecutionId", - p.stageExecutionId, - "OperatorStats", - "int", - "stageExecutionId"); + j["@type"] = "hive"; to_json_key( - j, "pipelineId", p.pipelineId, "OperatorStats", "int", "pipelineId"); + 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, "operatorId", p.operatorId, "OperatorStats", "int", "operatorId"); + j, + "partitionName", + p.partitionName, + "HiveSplit", + "String", + "partitionName"); + to_json_key(j, "storage", p.storage, "HiveSplit", "Storage", "storage"); to_json_key( j, - "planNodeId", - p.planNodeId, - "OperatorStats", - "PlanNodeId", - "planNodeId"); - to_json_key( - j, - "operatorType", - p.operatorType, - "OperatorStats", - "String", - "operatorType"); + "partitionKeys", + p.partitionKeys, + "HiveSplit", + "List", + "partitionKeys"); to_json_key( j, - "totalDrivers", - p.totalDrivers, - "OperatorStats", - "int64_t", - "totalDrivers"); + "addresses", + p.addresses, + "HiveSplit", + "List", + "addresses"); to_json_key( j, - "addInputCalls", - p.addInputCalls, - "OperatorStats", - "int64_t", - "addInputCalls"); + "readBucketNumber", + p.readBucketNumber, + "HiveSplit", + "int", + "readBucketNumber"); to_json_key( j, - "addInputWall", - p.addInputWall, - "OperatorStats", - "Duration", - "addInputWall"); + "tableBucketNumber", + p.tableBucketNumber, + "HiveSplit", + "int", + "tableBucketNumber"); to_json_key( j, - "addInputCpu", - p.addInputCpu, - "OperatorStats", - "Duration", - "addInputCpu"); + "nodeSelectionStrategy", + p.nodeSelectionStrategy, + "HiveSplit", + "NodeSelectionStrategy", + "nodeSelectionStrategy"); to_json_key( j, - "addInputAllocation", - p.addInputAllocation, - "OperatorStats", - "DataSize", - "addInputAllocation"); + "partitionDataColumnCount", + p.partitionDataColumnCount, + "HiveSplit", + "int", + "partitionDataColumnCount"); to_json_key( j, - "rawInputDataSize", - p.rawInputDataSize, - "OperatorStats", - "DataSize", - "rawInputDataSize"); + "tableToPartitionMapping", + p.tableToPartitionMapping, + "HiveSplit", + "TableToPartitionMapping", + "tableToPartitionMapping"); to_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "OperatorStats", - "int64_t", - "rawInputPositions"); + "bucketConversion", + p.bucketConversion, + "HiveSplit", + "BucketConversion", + "bucketConversion"); to_json_key( j, - "inputDataSize", - p.inputDataSize, - "OperatorStats", - "DataSize", - "inputDataSize"); + "s3SelectPushdownEnabled", + p.s3SelectPushdownEnabled, + "HiveSplit", + "bool", + "s3SelectPushdownEnabled"); to_json_key( j, - "inputPositions", - p.inputPositions, - "OperatorStats", - "int64_t", - "inputPositions"); + "cacheQuota", + p.cacheQuota, + "HiveSplit", + "CacheQuotaRequirement", + "cacheQuota"); to_json_key( j, - "sumSquaredInputPositions", - p.sumSquaredInputPositions, - "OperatorStats", - "double", - "sumSquaredInputPositions"); + "encryptionMetadata", + p.encryptionMetadata, + "HiveSplit", + "EncryptionInformation", + "encryptionMetadata"); to_json_key( j, - "getOutputCalls", - p.getOutputCalls, - "OperatorStats", - "int64_t", - "getOutputCalls"); + "redundantColumnDomains", + p.redundantColumnDomains, + "HiveSplit", + "List>", + "redundantColumnDomains"); to_json_key( j, - "getOutputWall", - p.getOutputWall, - "OperatorStats", - "Duration", - "getOutputWall"); + "splitWeight", + p.splitWeight, + "HiveSplit", + "SplitWeight", + "splitWeight"); to_json_key( j, - "getOutputCpu", - p.getOutputCpu, - "OperatorStats", - "Duration", - "getOutputCpu"); - to_json_key( + "rowIdPartitionComponent", + p.rowIdPartitionComponent, + "HiveSplit", + "String", + "rowIdPartitionComponent"); +} + +void from_json(const json& j, HiveSplit& p) { + p._type = j["@type"]; + from_json_key( + j, "fileSplit", p.fileSplit, "HiveSplit", "HiveFileSplit", "fileSplit"); + from_json_key(j, "database", p.database, "HiveSplit", "String", "database"); + from_json_key(j, "table", p.table, "HiveSplit", "String", "table"); + from_json_key( j, - "getOutputAllocation", - p.getOutputAllocation, - "OperatorStats", - "DataSize", - "getOutputAllocation"); - to_json_key( + "partitionName", + p.partitionName, + "HiveSplit", + "String", + "partitionName"); + from_json_key(j, "storage", p.storage, "HiveSplit", "Storage", "storage"); + from_json_key( j, - "outputDataSize", - p.outputDataSize, - "OperatorStats", - "DataSize", - "outputDataSize"); - to_json_key( + "partitionKeys", + p.partitionKeys, + "HiveSplit", + "List", + "partitionKeys"); + from_json_key( j, - "outputPositions", - p.outputPositions, - "OperatorStats", - "int64_t", - "outputPositions"); - to_json_key( + "addresses", + p.addresses, + "HiveSplit", + "List", + "addresses"); + from_json_key( j, - "physicalWrittenDataSize", - p.physicalWrittenDataSize, - "OperatorStats", - "DataSize", - "physicalWrittenDataSize"); - to_json_key( + "readBucketNumber", + p.readBucketNumber, + "HiveSplit", + "int", + "readBucketNumber"); + from_json_key( j, - "additionalCpu", - p.additionalCpu, - "OperatorStats", - "Duration", - "additionalCpu"); - to_json_key( + "tableBucketNumber", + p.tableBucketNumber, + "HiveSplit", + "int", + "tableBucketNumber"); + from_json_key( j, - "blockedWall", - p.blockedWall, - "OperatorStats", - "Duration", - "blockedWall"); - to_json_key( + "nodeSelectionStrategy", + p.nodeSelectionStrategy, + "HiveSplit", + "NodeSelectionStrategy", + "nodeSelectionStrategy"); + from_json_key( j, - "finishCalls", - p.finishCalls, - "OperatorStats", - "int64_t", - "finishCalls"); - to_json_key( - j, "finishWall", p.finishWall, "OperatorStats", "Duration", "finishWall"); - to_json_key( - j, "finishCpu", p.finishCpu, "OperatorStats", "Duration", "finishCpu"); - to_json_key( + "partitionDataColumnCount", + p.partitionDataColumnCount, + "HiveSplit", + "int", + "partitionDataColumnCount"); + from_json_key( j, - "finishAllocation", - p.finishAllocation, - "OperatorStats", - "DataSize", - "finishAllocation"); - to_json_key( + "tableToPartitionMapping", + p.tableToPartitionMapping, + "HiveSplit", + "TableToPartitionMapping", + "tableToPartitionMapping"); + from_json_key( j, - "userMemoryReservation", - p.userMemoryReservation, - "OperatorStats", - "DataSize", - "userMemoryReservation"); - to_json_key( - j, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "OperatorStats", - "DataSize", - "revocableMemoryReservation"); - to_json_key( + "bucketConversion", + p.bucketConversion, + "HiveSplit", + "BucketConversion", + "bucketConversion"); + from_json_key( j, - "systemMemoryReservation", - p.systemMemoryReservation, - "OperatorStats", - "DataSize", - "systemMemoryReservation"); - to_json_key( + "s3SelectPushdownEnabled", + p.s3SelectPushdownEnabled, + "HiveSplit", + "bool", + "s3SelectPushdownEnabled"); + from_json_key( j, - "peakUserMemoryReservation", - p.peakUserMemoryReservation, - "OperatorStats", - "DataSize", - "peakUserMemoryReservation"); - to_json_key( + "cacheQuota", + p.cacheQuota, + "HiveSplit", + "CacheQuotaRequirement", + "cacheQuota"); + from_json_key( j, - "peakSystemMemoryReservation", - p.peakSystemMemoryReservation, - "OperatorStats", - "DataSize", - "peakSystemMemoryReservation"); - to_json_key( + "encryptionMetadata", + p.encryptionMetadata, + "HiveSplit", + "EncryptionInformation", + "encryptionMetadata"); + from_json_key( j, - "peakTotalMemoryReservation", - p.peakTotalMemoryReservation, - "OperatorStats", - "DataSize", - "peakTotalMemoryReservation"); - to_json_key( + "redundantColumnDomains", + p.redundantColumnDomains, + "HiveSplit", + "List>", + "redundantColumnDomains"); + from_json_key( j, - "spilledDataSize", - p.spilledDataSize, - "OperatorStats", - "DataSize", - "spilledDataSize"); - to_json_key( + "splitWeight", + p.splitWeight, + "HiveSplit", + "SplitWeight", + "splitWeight"); + from_json_key( j, - "blockedReason", - p.blockedReason, - "OperatorStats", - "BlockedReason", - "blockedReason"); - to_json_key(j, "info", p.info, "OperatorStats", "OperatorInfo", "info"); + "rowIdPartitionComponent", + p.rowIdPartitionComponent, + "HiveSplit", + "String", + "rowIdPartitionComponent"); +} +} // 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 PageBufferInfo& p) { + j = json::object(); to_json_key( - j, - "runtimeStats", - p.runtimeStats, - "OperatorStats", - "RuntimeStats", - "runtimeStats"); + j, "partition", p.partition, "PageBufferInfo", "int", "partition"); to_json_key( j, - "nullJoinBuildKeyCount", - p.nullJoinBuildKeyCount, - "OperatorStats", + "bufferedPages", + p.bufferedPages, + "PageBufferInfo", "int64_t", - "nullJoinBuildKeyCount"); + "bufferedPages"); to_json_key( j, - "joinBuildKeyCount", - p.joinBuildKeyCount, - "OperatorStats", + "bufferedBytes", + p.bufferedBytes, + "PageBufferInfo", "int64_t", - "joinBuildKeyCount"); + "bufferedBytes"); to_json_key( - j, - "nullJoinProbeKeyCount", - p.nullJoinProbeKeyCount, - "OperatorStats", - "int64_t", - "nullJoinProbeKeyCount"); + j, "rowsAdded", p.rowsAdded, "PageBufferInfo", "int64_t", "rowsAdded"); to_json_key( - j, - "joinProbeKeyCount", - p.joinProbeKeyCount, - "OperatorStats", - "int64_t", - "joinProbeKeyCount"); + j, "pagesAdded", p.pagesAdded, "PageBufferInfo", "int64_t", "pagesAdded"); } -void from_json(const json& j, OperatorStats& p) { - from_json_key(j, "stageId", p.stageId, "OperatorStats", "int", "stageId"); +void from_json(const json& j, PageBufferInfo& p) { + from_json_key( + j, "partition", p.partition, "PageBufferInfo", "int", "partition"); from_json_key( j, - "stageExecutionId", - p.stageExecutionId, - "OperatorStats", - "int", - "stageExecutionId"); + "bufferedPages", + p.bufferedPages, + "PageBufferInfo", + "int64_t", + "bufferedPages"); from_json_key( - j, "pipelineId", p.pipelineId, "OperatorStats", "int", "pipelineId"); + j, + "bufferedBytes", + p.bufferedBytes, + "PageBufferInfo", + "int64_t", + "bufferedBytes"); from_json_key( - j, "operatorId", p.operatorId, "OperatorStats", "int", "operatorId"); + 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, "bufferId", p.bufferId, "BufferInfo", "OutputBufferId", "bufferId"); + to_json_key(j, "finished", p.finished, "BufferInfo", "bool", "finished"); + to_json_key( j, - "planNodeId", - p.planNodeId, - "OperatorStats", - "PlanNodeId", - "planNodeId"); - from_json_key( + "bufferedPages", + p.bufferedPages, + "BufferInfo", + "int", + "bufferedPages"); + to_json_key( + j, "pagesSent", p.pagesSent, "BufferInfo", "int64_t", "pagesSent"); + to_json_key( j, - "operatorType", - p.operatorType, - "OperatorStats", - "String", - "operatorType"); + "pageBufferInfo", + p.pageBufferInfo, + "BufferInfo", + "PageBufferInfo", + "pageBufferInfo"); +} + +void from_json(const json& j, BufferInfo& p) { from_json_key( - j, - "totalDrivers", - p.totalDrivers, - "OperatorStats", - "int64_t", - "totalDrivers"); + j, "bufferId", p.bufferId, "BufferInfo", "OutputBufferId", "bufferId"); + from_json_key(j, "finished", p.finished, "BufferInfo", "bool", "finished"); from_json_key( j, - "addInputCalls", - p.addInputCalls, - "OperatorStats", - "int64_t", - "addInputCalls"); + "bufferedPages", + p.bufferedPages, + "BufferInfo", + "int", + "bufferedPages"); + from_json_key( + j, "pagesSent", p.pagesSent, "BufferInfo", "int64_t", "pagesSent"); from_json_key( j, - "addInputWall", - p.addInputWall, - "OperatorStats", - "Duration", - "addInputWall"); - from_json_key( - j, - "addInputCpu", - p.addInputCpu, - "OperatorStats", - "Duration", - "addInputCpu"); - from_json_key( + "pageBufferInfo", + p.pageBufferInfo, + "BufferInfo", + "PageBufferInfo", + "pageBufferInfo"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const OutputBufferInfo& p) { + j = json::object(); + to_json_key(j, "type", p.type, "OutputBufferInfo", "String", "type"); + to_json_key(j, "state", p.state, "OutputBufferInfo", "BufferState", "state"); + to_json_key( j, - "addInputAllocation", - p.addInputAllocation, - "OperatorStats", - "DataSize", - "addInputAllocation"); - from_json_key( + "canAddBuffers", + p.canAddBuffers, + "OutputBufferInfo", + "bool", + "canAddBuffers"); + to_json_key( j, - "rawInputDataSize", - p.rawInputDataSize, - "OperatorStats", - "DataSize", - "rawInputDataSize"); - from_json_key( + "canAddPages", + p.canAddPages, + "OutputBufferInfo", + "bool", + "canAddPages"); + to_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "OperatorStats", + "totalBufferedBytes", + p.totalBufferedBytes, + "OutputBufferInfo", "int64_t", - "rawInputPositions"); - from_json_key( - j, - "inputDataSize", - p.inputDataSize, - "OperatorStats", - "DataSize", - "inputDataSize"); - from_json_key( + "totalBufferedBytes"); + to_json_key( j, - "inputPositions", - p.inputPositions, - "OperatorStats", + "totalBufferedPages", + p.totalBufferedPages, + "OutputBufferInfo", "int64_t", - "inputPositions"); - from_json_key( + "totalBufferedPages"); + to_json_key( j, - "sumSquaredInputPositions", - p.sumSquaredInputPositions, - "OperatorStats", - "double", - "sumSquaredInputPositions"); - from_json_key( + "totalRowsSent", + p.totalRowsSent, + "OutputBufferInfo", + "int64_t", + "totalRowsSent"); + to_json_key( j, - "getOutputCalls", - p.getOutputCalls, - "OperatorStats", + "totalPagesSent", + p.totalPagesSent, + "OutputBufferInfo", "int64_t", - "getOutputCalls"); - from_json_key( + "totalPagesSent"); + to_json_key( j, - "getOutputWall", - p.getOutputWall, - "OperatorStats", - "Duration", - "getOutputWall"); + "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, - "getOutputCpu", - p.getOutputCpu, - "OperatorStats", - "Duration", - "getOutputCpu"); + j, "state", p.state, "OutputBufferInfo", "BufferState", "state"); from_json_key( j, - "getOutputAllocation", - p.getOutputAllocation, - "OperatorStats", - "DataSize", - "getOutputAllocation"); + "canAddBuffers", + p.canAddBuffers, + "OutputBufferInfo", + "bool", + "canAddBuffers"); from_json_key( j, - "outputDataSize", - p.outputDataSize, - "OperatorStats", - "DataSize", - "outputDataSize"); + "canAddPages", + p.canAddPages, + "OutputBufferInfo", + "bool", + "canAddPages"); from_json_key( j, - "outputPositions", - p.outputPositions, - "OperatorStats", + "totalBufferedBytes", + p.totalBufferedBytes, + "OutputBufferInfo", "int64_t", - "outputPositions"); - from_json_key( - j, - "physicalWrittenDataSize", - p.physicalWrittenDataSize, - "OperatorStats", - "DataSize", - "physicalWrittenDataSize"); - from_json_key( - j, - "additionalCpu", - p.additionalCpu, - "OperatorStats", - "Duration", - "additionalCpu"); + "totalBufferedBytes"); from_json_key( j, - "blockedWall", - p.blockedWall, - "OperatorStats", - "Duration", - "blockedWall"); + "totalBufferedPages", + p.totalBufferedPages, + "OutputBufferInfo", + "int64_t", + "totalBufferedPages"); from_json_key( j, - "finishCalls", - p.finishCalls, - "OperatorStats", + "totalRowsSent", + p.totalRowsSent, + "OutputBufferInfo", "int64_t", - "finishCalls"); - from_json_key( - j, "finishWall", p.finishWall, "OperatorStats", "Duration", "finishWall"); - from_json_key( - j, "finishCpu", p.finishCpu, "OperatorStats", "Duration", "finishCpu"); + "totalRowsSent"); from_json_key( j, - "finishAllocation", - p.finishAllocation, - "OperatorStats", - "DataSize", - "finishAllocation"); + "totalPagesSent", + p.totalPagesSent, + "OutputBufferInfo", + "int64_t", + "totalPagesSent"); from_json_key( j, - "userMemoryReservation", - p.userMemoryReservation, - "OperatorStats", - "DataSize", - "userMemoryReservation"); - from_json_key( + "buffers", + p.buffers, + "OutputBufferInfo", + "List", + "buffers"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +SystemTableHandle::SystemTableHandle() noexcept { + _type = "$system@system"; +} + +void to_json(json& j, const SystemTableHandle& p) { + j = json::object(); + j["@type"] = "$system@system"; + to_json_key( j, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "OperatorStats", - "DataSize", - "revocableMemoryReservation"); - from_json_key( + "connectorId", + p.connectorId, + "SystemTableHandle", + "ConnectorId", + "connectorId"); + to_json_key( j, - "systemMemoryReservation", - p.systemMemoryReservation, - "OperatorStats", - "DataSize", - "systemMemoryReservation"); + "schemaName", + p.schemaName, + "SystemTableHandle", + "String", + "schemaName"); + to_json_key( + j, "tableName", p.tableName, "SystemTableHandle", "String", "tableName"); +} + +void from_json(const json& j, SystemTableHandle& p) { + p._type = j["@type"]; from_json_key( j, - "peakUserMemoryReservation", - p.peakUserMemoryReservation, - "OperatorStats", - "DataSize", - "peakUserMemoryReservation"); + "connectorId", + p.connectorId, + "SystemTableHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "peakSystemMemoryReservation", - p.peakSystemMemoryReservation, - "OperatorStats", - "DataSize", - "peakSystemMemoryReservation"); + "schemaName", + p.schemaName, + "SystemTableHandle", + "String", + "schemaName"); from_json_key( - j, - "peakTotalMemoryReservation", - p.peakTotalMemoryReservation, - "OperatorStats", - "DataSize", - "peakTotalMemoryReservation"); + j, "tableName", p.tableName, "SystemTableHandle", "String", "tableName"); +} +} // 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 { +SystemSplit::SystemSplit() noexcept { + _type = "$system@system"; +} + +void to_json(json& j, const SystemSplit& p) { + j = json::object(); + j["@type"] = "$system@system"; + to_json_key( j, - "spilledDataSize", - p.spilledDataSize, - "OperatorStats", - "DataSize", - "spilledDataSize"); - from_json_key( + "connectorId", + p.connectorId, + "SystemSplit", + "ConnectorId", + "connectorId"); + to_json_key( j, - "blockedReason", - p.blockedReason, - "OperatorStats", - "BlockedReason", - "blockedReason"); - from_json_key(j, "info", p.info, "OperatorStats", "OperatorInfo", "info"); - from_json_key( + "tableHandle", + p.tableHandle, + "SystemSplit", + "SystemTableHandle", + "tableHandle"); + to_json_key( j, - "runtimeStats", - p.runtimeStats, - "OperatorStats", - "RuntimeStats", - "runtimeStats"); + "addresses", + p.addresses, + "SystemSplit", + "List", + "addresses"); + to_json_key( + j, + "constraint", + p.constraint, + "SystemSplit", + "TupleDomain>", + "constraint"); +} + +void from_json(const json& j, SystemSplit& p) { + p._type = j["@type"]; from_json_key( j, - "nullJoinBuildKeyCount", - p.nullJoinBuildKeyCount, - "OperatorStats", - "int64_t", - "nullJoinBuildKeyCount"); + "connectorId", + p.connectorId, + "SystemSplit", + "ConnectorId", + "connectorId"); from_json_key( j, - "joinBuildKeyCount", - p.joinBuildKeyCount, - "OperatorStats", - "int64_t", - "joinBuildKeyCount"); + "tableHandle", + p.tableHandle, + "SystemSplit", + "SystemTableHandle", + "tableHandle"); from_json_key( j, - "nullJoinProbeKeyCount", - p.nullJoinProbeKeyCount, - "OperatorStats", - "int64_t", - "nullJoinProbeKeyCount"); + "addresses", + p.addresses, + "SystemSplit", + "List", + "addresses"); from_json_key( j, - "joinProbeKeyCount", - p.joinProbeKeyCount, - "OperatorStats", - "int64_t", - "joinProbeKeyCount"); + "constraint", + p.constraint, + "SystemSplit", + "TupleDomain>", + "constraint"); +} +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +namespace facebook::presto::protocol { + +void to_json(json& j, const Block& p) { + j = p.data; +} + +void from_json(const json& j, Block& p) { + p.data = std::string(j); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const DriverStats& p) { +void to_json(json& j, const ValueEntry& p) { j = json::object(); - to_json_key(j, "lifespan", p.lifespan, "DriverStats", "Lifespan", "lifespan"); - to_json_key( - j, "createTime", p.createTime, "DriverStats", "DateTime", "createTime"); - to_json_key( - j, "startTime", p.startTime, "DriverStats", "DateTime", "startTime"); - to_json_key(j, "endTime", p.endTime, "DriverStats", "DateTime", "endTime"); - to_json_key( - j, "queuedTime", p.queuedTime, "DriverStats", "Duration", "queuedTime"); - to_json_key( - j, - "elapsedTime", - p.elapsedTime, - "DriverStats", - "Duration", - "elapsedTime"); - to_json_key( - j, - "userMemoryReservation", - p.userMemoryReservation, - "DriverStats", - "DataSize", - "userMemoryReservation"); - to_json_key( - j, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "DriverStats", - "DataSize", - "revocableMemoryReservation"); - to_json_key( - j, - "systemMemoryReservation", - p.systemMemoryReservation, - "DriverStats", - "DataSize", - "systemMemoryReservation"); + to_json_key(j, "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, - "totalScheduledTime", - p.totalScheduledTime, - "DriverStats", - "Duration", - "totalScheduledTime"); + j, "whiteList", p.whiteList, "EquatableValueSet", "bool", "whiteList"); to_json_key( j, - "totalCpuTime", - p.totalCpuTime, - "DriverStats", - "Duration", - "totalCpuTime"); - to_json_key( + "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, "whiteList", p.whiteList, "EquatableValueSet", "bool", "whiteList"); + from_json_key( j, - "totalBlockedTime", - p.totalBlockedTime, - "DriverStats", - "Duration", - "totalBlockedTime"); - to_json_key( - j, "fullyBlocked", p.fullyBlocked, "DriverStats", "bool", "fullyBlocked"); - to_json_key( - j, - "blockedReasons", - p.blockedReasons, - "DriverStats", - "List", - "blockedReasons"); - to_json_key( - j, - "totalAllocation", - p.totalAllocation, - "DriverStats", - "DataSize", - "totalAllocation"); - to_json_key( - j, - "rawInputDataSize", - p.rawInputDataSize, - "DriverStats", - "DataSize", - "rawInputDataSize"); - to_json_key( - j, - "rawInputPositions", - p.rawInputPositions, - "DriverStats", - "int64_t", - "rawInputPositions"); + "entries", + p.entries, + "EquatableValueSet", + "List", + "entries"); +} +} // 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 { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair PrestoTableType_enum_table[] = + { // NOLINT: cert-err58-cpp + {PrestoTableType::MANAGED_TABLE, "MANAGED_TABLE"}, + {PrestoTableType::EXTERNAL_TABLE, "EXTERNAL_TABLE"}, + {PrestoTableType::VIRTUAL_VIEW, "VIRTUAL_VIEW"}, + {PrestoTableType::MATERIALIZED_VIEW, "MATERIALIZED_VIEW"}, + {PrestoTableType::TEMPORARY_TABLE, "TEMPORARY_TABLE"}, + {PrestoTableType::OTHER, "OTHER"}}; +void to_json(json& j, const PrestoTableType& e) { + static_assert( + std::is_enum::value, "PrestoTableType must be an enum!"); + const auto* it = std::find_if( + std::begin(PrestoTableType_enum_table), + std::end(PrestoTableType_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(PrestoTableType_enum_table)) + ? it + : std::begin(PrestoTableType_enum_table)) + ->second; +} +void from_json(const json& j, PrestoTableType& e) { + static_assert( + std::is_enum::value, "PrestoTableType must be an enum!"); + const auto* it = std::find_if( + std::begin(PrestoTableType_enum_table), + std::end(PrestoTableType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(PrestoTableType_enum_table)) + ? it + : std::begin(PrestoTableType_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const Table& p) { + j = json::object(); to_json_key( - j, - "rawInputReadTime", - p.rawInputReadTime, - "DriverStats", - "Duration", - "rawInputReadTime"); + 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, - "processedInputDataSize", - p.processedInputDataSize, - "DriverStats", - "DataSize", - "processedInputDataSize"); + j, "tableType", p.tableType, "Table", "PrestoTableType", "tableType"); + to_json_key(j, "storage", p.storage, "Table", "Storage", "storage"); to_json_key( - j, - "processedInputPositions", - p.processedInputPositions, - "DriverStats", - "int64_t", - "processedInputPositions"); + j, "dataColumns", p.dataColumns, "Table", "List", "dataColumns"); to_json_key( j, - "outputDataSize", - p.outputDataSize, - "DriverStats", - "DataSize", - "outputDataSize"); + "partitionColumns", + p.partitionColumns, + "Table", + "List", + "partitionColumns"); to_json_key( j, - "outputPositions", - p.outputPositions, - "DriverStats", - "int64_t", - "outputPositions"); + "parameters", + p.parameters, + "Table", + "Map", + "parameters"); to_json_key( j, - "physicalWrittenDataSize", - p.physicalWrittenDataSize, - "DriverStats", - "DataSize", - "physicalWrittenDataSize"); + "viewOriginalText", + p.viewOriginalText, + "Table", + "String", + "viewOriginalText"); to_json_key( j, - "operatorStats", - p.operatorStats, - "DriverStats", - "List", - "operatorStats"); + "viewExpandedText", + p.viewExpandedText, + "Table", + "String", + "viewExpandedText"); } -void from_json(const json& j, DriverStats& p) { - from_json_key( - j, "lifespan", p.lifespan, "DriverStats", "Lifespan", "lifespan"); +void from_json(const json& j, Table& p) { from_json_key( - j, "createTime", p.createTime, "DriverStats", "DateTime", "createTime"); + 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, "startTime", p.startTime, "DriverStats", "DateTime", "startTime"); - from_json_key(j, "endTime", p.endTime, "DriverStats", "DateTime", "endTime"); + j, "tableType", p.tableType, "Table", "PrestoTableType", "tableType"); + from_json_key(j, "storage", p.storage, "Table", "Storage", "storage"); from_json_key( - j, "queuedTime", p.queuedTime, "DriverStats", "Duration", "queuedTime"); + j, "dataColumns", p.dataColumns, "Table", "List", "dataColumns"); from_json_key( j, - "elapsedTime", - p.elapsedTime, - "DriverStats", - "Duration", - "elapsedTime"); + "partitionColumns", + p.partitionColumns, + "Table", + "List", + "partitionColumns"); from_json_key( j, - "userMemoryReservation", - p.userMemoryReservation, - "DriverStats", - "DataSize", - "userMemoryReservation"); + "parameters", + p.parameters, + "Table", + "Map", + "parameters"); from_json_key( j, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "DriverStats", - "DataSize", - "revocableMemoryReservation"); + "viewOriginalText", + p.viewOriginalText, + "Table", + "String", + "viewOriginalText"); from_json_key( j, - "systemMemoryReservation", - p.systemMemoryReservation, - "DriverStats", - "DataSize", - "systemMemoryReservation"); - from_json_key( + "viewExpandedText", + p.viewExpandedText, + "Table", + "String", + "viewExpandedText"); +} +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +namespace facebook::presto::protocol { + +void to_json(json& j, const HivePageSinkMetadata& p) { + j = json::object(); + to_json_key( j, - "totalScheduledTime", - p.totalScheduledTime, - "DriverStats", - "Duration", - "totalScheduledTime"); + "schemaTableName", + p.schemaTableName, + "HivePageSinkMetadata", + "SchemaTableName", + "schemaTableName"); + to_json_key(j, "table", p.table, "HivePageSinkMetadata", "Table", "table"); +} + +void from_json(const json& j, HivePageSinkMetadata& p) { from_json_key( j, - "totalCpuTime", - p.totalCpuTime, - "DriverStats", - "Duration", - "totalCpuTime"); - from_json_key( - j, - "totalBlockedTime", - p.totalBlockedTime, - "DriverStats", - "Duration", - "totalBlockedTime"); - from_json_key( - j, "fullyBlocked", p.fullyBlocked, "DriverStats", "bool", "fullyBlocked"); - from_json_key( - j, - "blockedReasons", - p.blockedReasons, - "DriverStats", - "List", - "blockedReasons"); - from_json_key( - j, - "totalAllocation", - p.totalAllocation, - "DriverStats", - "DataSize", - "totalAllocation"); - from_json_key( - j, - "rawInputDataSize", - p.rawInputDataSize, - "DriverStats", - "DataSize", - "rawInputDataSize"); - from_json_key( - j, - "rawInputPositions", - p.rawInputPositions, - "DriverStats", - "int64_t", - "rawInputPositions"); - from_json_key( - j, - "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"); + "schemaTableName", + p.schemaTableName, + "HivePageSinkMetadata", + "SchemaTableName", + "schemaTableName"); + from_json_key(j, "table", p.table, "HivePageSinkMetadata", "Table", "table"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const PipelineStats& p) { +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair TableType_enum_table[] = + { // NOLINT: cert-err58-cpp + {TableType::NEW, "NEW"}, + {TableType::EXISTING, "EXISTING"}, + {TableType::TEMPORARY, "TEMPORARY"}}; +void to_json(json& j, const TableType& e) { + static_assert(std::is_enum::value, "TableType must be an enum!"); + const auto* it = std::find_if( + std::begin(TableType_enum_table), + std::end(TableType_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(TableType_enum_table)) + ? it + : std::begin(TableType_enum_table)) + ->second; +} +void from_json(const json& j, TableType& e) { + static_assert(std::is_enum::value, "TableType must be an enum!"); + const auto* it = std::find_if( + std::begin(TableType_enum_table), + std::end(TableType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(TableType_enum_table)) + ? it + : std::begin(TableType_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair WriteMode_enum_table[] = + { // NOLINT: cert-err58-cpp + {WriteMode::STAGE_AND_MOVE_TO_TARGET_DIRECTORY, + "STAGE_AND_MOVE_TO_TARGET_DIRECTORY"}, + {WriteMode::DIRECT_TO_TARGET_NEW_DIRECTORY, + "DIRECT_TO_TARGET_NEW_DIRECTORY"}, + {WriteMode::DIRECT_TO_TARGET_EXISTING_DIRECTORY, + "DIRECT_TO_TARGET_EXISTING_DIRECTORY"}}; +void to_json(json& j, const WriteMode& e) { + static_assert(std::is_enum::value, "WriteMode must be an enum!"); + const auto* it = std::find_if( + std::begin(WriteMode_enum_table), + std::end(WriteMode_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(WriteMode_enum_table)) + ? it + : std::begin(WriteMode_enum_table)) + ->second; +} +void from_json(const json& j, WriteMode& e) { + static_assert(std::is_enum::value, "WriteMode must be an enum!"); + const auto* it = std::find_if( + std::begin(WriteMode_enum_table), + std::end(WriteMode_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(WriteMode_enum_table)) + ? it + : std::begin(WriteMode_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const LocationHandle& 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"); + j, "targetPath", p.targetPath, "LocationHandle", "String", "targetPath"); to_json_key( - j, - "runningDrivers", - p.runningDrivers, - "PipelineStats", - "int", - "runningDrivers"); + j, "writePath", p.writePath, "LocationHandle", "String", "writePath"); to_json_key( - j, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "PipelineStats", - "int", - "runningPartitionedDrivers"); + j, "tempPath", p.tempPath, "LocationHandle", "String", "tempPath"); to_json_key( - j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "PipelineStats", - "int64_t", - "runningPartitionedSplitsWeight"); + j, "tableType", p.tableType, "LocationHandle", "TableType", "tableType"); to_json_key( - j, - "blockedDrivers", - p.blockedDrivers, - "PipelineStats", - "int", - "blockedDrivers"); + j, "writeMode", p.writeMode, "LocationHandle", "WriteMode", "writeMode"); +} + +void from_json(const json& j, LocationHandle& p) { + from_json_key( + j, "targetPath", p.targetPath, "LocationHandle", "String", "targetPath"); + from_json_key( + j, "writePath", p.writePath, "LocationHandle", "String", "writePath"); + from_json_key( + j, "tempPath", p.tempPath, "LocationHandle", "String", "tempPath"); + from_json_key( + j, "tableType", p.tableType, "LocationHandle", "TableType", "tableType"); + from_json_key( + j, "writeMode", p.writeMode, "LocationHandle", "WriteMode", "writeMode"); +} +} // namespace facebook::presto::protocol +/* + * 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() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair + HiveCompressionCodec_enum_table[] = { // NOLINT: cert-err58-cpp + {HiveCompressionCodec::NONE, "NONE"}, + {HiveCompressionCodec::SNAPPY, "SNAPPY"}, + {HiveCompressionCodec::GZIP, "GZIP"}, + {HiveCompressionCodec::LZ4, "LZ4"}, + {HiveCompressionCodec::ZSTD, "ZSTD"}}; +void to_json(json& j, const HiveCompressionCodec& e) { + static_assert( + std::is_enum::value, + "HiveCompressionCodec must be an enum!"); + const auto* it = std::find_if( + std::begin(HiveCompressionCodec_enum_table), + std::end(HiveCompressionCodec_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(HiveCompressionCodec_enum_table)) + ? it + : std::begin(HiveCompressionCodec_enum_table)) + ->second; +} +void from_json(const json& j, HiveCompressionCodec& e) { + static_assert( + std::is_enum::value, + "HiveCompressionCodec must be an enum!"); + const auto* it = std::find_if( + std::begin(HiveCompressionCodec_enum_table), + std::end(HiveCompressionCodec_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(HiveCompressionCodec_enum_table)) + ? it + : std::begin(HiveCompressionCodec_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +HiveOutputTableHandle::HiveOutputTableHandle() noexcept { + _type = "hive"; +} + +void to_json(json& j, const HiveOutputTableHandle& p) { + j = json::object(); + j["@type"] = "hive"; to_json_key( j, - "completedDrivers", - p.completedDrivers, - "PipelineStats", - "int", - "completedDrivers"); + "schemaName", + p.schemaName, + "HiveOutputTableHandle", + "String", + "schemaName"); to_json_key( j, - "userMemoryReservationInBytes", - p.userMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "userMemoryReservationInBytes"); + "tableName", + p.tableName, + "HiveOutputTableHandle", + "String", + "tableName"); to_json_key( j, - "revocableMemoryReservationInBytes", - p.revocableMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "revocableMemoryReservationInBytes"); + "inputColumns", + p.inputColumns, + "HiveOutputTableHandle", + "List", + "inputColumns"); to_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "systemMemoryReservationInBytes"); + "pageSinkMetadata", + p.pageSinkMetadata, + "HiveOutputTableHandle", + "HivePageSinkMetadata", + "pageSinkMetadata"); to_json_key( j, - "queuedTime", - p.queuedTime, - "PipelineStats", - "DistributionSnapshot", - "queuedTime"); + "locationHandle", + p.locationHandle, + "HiveOutputTableHandle", + "LocationHandle", + "locationHandle"); to_json_key( j, - "elapsedTime", - p.elapsedTime, - "PipelineStats", - "DistributionSnapshot", - "elapsedTime"); + "tableStorageFormat", + p.tableStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "tableStorageFormat"); to_json_key( j, - "totalScheduledTimeInNanos", - p.totalScheduledTimeInNanos, - "PipelineStats", - "int64_t", - "totalScheduledTimeInNanos"); + "partitionStorageFormat", + p.partitionStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "partitionStorageFormat"); to_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "PipelineStats", - "int64_t", - "totalCpuTimeInNanos"); + "actualStorageFormat", + p.actualStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "actualStorageFormat"); to_json_key( j, - "totalBlockedTimeInNanos", - p.totalBlockedTimeInNanos, - "PipelineStats", - "int64_t", - "totalBlockedTimeInNanos"); + "compressionCodec", + p.compressionCodec, + "HiveOutputTableHandle", + "HiveCompressionCodec", + "compressionCodec"); to_json_key( j, - "fullyBlocked", - p.fullyBlocked, - "PipelineStats", - "bool", - "fullyBlocked"); + "partitionedBy", + p.partitionedBy, + "HiveOutputTableHandle", + "List", + "partitionedBy"); to_json_key( j, - "blockedReasons", - p.blockedReasons, - "PipelineStats", - "List", - "blockedReasons"); + "bucketProperty", + p.bucketProperty, + "HiveOutputTableHandle", + "HiveBucketProperty", + "bucketProperty"); to_json_key( j, - "totalAllocationInBytes", - p.totalAllocationInBytes, - "PipelineStats", - "int64_t", - "totalAllocationInBytes"); + "preferredOrderingColumns", + p.preferredOrderingColumns, + "HiveOutputTableHandle", + "List", + "preferredOrderingColumns"); to_json_key( j, - "rawInputDataSizeInBytes", - p.rawInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "rawInputDataSizeInBytes"); - to_json_key( - j, - "rawInputPositions", - p.rawInputPositions, - "PipelineStats", - "int64_t", - "rawInputPositions"); - to_json_key( - j, - "processedInputDataSizeInBytes", - p.processedInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "processedInputDataSizeInBytes"); - to_json_key( - j, - "processedInputPositions", - p.processedInputPositions, - "PipelineStats", - "int64_t", - "processedInputPositions"); - to_json_key( - j, - "outputDataSizeInBytes", - p.outputDataSizeInBytes, - "PipelineStats", - "int64_t", - "outputDataSizeInBytes"); - to_json_key( - j, - "outputPositions", - p.outputPositions, - "PipelineStats", - "int64_t", - "outputPositions"); + "tableOwner", + p.tableOwner, + "HiveOutputTableHandle", + "String", + "tableOwner"); to_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "PipelineStats", - "int64_t", - "physicalWrittenDataSizeInBytes"); + "additionalTableParameters", + p.additionalTableParameters, + "HiveOutputTableHandle", + "Map", + "additionalTableParameters"); to_json_key( j, - "operatorSummaries", - p.operatorSummaries, - "PipelineStats", - "List", - "operatorSummaries"); - to_json_key( - j, "drivers", p.drivers, "PipelineStats", "List", "drivers"); + "encryptionInformation", + p.encryptionInformation, + "HiveOutputTableHandle", + "EncryptionInformation", + "encryptionInformation"); } -void from_json(const json& j, PipelineStats& p) { - from_json_key( - j, "pipelineId", p.pipelineId, "PipelineStats", "int", "pipelineId"); +void from_json(const json& j, HiveOutputTableHandle& p) { + p._type = j["@type"]; from_json_key( j, - "firstStartTime", - p.firstStartTime, - "PipelineStats", - "DateTime", - "firstStartTime"); + "schemaName", + p.schemaName, + "HiveOutputTableHandle", + "String", + "schemaName"); from_json_key( j, - "lastStartTime", - p.lastStartTime, - "PipelineStats", - "DateTime", - "lastStartTime"); + "tableName", + p.tableName, + "HiveOutputTableHandle", + "String", + "tableName"); from_json_key( j, - "lastEndTime", - p.lastEndTime, - "PipelineStats", - "DateTime", - "lastEndTime"); + "inputColumns", + p.inputColumns, + "HiveOutputTableHandle", + "List", + "inputColumns"); from_json_key( j, - "inputPipeline", - p.inputPipeline, - "PipelineStats", - "bool", - "inputPipeline"); + "pageSinkMetadata", + p.pageSinkMetadata, + "HiveOutputTableHandle", + "HivePageSinkMetadata", + "pageSinkMetadata"); from_json_key( j, - "outputPipeline", - p.outputPipeline, - "PipelineStats", - "bool", - "outputPipeline"); + "locationHandle", + p.locationHandle, + "HiveOutputTableHandle", + "LocationHandle", + "locationHandle"); from_json_key( j, - "totalDrivers", - p.totalDrivers, - "PipelineStats", - "int", - "totalDrivers"); + "tableStorageFormat", + p.tableStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "tableStorageFormat"); from_json_key( j, - "queuedDrivers", - p.queuedDrivers, - "PipelineStats", - "int", - "queuedDrivers"); + "partitionStorageFormat", + p.partitionStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "partitionStorageFormat"); from_json_key( j, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "PipelineStats", - "int", - "queuedPartitionedDrivers"); + "actualStorageFormat", + p.actualStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "actualStorageFormat"); from_json_key( j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "PipelineStats", - "int64_t", - "queuedPartitionedSplitsWeight"); + "compressionCodec", + p.compressionCodec, + "HiveOutputTableHandle", + "HiveCompressionCodec", + "compressionCodec"); from_json_key( j, - "runningDrivers", - p.runningDrivers, - "PipelineStats", - "int", - "runningDrivers"); + "partitionedBy", + p.partitionedBy, + "HiveOutputTableHandle", + "List", + "partitionedBy"); from_json_key( j, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "PipelineStats", - "int", - "runningPartitionedDrivers"); + "bucketProperty", + p.bucketProperty, + "HiveOutputTableHandle", + "HiveBucketProperty", + "bucketProperty"); from_json_key( j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "PipelineStats", - "int64_t", - "runningPartitionedSplitsWeight"); + "preferredOrderingColumns", + p.preferredOrderingColumns, + "HiveOutputTableHandle", + "List", + "preferredOrderingColumns"); from_json_key( j, - "blockedDrivers", - p.blockedDrivers, - "PipelineStats", - "int", - "blockedDrivers"); + "tableOwner", + p.tableOwner, + "HiveOutputTableHandle", + "String", + "tableOwner"); from_json_key( j, - "completedDrivers", - p.completedDrivers, - "PipelineStats", - "int", - "completedDrivers"); + "additionalTableParameters", + p.additionalTableParameters, + "HiveOutputTableHandle", + "Map", + "additionalTableParameters"); from_json_key( j, - "userMemoryReservationInBytes", - p.userMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "userMemoryReservationInBytes"); - from_json_key( + "encryptionInformation", + p.encryptionInformation, + "HiveOutputTableHandle", + "EncryptionInformation", + "encryptionInformation"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const PartialAggregationStatsEstimate& p) { + j = json::object(); + to_json_key( j, - "revocableMemoryReservationInBytes", - p.revocableMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "revocableMemoryReservationInBytes"); - from_json_key( + "inputBytes", + p.inputBytes, + "PartialAggregationStatsEstimate", + "double", + "inputBytes"); + to_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "systemMemoryReservationInBytes"); - from_json_key( + "outputBytes", + p.outputBytes, + "PartialAggregationStatsEstimate", + "double", + "outputBytes"); + to_json_key( j, - "queuedTime", - p.queuedTime, - "PipelineStats", - "DistributionSnapshot", - "queuedTime"); - from_json_key( + "inputRowCount", + p.inputRowCount, + "PartialAggregationStatsEstimate", + "double", + "inputRowCount"); + to_json_key( j, - "elapsedTime", - p.elapsedTime, - "PipelineStats", - "DistributionSnapshot", - "elapsedTime"); + "outputRowCount", + p.outputRowCount, + "PartialAggregationStatsEstimate", + "double", + "outputRowCount"); +} + +void from_json(const json& j, PartialAggregationStatsEstimate& p) { from_json_key( j, - "totalScheduledTimeInNanos", - p.totalScheduledTimeInNanos, - "PipelineStats", - "int64_t", - "totalScheduledTimeInNanos"); + "inputBytes", + p.inputBytes, + "PartialAggregationStatsEstimate", + "double", + "inputBytes"); from_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "PipelineStats", - "int64_t", - "totalCpuTimeInNanos"); + "outputBytes", + p.outputBytes, + "PartialAggregationStatsEstimate", + "double", + "outputBytes"); from_json_key( j, - "totalBlockedTimeInNanos", - p.totalBlockedTimeInNanos, - "PipelineStats", - "int64_t", - "totalBlockedTimeInNanos"); + "inputRowCount", + p.inputRowCount, + "PartialAggregationStatsEstimate", + "double", + "inputRowCount"); from_json_key( j, - "fullyBlocked", - p.fullyBlocked, - "PipelineStats", - "bool", - "fullyBlocked"); - from_json_key( + "outputRowCount", + p.outputRowCount, + "PartialAggregationStatsEstimate", + "double", + "outputRowCount"); +} +} // 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 == "uniform-distribution") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ConnectorHistogram "); +} + +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()) + " ConnectorHistogram ConnectorHistogram"); + } + + if (type == "uniform-distribution") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + + throw TypeError(type + " no abstract type ConnectorHistogram "); +} +} // 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, - "blockedReasons", - p.blockedReasons, - "PipelineStats", - "List", - "blockedReasons"); - from_json_key( + "highValue", + p.highValue, + "VariableStatsEstimate", + "double", + "highValue"); + to_json_key( j, - "totalAllocationInBytes", - p.totalAllocationInBytes, - "PipelineStats", - "int64_t", - "totalAllocationInBytes"); - from_json_key( + "nullsFraction", + p.nullsFraction, + "VariableStatsEstimate", + "double", + "nullsFraction"); + to_json_key( j, - "rawInputDataSizeInBytes", - p.rawInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "rawInputDataSizeInBytes"); - from_json_key( + "averageRowSize", + p.averageRowSize, + "VariableStatsEstimate", + "double", + "averageRowSize"); + to_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "PipelineStats", - "int64_t", - "rawInputPositions"); - from_json_key( + "distinctValuesCount", + p.distinctValuesCount, + "VariableStatsEstimate", + "double", + "distinctValuesCount"); + to_json_key( j, - "processedInputDataSizeInBytes", - p.processedInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "processedInputDataSizeInBytes"); + "histogram", + p.histogram, + "VariableStatsEstimate", + "ConnectorHistogram", + "histogram"); +} + +void from_json(const json& j, VariableStatsEstimate& p) { from_json_key( - j, - "processedInputPositions", - p.processedInputPositions, - "PipelineStats", - "int64_t", - "processedInputPositions"); + j, "lowValue", p.lowValue, "VariableStatsEstimate", "double", "lowValue"); from_json_key( j, - "outputDataSizeInBytes", - p.outputDataSizeInBytes, - "PipelineStats", - "int64_t", - "outputDataSizeInBytes"); + "highValue", + p.highValue, + "VariableStatsEstimate", + "double", + "highValue"); from_json_key( j, - "outputPositions", - p.outputPositions, - "PipelineStats", - "int64_t", - "outputPositions"); + "nullsFraction", + p.nullsFraction, + "VariableStatsEstimate", + "double", + "nullsFraction"); from_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "PipelineStats", - "int64_t", - "physicalWrittenDataSizeInBytes"); + "averageRowSize", + p.averageRowSize, + "VariableStatsEstimate", + "double", + "averageRowSize"); from_json_key( j, - "operatorSummaries", - p.operatorSummaries, - "PipelineStats", - "List", - "operatorSummaries"); + "distinctValuesCount", + p.distinctValuesCount, + "VariableStatsEstimate", + "double", + "distinctValuesCount"); from_json_key( - j, "drivers", p.drivers, "PipelineStats", "List", "drivers"); + j, + "histogram", + p.histogram, + "VariableStatsEstimate", + "ConnectorHistogram", + "histogram"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair Bound_enum_table[] = - { // NOLINT: cert-err58-cpp - {Bound::BELOW, "BELOW"}, - {Bound::EXACTLY, "EXACTLY"}, - {Bound::ABOVE, "ABOVE"}}; -void to_json(json& j, const Bound& e) { - static_assert(std::is_enum::value, "Bound must be an enum!"); - const auto* it = std::find_if( - std::begin(Bound_enum_table), - std::end(Bound_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(Bound_enum_table)) ? it : std::begin(Bound_enum_table)) - ->second; -} -void from_json(const json& j, Bound& e) { - static_assert(std::is_enum::value, "Bound must be an enum!"); - const auto* it = std::find_if( - std::begin(Bound_enum_table), - std::end(Bound_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(Bound_enum_table)) ? it : std::begin(Bound_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { - -void to_json(json& j, const Block& p) { - j = p.data; +void to_json(json& j, const JoinNodeStatsEstimate& p) { + j = json::object(); + to_json_key( + j, + "nullJoinBuildKeyCount", + p.nullJoinBuildKeyCount, + "JoinNodeStatsEstimate", + "double", + "nullJoinBuildKeyCount"); + to_json_key( + 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, Block& p) { - p.data = std::string(j); +void from_json(const json& j, JoinNodeStatsEstimate& p) { + from_json_key( + j, + "nullJoinBuildKeyCount", + p.nullJoinBuildKeyCount, + "JoinNodeStatsEstimate", + "double", + "nullJoinBuildKeyCount"); + from_json_key( + j, + "joinBuildKeyCount", + p.joinBuildKeyCount, + "JoinNodeStatsEstimate", + "double", + "joinBuildKeyCount"); + from_json_key( + j, + "nullJoinProbeKeyCount", + p.nullJoinProbeKeyCount, + "JoinNodeStatsEstimate", + "double", + "nullJoinProbeKeyCount"); + from_json_key( + j, + "joinProbeKeyCount", + p.joinProbeKeyCount, + "JoinNodeStatsEstimate", + "double", + "joinProbeKeyCount"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Marker& p) { +void to_json(json& j, const TableWriterNodeStatsEstimate& p) { j = json::object(); - to_json_key(j, "type", p.type, "Marker", "Type", "type"); - to_json_key(j, "valueBlock", p.valueBlock, "Marker", "Block", "valueBlock"); - to_json_key(j, "bound", p.bound, "Marker", "Bound", "bound"); + to_json_key( + j, + "taskCountIfScaledWriter", + p.taskCountIfScaledWriter, + "TableWriterNodeStatsEstimate", + "double", + "taskCountIfScaledWriter"); } -void from_json(const json& j, Marker& p) { - from_json_key(j, "type", p.type, "Marker", "Type", "type"); - from_json_key(j, "valueBlock", p.valueBlock, "Marker", "Block", "valueBlock"); - from_json_key(j, "bound", p.bound, "Marker", "Bound", "bound"); +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 SourceLocation& p) { +void to_json(json& j, const PlanNodeStatsEstimate& p) { j = json::object(); - to_json_key(j, "line", p.line, "SourceLocation", "int", "line"); - to_json_key(j, "column", p.column, "SourceLocation", "int", "column"); + to_json_key( + j, + "outputRowCount", + p.outputRowCount, + "PlanNodeStatsEstimate", + "double", + "outputRowCount"); + to_json_key( + j, + "totalSize", + p.totalSize, + "PlanNodeStatsEstimate", + "double", + "totalSize"); + to_json_key( + j, + "confident", + p.confident, + "PlanNodeStatsEstimate", + "bool", + "confident"); + to_json_key( + j, + "variableStatistics", + p.variableStatistics, + "PlanNodeStatsEstimate", + "Map", + "variableStatistics"); + 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, SourceLocation& p) { - from_json_key(j, "line", p.line, "SourceLocation", "int", "line"); - from_json_key(j, "column", p.column, "SourceLocation", "int", "column"); +void from_json(const json& j, 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( + 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, + "partialAggregationStatsEstimate", + p.partialAggregationStatsEstimate, + "PlanNodeStatsEstimate", + "PartialAggregationStatsEstimate", + "partialAggregationStatsEstimate"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -VariableReferenceExpression::VariableReferenceExpression() noexcept { - _type = "variable"; +EmptySplit::EmptySplit() noexcept { + _type = "$empty"; } -void to_json(json& j, const VariableReferenceExpression& p) { +void to_json(json& j, const EmptySplit& p) { j = json::object(); - j["@type"] = "variable"; + j["@type"] = "$empty"; to_json_key( j, - "sourceLocation", - p.sourceLocation, - "VariableReferenceExpression", - "SourceLocation", - "sourceLocation"); - to_json_key( - j, "name", p.name, "VariableReferenceExpression", "String", "name"); - to_json_key(j, "type", p.type, "VariableReferenceExpression", "Type", "type"); + "connectorId", + p.connectorId, + "EmptySplit", + "ConnectorId", + "connectorId"); } -void from_json(const json& j, VariableReferenceExpression& p) { +void from_json(const json& j, EmptySplit& 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"); + "connectorId", + p.connectorId, + "EmptySplit", + "ConnectorId", + "connectorId"); } } // 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)) +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, SortOrder& e) { - static_assert(std::is_enum::value, "SortOrder must be an enum!"); +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(SortOrder_enum_table), - std::end(SortOrder_enum_table), - [&j](const std::pair& ej_pair) -> bool { + 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(SortOrder_enum_table)) + e = ((it != std::end(ChangelogOperation_enum_table)) ? it - : std::begin(SortOrder_enum_table)) + : std::begin(ChangelogOperation_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const Ordering& p) { +// 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, - "variable", - p.variable, - "Ordering", - "VariableReferenceExpression", - "variable"); + "typeCategory", + p.typeCategory, + "ColumnIdentity", + "TypeCategory", + "typeCategory"); to_json_key( - j, "sortOrder", p.sortOrder, "Ordering", "SortOrder", "sortOrder"); + j, + "children", + p.children, + "ColumnIdentity", + "List", + "children"); } -void from_json(const json& j, Ordering& p) { +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, - "variable", - p.variable, - "Ordering", - "VariableReferenceExpression", - "variable"); + "typeCategory", + p.typeCategory, + "ColumnIdentity", + "TypeCategory", + "typeCategory"); from_json_key( - j, "sortOrder", p.sortOrder, "Ordering", "SortOrder", "sortOrder"); + j, + "children", + p.children, + "ColumnIdentity", + "List", + "children"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +IcebergColumnHandle::IcebergColumnHandle() noexcept { + _type = "hive-iceberg"; +} -void to_json(json& j, const OrderingScheme& p) { +void to_json(json& j, const IcebergColumnHandle& p) { j = json::object(); + j["@type"] = "hive-iceberg"; to_json_key( - j, "orderBy", p.orderBy, "OrderingScheme", "List", "orderBy"); + j, + "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, + "columnType", + p.columnType, + "IcebergColumnHandle", + "ColumnType", + "columnType"); + to_json_key( + j, + "requiredSubfields", + p.requiredSubfields, + "IcebergColumnHandle", + "List", + "requiredSubfields"); } -void from_json(const json& j, OrderingScheme& p) { +void from_json(const json& j, IcebergColumnHandle& p) { + p._type = j["@type"]; from_json_key( - j, "orderBy", p.orderBy, "OrderingScheme", "List", "orderBy"); + 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 { -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 to_json(json& j, const ChangelogSplitInfo& p) { + j = json::object(); + to_json_key( + j, + "operation", + p.operation, + "ChangelogSplitInfo", + "ChangelogOperation", + "operation"); + to_json_key( + j, "ordinal", p.ordinal, "ChangelogSplitInfo", "int64_t", "ordinal"); + to_json_key( + j, + "snapshotId", + p.snapshotId, + "ChangelogSplitInfo", + "int64_t", + "snapshotId"); + to_json_key( + j, + "icebergColumns", + p.icebergColumns, + "ChangelogSplitInfo", + "List", + "icebergColumns"); } -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"); - } +void from_json(const json& j, ChangelogSplitInfo& p) { + from_json_key( + j, + "operation", + p.operation, + "ChangelogSplitInfo", + "ChangelogOperation", + "operation"); + from_json_key( + j, "ordinal", p.ordinal, "ChangelogSplitInfo", "int64_t", "ordinal"); + from_json_key( + j, + "snapshotId", + p.snapshotId, + "ChangelogSplitInfo", + "int64_t", + "snapshotId"); + from_json_key( + j, + "icebergColumns", + p.icebergColumns, + "ChangelogSplitInfo", + "List", + "icebergColumns"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { - 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 { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (type == "call") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "constant") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "special") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "lambda") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "variable") { - j = *std::static_pointer_cast(p); - return; - } - - throw TypeError(type + " no abstract type RowExpression "); -} - -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " RowExpression RowExpression"); - } - - if (type == "call") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "constant") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "special") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "lambda") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "variable") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - - throw TypeError(type + " no abstract type RowExpression "); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -CallExpression::CallExpression() noexcept { - _type = "call"; -} - -void to_json(json& j, const CallExpression& p) { +void to_json(json& j, const TypeVariableConstraint& p) { j = json::object(); - j["@type"] = "call"; + to_json_key(j, "name", p.name, "TypeVariableConstraint", "String", "name"); to_json_key( j, - "sourceLocation", - p.sourceLocation, - "CallExpression", - "SourceLocation", - "sourceLocation"); + "comparableRequired", + p.comparableRequired, + "TypeVariableConstraint", + "bool", + "comparableRequired"); to_json_key( j, - "displayName", - p.displayName, - "CallExpression", - "String", - "displayName"); + "orderableRequired", + p.orderableRequired, + "TypeVariableConstraint", + "bool", + "orderableRequired"); to_json_key( j, - "functionHandle", - p.functionHandle, - "CallExpression", - "FunctionHandle", - "functionHandle"); - to_json_key( - j, "returnType", p.returnType, "CallExpression", "Type", "returnType"); + "variadicBound", + p.variadicBound, + "TypeVariableConstraint", + "String", + "variadicBound"); to_json_key( j, - "arguments", - p.arguments, - "CallExpression", - "List>", - "arguments"); + "nonDecimalNumericRequired", + p.nonDecimalNumericRequired, + "TypeVariableConstraint", + "bool", + "nonDecimalNumericRequired"); } -void from_json(const json& j, CallExpression& p) { - p._type = j["@type"]; +void from_json(const json& j, TypeVariableConstraint& p) { + from_json_key(j, "name", p.name, "TypeVariableConstraint", "String", "name"); from_json_key( j, - "sourceLocation", - p.sourceLocation, - "CallExpression", - "SourceLocation", - "sourceLocation"); + "comparableRequired", + p.comparableRequired, + "TypeVariableConstraint", + "bool", + "comparableRequired"); from_json_key( j, - "displayName", - p.displayName, - "CallExpression", - "String", - "displayName"); + "orderableRequired", + p.orderableRequired, + "TypeVariableConstraint", + "bool", + "orderableRequired"); from_json_key( j, - "functionHandle", - p.functionHandle, - "CallExpression", - "FunctionHandle", - "functionHandle"); - from_json_key( - j, "returnType", p.returnType, "CallExpression", "Type", "returnType"); + "variadicBound", + p.variadicBound, + "TypeVariableConstraint", + "String", + "variadicBound"); from_json_key( j, - "arguments", - p.arguments, - "CallExpression", - "List>", - "arguments"); + "nonDecimalNumericRequired", + p.nonDecimalNumericRequired, + "TypeVariableConstraint", + "bool", + "nonDecimalNumericRequired"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const Aggregation& p) { +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair FunctionKind_enum_table[] = + { // NOLINT: cert-err58-cpp + {FunctionKind::SCALAR, "SCALAR"}, + {FunctionKind::AGGREGATE, "AGGREGATE"}, + {FunctionKind::WINDOW, "WINDOW"}}; +void to_json(json& j, const FunctionKind& e) { + static_assert( + std::is_enum::value, "FunctionKind must be an enum!"); + const auto* it = std::find_if( + std::begin(FunctionKind_enum_table), + std::end(FunctionKind_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(FunctionKind_enum_table)) + ? it + : std::begin(FunctionKind_enum_table)) + ->second; +} +void from_json(const json& j, FunctionKind& e) { + static_assert( + std::is_enum::value, "FunctionKind must be an enum!"); + const auto* it = std::find_if( + std::begin(FunctionKind_enum_table), + std::end(FunctionKind_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(FunctionKind_enum_table)) + ? it + : std::begin(FunctionKind_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const LongVariableConstraint& p) { j = json::object(); - to_json_key(j, "call", p.call, "Aggregation", "CallExpression", "call"); - to_json_key( - j, - "filter", - p.filter, - "Aggregation", - "std::shared_ptr", - "filter"); - to_json_key( - 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, "name", p.name, "LongVariableConstraint", "String", "name"); to_json_key( j, - "arguments", - p.arguments, - "Aggregation", - "List>", - "arguments"); + "expression", + p.expression, + "LongVariableConstraint", + "String", + "expression"); } -void from_json(const json& j, Aggregation& p) { - from_json_key(j, "call", p.call, "Aggregation", "CallExpression", "call"); - from_json_key( - j, - "filter", - p.filter, - "Aggregation", - "std::shared_ptr", - "filter"); - from_json_key( - 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"); +void from_json(const json& j, LongVariableConstraint& p) { + from_json_key(j, "name", p.name, "LongVariableConstraint", "String", "name"); from_json_key( j, - "arguments", - p.arguments, - "Aggregation", - "List>", - "arguments"); + "expression", + p.expression, + "LongVariableConstraint", + "String", + "expression"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveColumnHandle::HiveColumnHandle() noexcept { - _type = "hive"; -} -void to_json(json& j, const HiveColumnHandle& p) { +void to_json(json& j, const Signature& 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, "name", p.name, "Signature", "QualifiedObjectName", "name"); + to_json_key(j, "kind", p.kind, "Signature", "FunctionKind", "kind"); to_json_key( j, - "typeSignature", - p.typeSignature, - "HiveColumnHandle", - "TypeSignature", - "typeSignature"); + "typeVariableConstraints", + p.typeVariableConstraints, + "Signature", + "List", + "typeVariableConstraints"); to_json_key( j, - "hiveColumnIndex", - p.hiveColumnIndex, - "HiveColumnHandle", - "int", - "hiveColumnIndex"); + "longVariableConstraints", + p.longVariableConstraints, + "Signature", + "List", + "longVariableConstraints"); to_json_key( j, - "columnType", - p.columnType, - "HiveColumnHandle", - "ColumnType", - "columnType"); - to_json_key(j, "comment", p.comment, "HiveColumnHandle", "String", "comment"); + "returnType", + p.returnType, + "Signature", + "TypeSignature", + "returnType"); to_json_key( j, - "requiredSubfields", - p.requiredSubfields, - "HiveColumnHandle", - "List", - "requiredSubfields"); + "argumentTypes", + p.argumentTypes, + "Signature", + "List", + "argumentTypes"); to_json_key( j, - "partialAggregation", - p.partialAggregation, - "HiveColumnHandle", - "Aggregation", - "partialAggregation"); + "variableArity", + p.variableArity, + "Signature", + "bool", + "variableArity"); } -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"); +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, - "typeSignature", - p.typeSignature, - "HiveColumnHandle", - "TypeSignature", - "typeSignature"); + "typeVariableConstraints", + p.typeVariableConstraints, + "Signature", + "List", + "typeVariableConstraints"); from_json_key( j, - "hiveColumnIndex", - p.hiveColumnIndex, - "HiveColumnHandle", - "int", - "hiveColumnIndex"); + "longVariableConstraints", + p.longVariableConstraints, + "Signature", + "List", + "longVariableConstraints"); from_json_key( j, - "columnType", - p.columnType, - "HiveColumnHandle", - "ColumnType", - "columnType"); - from_json_key( - j, "comment", p.comment, "HiveColumnHandle", "String", "comment"); + "returnType", + p.returnType, + "Signature", + "TypeSignature", + "returnType"); from_json_key( j, - "requiredSubfields", - p.requiredSubfields, - "HiveColumnHandle", - "List", - "requiredSubfields"); + "argumentTypes", + p.argumentTypes, + "Signature", + "List", + "argumentTypes"); from_json_key( j, - "partialAggregation", - p.partialAggregation, - "HiveColumnHandle", - "Aggregation", - "partialAggregation"); + "variableArity", + p.variableArity, + "Signature", + "bool", + "variableArity"); } } // 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; + getConnectorProtocol(type).to_json(j, p); +} -void to_json(json& j, const BucketConversion& 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()) + " ConnectorTableHandle ConnectorTableHandle"); + } + getConnectorProtocol(type).from_json(j, 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 { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + getConnectorProtocol(type).to_json(j, 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()) + + " ConnectorTableLayoutHandle ConnectorTableLayoutHandle"); + } + getConnectorProtocol(type).from_json(j, p); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const TableHandle& p) { j = json::object(); to_json_key( j, - "tableBucketCount", - p.tableBucketCount, - "BucketConversion", - "int", - "tableBucketCount"); + "connectorId", + p.connectorId, + "TableHandle", + "ConnectorId", + "connectorId"); to_json_key( j, - "partitionBucketCount", - p.partitionBucketCount, - "BucketConversion", - "int", - "partitionBucketCount"); + "connectorHandle", + p.connectorHandle, + "TableHandle", + "ConnectorTableHandle", + "connectorHandle"); to_json_key( j, - "bucketColumnHandles", - p.bucketColumnHandles, - "BucketConversion", - "List", - "bucketColumnHandles"); -} - -void from_json(const json& j, BucketConversion& p) { - from_json_key( - j, - "tableBucketCount", - p.tableBucketCount, - "BucketConversion", - "int", - "tableBucketCount"); + "transaction", + p.transaction, + "TableHandle", + "ConnectorTransactionHandle", + "transaction"); + to_json_key( + j, + "connectorTableLayout", + p.connectorTableLayout, + "TableHandle", + "ConnectorTableLayoutHandle", + "connectorTableLayout"); +} + +void from_json(const json& j, TableHandle& p) { from_json_key( j, - "partitionBucketCount", - p.partitionBucketCount, - "BucketConversion", - "int", - "partitionBucketCount"); + "connectorId", + p.connectorId, + "TableHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "bucketColumnHandles", - p.bucketColumnHandles, - "BucketConversion", - "List", - "bucketColumnHandles"); + "connectorHandle", + p.connectorHandle, + "TableHandle", + "ConnectorTableHandle", + "connectorHandle"); + from_json_key( + j, + "transaction", + p.transaction, + "TableHandle", + "ConnectorTransactionHandle", + "transaction"); + from_json_key( + j, + "connectorTableLayout", + p.connectorTableLayout, + "TableHandle", + "ConnectorTableLayoutHandle", + "connectorTableLayout"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TpchTableHandle::TpchTableHandle() noexcept { - _type = "tpch"; -} -void to_json(json& j, const TpchTableHandle& p) { +void to_json(json& j, const UpdateHandle& p) { j = json::object(); - j["@type"] = "tpch"; - to_json_key( - j, "tableName", p.tableName, "TpchTableHandle", "String", "tableName"); + to_json_key(j, "handle", p.handle, "UpdateHandle", "TableHandle", "handle"); to_json_key( j, - "scaleFactor", - p.scaleFactor, - "TpchTableHandle", - "double", - "scaleFactor"); + "schemaTableName", + p.schemaTableName, + "UpdateHandle", + "SchemaTableName", + "schemaTableName"); } -void from_json(const json& j, TpchTableHandle& p) { - p._type = j["@type"]; - from_json_key( - j, "tableName", p.tableName, "TpchTableHandle", "String", "tableName"); +void from_json(const json& j, UpdateHandle& p) { + from_json_key(j, "handle", p.handle, "UpdateHandle", "TableHandle", "handle"); from_json_key( j, - "scaleFactor", - p.scaleFactor, - "TpchTableHandle", - "double", - "scaleFactor"); + "schemaTableName", + p.schemaTableName, + "UpdateHandle", + "SchemaTableName", + "schemaTableName"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair ExchangeNodeType_enum_table[] = { + // NOLINT: cert-err58-cpp + {ExchangeNodeType::GATHER, "GATHER"}, + {ExchangeNodeType::REPARTITION, "REPARTITION"}, + {ExchangeNodeType::REPLICATE, "REPLICATE"}, +}; +void to_json(json& j, const ExchangeNodeType& e) { + static_assert( + std::is_enum::value, + "ExchangeNodeType must be an enum!"); + const auto* it = std::find_if( + std::begin(ExchangeNodeType_enum_table), + std::end(ExchangeNodeType_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(ExchangeNodeType_enum_table)) + ? it + : std::begin(ExchangeNodeType_enum_table)) + ->second; +} +void from_json(const json& j, ExchangeNodeType& e) { + static_assert( + std::is_enum::value, + "ExchangeNodeType must be an enum!"); + const auto* it = std::find_if( + std::begin(ExchangeNodeType_enum_table), + std::end(ExchangeNodeType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(ExchangeNodeType_enum_table)) + ? it + : std::begin(ExchangeNodeType_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +/* + * 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 == ".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 == ".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); + if (type == "$remote") { + j = *std::static_pointer_cast(p); return; } - - throw TypeError(type + " no abstract type PlanNode "); + getConnectorProtocol(type).to_json(j, p); } -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()) + " PlanNode PlanNode"); + throw ParseError(std::string(e.what()) + " ConnectorPartitioningHandle"); } - if (type == ".AggregationNode") { - std::shared_ptr k = std::make_shared(); + if (type == "$remote") { + auto 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 == ".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); + p = k; return; } + getConnectorProtocol(type).from_json(j, p); +} +} // 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 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"); + from_json_key( + j, + "arguments", + p.arguments, + "Partitioning", + "List>", + "arguments"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const PartitioningScheme& p) { + j = json::object(); + to_json_key( + j, + "partitioning", + p.partitioning, + "PartitioningScheme", + "Partitioning", + "partitioning"); + to_json_key( + j, + "outputLayout", + p.outputLayout, + "PartitioningScheme", + "List", + "outputLayout"); + to_json_key( + j, + "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, PartitioningScheme& p) { + from_json_key( + j, + "partitioning", + p.partitioning, + "PartitioningScheme", + "Partitioning", + "partitioning"); + from_json_key( + j, + "outputLayout", + p.outputLayout, + "PartitioningScheme", + "List", + "outputLayout"); + from_json_key( + j, + "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 { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair ExchangeNodeScope_enum_table[] = + { // NOLINT: cert-err58-cpp + {ExchangeNodeScope::LOCAL, "LOCAL"}, + {ExchangeNodeScope::REMOTE_STREAMING, "REMOTE_STREAMING"}, + {ExchangeNodeScope::REMOTE_MATERIALIZED, "REMOTE_MATERIALIZED"}}; +void to_json(json& j, const ExchangeNodeScope& e) { + static_assert( + std::is_enum::value, + "ExchangeNodeScope must be an enum!"); + const auto* it = std::find_if( + std::begin(ExchangeNodeScope_enum_table), + std::end(ExchangeNodeScope_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(ExchangeNodeScope_enum_table)) + ? it + : std::begin(ExchangeNodeScope_enum_table)) + ->second; +} +void from_json(const json& j, ExchangeNodeScope& e) { + static_assert( + std::is_enum::value, + "ExchangeNodeScope must be an enum!"); + const auto* it = std::find_if( + std::begin(ExchangeNodeScope_enum_table), + std::end(ExchangeNodeScope_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(ExchangeNodeScope_enum_table)) + ? it + : std::begin(ExchangeNodeScope_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +ExchangeNode::ExchangeNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.ExchangeNode"; +} + +void to_json(json& j, const ExchangeNode& p) { + j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.ExchangeNode"; + to_json_key(j, "id", p.id, "ExchangeNode", "PlanNodeId", "id"); + to_json_key(j, "type", p.type, "ExchangeNode", "ExchangeNodeType", "type"); + to_json_key( + j, "scope", p.scope, "ExchangeNode", "ExchangeNodeScope", "scope"); + to_json_key( + j, + "partitioningScheme", + p.partitioningScheme, + "ExchangeNode", + "PartitioningScheme", + "partitioningScheme"); + to_json_key( + j, + "sources", + p.sources, + "ExchangeNode", + "List>", + "sources"); + to_json_key( + j, + "inputs", + p.inputs, + "ExchangeNode", + "List>", + "inputs"); + to_json_key( + j, + "ensureSourceOrdering", + p.ensureSourceOrdering, + "ExchangeNode", + "bool", + "ensureSourceOrdering"); + to_json_key( + j, + "orderingScheme", + p.orderingScheme, + "ExchangeNode", + "OrderingScheme", + "orderingScheme"); +} + +void from_json(const json& j, ExchangeNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "ExchangeNode", "PlanNodeId", "id"); + from_json_key(j, "type", p.type, "ExchangeNode", "ExchangeNodeType", "type"); + from_json_key( + j, "scope", p.scope, "ExchangeNode", "ExchangeNodeScope", "scope"); + from_json_key( + j, + "partitioningScheme", + p.partitioningScheme, + "ExchangeNode", + "PartitioningScheme", + "partitioningScheme"); + from_json_key( + j, + "sources", + p.sources, + "ExchangeNode", + "List>", + "sources"); + from_json_key( + j, + "inputs", + p.inputs, + "ExchangeNode", + "List>", + "inputs"); + from_json_key( + j, + "ensureSourceOrdering", + p.ensureSourceOrdering, + "ExchangeNode", + "bool", + "ensureSourceOrdering"); + from_json_key( + j, + "orderingScheme", + p.orderingScheme, + "ExchangeNode", + "OrderingScheme", + "orderingScheme"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +DeleteHandle::DeleteHandle() noexcept { + _type = "DeleteHandle"; +} + +void to_json(json& j, const DeleteHandle& p) { + j = json::object(); + j["@type"] = "DeleteHandle"; + to_json_key(j, "handle", p.handle, "DeleteHandle", "TableHandle", "handle"); + to_json_key( + j, + "schemaTableName", + p.schemaTableName, + "DeleteHandle", + "SchemaTableName", + "schemaTableName"); +} + +void from_json(const json& j, DeleteHandle& p) { + p._type = j["@type"]; + from_json_key(j, "handle", p.handle, "DeleteHandle", "TableHandle", "handle"); + from_json_key( + j, + "schemaTableName", + p.schemaTableName, + "DeleteHandle", + "SchemaTableName", + "schemaTableName"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const HiveBucketHandle& p) { + j = json::object(); + to_json_key( + j, + "columns", + p.columns, + "HiveBucketHandle", + "List", + "columns"); + to_json_key( + j, + "tableBucketCount", + p.tableBucketCount, + "HiveBucketHandle", + "int", + "tableBucketCount"); + to_json_key( + j, + "readBucketCount", + p.readBucketCount, + "HiveBucketHandle", + "int", + "readBucketCount"); +} + +void from_json(const json& j, HiveBucketHandle& p) { + from_json_key( + j, + "columns", + p.columns, + "HiveBucketHandle", + "List", + "columns"); + from_json_key( + j, + "tableBucketCount", + p.tableBucketCount, + "HiveBucketHandle", + "int", + "tableBucketCount"); + from_json_key( + j, + "readBucketCount", + p.readBucketCount, + "HiveBucketHandle", + "int", + "readBucketCount"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const Assignments& p) { + j = json::object(); + to_json_key( + j, + "assignments", + p.assignments, + "Assignments", + "Map>", + "assignments"); +} + +void from_json(const json& j, Assignments& p) { + from_json_key( + j, + "assignments", + p.assignments, + "Assignments", + "Map>", + "assignments"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair Locality_enum_table[] = + { // NOLINT: cert-err58-cpp + {Locality::UNKNOWN, "UNKNOWN"}, + {Locality::LOCAL, "LOCAL"}, + {Locality::REMOTE, "REMOTE"}}; +void to_json(json& j, const Locality& e) { + static_assert(std::is_enum::value, "Locality must be an enum!"); + const auto* it = std::find_if( + std::begin(Locality_enum_table), + std::end(Locality_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(Locality_enum_table)) ? it + : std::begin(Locality_enum_table)) + ->second; +} +void from_json(const json& j, Locality& e) { + static_assert(std::is_enum::value, "Locality must be an enum!"); + const auto* it = std::find_if( + std::begin(Locality_enum_table), + std::end(Locality_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(Locality_enum_table)) ? it + : std::begin(Locality_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +ProjectNode::ProjectNode() noexcept { + _type = ".ProjectNode"; +} + +void to_json(json& j, const ProjectNode& p) { + j = json::object(); + j["@type"] = ".ProjectNode"; + to_json_key(j, "id", p.id, "ProjectNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "ProjectNode", "PlanNode", "source"); + to_json_key( + j, + "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 { + +void to_json(json& j, const HiveBucketFilter& p) { + j = json::object(); + to_json_key( + j, + "bucketsToKeep", + p.bucketsToKeep, + "HiveBucketFilter", + "List", + "bucketsToKeep"); +} + +void from_json(const json& j, HiveBucketFilter& p) { + from_json_key( + j, + "bucketsToKeep", + p.bucketsToKeep, + "HiveBucketFilter", + "List", + "bucketsToKeep"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +HiveTableLayoutHandle::HiveTableLayoutHandle() noexcept { + _type = "hive"; +} + +void to_json(json& j, const HiveTableLayoutHandle& p) { + j = json::object(); + j["@type"] = "hive"; + to_json_key( + j, + "schemaTableName", + p.schemaTableName, + "HiveTableLayoutHandle", + "SchemaTableName", + "schemaTableName"); + to_json_key( + j, + "tablePath", + p.tablePath, + "HiveTableLayoutHandle", + "String", + "tablePath"); + to_json_key( + j, + "partitionColumns", + p.partitionColumns, + "HiveTableLayoutHandle", + "List", + "partitionColumns"); + to_json_key( + j, + "dataColumns", + p.dataColumns, + "HiveTableLayoutHandle", + "List", + "dataColumns"); + to_json_key( + j, + "tableParameters", + p.tableParameters, + "HiveTableLayoutHandle", + "Map", + "tableParameters"); + to_json_key( + j, + "domainPredicate", + p.domainPredicate, + "HiveTableLayoutHandle", + "TupleDomain", + "domainPredicate"); + to_json_key( + j, + "remainingPredicate", + p.remainingPredicate, + "HiveTableLayoutHandle", + "RowExpression", + "remainingPredicate"); + to_json_key( + j, + "predicateColumns", + p.predicateColumns, + "HiveTableLayoutHandle", + "Map", + "predicateColumns"); + to_json_key( + j, + "partitionColumnPredicate", + p.partitionColumnPredicate, + "HiveTableLayoutHandle", + "TupleDomain>", + "partitionColumnPredicate"); + to_json_key( + j, + "bucketHandle", + p.bucketHandle, + "HiveTableLayoutHandle", + "HiveBucketHandle", + "bucketHandle"); + to_json_key( + j, + "bucketFilter", + p.bucketFilter, + "HiveTableLayoutHandle", + "HiveBucketFilter", + "bucketFilter"); + to_json_key( + j, + "pushdownFilterEnabled", + p.pushdownFilterEnabled, + "HiveTableLayoutHandle", + "bool", + "pushdownFilterEnabled"); + to_json_key( + j, + "layoutString", + p.layoutString, + "HiveTableLayoutHandle", + "String", + "layoutString"); + to_json_key( + j, + "requestedColumns", + p.requestedColumns, + "HiveTableLayoutHandle", + "List", + "requestedColumns"); + to_json_key( + j, + "partialAggregationsPushedDown", + p.partialAggregationsPushedDown, + "HiveTableLayoutHandle", + "bool", + "partialAggregationsPushedDown"); + to_json_key( + j, + "appendRowNumber", + p.appendRowNumber, + "HiveTableLayoutHandle", + "bool", + "appendRowNumber"); + to_json_key( + j, + "footerStatsUnreliable", + p.footerStatsUnreliable, + "HiveTableLayoutHandle", + "bool", + "footerStatsUnreliable"); +} + +void from_json(const json& j, HiveTableLayoutHandle& p) { + p._type = j["@type"]; + from_json_key( + j, + "schemaTableName", + p.schemaTableName, + "HiveTableLayoutHandle", + "SchemaTableName", + "schemaTableName"); + from_json_key( + j, + "tablePath", + p.tablePath, + "HiveTableLayoutHandle", + "String", + "tablePath"); + from_json_key( + j, + "partitionColumns", + p.partitionColumns, + "HiveTableLayoutHandle", + "List", + "partitionColumns"); + from_json_key( + j, + "dataColumns", + p.dataColumns, + "HiveTableLayoutHandle", + "List", + "dataColumns"); + from_json_key( + j, + "tableParameters", + p.tableParameters, + "HiveTableLayoutHandle", + "Map", + "tableParameters"); + from_json_key( + j, + "domainPredicate", + p.domainPredicate, + "HiveTableLayoutHandle", + "TupleDomain", + "domainPredicate"); + from_json_key( + j, + "remainingPredicate", + p.remainingPredicate, + "HiveTableLayoutHandle", + "RowExpression", + "remainingPredicate"); + from_json_key( + j, + "predicateColumns", + p.predicateColumns, + "HiveTableLayoutHandle", + "Map", + "predicateColumns"); + from_json_key( + j, + "partitionColumnPredicate", + p.partitionColumnPredicate, + "HiveTableLayoutHandle", + "TupleDomain>", + "partitionColumnPredicate"); + from_json_key( + j, + "bucketHandle", + p.bucketHandle, + "HiveTableLayoutHandle", + "HiveBucketHandle", + "bucketHandle"); + from_json_key( + j, + "bucketFilter", + p.bucketFilter, + "HiveTableLayoutHandle", + "HiveBucketFilter", + "bucketFilter"); + from_json_key( + j, + "pushdownFilterEnabled", + p.pushdownFilterEnabled, + "HiveTableLayoutHandle", + "bool", + "pushdownFilterEnabled"); + from_json_key( + j, + "layoutString", + p.layoutString, + "HiveTableLayoutHandle", + "String", + "layoutString"); + from_json_key( + j, + "requestedColumns", + p.requestedColumns, + "HiveTableLayoutHandle", + "List", + "requestedColumns"); + from_json_key( + j, + "partialAggregationsPushedDown", + p.partialAggregationsPushedDown, + "HiveTableLayoutHandle", + "bool", + "partialAggregationsPushedDown"); + from_json_key( + j, + "appendRowNumber", + p.appendRowNumber, + "HiveTableLayoutHandle", + "bool", + "appendRowNumber"); + from_json_key( + j, + "footerStatsUnreliable", + p.footerStatsUnreliable, + "HiveTableLayoutHandle", + "bool", + "footerStatsUnreliable"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - throw TypeError(type + " no abstract type PlanNode "); +// 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 { -MarkDistinctNode::MarkDistinctNode() noexcept { - _type = ".MarkDistinctNode"; + +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 to_json(json& j, const MarkDistinctNode& p) { +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 Location& 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, "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, - "markerVariable", - p.markerVariable, - "MarkDistinctNode", + "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 +namespace facebook::presto::protocol { +RowNumberNode::RowNumberNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.RowNumberNode"; +} + +void to_json(json& j, const RowNumberNode& p) { + j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.RowNumberNode"; + to_json_key(j, "id", p.id, "RowNumberNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "RowNumberNode", "PlanNode", "source"); + to_json_key( + j, + "partitionBy", + p.partitionBy, + "RowNumberNode", + "List", + "partitionBy"); + to_json_key( + j, + "rowNumberVariable", + p.rowNumberVariable, + "RowNumberNode", "VariableReferenceExpression", - "markerVariable"); + "rowNumberVariable"); + to_json_key( + 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, RowNumberNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "RowNumberNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "RowNumberNode", "PlanNode", "source"); + from_json_key( + j, + "partitionBy", + p.partitionBy, + "RowNumberNode", + "List", + "partitionBy"); + from_json_key( + j, + "rowNumberVariable", + p.rowNumberVariable, + "RowNumberNode", + "VariableReferenceExpression", + "rowNumberVariable"); + from_json_key( + j, + "maxRowCountPerPartition", + p.maxRowCountPerPartition, + "RowNumberNode", + "Integer", + "maxRowCountPerPartition"); + from_json_key(j, "partial", p.partial, "RowNumberNode", "bool", "partial"); + from_json_key( + j, + "hashVariable", + p.hashVariable, + "RowNumberNode", + "VariableReferenceExpression", + "hashVariable"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair + 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, - "distinctVariables", - p.distinctVariables, - "MarkDistinctNode", - "List", - "distinctVariables"); + "groupedExecutionScanNodes", + p.groupedExecutionScanNodes, + "StageExecutionDescriptor", + "List", + "groupedExecutionScanNodes"); to_json_key( j, - "hashVariable", - p.hashVariable, - "MarkDistinctNode", - "VariableReferenceExpression", - "hashVariable"); + "totalLifespans", + p.totalLifespans, + "StageExecutionDescriptor", + "int", + "totalLifespans"); } -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, "source", p.source, "MarkDistinctNode", "PlanNode", "source"); +void from_json(const json& j, StageExecutionDescriptor& p) { from_json_key( j, - "markerVariable", - p.markerVariable, - "MarkDistinctNode", - "VariableReferenceExpression", - "markerVariable"); + "stageExecutionStrategy", + p.stageExecutionStrategy, + "StageExecutionDescriptor", + "StageExecutionStrategy", + "stageExecutionStrategy"); from_json_key( j, - "distinctVariables", - p.distinctVariables, - "MarkDistinctNode", - "List", - "distinctVariables"); + "groupedExecutionScanNodes", + p.groupedExecutionScanNodes, + "StageExecutionDescriptor", + "List", + "groupedExecutionScanNodes"); from_json_key( j, - "hashVariable", - p.hashVariable, - "MarkDistinctNode", - "VariableReferenceExpression", - "hashVariable"); -} -} // 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 "); + "totalLifespans", + p.totalLifespans, + "StageExecutionDescriptor", + "int", + "totalLifespans"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Domain& p) { +void to_json(json& j, const PlanCostEstimate& p) { j = json::object(); - to_json_key(j, "values", p.values, "Domain", "ValueSet", "values"); - to_json_key(j, "nullAllowed", p.nullAllowed, "Domain", "bool", "nullAllowed"); + to_json_key(j, "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, Domain& p) { - from_json_key(j, "values", p.values, "Domain", "ValueSet", "values"); +void from_json(const json& j, PlanCostEstimate& p) { from_json_key( - j, "nullAllowed", p.nullAllowed, "Domain", "bool", "nullAllowed"); + j, "cpuCost", p.cpuCost, "PlanCostEstimate", "double", "cpuCost"); + from_json_key( + j, "maxMemory", p.maxMemory, "PlanCostEstimate", "double", "maxMemory"); + from_json_key( + j, + "maxMemoryWhenOutputting", + p.maxMemoryWhenOutputting, + "PlanCostEstimate", + "double", + "maxMemoryWhenOutputting"); + from_json_key( + j, + "networkCost", + p.networkCost, + "PlanCostEstimate", + "double", + "networkCost"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const ValueEntry& p) { +void to_json(json& j, const StatsAndCosts& p) { j = json::object(); - to_json_key(j, "type", p.type, "ValueEntry", "Type", "type"); - to_json_key(j, "block", p.block, "ValueEntry", "Block", "block"); + to_json_key( + j, + "stats", + p.stats, + "StatsAndCosts", + "Map", + "stats"); + to_json_key( + j, + "costs", + p.costs, + "StatsAndCosts", + "Map", + "costs"); } -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"); +void from_json(const json& j, StatsAndCosts& p) { + from_json_key( + j, + "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 { -EquatableValueSet::EquatableValueSet() noexcept { - _type = "equatable"; -} -void to_json(json& j, const EquatableValueSet& p) { +void to_json(json& j, const PlanFragment& p) { j = json::object(); - j["@type"] = "equatable"; - to_json_key(j, "type", p.type, "EquatableValueSet", "Type", "type"); + to_json_key(j, "id", p.id, "PlanFragment", "PlanFragmentId", "id"); + to_json_key(j, "root", p.root, "PlanFragment", "PlanNode", "root"); to_json_key( - j, "whiteList", p.whiteList, "EquatableValueSet", "bool", "whiteList"); + j, + "variables", + p.variables, + "PlanFragment", + "List", + "variables"); to_json_key( j, - "entries", - p.entries, - "EquatableValueSet", - "List", - "entries"); + "partitioning", + p.partitioning, + "PlanFragment", + "PartitioningHandle", + "partitioning"); + to_json_key( + j, + "tableScanSchedulingOrder", + p.tableScanSchedulingOrder, + "PlanFragment", + "List", + "tableScanSchedulingOrder"); + to_json_key( + j, + "partitioningScheme", + p.partitioningScheme, + "PlanFragment", + "PartitioningScheme", + "partitioningScheme"); + to_json_key( + j, + "stageExecutionDescriptor", + p.stageExecutionDescriptor, + "PlanFragment", + "StageExecutionDescriptor", + "stageExecutionDescriptor"); + to_json_key( + j, + "outputTableWriterFragment", + p.outputTableWriterFragment, + "PlanFragment", + "bool", + "outputTableWriterFragment"); + to_json_key( + j, + "jsonRepresentation", + p.jsonRepresentation, + "PlanFragment", + "String", + "jsonRepresentation"); } -void from_json(const json& j, EquatableValueSet& p) { - p._type = j["@type"]; - from_json_key(j, "type", p.type, "EquatableValueSet", "Type", "type"); +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, "whiteList", p.whiteList, "EquatableValueSet", "bool", "whiteList"); + j, + "variables", + p.variables, + "PlanFragment", + "List", + "variables"); from_json_key( j, - "entries", - p.entries, - "EquatableValueSet", - "List", - "entries"); + "partitioning", + p.partitioning, + "PlanFragment", + "PartitioningHandle", + "partitioning"); + from_json_key( + j, + "tableScanSchedulingOrder", + p.tableScanSchedulingOrder, + "PlanFragment", + "List", + "tableScanSchedulingOrder"); + from_json_key( + j, + "partitioningScheme", + p.partitioningScheme, + "PlanFragment", + "PartitioningScheme", + "partitioningScheme"); + from_json_key( + j, + "stageExecutionDescriptor", + p.stageExecutionDescriptor, + "PlanFragment", + "StageExecutionDescriptor", + "stageExecutionDescriptor"); + from_json_key( + j, + "outputTableWriterFragment", + p.outputTableWriterFragment, + "PlanFragment", + "bool", + "outputTableWriterFragment"); + from_json_key( + j, + "jsonRepresentation", + p.jsonRepresentation, + "PlanFragment", + "String", + "jsonRepresentation"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +OutputNode::OutputNode() noexcept { + _type = ".OutputNode"; +} -void to_json(json& j, const Specification& p) { +void to_json(json& j, const OutputNode& 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, - "partitionBy", - p.partitionBy, - "Specification", - "List", - "partitionBy"); + "columnNames", + p.columnNames, + "OutputNode", + "List", + "columnNames"); to_json_key( j, - "orderingScheme", - p.orderingScheme, - "Specification", - "OrderingScheme", - "orderingScheme"); + "outputVariables", + p.outputVariables, + "OutputNode", + "List", + "outputVariables"); } -void from_json(const json& j, Specification& p) { +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"); from_json_key( j, - "partitionBy", - p.partitionBy, - "Specification", - "List", - "partitionBy"); + "columnNames", + p.columnNames, + "OutputNode", + "List", + "columnNames"); from_json_key( j, - "orderingScheme", - p.orderingScheme, - "Specification", - "OrderingScheme", - "orderingScheme"); + "outputVariables", + p.outputVariables, + "OutputNode", + "List", + "outputVariables"); } } // 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!"); +static const std::pair Bound_enum_table[] = + { // NOLINT: cert-err58-cpp + {Bound::BELOW, "BELOW"}, + {Bound::EXACTLY, "EXACTLY"}, + {Bound::ABOVE, "ABOVE"}}; +void to_json(json& j, const Bound& e) { + static_assert(std::is_enum::value, "Bound must be an enum!"); const auto* it = std::find_if( - std::begin(WindowType_enum_table), - std::end(WindowType_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(Bound_enum_table), + std::end(Bound_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(WindowType_enum_table)) - ? it - : std::begin(WindowType_enum_table)) + j = ((it != std::end(Bound_enum_table)) ? it : std::begin(Bound_enum_table)) ->second; } -void from_json(const json& j, WindowType& e) { - static_assert(std::is_enum::value, "WindowType must be an enum!"); +void from_json(const json& j, Bound& e) { + static_assert(std::is_enum::value, "Bound must be an enum!"); const auto* it = std::find_if( - std::begin(WindowType_enum_table), - std::end(WindowType_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(Bound_enum_table), + std::end(Bound_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(WindowType_enum_table)) - ? it - : std::begin(WindowType_enum_table)) + e = ((it != std::end(Bound_enum_table)) ? it : std::begin(Bound_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 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 to_json(json& j, const Marker& p) { + j = json::object(); + to_json_key(j, "type", p.type, "Marker", "Type", "type"); + to_json_key(j, "valueBlock", p.valueBlock, "Marker", "Block", "valueBlock"); + to_json_key(j, "bound", p.bound, "Marker", "Bound", "bound"); } -void from_json(const json& j, 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, Marker& p) { + from_json_key(j, "type", p.type, "Marker", "Type", "type"); + from_json_key(j, "valueBlock", p.valueBlock, "Marker", "Block", "valueBlock"); + from_json_key(j, "bound", p.bound, "Marker", "Bound", "bound"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Frame& p) { +void to_json(json& j, const Range& 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"); - to_json_key( - j, - "sortKeyCoercedForFrameEndComparison", - p.sortKeyCoercedForFrameEndComparison, - "Frame", - "VariableReferenceExpression", - "sortKeyCoercedForFrameEndComparison"); - to_json_key( - j, - "originalStartValue", - p.originalStartValue, - "Frame", - "String", - "originalStartValue"); - to_json_key( - j, - "originalEndValue", - p.originalEndValue, - "Frame", - "String", - "originalEndValue"); + 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 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"); - from_json_key( - j, - "endValue", - p.endValue, - "Frame", - "VariableReferenceExpression", - "endValue"); - from_json_key( - j, - "sortKeyCoercedForFrameEndComparison", - p.sortKeyCoercedForFrameEndComparison, - "Frame", - "VariableReferenceExpression", - "sortKeyCoercedForFrameEndComparison"); - from_json_key( - j, - "originalStartValue", - p.originalStartValue, - "Frame", - "String", - "originalStartValue"); +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, - "originalEndValue", - p.originalEndValue, - "Frame", - "String", - "originalEndValue"); + j, "ranges", p.ranges, "SortedRangeSet", "List", "ranges"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +TableScanNode::TableScanNode() noexcept { + _type = ".TableScanNode"; +} -void to_json(json& j, const Function& p) { +void to_json(json& j, const TableScanNode& p) { j = json::object(); + j["@type"] = ".TableScanNode"; + to_json_key(j, "id", p.id, "TableScanNode", "PlanNodeId", "id"); + to_json_key(j, "table", p.table, "TableScanNode", "TableHandle", "table"); to_json_key( j, - "functionCall", - p.functionCall, - "Function", - "CallExpression", - "functionCall"); - to_json_key(j, "frame", p.frame, "Function", "Frame", "frame"); + "outputVariables", + p.outputVariables, + "TableScanNode", + "List", + "outputVariables"); to_json_key( - j, "ignoreNulls", p.ignoreNulls, "Function", "bool", "ignoreNulls"); + j, + "assignments", + p.assignments, + "TableScanNode", + "Map>", + "assignments"); } -void from_json(const json& j, Function& p) { +void from_json(const json& j, TableScanNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "TableScanNode", "PlanNodeId", "id"); + from_json_key(j, "table", p.table, "TableScanNode", "TableHandle", "table"); from_json_key( j, - "functionCall", - p.functionCall, - "Function", - "CallExpression", - "functionCall"); - from_json_key(j, "frame", p.frame, "Function", "Frame", "frame"); + "outputVariables", + p.outputVariables, + "TableScanNode", + "List", + "outputVariables"); from_json_key( - j, "ignoreNulls", p.ignoreNulls, "Function", "bool", "ignoreNulls"); + j, + "assignments", + p.assignments, + "TableScanNode", + "Map>", + "assignments"); } } // 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 { -WindowNode::WindowNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.WindowNode"; +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + getConnectorProtocol(type).to_json(j, p); } -void to_json(json& j, const WindowNode& 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"); + } + getConnectorProtocol(type).from_json(j, p); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const InsertTableHandle& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.WindowNode"; - 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"); - to_json_key( - j, - "specification", - p.specification, - "WindowNode", - "Specification", - "specification"); - to_json_key( - j, - "windowFunctions", - p.windowFunctions, - "WindowNode", - "Map", - "windowFunctions"); to_json_key( j, - "hashVariable", - p.hashVariable, - "WindowNode", - "VariableReferenceExpression", - "hashVariable"); + "connectorId", + p.connectorId, + "InsertTableHandle", + "ConnectorId", + "connectorId"); to_json_key( j, - "prePartitionedInputs", - p.prePartitionedInputs, - "WindowNode", - "List", - "prePartitionedInputs"); + "transactionHandle", + p.transactionHandle, + "InsertTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); to_json_key( j, - "preSortedOrderPrefix", - p.preSortedOrderPrefix, - "WindowNode", - "int", - "preSortedOrderPrefix"); + "connectorHandle", + p.connectorHandle, + "InsertTableHandle", + "ConnectorInsertTableHandle", + "connectorHandle"); } -void from_json(const json& j, WindowNode& p) { - p._type = j["@type"]; +void from_json(const json& j, InsertTableHandle& p) { 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"); + "connectorId", + p.connectorId, + "InsertTableHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "specification", - p.specification, - "WindowNode", - "Specification", - "specification"); + "transactionHandle", + p.transactionHandle, + "InsertTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); from_json_key( j, - "windowFunctions", - p.windowFunctions, - "WindowNode", - "Map", - "windowFunctions"); - from_json_key( + "connectorHandle", + p.connectorHandle, + "InsertTableHandle", + "ConnectorInsertTableHandle", + "connectorHandle"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const RefreshMaterializedViewHandle& p) { + j = json::object(); + to_json_key( j, - "hashVariable", - p.hashVariable, - "WindowNode", - "VariableReferenceExpression", - "hashVariable"); + "handle", + p.handle, + "RefreshMaterializedViewHandle", + "InsertTableHandle", + "handle"); + to_json_key( + j, + "schemaTableName", + p.schemaTableName, + "RefreshMaterializedViewHandle", + "SchemaTableName", + "schemaTableName"); +} + +void from_json(const json& j, RefreshMaterializedViewHandle& p) { from_json_key( j, - "prePartitionedInputs", - p.prePartitionedInputs, - "WindowNode", - "List", - "prePartitionedInputs"); + "handle", + p.handle, + "RefreshMaterializedViewHandle", + "InsertTableHandle", + "handle"); from_json_key( j, - "preSortedOrderPrefix", - p.preSortedOrderPrefix, - "WindowNode", - "int", - "preSortedOrderPrefix"); + "schemaTableName", + p.schemaTableName, + "RefreshMaterializedViewHandle", + "SchemaTableName", + "schemaTableName"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair 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!"); +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(JoinType_enum_table), - std::end(JoinType_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(JoinType_enum_table)) ? it - : std::begin(JoinType_enum_table)) + j = ((it != std::end(WindowType_enum_table)) + ? it + : std::begin(WindowType_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, WindowType& e) { + static_assert(std::is_enum::value, "WindowType 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(WindowType_enum_table), + std::end(WindowType_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)) + e = ((it != std::end(WindowType_enum_table)) + ? it + : std::begin(WindowType_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const 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"); +// 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, 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, BoundType& e) { + static_assert(std::is_enum::value, "BoundType must be an enum!"); + const auto* it = std::find_if( + std::begin(BoundType_enum_table), + std::end(BoundType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(BoundType_enum_table)) + ? it + : std::begin(BoundType_enum_table)) + ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -MergeJoinNode::MergeJoinNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.MergeJoinNode"; -} -void to_json(json& j, const MergeJoinNode& p) { +void to_json(json& j, const Frame& 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, "type", p.type, "Frame", "WindowType", "type"); + to_json_key(j, "startType", p.startType, "Frame", "BoundType", "startType"); to_json_key( j, - "criteria", - p.criteria, - "MergeJoinNode", - "List", - "criteria"); + "startValue", + p.startValue, + "Frame", + "VariableReferenceExpression", + "startValue"); to_json_key( j, - "outputVariables", - p.outputVariables, - "MergeJoinNode", - "List", - "outputVariables"); + "sortKeyCoercedForFrameStartComparison", + p.sortKeyCoercedForFrameStartComparison, + "Frame", + "VariableReferenceExpression", + "sortKeyCoercedForFrameStartComparison"); + to_json_key(j, "endType", p.endType, "Frame", "BoundType", "endType"); to_json_key( j, - "filter", - p.filter, - "MergeJoinNode", - "std::shared_ptr", - "filter"); + "endValue", + p.endValue, + "Frame", + "VariableReferenceExpression", + "endValue"); to_json_key( j, - "leftHashVariable", - p.leftHashVariable, - "MergeJoinNode", + "sortKeyCoercedForFrameEndComparison", + p.sortKeyCoercedForFrameEndComparison, + "Frame", "VariableReferenceExpression", - "leftHashVariable"); + "sortKeyCoercedForFrameEndComparison"); to_json_key( j, - "rightHashVariable", - p.rightHashVariable, - "MergeJoinNode", - "VariableReferenceExpression", - "rightHashVariable"); + "originalStartValue", + p.originalStartValue, + "Frame", + "String", + "originalStartValue"); + to_json_key( + j, + "originalEndValue", + p.originalEndValue, + "Frame", + "String", + "originalEndValue"); } -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"); +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, - "criteria", - p.criteria, - "MergeJoinNode", - "List", - "criteria"); + "startValue", + p.startValue, + "Frame", + "VariableReferenceExpression", + "startValue"); from_json_key( j, - "outputVariables", - p.outputVariables, - "MergeJoinNode", - "List", - "outputVariables"); + "sortKeyCoercedForFrameStartComparison", + p.sortKeyCoercedForFrameStartComparison, + "Frame", + "VariableReferenceExpression", + "sortKeyCoercedForFrameStartComparison"); + from_json_key(j, "endType", p.endType, "Frame", "BoundType", "endType"); from_json_key( j, - "filter", - p.filter, - "MergeJoinNode", - "std::shared_ptr", - "filter"); + "endValue", + p.endValue, + "Frame", + "VariableReferenceExpression", + "endValue"); from_json_key( j, - "leftHashVariable", - p.leftHashVariable, - "MergeJoinNode", + "sortKeyCoercedForFrameEndComparison", + p.sortKeyCoercedForFrameEndComparison, + "Frame", "VariableReferenceExpression", - "leftHashVariable"); + "sortKeyCoercedForFrameEndComparison"); + from_json_key( + j, + "originalStartValue", + p.originalStartValue, + "Frame", + "String", + "originalStartValue"); from_json_key( j, - "rightHashVariable", - p.rightHashVariable, - "MergeJoinNode", - "VariableReferenceExpression", - "rightHashVariable"); -} -} // 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"); -} - -void from_json(const json& j, RemoteTransactionHandle& p) { - p._type = j["@type"]; - from_json_key( - j, "dummy", p.dummy, "RemoteTransactionHandle", "String", "dummy"); + "originalEndValue", + p.originalEndValue, + "Frame", + "String", + "originalEndValue"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -ValuesNode::ValuesNode() noexcept { - _type = ".ValuesNode"; -} -void to_json(json& j, const ValuesNode& p) { +void to_json(json& j, const Function& p) { j = json::object(); - j["@type"] = ".ValuesNode"; - to_json_key( - j, "location", p.location, "ValuesNode", "SourceLocation", "location"); - to_json_key(j, "id", p.id, "ValuesNode", "PlanNodeId", "id"); - to_json_key( - j, - "outputVariables", - p.outputVariables, - "ValuesNode", - "List", - "outputVariables"); to_json_key( j, - "rows", - p.rows, - "ValuesNode", - "List>>", - "rows"); + "functionCall", + p.functionCall, + "Function", + "CallExpression", + "functionCall"); + to_json_key(j, "frame", p.frame, "Function", "Frame", "frame"); to_json_key( - j, - "valuesNodeLabel", - p.valuesNodeLabel, - "ValuesNode", - "String", - "valuesNodeLabel"); + j, "ignoreNulls", p.ignoreNulls, "Function", "bool", "ignoreNulls"); } -void from_json(const json& j, ValuesNode& p) { - p._type = j["@type"]; - from_json_key( - j, "location", p.location, "ValuesNode", "SourceLocation", "location"); - from_json_key(j, "id", p.id, "ValuesNode", "PlanNodeId", "id"); - from_json_key( - j, - "outputVariables", - p.outputVariables, - "ValuesNode", - "List", - "outputVariables"); +void from_json(const json& j, Function& p) { from_json_key( j, - "rows", - p.rows, - "ValuesNode", - "List>>", - "rows"); + "functionCall", + p.functionCall, + "Function", + "CallExpression", + "functionCall"); + from_json_key(j, "frame", p.frame, "Function", "Frame", "frame"); from_json_key( - j, - "valuesNodeLabel", - p.valuesNodeLabel, - "ValuesNode", - "String", - "valuesNodeLabel"); + j, "ignoreNulls", p.ignoreNulls, "Function", "bool", "ignoreNulls"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -GroupIdNode::GroupIdNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.GroupIdNode"; +WindowNode::WindowNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.WindowNode"; } -void to_json(json& j, const GroupIdNode& p) { +void to_json(json& j, const WindowNode& 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"); + j["@type"] = "com.facebook.presto.sql.planner.plan.WindowNode"; to_json_key( j, - "groupingSets", - p.groupingSets, - "GroupIdNode", - "List>", - "groupingSets"); + "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, - "groupingColumns", - p.groupingColumns, - "GroupIdNode", - "Map", - "groupingColumns"); + "specification", + p.specification, + "WindowNode", + "Specification", + "specification"); to_json_key( j, - "aggregationArguments", - p.aggregationArguments, - "GroupIdNode", - "List", - "aggregationArguments"); + "windowFunctions", + p.windowFunctions, + "WindowNode", + "Map", + "windowFunctions"); to_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "GroupIdNode", + "hashVariable", + p.hashVariable, + "WindowNode", "VariableReferenceExpression", - "groupIdVariable"); + "hashVariable"); + to_json_key( + j, + "prePartitionedInputs", + p.prePartitionedInputs, + "WindowNode", + "List", + "prePartitionedInputs"); + to_json_key( + j, + "preSortedOrderPrefix", + p.preSortedOrderPrefix, + "WindowNode", + "int", + "preSortedOrderPrefix"); } -void from_json(const json& j, GroupIdNode& p) { +void from_json(const json& j, WindowNode& 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"); + "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, - "groupingColumns", - p.groupingColumns, - "GroupIdNode", - "Map", - "groupingColumns"); + "specification", + p.specification, + "WindowNode", + "Specification", + "specification"); from_json_key( j, - "aggregationArguments", - p.aggregationArguments, - "GroupIdNode", - "List", - "aggregationArguments"); + "windowFunctions", + p.windowFunctions, + "WindowNode", + "Map", + "windowFunctions"); from_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "GroupIdNode", - "VariableReferenceExpression", - "groupIdVariable"); -} -} // 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 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; + "hashVariable", + p.hashVariable, + "WindowNode", + "VariableReferenceExpression", + "hashVariable"); + from_json_key( + j, + "prePartitionedInputs", + p.prePartitionedInputs, + "WindowNode", + "List", + "prePartitionedInputs"); + from_json_key( + j, + "preSortedOrderPrefix", + p.preSortedOrderPrefix, + "WindowNode", + "int", + "preSortedOrderPrefix"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const CacheQuotaRequirement& 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, - "cacheQuotaScope", - p.cacheQuotaScope, - "CacheQuotaRequirement", - "CacheQuotaScope", - "cacheQuotaScope"); - to_json_key( - j, "quota", p.quota, "CacheQuotaRequirement", "DataSize", "quota"); + "tableHandle", + p.tableHandle, + "DeleteScanInfo", + "TableHandle", + "tableHandle"); } -void from_json(const json& j, CacheQuotaRequirement& p) { +void from_json(const json& j, DeleteScanInfo& p) { + from_json_key(j, "id", p.id, "DeleteScanInfo", "PlanNodeId", "id"); from_json_key( j, - "cacheQuotaScope", - p.cacheQuotaScope, - "CacheQuotaRequirement", - "CacheQuotaScope", - "cacheQuotaScope"); - from_json_key( - j, "quota", p.quota, "CacheQuotaRequirement", "DataSize", "quota"); -} -} // 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; - getConnectorProtocol(type).to_json(j, 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()) + " ColumnHandle ColumnHandle"); - } - getConnectorProtocol(type).from_json(j, p); + "tableHandle", + p.tableHandle, + "DeleteScanInfo", + "TableHandle", + "tableHandle"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TpchSplit::TpchSplit() noexcept { - _type = "tpch"; -} -void to_json(json& j, const TpchSplit& p) { +void to_json(json& j, const AnalyzeTableHandle& p) { j = json::object(); - j["@type"] = "tpch"; to_json_key( j, - "tableHandle", - p.tableHandle, - "TpchSplit", - "TpchTableHandle", - "tableHandle"); - to_json_key(j, "partNumber", p.partNumber, "TpchSplit", "int", "partNumber"); - to_json_key(j, "totalParts", p.totalParts, "TpchSplit", "int", "totalParts"); + "connectorId", + p.connectorId, + "AnalyzeTableHandle", + "ConnectorId", + "connectorId"); to_json_key( j, - "addresses", - p.addresses, - "TpchSplit", - "List", - "addresses"); + "transactionHandle", + p.transactionHandle, + "AnalyzeTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); to_json_key( j, - "predicate", - p.predicate, - "TpchSplit", - "TupleDomain>", - "predicate"); + "connectorHandle", + p.connectorHandle, + "AnalyzeTableHandle", + "ConnectorTableHandle", + "connectorHandle"); } -void from_json(const json& j, TpchSplit& p) { - p._type = j["@type"]; +void from_json(const json& j, AnalyzeTableHandle& p) { from_json_key( j, - "tableHandle", - p.tableHandle, - "TpchSplit", - "TpchTableHandle", - "tableHandle"); - from_json_key( - j, "partNumber", p.partNumber, "TpchSplit", "int", "partNumber"); - from_json_key( - j, "totalParts", p.totalParts, "TpchSplit", "int", "totalParts"); + "connectorId", + p.connectorId, + "AnalyzeTableHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "addresses", - p.addresses, - "TpchSplit", - "List", - "addresses"); + "transactionHandle", + p.transactionHandle, + "AnalyzeTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); from_json_key( j, - "predicate", - p.predicate, - "TpchSplit", - "TupleDomain>", - "predicate"); + "connectorHandle", + p.connectorHandle, + "AnalyzeTableHandle", + "ConnectorTableHandle", + "connectorHandle"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; -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"); -} + 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; + } -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"); + throw TypeError(type + " no abstract type ExecutionWriterTarget "); } -} // 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 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; +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 IcebergTableName& p) { +void to_json(json& j, const TableWriteInfo& p) { j = json::object(); - to_json_key( - j, "tableName", p.tableName, "IcebergTableName", "String", "tableName"); to_json_key( j, - "tableType", - p.tableType, - "IcebergTableName", - "IcebergTableType", - "tableType"); + "writerTarget", + p.writerTarget, + "TableWriteInfo", + "ExecutionWriterTarget", + "writerTarget"); to_json_key( - j, "snapshotId", p.snapshotId, "IcebergTableName", "Long", "snapshotId"); + j, + "analyzeTableHandle", + p.analyzeTableHandle, + "TableWriteInfo", + "AnalyzeTableHandle", + "analyzeTableHandle"); to_json_key( j, - "changelogEndSnapshot", - p.changelogEndSnapshot, - "IcebergTableName", - "Long", - "changelogEndSnapshot"); + "deleteScanInfo", + p.deleteScanInfo, + "TableWriteInfo", + "DeleteScanInfo", + "deleteScanInfo"); } -void from_json(const json& j, IcebergTableName& p) { - from_json_key( - j, "tableName", p.tableName, "IcebergTableName", "String", "tableName"); +void from_json(const json& j, TableWriteInfo& p) { from_json_key( j, - "tableType", - p.tableType, - "IcebergTableName", - "IcebergTableType", - "tableType"); + "writerTarget", + p.writerTarget, + "TableWriteInfo", + "ExecutionWriterTarget", + "writerTarget"); from_json_key( - j, "snapshotId", p.snapshotId, "IcebergTableName", "Long", "snapshotId"); + j, + "analyzeTableHandle", + p.analyzeTableHandle, + "TableWriteInfo", + "AnalyzeTableHandle", + "analyzeTableHandle"); from_json_key( j, - "changelogEndSnapshot", - p.changelogEndSnapshot, - "IcebergTableName", - "Long", - "changelogEndSnapshot"); + "deleteScanInfo", + p.deleteScanInfo, + "TableWriteInfo", + "DeleteScanInfo", + "deleteScanInfo"); } } // 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 { -IcebergTableHandle::IcebergTableHandle() noexcept { - _type = "hive-iceberg"; + +void to_json(json& j, const Duration& p) { + j = p.toString(); } -void to_json(json& j, const IcebergTableHandle& p) { +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 ResourceEstimates& p) { j = json::object(); - j["@type"] = "hive-iceberg"; - to_json_key( - j, - "schemaName", - p.schemaName, - "IcebergTableHandle", - "String", - "schemaName"); - to_json_key( - j, - "icebergTableName", - p.icebergTableName, - "IcebergTableHandle", - "IcebergTableName", - "icebergTableName"); - to_json_key( - j, - "snapshotSpecified", - p.snapshotSpecified, - "IcebergTableHandle", - "bool", - "snapshotSpecified"); - to_json_key( - j, - "predicate", - p.predicate, - "IcebergTableHandle", - "TupleDomain", - "predicate"); - to_json_key( - j, - "outputPath", - p.outputPath, - "IcebergTableHandle", - "String", - "outputPath"); to_json_key( j, - "storageProperties", - p.storageProperties, - "IcebergTableHandle", - "Map", - "storageProperties"); + "executionTime", + p.executionTime, + "ResourceEstimates", + "Duration", + "executionTime"); to_json_key( - j, - "tableSchemaJson", - p.tableSchemaJson, - "IcebergTableHandle", - "String", - "tableSchemaJson"); + j, "cpuTime", p.cpuTime, "ResourceEstimates", "Duration", "cpuTime"); to_json_key( j, - "partitionFieldIds", - p.partitionFieldIds, - "IcebergTableHandle", - "List", - "partitionFieldIds"); + "peakMemory", + p.peakMemory, + "ResourceEstimates", + "DataSize", + "peakMemory"); to_json_key( j, - "equalityFieldIds", - p.equalityFieldIds, - "IcebergTableHandle", - "List", - "equalityFieldIds"); + "peakTaskMemory", + p.peakTaskMemory, + "ResourceEstimates", + "DataSize", + "peakTaskMemory"); } -void from_json(const json& j, IcebergTableHandle& p) { - p._type = j["@type"]; - from_json_key( - j, - "schemaName", - p.schemaName, - "IcebergTableHandle", - "String", - "schemaName"); - from_json_key( - j, - "icebergTableName", - p.icebergTableName, - "IcebergTableHandle", - "IcebergTableName", - "icebergTableName"); - from_json_key( - j, - "snapshotSpecified", - p.snapshotSpecified, - "IcebergTableHandle", - "bool", - "snapshotSpecified"); - from_json_key( - j, - "predicate", - p.predicate, - "IcebergTableHandle", - "TupleDomain", - "predicate"); - from_json_key( - j, - "outputPath", - p.outputPath, - "IcebergTableHandle", - "String", - "outputPath"); +void from_json(const json& j, ResourceEstimates& p) { from_json_key( j, - "storageProperties", - p.storageProperties, - "IcebergTableHandle", - "Map", - "storageProperties"); + "executionTime", + p.executionTime, + "ResourceEstimates", + "Duration", + "executionTime"); from_json_key( - j, - "tableSchemaJson", - p.tableSchemaJson, - "IcebergTableHandle", - "String", - "tableSchemaJson"); + j, "cpuTime", p.cpuTime, "ResourceEstimates", "Duration", "cpuTime"); from_json_key( j, - "partitionFieldIds", - p.partitionFieldIds, - "IcebergTableHandle", - "List", - "partitionFieldIds"); + "peakMemory", + p.peakMemory, + "ResourceEstimates", + "DataSize", + "peakMemory"); from_json_key( j, - "equalityFieldIds", - p.equalityFieldIds, - "IcebergTableHandle", - "List", - "equalityFieldIds"); + "peakTaskMemory", + p.peakTaskMemory, + "ResourceEstimates", + "DataSize", + "peakTaskMemory"); +} +} // 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 Determinism_enum_table[] = { + // NOLINT: cert-err58-cpp + {Determinism::DETERMINISTIC, "DETERMINISTIC"}, + {Determinism::NOT_DETERMINISTIC, "NOT_DETERMINISTIC"}, +}; +void to_json(json& j, const Determinism& e) { + static_assert( + std::is_enum::value, "Determinism must be an enum!"); + const auto* it = std::find_if( + std::begin(Determinism_enum_table), + std::end(Determinism_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(Determinism_enum_table)) + ? it + : std::begin(Determinism_enum_table)) + ->second; +} +void from_json(const json& j, Determinism& e) { + static_assert( + std::is_enum::value, "Determinism must be an enum!"); + const auto* it = std::find_if( + std::begin(Determinism_enum_table), + std::end(Determinism_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(Determinism_enum_table)) + ? it + : std::begin(Determinism_enum_table)) + ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const Column& p) { +// 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 { + +void to_json(json& j, const Language& 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, "language", p.language, "Language", "String", "language"); } -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, Language& p) { + from_json_key(j, "language", p.language, "Language", "String", "language"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -IcebergTableLayoutHandle::IcebergTableLayoutHandle() noexcept { - _type = "hive-iceberg"; -} -void to_json(json& j, const IcebergTableLayoutHandle& p) { +void to_json(json& j, const RoutineCharacteristics& 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"); + "language", + p.language, + "RoutineCharacteristics", + "Language", + "language"); to_json_key( j, - "partitionColumnPredicate", - p.partitionColumnPredicate, - "IcebergTableLayoutHandle", - "TupleDomain>", - "partitionColumnPredicate"); + "determinism", + p.determinism, + "RoutineCharacteristics", + "Determinism", + "determinism"); to_json_key( j, - "table", - p.table, - "IcebergTableLayoutHandle", - "IcebergTableHandle", - "table"); + "nullCallClause", + p.nullCallClause, + "RoutineCharacteristics", + "NullCallClause", + "nullCallClause"); } -void from_json(const json& j, IcebergTableLayoutHandle& p) { - p._type = j["@type"]; - from_json_key( - j, - "partitionColumns", - p.partitionColumns, - "IcebergTableLayoutHandle", - "List", - "partitionColumns"); - from_json_key( - j, - "dataColumns", - p.dataColumns, - "IcebergTableLayoutHandle", - "List", - "dataColumns"); - from_json_key( - j, - "domainPredicate", - p.domainPredicate, - "IcebergTableLayoutHandle", - "TupleDomain", - "domainPredicate"); - from_json_key( - j, - "remainingPredicate", - p.remainingPredicate, - "IcebergTableLayoutHandle", - "RowExpression", - "remainingPredicate"); - from_json_key( - j, - "predicateColumns", - p.predicateColumns, - "IcebergTableLayoutHandle", - "Map", - "predicateColumns"); - from_json_key( - j, - "requestedColumns", - p.requestedColumns, - "IcebergTableLayoutHandle", - "List", - "requestedColumns"); +void from_json(const json& j, RoutineCharacteristics& p) { from_json_key( j, - "pushdownFilterEnabled", - p.pushdownFilterEnabled, - "IcebergTableLayoutHandle", - "bool", - "pushdownFilterEnabled"); + "language", + p.language, + "RoutineCharacteristics", + "Language", + "language"); from_json_key( j, - "partitionColumnPredicate", - p.partitionColumnPredicate, - "IcebergTableLayoutHandle", - "TupleDomain>", - "partitionColumnPredicate"); + "determinism", + p.determinism, + "RoutineCharacteristics", + "Determinism", + "determinism"); from_json_key( j, - "table", - p.table, - "IcebergTableLayoutHandle", - "IcebergTableHandle", - "table"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -FilterNode::FilterNode() noexcept { - _type = ".FilterNode"; -} - -void to_json(json& j, const FilterNode& p) { - j = json::object(); - j["@type"] = ".FilterNode"; - to_json_key(j, "id", p.id, "FilterNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "FilterNode", "PlanNode", "source"); - to_json_key( - j, "predicate", p.predicate, "FilterNode", "RowExpression", "predicate"); -} - -void from_json(const json& j, FilterNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "FilterNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "FilterNode", "PlanNode", "source"); - from_json_key( - j, "predicate", p.predicate, "FilterNode", "RowExpression", "predicate"); + "nullCallClause", + p.nullCallClause, + "RoutineCharacteristics", + "NullCallClause", + "nullCallClause"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const StatisticAggregations& p) { +void to_json(json& j, const SqlInvokedFunction& p) { j = json::object(); to_json_key( j, - "outputVariables", - p.outputVariables, - "StatisticAggregations", - "List", - "outputVariables"); + "parameters", + p.parameters, + "SqlInvokedFunction", + "List", + "parameters"); to_json_key( j, - "aggregations", - p.aggregations, - "StatisticAggregations", - "Map", - "aggregations"); + "description", + p.description, + "SqlInvokedFunction", + "String", + "description"); to_json_key( j, - "groupingVariables", - p.groupingVariables, - "StatisticAggregations", - "List", - "groupingVariables"); + "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, StatisticAggregations& p) { +void from_json(const json& j, SqlInvokedFunction& p) { from_json_key( j, - "outputVariables", - p.outputVariables, - "StatisticAggregations", - "List", - "outputVariables"); + "parameters", + p.parameters, + "SqlInvokedFunction", + "List", + "parameters"); from_json_key( j, - "aggregations", - p.aggregations, - "StatisticAggregations", - "Map", - "aggregations"); + "description", + p.description, + "SqlInvokedFunction", + "String", + "description"); from_json_key( j, - "groupingVariables", - p.groupingVariables, - "StatisticAggregations", - "List", - "groupingVariables"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Assignments& p) { - j = json::object(); - to_json_key( + "routineCharacteristics", + p.routineCharacteristics, + "SqlInvokedFunction", + "RoutineCharacteristics", + "routineCharacteristics"); + from_json_key(j, "body", p.body, "SqlInvokedFunction", "String", "body"); + from_json_key( j, - "assignments", - p.assignments, - "Assignments", - "Map>", - "assignments"); -} - -void from_json(const json& j, Assignments& p) { + "signature", + p.signature, + "SqlInvokedFunction", + "Signature", + "signature"); from_json_key( j, - "assignments", - p.assignments, - "Assignments", - "Map>", - "assignments"); -} -} // 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; - } - getConnectorProtocol(type).to_json(j, 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"); - } - - if (type == "$remote") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - getConnectorProtocol(type).from_json(j, p); + "functionId", + p.functionId, + "SqlInvokedFunction", + "SqlFunctionId", + "functionId"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const PartitioningHandle& p) { +void to_json(json& j, const SessionRepresentation& p) { j = json::object(); + to_json_key( + j, "queryId", p.queryId, "SessionRepresentation", "String", "queryId"); to_json_key( j, - "connectorId", - p.connectorId, - "PartitioningHandle", - "ConnectorId", - "connectorId"); + "transactionId", + p.transactionId, + "SessionRepresentation", + "TransactionId", + "transactionId"); to_json_key( j, - "transactionHandle", - p.transactionHandle, - "PartitioningHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "clientTransactionSupport", + p.clientTransactionSupport, + "SessionRepresentation", + "bool", + "clientTransactionSupport"); + to_json_key(j, "user", p.user, "SessionRepresentation", "String", "user"); to_json_key( j, - "connectorHandle", - p.connectorHandle, - "PartitioningHandle", - "ConnectorPartitioningHandle", - "connectorHandle"); -} - -void from_json(const json& j, PartitioningHandle& p) { - from_json_key( + "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, - "connectorId", - p.connectorId, - "PartitioningHandle", - "ConnectorId", - "connectorId"); - from_json_key( + "traceToken", + p.traceToken, + "SessionRepresentation", + "String", + "traceToken"); + to_json_key( j, - "transactionHandle", - p.transactionHandle, - "PartitioningHandle", - "ConnectorTransactionHandle", - "transactionHandle"); - from_json_key( + "timeZoneKey", + p.timeZoneKey, + "SessionRepresentation", + "TimeZoneKey", + "timeZoneKey"); + to_json_key( + j, "locale", p.locale, "SessionRepresentation", "Locale", "locale"); + to_json_key( j, - "connectorHandle", - p.connectorHandle, - "PartitioningHandle", - "ConnectorPartitioningHandle", - "connectorHandle"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Partitioning& p) { - j = json::object(); + "remoteUserAddress", + p.remoteUserAddress, + "SessionRepresentation", + "String", + "remoteUserAddress"); + to_json_key( + j, + "userAgent", + p.userAgent, + "SessionRepresentation", + "String", + "userAgent"); + to_json_key( + j, + "clientInfo", + p.clientInfo, + "SessionRepresentation", + "String", + "clientInfo"); to_json_key( - j, "handle", p.handle, "Partitioning", "PartitioningHandle", "handle"); + j, + "clientTags", + p.clientTags, + "SessionRepresentation", + "List", + "clientTags"); to_json_key( j, - "arguments", - p.arguments, - "Partitioning", - "List>", - "arguments"); -} - -void from_json(const json& j, Partitioning& p) { - from_json_key( - j, "handle", p.handle, "Partitioning", "PartitioningHandle", "handle"); - from_json_key( + "resourceEstimates", + p.resourceEstimates, + "SessionRepresentation", + "ResourceEstimates", + "resourceEstimates"); + to_json_key( j, - "arguments", - p.arguments, - "Partitioning", - "List>", - "arguments"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const PartitioningScheme& p) { - j = json::object(); + "startTime", + p.startTime, + "SessionRepresentation", + "int64_t", + "startTime"); to_json_key( j, - "partitioning", - p.partitioning, - "PartitioningScheme", - "Partitioning", - "partitioning"); + "systemProperties", + p.systemProperties, + "SessionRepresentation", + "Map", + "systemProperties"); to_json_key( j, - "outputLayout", - p.outputLayout, - "PartitioningScheme", - "List", - "outputLayout"); + "catalogProperties", + p.catalogProperties, + "SessionRepresentation", + "Map>", + "catalogProperties"); to_json_key( j, - "hashColumn", - p.hashColumn, - "PartitioningScheme", - "VariableReferenceExpression", - "hashColumn"); + "unprocessedCatalogProperties", + p.unprocessedCatalogProperties, + "SessionRepresentation", + "Map>", + "unprocessedCatalogProperties"); to_json_key( j, - "replicateNullsAndAny", - p.replicateNullsAndAny, - "PartitioningScheme", - "bool", - "replicateNullsAndAny"); + "roles", + p.roles, + "SessionRepresentation", + "Map", + "roles"); to_json_key( j, - "bucketToPartition", - p.bucketToPartition, - "PartitioningScheme", - "List", - "bucketToPartition"); + "preparedStatements", + p.preparedStatements, + "SessionRepresentation", + "Map", + "preparedStatements"); + to_json_key( + j, + "sessionFunctions", + p.sessionFunctions, + "SessionRepresentation", + "Map", + "sessionFunctions"); } -void from_json(const json& j, PartitioningScheme& p) { - from_json_key( - j, - "partitioning", - p.partitioning, - "PartitioningScheme", - "Partitioning", - "partitioning"); +void from_json(const json& j, SessionRepresentation& p) { from_json_key( - j, - "outputLayout", - p.outputLayout, - "PartitioningScheme", - "List", - "outputLayout"); + j, "queryId", p.queryId, "SessionRepresentation", "String", "queryId"); from_json_key( j, - "hashColumn", - p.hashColumn, - "PartitioningScheme", - "VariableReferenceExpression", - "hashColumn"); + "transactionId", + p.transactionId, + "SessionRepresentation", + "TransactionId", + "transactionId"); from_json_key( j, - "replicateNullsAndAny", - p.replicateNullsAndAny, - "PartitioningScheme", + "clientTransactionSupport", + p.clientTransactionSupport, + "SessionRepresentation", "bool", - "replicateNullsAndAny"); + "clientTransactionSupport"); + from_json_key(j, "user", p.user, "SessionRepresentation", "String", "user"); from_json_key( j, - "bucketToPartition", - p.bucketToPartition, - "PartitioningScheme", - "List", - "bucketToPartition"); -} -} // 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, - "rowCountVariable", - p.rowCountVariable, - "TableWriterNode", - "VariableReferenceExpression", - "rowCountVariable"); - to_json_key( - j, - "fragmentVariable", - p.fragmentVariable, - "TableWriterNode", - "VariableReferenceExpression", - "fragmentVariable"); - to_json_key( - j, - "tableCommitContextVariable", - p.tableCommitContextVariable, - "TableWriterNode", - "VariableReferenceExpression", - "tableCommitContextVariable"); - to_json_key( - j, - "columns", - p.columns, - "TableWriterNode", - "List", - "columns"); - to_json_key( + "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, - "columnNames", - p.columnNames, - "TableWriterNode", - "List", - "columnNames"); - to_json_key( + "traceToken", + p.traceToken, + "SessionRepresentation", + "String", + "traceToken"); + from_json_key( j, - "notNullColumnVariables", - p.notNullColumnVariables, - "TableWriterNode", - "List", - "notNullColumnVariables"); - to_json_key( + "timeZoneKey", + p.timeZoneKey, + "SessionRepresentation", + "TimeZoneKey", + "timeZoneKey"); + from_json_key( + j, "locale", p.locale, "SessionRepresentation", "Locale", "locale"); + from_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "partitioningScheme"); - to_json_key( + "remoteUserAddress", + p.remoteUserAddress, + "SessionRepresentation", + "String", + "remoteUserAddress"); + from_json_key( j, - "preferredShufflePartitioningScheme", - p.preferredShufflePartitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "preferredShufflePartitioningScheme"); - to_json_key( + "userAgent", + p.userAgent, + "SessionRepresentation", + "String", + "userAgent"); + from_json_key( j, - "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"); + "clientInfo", + p.clientInfo, + "SessionRepresentation", + "String", + "clientInfo"); from_json_key( j, - "rowCountVariable", - p.rowCountVariable, - "TableWriterNode", - "VariableReferenceExpression", - "rowCountVariable"); + "clientTags", + p.clientTags, + "SessionRepresentation", + "List", + "clientTags"); from_json_key( j, - "fragmentVariable", - p.fragmentVariable, - "TableWriterNode", - "VariableReferenceExpression", - "fragmentVariable"); + "resourceEstimates", + p.resourceEstimates, + "SessionRepresentation", + "ResourceEstimates", + "resourceEstimates"); from_json_key( j, - "tableCommitContextVariable", - p.tableCommitContextVariable, - "TableWriterNode", - "VariableReferenceExpression", - "tableCommitContextVariable"); + "startTime", + p.startTime, + "SessionRepresentation", + "int64_t", + "startTime"); from_json_key( j, - "columns", - p.columns, - "TableWriterNode", - "List", - "columns"); + "systemProperties", + p.systemProperties, + "SessionRepresentation", + "Map", + "systemProperties"); from_json_key( j, - "columnNames", - p.columnNames, - "TableWriterNode", - "List", - "columnNames"); + "catalogProperties", + p.catalogProperties, + "SessionRepresentation", + "Map>", + "catalogProperties"); from_json_key( j, - "notNullColumnVariables", - p.notNullColumnVariables, - "TableWriterNode", - "List", - "notNullColumnVariables"); + "unprocessedCatalogProperties", + p.unprocessedCatalogProperties, + "SessionRepresentation", + "Map>", + "unprocessedCatalogProperties"); from_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "partitioningScheme"); + "roles", + p.roles, + "SessionRepresentation", + "Map", + "roles"); from_json_key( j, - "preferredShufflePartitioningScheme", - p.preferredShufflePartitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "preferredShufflePartitioningScheme"); + "preparedStatements", + p.preparedStatements, + "SessionRepresentation", + "Map", + "preparedStatements"); from_json_key( j, - "statisticsAggregation", - p.statisticsAggregation, - "TableWriterNode", - "StatisticAggregations", - "statisticsAggregation"); + "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 - 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 BufferType_enum_table[] = + { // NOLINT: cert-err58-cpp + {BufferType::PARTITIONED, "PARTITIONED"}, + {BufferType::BROADCAST, "BROADCAST"}, + {BufferType::ARBITRARY, "ARBITRARY"}, + {BufferType::DISCARDING, "DISCARDING"}, + {BufferType::SPOOLING, "SPOOLING"}}; +void to_json(json& j, const BufferType& e) { + static_assert(std::is_enum::value, "BufferType must be an enum!"); 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(BufferType_enum_table), + std::end(BufferType_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(JoinDistributionType_enum_table)) + j = ((it != std::end(BufferType_enum_table)) ? it - : std::begin(JoinDistributionType_enum_table)) + : std::begin(BufferType_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, BufferType& e) { + static_assert(std::is_enum::value, "BufferType 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(BufferType_enum_table), + std::end(BufferType_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(JoinDistributionType_enum_table)) + e = ((it != std::end(BufferType_enum_table)) ? it - : std::begin(JoinDistributionType_enum_table)) + : std::begin(BufferType_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -JoinNode::JoinNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.JoinNode"; -} -void to_json(json& j, const JoinNode& p) { +void to_json(json& j, const OutputBuffers& 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, "type", p.type, "OutputBuffers", "BufferType", "type"); + to_json_key(j, "version", p.version, "OutputBuffers", "int64_t", "version"); to_json_key( j, - "criteria", - p.criteria, - "JoinNode", - "List", - "criteria"); + "noMoreBufferIds", + p.noMoreBufferIds, + "OutputBuffers", + "bool", + "noMoreBufferIds"); to_json_key( j, - "outputVariables", - p.outputVariables, - "JoinNode", - "List", - "outputVariables"); + "buffers", + p.buffers, + "OutputBuffers", + "Map", + "buffers"); +} + +void from_json(const json& j, OutputBuffers& p) { + from_json_key(j, "type", p.type, "OutputBuffers", "BufferType", "type"); + from_json_key(j, "version", p.version, "OutputBuffers", "int64_t", "version"); + from_json_key( + j, + "noMoreBufferIds", + p.noMoreBufferIds, + "OutputBuffers", + "bool", + "noMoreBufferIds"); + from_json_key( + j, + "buffers", + p.buffers, + "OutputBuffers", + "Map", + "buffers"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const TaskUpdateRequest& p) { + j = json::object(); to_json_key( j, - "filter", - p.filter, - "JoinNode", - "std::shared_ptr", - "filter"); + "session", + p.session, + "TaskUpdateRequest", + "SessionRepresentation", + "session"); to_json_key( j, - "leftHashVariable", - p.leftHashVariable, - "JoinNode", - "VariableReferenceExpression", - "leftHashVariable"); + "extraCredentials", + p.extraCredentials, + "TaskUpdateRequest", + "Map", + "extraCredentials"); + to_json_key( + j, "fragment", p.fragment, "TaskUpdateRequest", "String", "fragment"); to_json_key( j, - "rightHashVariable", - p.rightHashVariable, - "JoinNode", - "VariableReferenceExpression", - "rightHashVariable"); + "sources", + p.sources, + "TaskUpdateRequest", + "List", + "sources"); to_json_key( j, - "distributionType", - p.distributionType, - "JoinNode", - "JoinDistributionType", - "distributionType"); + "outputIds", + p.outputIds, + "TaskUpdateRequest", + "OutputBuffers", + "outputIds"); to_json_key( j, - "dynamicFilters", - p.dynamicFilters, - "JoinNode", - "Map", - "dynamicFilters"); + "tableWriteInfo", + p.tableWriteInfo, + "TaskUpdateRequest", + "TableWriteInfo", + "tableWriteInfo"); } -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"); +void from_json(const json& j, TaskUpdateRequest& p) { from_json_key( j, - "outputVariables", - p.outputVariables, - "JoinNode", - "List", - "outputVariables"); + "session", + p.session, + "TaskUpdateRequest", + "SessionRepresentation", + "session"); from_json_key( j, - "filter", - p.filter, - "JoinNode", - "std::shared_ptr", - "filter"); + "extraCredentials", + p.extraCredentials, + "TaskUpdateRequest", + "Map", + "extraCredentials"); from_json_key( - j, - "leftHashVariable", - p.leftHashVariable, - "JoinNode", - "VariableReferenceExpression", - "leftHashVariable"); + j, "fragment", p.fragment, "TaskUpdateRequest", "String", "fragment"); from_json_key( j, - "rightHashVariable", - p.rightHashVariable, - "JoinNode", - "VariableReferenceExpression", - "rightHashVariable"); + "sources", + p.sources, + "TaskUpdateRequest", + "List", + "sources"); from_json_key( j, - "distributionType", - p.distributionType, - "JoinNode", - "JoinDistributionType", - "distributionType"); + "outputIds", + p.outputIds, + "TaskUpdateRequest", + "OutputBuffers", + "outputIds"); from_json_key( j, - "dynamicFilters", - p.dynamicFilters, - "JoinNode", - "Map", - "dynamicFilters"); + "tableWriteInfo", + p.tableWriteInfo, + "TaskUpdateRequest", + "TableWriteInfo", + "tableWriteInfo"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +SortNode::SortNode() noexcept { + _type = ".SortNode"; +} -void to_json(json& j, const TableWriterNodeStatsEstimate& p) { +void to_json(json& j, const SortNode& p) { j = json::object(); + j["@type"] = ".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, - "taskCountIfScaledWriter", - p.taskCountIfScaledWriter, - "TableWriterNodeStatsEstimate", - "double", - "taskCountIfScaledWriter"); + "orderingScheme", + p.orderingScheme, + "SortNode", + "OrderingScheme", + "orderingScheme"); + to_json_key(j, "isPartial", p.isPartial, "SortNode", "bool", "isPartial"); } -void from_json(const json& j, TableWriterNodeStatsEstimate& p) { +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, - "taskCountIfScaledWriter", - p.taskCountIfScaledWriter, - "TableWriterNodeStatsEstimate", - "double", - "taskCountIfScaledWriter"); + "orderingScheme", + p.orderingScheme, + "SortNode", + "OrderingScheme", + "orderingScheme"); + from_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() -void to_json(json& j, const PartialAggregationStatsEstimate& p) { +// 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, - "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"); + j, "lineNumber", p.lineNumber, "ErrorLocation", "int", "lineNumber"); to_json_key( j, - "outputRowCount", - p.outputRowCount, - "PartialAggregationStatsEstimate", - "double", - "outputRowCount"); + "columnNumber", + p.columnNumber, + "ErrorLocation", + "int", + "columnNumber"); } -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"); +void from_json(const json& j, ErrorLocation& p) { from_json_key( - j, - "inputRowCount", - p.inputRowCount, - "PartialAggregationStatsEstimate", - "double", - "inputRowCount"); + j, "lineNumber", p.lineNumber, "ErrorLocation", "int", "lineNumber"); from_json_key( j, - "outputRowCount", - p.outputRowCount, - "PartialAggregationStatsEstimate", - "double", - "outputRowCount"); + "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 JoinNodeStatsEstimate& p) { +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, - "nullJoinBuildKeyCount", - p.nullJoinBuildKeyCount, - "JoinNodeStatsEstimate", - "double", - "nullJoinBuildKeyCount"); + "cause", + p.cause, + "ExecutionFailureInfo", + "ExecutionFailureInfo", + "cause"); to_json_key( j, - "joinBuildKeyCount", - p.joinBuildKeyCount, - "JoinNodeStatsEstimate", - "double", - "joinBuildKeyCount"); + "suppressed", + p.suppressed, + "ExecutionFailureInfo", + "List", + "suppressed"); + to_json_key( + j, "stack", p.stack, "ExecutionFailureInfo", "List", "stack"); to_json_key( j, - "nullJoinProbeKeyCount", - p.nullJoinProbeKeyCount, - "JoinNodeStatsEstimate", - "double", - "nullJoinProbeKeyCount"); + "errorLocation", + p.errorLocation, + "ExecutionFailureInfo", + "ErrorLocation", + "errorLocation"); to_json_key( j, - "joinProbeKeyCount", - p.joinProbeKeyCount, - "JoinNodeStatsEstimate", - "double", - "joinProbeKeyCount"); + "errorCode", + p.errorCode, + "ExecutionFailureInfo", + "ErrorCode", + "errorCode"); + to_json_key( + j, + "remoteHost", + p.remoteHost, + "ExecutionFailureInfo", + "HostAddress", + "remoteHost"); + to_json_key( + j, + "errorCause", + p.errorCause, + "ExecutionFailureInfo", + "ErrorCause", + "errorCause"); } -void from_json(const json& j, JoinNodeStatsEstimate& p) { +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, - "nullJoinBuildKeyCount", - p.nullJoinBuildKeyCount, - "JoinNodeStatsEstimate", - "double", - "nullJoinBuildKeyCount"); + "cause", + p.cause, + "ExecutionFailureInfo", + "ExecutionFailureInfo", + "cause"); from_json_key( j, - "joinBuildKeyCount", - p.joinBuildKeyCount, - "JoinNodeStatsEstimate", - "double", - "joinBuildKeyCount"); + "suppressed", + p.suppressed, + "ExecutionFailureInfo", + "List", + "suppressed"); + from_json_key( + j, "stack", p.stack, "ExecutionFailureInfo", "List", "stack"); from_json_key( j, - "nullJoinProbeKeyCount", - p.nullJoinProbeKeyCount, - "JoinNodeStatsEstimate", - "double", - "nullJoinProbeKeyCount"); + "errorLocation", + p.errorLocation, + "ExecutionFailureInfo", + "ErrorLocation", + "errorLocation"); from_json_key( j, - "joinProbeKeyCount", - p.joinProbeKeyCount, - "JoinNodeStatsEstimate", - "double", - "joinProbeKeyCount"); + "errorCode", + p.errorCode, + "ExecutionFailureInfo", + "ErrorCode", + "errorCode"); + from_json_key( + j, + "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 std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (type == "uniform-distribution") { - j = *std::static_pointer_cast(p); - return; - } +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - throw TypeError(type + " no abstract type ConnectorHistogram "); +// 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, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError( - std::string(e.what()) + " ConnectorHistogram ConnectorHistogram"); - } - - if (type == "uniform-distribution") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - - throw TypeError(type + " no abstract type ConnectorHistogram "); +void from_json(const json& j, TaskState& e) { + static_assert(std::is_enum::value, "TaskState must be an enum!"); + const auto* it = std::find_if( + std::begin(TaskState_enum_table), + std::end(TaskState_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(TaskState_enum_table)) + ? it + : std::begin(TaskState_enum_table)) + ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const VariableStatsEstimate& p) { +void to_json(json& j, const TaskStatus& p) { j = json::object(); - to_json_key( - j, "lowValue", p.lowValue, "VariableStatsEstimate", "double", "lowValue"); to_json_key( j, - "highValue", - p.highValue, - "VariableStatsEstimate", - "double", - "highValue"); + "taskInstanceIdLeastSignificantBits", + p.taskInstanceIdLeastSignificantBits, + "TaskStatus", + "int64_t", + "taskInstanceIdLeastSignificantBits"); to_json_key( j, - "nullsFraction", - p.nullsFraction, - "VariableStatsEstimate", - "double", - "nullsFraction"); + "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, - "averageRowSize", - p.averageRowSize, - "VariableStatsEstimate", - "double", - "averageRowSize"); + "completedDriverGroups", + p.completedDriverGroups, + "TaskStatus", + "List", + "completedDriverGroups"); to_json_key( j, - "distinctValuesCount", - p.distinctValuesCount, - "VariableStatsEstimate", - "double", - "distinctValuesCount"); + "failures", + p.failures, + "TaskStatus", + "List", + "failures"); to_json_key( j, - "histogram", - p.histogram, - "VariableStatsEstimate", - "ConnectorHistogram", - "histogram"); -} - -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( + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "TaskStatus", + "int", + "queuedPartitionedDrivers"); + to_json_key( j, - "nullsFraction", - p.nullsFraction, - "VariableStatsEstimate", - "double", - "nullsFraction"); - from_json_key( + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "TaskStatus", + "int", + "runningPartitionedDrivers"); + to_json_key( j, - "averageRowSize", - p.averageRowSize, - "VariableStatsEstimate", + "outputBufferUtilization", + p.outputBufferUtilization, + "TaskStatus", "double", - "averageRowSize"); - from_json_key( + "outputBufferUtilization"); + to_json_key( j, - "distinctValuesCount", - p.distinctValuesCount, - "VariableStatsEstimate", - "double", - "distinctValuesCount"); - from_json_key( + "outputBufferOverutilized", + p.outputBufferOverutilized, + "TaskStatus", + "bool", + "outputBufferOverutilized"); + to_json_key( j, - "histogram", - p.histogram, - "VariableStatsEstimate", - "ConnectorHistogram", - "histogram"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const PlanNodeStatsEstimate& p) { - j = json::object(); + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "TaskStatus", + "int64_t", + "physicalWrittenDataSizeInBytes"); to_json_key( j, - "outputRowCount", - p.outputRowCount, - "PlanNodeStatsEstimate", - "double", - "outputRowCount"); + "memoryReservationInBytes", + p.memoryReservationInBytes, + "TaskStatus", + "int64_t", + "memoryReservationInBytes"); to_json_key( j, - "totalSize", - p.totalSize, - "PlanNodeStatsEstimate", - "double", - "totalSize"); + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "TaskStatus", + "int64_t", + "systemMemoryReservationInBytes"); to_json_key( j, - "confident", - p.confident, - "PlanNodeStatsEstimate", - "bool", - "confident"); + "peakNodeTotalMemoryReservationInBytes", + p.peakNodeTotalMemoryReservationInBytes, + "TaskStatus", + "int64_t", + "peakNodeTotalMemoryReservationInBytes"); + to_json_key( + j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); to_json_key( j, - "variableStatistics", - p.variableStatistics, - "PlanNodeStatsEstimate", - "Map", - "variableStatistics"); + "fullGcTimeInMillis", + p.fullGcTimeInMillis, + "TaskStatus", + "int64_t", + "fullGcTimeInMillis"); to_json_key( j, - "joinNodeStatsEstimate", - p.joinNodeStatsEstimate, - "PlanNodeStatsEstimate", - "JoinNodeStatsEstimate", - "joinNodeStatsEstimate"); + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "TaskStatus", + "int64_t", + "totalCpuTimeInNanos"); to_json_key( j, - "tableWriterNodeStatsEstimate", - p.tableWriterNodeStatsEstimate, - "PlanNodeStatsEstimate", - "TableWriterNodeStatsEstimate", - "tableWriterNodeStatsEstimate"); + "taskAgeInMillis", + p.taskAgeInMillis, + "TaskStatus", + "int64_t", + "taskAgeInMillis"); to_json_key( j, - "partialAggregationStatsEstimate", - p.partialAggregationStatsEstimate, - "PlanNodeStatsEstimate", - "PartialAggregationStatsEstimate", - "partialAggregationStatsEstimate"); + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "TaskStatus", + "int64_t", + "queuedPartitionedSplitsWeight"); + to_json_key( + j, + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "TaskStatus", + "int64_t", + "runningPartitionedSplitsWeight"); } -void from_json(const json& j, PlanNodeStatsEstimate& p) { +void from_json(const json& j, TaskStatus& p) { from_json_key( j, - "outputRowCount", - p.outputRowCount, - "PlanNodeStatsEstimate", - "double", - "outputRowCount"); + "taskInstanceIdLeastSignificantBits", + p.taskInstanceIdLeastSignificantBits, + "TaskStatus", + "int64_t", + "taskInstanceIdLeastSignificantBits"); from_json_key( j, - "totalSize", - p.totalSize, - "PlanNodeStatsEstimate", - "double", - "totalSize"); + "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, - "confident", - p.confident, - "PlanNodeStatsEstimate", - "bool", - "confident"); + "completedDriverGroups", + p.completedDriverGroups, + "TaskStatus", + "List", + "completedDriverGroups"); from_json_key( j, - "variableStatistics", - p.variableStatistics, - "PlanNodeStatsEstimate", - "Map", - "variableStatistics"); + "failures", + p.failures, + "TaskStatus", + "List", + "failures"); from_json_key( j, - "joinNodeStatsEstimate", - p.joinNodeStatsEstimate, - "PlanNodeStatsEstimate", - "JoinNodeStatsEstimate", - "joinNodeStatsEstimate"); + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "TaskStatus", + "int", + "queuedPartitionedDrivers"); from_json_key( j, - "tableWriterNodeStatsEstimate", - p.tableWriterNodeStatsEstimate, - "PlanNodeStatsEstimate", - "TableWriterNodeStatsEstimate", - "tableWriterNodeStatsEstimate"); + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "TaskStatus", + "int", + "runningPartitionedDrivers"); from_json_key( j, - "partialAggregationStatsEstimate", - p.partialAggregationStatsEstimate, - "PlanNodeStatsEstimate", - "PartialAggregationStatsEstimate", - "partialAggregationStatsEstimate"); -} -} // 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( + "outputBufferUtilization", + p.outputBufferUtilization, + "TaskStatus", + "double", + "outputBufferUtilization"); + from_json_key( j, - "distinctVariables", - p.distinctVariables, - "DistinctLimitNode", - "List", - "distinctVariables"); - to_json_key( + "outputBufferOverutilized", + p.outputBufferOverutilized, + "TaskStatus", + "bool", + "outputBufferOverutilized"); + from_json_key( + j, + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "TaskStatus", + "int64_t", + "physicalWrittenDataSizeInBytes"); + from_json_key( + j, + "memoryReservationInBytes", + p.memoryReservationInBytes, + "TaskStatus", + "int64_t", + "memoryReservationInBytes"); + from_json_key( j, - "hashVariable", - p.hashVariable, - "DistinctLimitNode", - "VariableReferenceExpression", - "hashVariable"); - to_json_key( + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "TaskStatus", + "int64_t", + "systemMemoryReservationInBytes"); + from_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"); + "peakNodeTotalMemoryReservationInBytes", + p.peakNodeTotalMemoryReservationInBytes, + "TaskStatus", + "int64_t", + "peakNodeTotalMemoryReservationInBytes"); from_json_key( - j, "source", p.source, "DistinctLimitNode", "PlanNode", "source"); - from_json_key(j, "limit", p.limit, "DistinctLimitNode", "int64_t", "limit"); + j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); from_json_key( - j, "partial", p.partial, "DistinctLimitNode", "bool", "partial"); + j, + "fullGcTimeInMillis", + p.fullGcTimeInMillis, + "TaskStatus", + "int64_t", + "fullGcTimeInMillis"); from_json_key( j, - "distinctVariables", - p.distinctVariables, - "DistinctLimitNode", - "List", - "distinctVariables"); + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "TaskStatus", + "int64_t", + "totalCpuTimeInNanos"); from_json_key( j, - "hashVariable", - p.hashVariable, - "DistinctLimitNode", - "VariableReferenceExpression", - "hashVariable"); + "taskAgeInMillis", + p.taskAgeInMillis, + "TaskStatus", + "int64_t", + "taskAgeInMillis"); from_json_key( j, - "timeoutMillis", - p.timeoutMillis, - "DistinctLimitNode", - "int", - "timeoutMillis"); + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "TaskStatus", + "int64_t", + "queuedPartitionedSplitsWeight"); + from_json_key( + j, + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "TaskStatus", + "int64_t", + "runningPartitionedSplitsWeight"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const SchemaTableName& p) { +// 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, "schema", p.schema, "SchemaTableName", "String", "schema"); - to_json_key(j, "table", p.table, "SchemaTableName", "String", "table"); + 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, SchemaTableName& p) { - from_json_key(j, "schema", p.schema, "SchemaTableName", "String", "schema"); - from_json_key(j, "table", p.table, "SchemaTableName", "String", "table"); +void from_json(const json& j, DistributionSnapshot& p) { + from_json_key( + j, "maxError", p.maxError, "DistributionSnapshot", "double", "maxError"); + from_json_key(j, "count", p.count, "DistributionSnapshot", "double", "count"); + from_json_key(j, "total", p.total, "DistributionSnapshot", "double", "total"); + from_json_key(j, "p01", p.p01, "DistributionSnapshot", "int64_t", "p01"); + from_json_key(j, "p05", p.p05, "DistributionSnapshot", "int64_t", "p05"); + from_json_key(j, "p10", p.p10, "DistributionSnapshot", "int64_t", "p10"); + from_json_key(j, "p25", p.p25, "DistributionSnapshot", "int64_t", "p25"); + from_json_key(j, "p50", p.p50, "DistributionSnapshot", "int64_t", "p50"); + from_json_key(j, "p75", p.p75, "DistributionSnapshot", "int64_t", "p75"); + from_json_key(j, "p90", p.p90, "DistributionSnapshot", "int64_t", "p90"); + from_json_key(j, "p95", p.p95, "DistributionSnapshot", "int64_t", "p95"); + from_json_key(j, "p99", p.p99, "DistributionSnapshot", "int64_t", "p99"); + from_json_key(j, "min", p.min, "DistributionSnapshot", "int64_t", "min"); + from_json_key(j, "max", p.max, "DistributionSnapshot", "int64_t", "max"); + from_json_key(j, "avg", p.avg, "DistributionSnapshot", "double", "avg"); } } // namespace facebook::presto::protocol /* @@ -7128,1415 +7830,1315 @@ void from_json(const json& j, SchemaTableName& p) { * 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; - getConnectorProtocol(type).to_json(j, p); -} - -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError( - std::string(e.what()) + - " ConnectorOutputTableHandle ConnectorOutputTableHandle"); - } - getConnectorProtocol(type).from_json(j, p); -} +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 OutputTableHandle& p) { +void to_json(json& j, const OperatorStats& p) { j = json::object(); + to_json_key(j, "stageId", p.stageId, "OperatorStats", "int", "stageId"); to_json_key( j, - "connectorId", - p.connectorId, - "OutputTableHandle", - "ConnectorId", - "connectorId"); + "stageExecutionId", + p.stageExecutionId, + "OperatorStats", + "int", + "stageExecutionId"); + to_json_key( + j, "pipelineId", p.pipelineId, "OperatorStats", "int", "pipelineId"); + to_json_key( + j, "operatorId", p.operatorId, "OperatorStats", "int", "operatorId"); to_json_key( j, - "transactionHandle", - p.transactionHandle, - "OutputTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "planNodeId", + p.planNodeId, + "OperatorStats", + "PlanNodeId", + "planNodeId"); to_json_key( j, - "connectorHandle", - p.connectorHandle, - "OutputTableHandle", - "ConnectorOutputTableHandle", - "connectorHandle"); -} - -void from_json(const json& j, OutputTableHandle& p) { - from_json_key( + "operatorType", + p.operatorType, + "OperatorStats", + "String", + "operatorType"); + to_json_key( + j, + "totalDrivers", + p.totalDrivers, + "OperatorStats", + "int64_t", + "totalDrivers"); + to_json_key( + j, + "addInputCalls", + p.addInputCalls, + "OperatorStats", + "int64_t", + "addInputCalls"); + to_json_key( + j, + "addInputWall", + p.addInputWall, + "OperatorStats", + "Duration", + "addInputWall"); + to_json_key( + j, + "addInputCpu", + p.addInputCpu, + "OperatorStats", + "Duration", + "addInputCpu"); + to_json_key( + j, + "addInputAllocation", + p.addInputAllocation, + "OperatorStats", + "DataSize", + "addInputAllocation"); + to_json_key( + j, + "rawInputDataSize", + p.rawInputDataSize, + "OperatorStats", + "DataSize", + "rawInputDataSize"); + to_json_key( j, - "connectorId", - p.connectorId, - "OutputTableHandle", - "ConnectorId", - "connectorId"); - from_json_key( + "rawInputPositions", + p.rawInputPositions, + "OperatorStats", + "int64_t", + "rawInputPositions"); + to_json_key( j, - "transactionHandle", - p.transactionHandle, - "OutputTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); - from_json_key( + "inputDataSize", + p.inputDataSize, + "OperatorStats", + "DataSize", + "inputDataSize"); + to_json_key( j, - "connectorHandle", - p.connectorHandle, - "OutputTableHandle", - "ConnectorOutputTableHandle", - "connectorHandle"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -CreateHandle::CreateHandle() noexcept { - _type = "CreateHandle"; -} - -void to_json(json& j, const CreateHandle& p) { - j = json::object(); - j["@type"] = "CreateHandle"; + "inputPositions", + p.inputPositions, + "OperatorStats", + "int64_t", + "inputPositions"); to_json_key( - j, "handle", p.handle, "CreateHandle", "OutputTableHandle", "handle"); + j, + "sumSquaredInputPositions", + p.sumSquaredInputPositions, + "OperatorStats", + "double", + "sumSquaredInputPositions"); to_json_key( j, - "schemaTableName", - p.schemaTableName, - "CreateHandle", - "SchemaTableName", - "schemaTableName"); -} - -void from_json(const json& j, CreateHandle& p) { - p._type = j["@type"]; - from_json_key( - j, "handle", p.handle, "CreateHandle", "OutputTableHandle", "handle"); - from_json_key( + "getOutputCalls", + p.getOutputCalls, + "OperatorStats", + "int64_t", + "getOutputCalls"); + to_json_key( j, - "schemaTableName", - p.schemaTableName, - "CreateHandle", - "SchemaTableName", - "schemaTableName"); -} -} // 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"; + "getOutputWall", + p.getOutputWall, + "OperatorStats", + "Duration", + "getOutputWall"); 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( + "getOutputCpu", + p.getOutputCpu, + "OperatorStats", + "Duration", + "getOutputCpu"); + to_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 GroupingSetDescriptor& p) { - j = json::object(); + "getOutputAllocation", + p.getOutputAllocation, + "OperatorStats", + "DataSize", + "getOutputAllocation"); to_json_key( j, - "groupingKeys", - p.groupingKeys, - "GroupingSetDescriptor", - "List", - "groupingKeys"); + "outputDataSize", + p.outputDataSize, + "OperatorStats", + "DataSize", + "outputDataSize"); to_json_key( j, - "groupingSetCount", - p.groupingSetCount, - "GroupingSetDescriptor", - "int", - "groupingSetCount"); + "outputPositions", + p.outputPositions, + "OperatorStats", + "int64_t", + "outputPositions"); to_json_key( j, - "globalGroupingSets", - p.globalGroupingSets, - "GroupingSetDescriptor", - "List", - "globalGroupingSets"); -} - -void from_json(const json& j, GroupingSetDescriptor& p) { - from_json_key( + "physicalWrittenDataSize", + p.physicalWrittenDataSize, + "OperatorStats", + "DataSize", + "physicalWrittenDataSize"); + to_json_key( j, - "groupingKeys", - p.groupingKeys, - "GroupingSetDescriptor", - "List", - "groupingKeys"); - from_json_key( + "additionalCpu", + p.additionalCpu, + "OperatorStats", + "Duration", + "additionalCpu"); + to_json_key( j, - "groupingSetCount", - p.groupingSetCount, - "GroupingSetDescriptor", - "int", - "groupingSetCount"); - from_json_key( + "blockedWall", + p.blockedWall, + "OperatorStats", + "Duration", + "blockedWall"); + to_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 - 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(); + "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, - "stageExecutionStrategy", - p.stageExecutionStrategy, - "StageExecutionDescriptor", - "StageExecutionStrategy", - "stageExecutionStrategy"); + "finishAllocation", + p.finishAllocation, + "OperatorStats", + "DataSize", + "finishAllocation"); to_json_key( j, - "groupedExecutionScanNodes", - p.groupedExecutionScanNodes, - "StageExecutionDescriptor", - "List", - "groupedExecutionScanNodes"); + "userMemoryReservation", + p.userMemoryReservation, + "OperatorStats", + "DataSize", + "userMemoryReservation"); 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( + "revocableMemoryReservation", + p.revocableMemoryReservation, + "OperatorStats", + "DataSize", + "revocableMemoryReservation"); + to_json_key( j, - "groupedExecutionScanNodes", - p.groupedExecutionScanNodes, - "StageExecutionDescriptor", - "List", - "groupedExecutionScanNodes"); - from_json_key( + "systemMemoryReservation", + p.systemMemoryReservation, + "OperatorStats", + "DataSize", + "systemMemoryReservation"); + to_json_key( j, - "totalLifespans", - p.totalLifespans, - "StageExecutionDescriptor", - "int", - "totalLifespans"); -} -} // 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 TableType_enum_table[] = - { // NOLINT: cert-err58-cpp - {TableType::NEW, "NEW"}, - {TableType::EXISTING, "EXISTING"}, - {TableType::TEMPORARY, "TEMPORARY"}}; -void to_json(json& j, const TableType& e) { - static_assert(std::is_enum::value, "TableType must be an enum!"); - const auto* it = std::find_if( - std::begin(TableType_enum_table), - std::end(TableType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(TableType_enum_table)) - ? it - : std::begin(TableType_enum_table)) - ->second; -} -void from_json(const json& j, TableType& e) { - static_assert(std::is_enum::value, "TableType must be an enum!"); - const auto* it = std::find_if( - std::begin(TableType_enum_table), - std::end(TableType_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(TableType_enum_table)) - ? it - : std::begin(TableType_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair WriteMode_enum_table[] = - { // NOLINT: cert-err58-cpp - {WriteMode::STAGE_AND_MOVE_TO_TARGET_DIRECTORY, - "STAGE_AND_MOVE_TO_TARGET_DIRECTORY"}, - {WriteMode::DIRECT_TO_TARGET_NEW_DIRECTORY, - "DIRECT_TO_TARGET_NEW_DIRECTORY"}, - {WriteMode::DIRECT_TO_TARGET_EXISTING_DIRECTORY, - "DIRECT_TO_TARGET_EXISTING_DIRECTORY"}}; -void to_json(json& j, const WriteMode& e) { - static_assert(std::is_enum::value, "WriteMode must be an enum!"); - const auto* it = std::find_if( - std::begin(WriteMode_enum_table), - std::end(WriteMode_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(WriteMode_enum_table)) - ? it - : std::begin(WriteMode_enum_table)) - ->second; -} -void from_json(const json& j, WriteMode& e) { - static_assert(std::is_enum::value, "WriteMode must be an enum!"); - const auto* it = std::find_if( - std::begin(WriteMode_enum_table), - std::end(WriteMode_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(WriteMode_enum_table)) - ? it - : std::begin(WriteMode_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const LocationHandle& p) { - j = json::object(); + "peakUserMemoryReservation", + p.peakUserMemoryReservation, + "OperatorStats", + "DataSize", + "peakUserMemoryReservation"); to_json_key( - j, "targetPath", p.targetPath, "LocationHandle", "String", "targetPath"); + j, + "peakSystemMemoryReservation", + p.peakSystemMemoryReservation, + "OperatorStats", + "DataSize", + "peakSystemMemoryReservation"); to_json_key( - j, "writePath", p.writePath, "LocationHandle", "String", "writePath"); + j, + "peakTotalMemoryReservation", + p.peakTotalMemoryReservation, + "OperatorStats", + "DataSize", + "peakTotalMemoryReservation"); to_json_key( - j, "tempPath", p.tempPath, "LocationHandle", "String", "tempPath"); + j, + "spilledDataSize", + p.spilledDataSize, + "OperatorStats", + "DataSize", + "spilledDataSize"); to_json_key( - j, "tableType", p.tableType, "LocationHandle", "TableType", "tableType"); + j, + "blockedReason", + p.blockedReason, + "OperatorStats", + "BlockedReason", + "blockedReason"); + to_json_key(j, "info", p.info, "OperatorStats", "OperatorInfo", "info"); to_json_key( - j, "writeMode", p.writeMode, "LocationHandle", "WriteMode", "writeMode"); -} - -void from_json(const json& j, LocationHandle& p) { - from_json_key( - j, "targetPath", p.targetPath, "LocationHandle", "String", "targetPath"); - from_json_key( - j, "writePath", p.writePath, "LocationHandle", "String", "writePath"); - from_json_key( - j, "tempPath", p.tempPath, "LocationHandle", "String", "tempPath"); - from_json_key( - j, "tableType", p.tableType, "LocationHandle", "TableType", "tableType"); - from_json_key( - j, "writeMode", p.writeMode, "LocationHandle", "WriteMode", "writeMode"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -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 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"); + j, + "runtimeStats", + p.runtimeStats, + "OperatorStats", + "RuntimeStats", + "runtimeStats"); to_json_key( - j, "fileSize", p.fileSize, "HiveFileSplit", "int64_t", "fileSize"); + j, + "nullJoinBuildKeyCount", + p.nullJoinBuildKeyCount, + "OperatorStats", + "int64_t", + "nullJoinBuildKeyCount"); to_json_key( j, - "fileModifiedTime", - p.fileModifiedTime, - "HiveFileSplit", + "joinBuildKeyCount", + p.joinBuildKeyCount, + "OperatorStats", "int64_t", - "fileModifiedTime"); + "joinBuildKeyCount"); to_json_key( j, - "extraFileInfo", - p.extraFileInfo, - "HiveFileSplit", - "String", - "extraFileInfo"); + "nullJoinProbeKeyCount", + p.nullJoinProbeKeyCount, + "OperatorStats", + "int64_t", + "nullJoinProbeKeyCount"); to_json_key( j, - "customSplitInfo", - p.customSplitInfo, - "HiveFileSplit", - "Map", - "customSplitInfo"); + "joinProbeKeyCount", + p.joinProbeKeyCount, + "OperatorStats", + "int64_t", + "joinProbeKeyCount"); } -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"); +void from_json(const json& j, OperatorStats& p) { + from_json_key(j, "stageId", p.stageId, "OperatorStats", "int", "stageId"); from_json_key( - j, "fileSize", p.fileSize, "HiveFileSplit", "int64_t", "fileSize"); + 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, - "fileModifiedTime", - p.fileModifiedTime, - "HiveFileSplit", - "int64_t", - "fileModifiedTime"); + "planNodeId", + p.planNodeId, + "OperatorStats", + "PlanNodeId", + "planNodeId"); from_json_key( j, - "extraFileInfo", - p.extraFileInfo, - "HiveFileSplit", + "operatorType", + p.operatorType, + "OperatorStats", "String", - "extraFileInfo"); + "operatorType"); from_json_key( j, - "customSplitInfo", - p.customSplitInfo, - "HiveFileSplit", - "Map", - "customSplitInfo"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -UniformDistributionHistogram::UniformDistributionHistogram() noexcept { - _type = "uniform-distribution"; -} - -void to_json(json& j, const UniformDistributionHistogram& p) { - j = json::object(); - j["@type"] = "uniform-distribution"; - to_json_key( + "totalDrivers", + p.totalDrivers, + "OperatorStats", + "int64_t", + "totalDrivers"); + from_json_key( j, - "lowValue", - p.lowValue, - "UniformDistributionHistogram", - "double", - "lowValue"); - to_json_key( + "addInputCalls", + p.addInputCalls, + "OperatorStats", + "int64_t", + "addInputCalls"); + from_json_key( j, - "highValue", - p.highValue, - "UniformDistributionHistogram", - "double", - "highValue"); -} - -void from_json(const json& j, UniformDistributionHistogram& p) { - p._type = j["@type"]; + "addInputWall", + p.addInputWall, + "OperatorStats", + "Duration", + "addInputWall"); from_json_key( j, - "lowValue", - p.lowValue, - "UniformDistributionHistogram", - "double", - "lowValue"); + "addInputCpu", + p.addInputCpu, + "OperatorStats", + "Duration", + "addInputCpu"); from_json_key( j, - "highValue", - p.highValue, - "UniformDistributionHistogram", - "double", - "highValue"); -} -} // 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 -namespace facebook::presto::protocol { - -void to_json(json& j, const PageBufferInfo& p) { - j = json::object(); - to_json_key( - j, "partition", p.partition, "PageBufferInfo", "int", "partition"); - to_json_key( + "addInputAllocation", + p.addInputAllocation, + "OperatorStats", + "DataSize", + "addInputAllocation"); + from_json_key( j, - "bufferedPages", - p.bufferedPages, - "PageBufferInfo", + "rawInputDataSize", + p.rawInputDataSize, + "OperatorStats", + "DataSize", + "rawInputDataSize"); + from_json_key( + j, + "rawInputPositions", + p.rawInputPositions, + "OperatorStats", "int64_t", - "bufferedPages"); - to_json_key( + "rawInputPositions"); + from_json_key( j, - "bufferedBytes", - p.bufferedBytes, - "PageBufferInfo", + "inputDataSize", + p.inputDataSize, + "OperatorStats", + "DataSize", + "inputDataSize"); + from_json_key( + j, + "inputPositions", + p.inputPositions, + "OperatorStats", "int64_t", - "bufferedBytes"); - to_json_key( - 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) { + "inputPositions"); from_json_key( - j, "partition", p.partition, "PageBufferInfo", "int", "partition"); + j, + "sumSquaredInputPositions", + p.sumSquaredInputPositions, + "OperatorStats", + "double", + "sumSquaredInputPositions"); from_json_key( j, - "bufferedPages", - p.bufferedPages, - "PageBufferInfo", + "getOutputCalls", + p.getOutputCalls, + "OperatorStats", "int64_t", - "bufferedPages"); + "getOutputCalls"); + from_json_key( + j, + "getOutputWall", + p.getOutputWall, + "OperatorStats", + "Duration", + "getOutputWall"); + from_json_key( + j, + "getOutputCpu", + p.getOutputCpu, + "OperatorStats", + "Duration", + "getOutputCpu"); + from_json_key( + j, + "getOutputAllocation", + p.getOutputAllocation, + "OperatorStats", + "DataSize", + "getOutputAllocation"); + from_json_key( + j, + "outputDataSize", + p.outputDataSize, + "OperatorStats", + "DataSize", + "outputDataSize"); from_json_key( j, - "bufferedBytes", - p.bufferedBytes, - "PageBufferInfo", + "outputPositions", + p.outputPositions, + "OperatorStats", "int64_t", - "bufferedBytes"); - from_json_key( - j, "rowsAdded", p.rowsAdded, "PageBufferInfo", "int64_t", "rowsAdded"); + "outputPositions"); from_json_key( - j, "pagesAdded", p.pagesAdded, "PageBufferInfo", "int64_t", "pagesAdded"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const BufferInfo& p) { - j = json::object(); - to_json_key( - j, "bufferId", p.bufferId, "BufferInfo", "OutputBufferId", "bufferId"); - to_json_key(j, "finished", p.finished, "BufferInfo", "bool", "finished"); - to_json_key( j, - "bufferedPages", - p.bufferedPages, - "BufferInfo", - "int", - "bufferedPages"); - to_json_key( - j, "pagesSent", p.pagesSent, "BufferInfo", "int64_t", "pagesSent"); - to_json_key( + "physicalWrittenDataSize", + p.physicalWrittenDataSize, + "OperatorStats", + "DataSize", + "physicalWrittenDataSize"); + from_json_key( j, - "pageBufferInfo", - p.pageBufferInfo, - "BufferInfo", - "PageBufferInfo", - "pageBufferInfo"); -} - -void from_json(const json& j, BufferInfo& p) { + "additionalCpu", + p.additionalCpu, + "OperatorStats", + "Duration", + "additionalCpu"); from_json_key( - j, "bufferId", p.bufferId, "BufferInfo", "OutputBufferId", "bufferId"); - from_json_key(j, "finished", p.finished, "BufferInfo", "bool", "finished"); + j, + "blockedWall", + p.blockedWall, + "OperatorStats", + "Duration", + "blockedWall"); from_json_key( j, - "bufferedPages", - p.bufferedPages, - "BufferInfo", - "int", - "bufferedPages"); + "finishCalls", + p.finishCalls, + "OperatorStats", + "int64_t", + "finishCalls"); from_json_key( - j, "pagesSent", p.pagesSent, "BufferInfo", "int64_t", "pagesSent"); + j, "finishWall", p.finishWall, "OperatorStats", "Duration", "finishWall"); + from_json_key( + j, "finishCpu", p.finishCpu, "OperatorStats", "Duration", "finishCpu"); 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, - "canAddBuffers", - p.canAddBuffers, - "OutputBufferInfo", - "bool", - "canAddBuffers"); - to_json_key( - j, - "canAddPages", - p.canAddPages, - "OutputBufferInfo", - "bool", - "canAddPages"); - to_json_key( + "finishAllocation", + p.finishAllocation, + "OperatorStats", + "DataSize", + "finishAllocation"); + from_json_key( j, - "totalBufferedBytes", - p.totalBufferedBytes, - "OutputBufferInfo", - "int64_t", - "totalBufferedBytes"); - to_json_key( + "userMemoryReservation", + p.userMemoryReservation, + "OperatorStats", + "DataSize", + "userMemoryReservation"); + from_json_key( j, - "totalBufferedPages", - p.totalBufferedPages, - "OutputBufferInfo", - "int64_t", - "totalBufferedPages"); - to_json_key( + "revocableMemoryReservation", + p.revocableMemoryReservation, + "OperatorStats", + "DataSize", + "revocableMemoryReservation"); + from_json_key( j, - "totalRowsSent", - p.totalRowsSent, - "OutputBufferInfo", - "int64_t", - "totalRowsSent"); - to_json_key( + "systemMemoryReservation", + p.systemMemoryReservation, + "OperatorStats", + "DataSize", + "systemMemoryReservation"); + from_json_key( j, - "totalPagesSent", - p.totalPagesSent, - "OutputBufferInfo", - "int64_t", - "totalPagesSent"); - to_json_key( + "peakUserMemoryReservation", + p.peakUserMemoryReservation, + "OperatorStats", + "DataSize", + "peakUserMemoryReservation"); + from_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"); + "peakSystemMemoryReservation", + p.peakSystemMemoryReservation, + "OperatorStats", + "DataSize", + "peakSystemMemoryReservation"); from_json_key( - j, "state", p.state, "OutputBufferInfo", "BufferState", "state"); + j, + "peakTotalMemoryReservation", + p.peakTotalMemoryReservation, + "OperatorStats", + "DataSize", + "peakTotalMemoryReservation"); from_json_key( j, - "canAddBuffers", - p.canAddBuffers, - "OutputBufferInfo", - "bool", - "canAddBuffers"); + "spilledDataSize", + p.spilledDataSize, + "OperatorStats", + "DataSize", + "spilledDataSize"); from_json_key( j, - "canAddPages", - p.canAddPages, - "OutputBufferInfo", - "bool", - "canAddPages"); + "blockedReason", + p.blockedReason, + "OperatorStats", + "BlockedReason", + "blockedReason"); + from_json_key(j, "info", p.info, "OperatorStats", "OperatorInfo", "info"); from_json_key( j, - "totalBufferedBytes", - p.totalBufferedBytes, - "OutputBufferInfo", - "int64_t", - "totalBufferedBytes"); + "runtimeStats", + p.runtimeStats, + "OperatorStats", + "RuntimeStats", + "runtimeStats"); from_json_key( j, - "totalBufferedPages", - p.totalBufferedPages, - "OutputBufferInfo", + "nullJoinBuildKeyCount", + p.nullJoinBuildKeyCount, + "OperatorStats", "int64_t", - "totalBufferedPages"); + "nullJoinBuildKeyCount"); from_json_key( j, - "totalRowsSent", - p.totalRowsSent, - "OutputBufferInfo", + "joinBuildKeyCount", + p.joinBuildKeyCount, + "OperatorStats", "int64_t", - "totalRowsSent"); + "joinBuildKeyCount"); from_json_key( j, - "totalPagesSent", - p.totalPagesSent, - "OutputBufferInfo", + "nullJoinProbeKeyCount", + p.nullJoinProbeKeyCount, + "OperatorStats", "int64_t", - "totalPagesSent"); + "nullJoinProbeKeyCount"); from_json_key( j, - "buffers", - p.buffers, - "OutputBufferInfo", - "List", - "buffers"); -} -} // 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 { -// 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"); + "joinProbeKeyCount", + p.joinProbeKeyCount, + "OperatorStats", + "int64_t", + "joinProbeKeyCount"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const ErrorLocation& 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, "lineNumber", p.lineNumber, "ErrorLocation", "int", "lineNumber"); + 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, - "columnNumber", - p.columnNumber, - "ErrorLocation", - "int", - "columnNumber"); -} - -void from_json(const json& j, ErrorLocation& p) { - from_json_key( - j, "lineNumber", p.lineNumber, "ErrorLocation", "int", "lineNumber"); - from_json_key( + "elapsedTime", + p.elapsedTime, + "DriverStats", + "Duration", + "elapsedTime"); + to_json_key( j, - "columnNumber", - p.columnNumber, - "ErrorLocation", - "int", - "columnNumber"); -} -} // 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"); + "userMemoryReservation", + p.userMemoryReservation, + "DriverStats", + "DataSize", + "userMemoryReservation"); to_json_key( - j, "message", p.message, "ExecutionFailureInfo", "String", "message"); + j, + "revocableMemoryReservation", + p.revocableMemoryReservation, + "DriverStats", + "DataSize", + "revocableMemoryReservation"); + to_json_key( + j, + "systemMemoryReservation", + p.systemMemoryReservation, + "DriverStats", + "DataSize", + "systemMemoryReservation"); + to_json_key( + j, + "totalScheduledTime", + p.totalScheduledTime, + "DriverStats", + "Duration", + "totalScheduledTime"); + to_json_key( + j, + "totalCpuTime", + p.totalCpuTime, + "DriverStats", + "Duration", + "totalCpuTime"); + to_json_key( + j, + "totalBlockedTime", + p.totalBlockedTime, + "DriverStats", + "Duration", + "totalBlockedTime"); + to_json_key( + j, "fullyBlocked", p.fullyBlocked, "DriverStats", "bool", "fullyBlocked"); + to_json_key( + j, + "blockedReasons", + p.blockedReasons, + "DriverStats", + "List", + "blockedReasons"); + to_json_key( + j, + "totalAllocation", + p.totalAllocation, + "DriverStats", + "DataSize", + "totalAllocation"); + to_json_key( + j, + "rawInputDataSize", + p.rawInputDataSize, + "DriverStats", + "DataSize", + "rawInputDataSize"); + to_json_key( + j, + "rawInputPositions", + p.rawInputPositions, + "DriverStats", + "int64_t", + "rawInputPositions"); to_json_key( j, - "cause", - p.cause, - "ExecutionFailureInfo", - "ExecutionFailureInfo", - "cause"); + "rawInputReadTime", + p.rawInputReadTime, + "DriverStats", + "Duration", + "rawInputReadTime"); to_json_key( j, - "suppressed", - p.suppressed, - "ExecutionFailureInfo", - "List", - "suppressed"); + "processedInputDataSize", + p.processedInputDataSize, + "DriverStats", + "DataSize", + "processedInputDataSize"); to_json_key( - j, "stack", p.stack, "ExecutionFailureInfo", "List", "stack"); + j, + "processedInputPositions", + p.processedInputPositions, + "DriverStats", + "int64_t", + "processedInputPositions"); to_json_key( j, - "errorLocation", - p.errorLocation, - "ExecutionFailureInfo", - "ErrorLocation", - "errorLocation"); + "outputDataSize", + p.outputDataSize, + "DriverStats", + "DataSize", + "outputDataSize"); to_json_key( j, - "errorCode", - p.errorCode, - "ExecutionFailureInfo", - "ErrorCode", - "errorCode"); + "outputPositions", + p.outputPositions, + "DriverStats", + "int64_t", + "outputPositions"); to_json_key( j, - "remoteHost", - p.remoteHost, - "ExecutionFailureInfo", - "HostAddress", - "remoteHost"); + "physicalWrittenDataSize", + p.physicalWrittenDataSize, + "DriverStats", + "DataSize", + "physicalWrittenDataSize"); to_json_key( j, - "errorCause", - p.errorCause, - "ExecutionFailureInfo", - "ErrorCause", - "errorCause"); + "operatorStats", + p.operatorStats, + "DriverStats", + "List", + "operatorStats"); } -void from_json(const json& j, ExecutionFailureInfo& p) { - from_json_key(j, "type", p.type, "ExecutionFailureInfo", "String", "type"); +void from_json(const json& j, DriverStats& p) { from_json_key( - j, "message", p.message, "ExecutionFailureInfo", "String", "message"); + 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, - "cause", - p.cause, - "ExecutionFailureInfo", - "ExecutionFailureInfo", - "cause"); + "elapsedTime", + p.elapsedTime, + "DriverStats", + "Duration", + "elapsedTime"); from_json_key( j, - "suppressed", - p.suppressed, - "ExecutionFailureInfo", - "List", - "suppressed"); + "userMemoryReservation", + p.userMemoryReservation, + "DriverStats", + "DataSize", + "userMemoryReservation"); from_json_key( - j, "stack", p.stack, "ExecutionFailureInfo", "List", "stack"); + j, + "revocableMemoryReservation", + p.revocableMemoryReservation, + "DriverStats", + "DataSize", + "revocableMemoryReservation"); from_json_key( j, - "errorLocation", - p.errorLocation, - "ExecutionFailureInfo", - "ErrorLocation", - "errorLocation"); + "systemMemoryReservation", + p.systemMemoryReservation, + "DriverStats", + "DataSize", + "systemMemoryReservation"); from_json_key( j, - "errorCode", - p.errorCode, - "ExecutionFailureInfo", - "ErrorCode", - "errorCode"); + "totalScheduledTime", + p.totalScheduledTime, + "DriverStats", + "Duration", + "totalScheduledTime"); from_json_key( j, - "remoteHost", - p.remoteHost, - "ExecutionFailureInfo", - "HostAddress", - "remoteHost"); + "totalCpuTime", + p.totalCpuTime, + "DriverStats", + "Duration", + "totalCpuTime"); from_json_key( j, - "errorCause", - p.errorCause, - "ExecutionFailureInfo", - "ErrorCause", - "errorCause"); + "totalBlockedTime", + p.totalBlockedTime, + "DriverStats", + "Duration", + "totalBlockedTime"); + from_json_key( + j, "fullyBlocked", p.fullyBlocked, "DriverStats", "bool", "fullyBlocked"); + from_json_key( + j, + "blockedReasons", + p.blockedReasons, + "DriverStats", + "List", + "blockedReasons"); + from_json_key( + j, + "totalAllocation", + p.totalAllocation, + "DriverStats", + "DataSize", + "totalAllocation"); + from_json_key( + j, + "rawInputDataSize", + p.rawInputDataSize, + "DriverStats", + "DataSize", + "rawInputDataSize"); + from_json_key( + j, + "rawInputPositions", + p.rawInputPositions, + "DriverStats", + "int64_t", + "rawInputPositions"); + from_json_key( + j, + "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 TaskStatus& p) { +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, - "taskInstanceIdLeastSignificantBits", - p.taskInstanceIdLeastSignificantBits, - "TaskStatus", - "int64_t", - "taskInstanceIdLeastSignificantBits"); + "inputPipeline", + p.inputPipeline, + "PipelineStats", + "bool", + "inputPipeline"); 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"); + "outputPipeline", + p.outputPipeline, + "PipelineStats", + "bool", + "outputPipeline"); to_json_key( j, - "completedDriverGroups", - p.completedDriverGroups, - "TaskStatus", - "List", - "completedDriverGroups"); + "totalDrivers", + p.totalDrivers, + "PipelineStats", + "int", + "totalDrivers"); to_json_key( j, - "failures", - p.failures, - "TaskStatus", - "List", - "failures"); + "queuedDrivers", + p.queuedDrivers, + "PipelineStats", + "int", + "queuedDrivers"); to_json_key( j, "queuedPartitionedDrivers", p.queuedPartitionedDrivers, - "TaskStatus", + "PipelineStats", "int", "queuedPartitionedDrivers"); + to_json_key( + j, + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "PipelineStats", + "int64_t", + "queuedPartitionedSplitsWeight"); + to_json_key( + j, + "runningDrivers", + p.runningDrivers, + "PipelineStats", + "int", + "runningDrivers"); to_json_key( j, "runningPartitionedDrivers", p.runningPartitionedDrivers, - "TaskStatus", + "PipelineStats", "int", "runningPartitionedDrivers"); to_json_key( j, - "outputBufferUtilization", - p.outputBufferUtilization, - "TaskStatus", - "double", - "outputBufferUtilization"); + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "PipelineStats", + "int64_t", + "runningPartitionedSplitsWeight"); to_json_key( j, - "outputBufferOverutilized", - p.outputBufferOverutilized, - "TaskStatus", - "bool", - "outputBufferOverutilized"); + "blockedDrivers", + p.blockedDrivers, + "PipelineStats", + "int", + "blockedDrivers"); to_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "TaskStatus", + "completedDrivers", + p.completedDrivers, + "PipelineStats", + "int", + "completedDrivers"); + to_json_key( + j, + "userMemoryReservationInBytes", + p.userMemoryReservationInBytes, + "PipelineStats", "int64_t", - "physicalWrittenDataSizeInBytes"); + "userMemoryReservationInBytes"); to_json_key( j, - "memoryReservationInBytes", - p.memoryReservationInBytes, - "TaskStatus", + "revocableMemoryReservationInBytes", + p.revocableMemoryReservationInBytes, + "PipelineStats", "int64_t", - "memoryReservationInBytes"); + "revocableMemoryReservationInBytes"); to_json_key( j, "systemMemoryReservationInBytes", p.systemMemoryReservationInBytes, - "TaskStatus", + "PipelineStats", "int64_t", "systemMemoryReservationInBytes"); to_json_key( j, - "peakNodeTotalMemoryReservationInBytes", - p.peakNodeTotalMemoryReservationInBytes, - "TaskStatus", - "int64_t", - "peakNodeTotalMemoryReservationInBytes"); + "queuedTime", + p.queuedTime, + "PipelineStats", + "DistributionSnapshot", + "queuedTime"); to_json_key( - j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); + j, + "elapsedTime", + p.elapsedTime, + "PipelineStats", + "DistributionSnapshot", + "elapsedTime"); to_json_key( j, - "fullGcTimeInMillis", - p.fullGcTimeInMillis, - "TaskStatus", + "totalScheduledTimeInNanos", + p.totalScheduledTimeInNanos, + "PipelineStats", "int64_t", - "fullGcTimeInMillis"); + "totalScheduledTimeInNanos"); to_json_key( j, "totalCpuTimeInNanos", p.totalCpuTimeInNanos, - "TaskStatus", + "PipelineStats", "int64_t", "totalCpuTimeInNanos"); to_json_key( j, - "taskAgeInMillis", - p.taskAgeInMillis, - "TaskStatus", + "totalBlockedTimeInNanos", + p.totalBlockedTimeInNanos, + "PipelineStats", "int64_t", - "taskAgeInMillis"); + "totalBlockedTimeInNanos"); to_json_key( j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "TaskStatus", + "fullyBlocked", + p.fullyBlocked, + "PipelineStats", + "bool", + "fullyBlocked"); + to_json_key( + j, + "blockedReasons", + p.blockedReasons, + "PipelineStats", + "List", + "blockedReasons"); + to_json_key( + j, + "totalAllocationInBytes", + p.totalAllocationInBytes, + "PipelineStats", "int64_t", - "queuedPartitionedSplitsWeight"); + "totalAllocationInBytes"); to_json_key( j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "TaskStatus", + "rawInputDataSizeInBytes", + p.rawInputDataSizeInBytes, + "PipelineStats", "int64_t", - "runningPartitionedSplitsWeight"); + "rawInputDataSizeInBytes"); + to_json_key( + j, + "rawInputPositions", + p.rawInputPositions, + "PipelineStats", + "int64_t", + "rawInputPositions"); + to_json_key( + j, + "processedInputDataSizeInBytes", + p.processedInputDataSizeInBytes, + "PipelineStats", + "int64_t", + "processedInputDataSizeInBytes"); + to_json_key( + j, + "processedInputPositions", + p.processedInputPositions, + "PipelineStats", + "int64_t", + "processedInputPositions"); + to_json_key( + j, + "outputDataSizeInBytes", + p.outputDataSizeInBytes, + "PipelineStats", + "int64_t", + "outputDataSizeInBytes"); + to_json_key( + j, + "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, "drivers", p.drivers, "PipelineStats", "List", "drivers"); } -void from_json(const json& j, TaskStatus& p) { +void from_json(const json& j, PipelineStats& p) { + from_json_key( + j, "pipelineId", p.pipelineId, "PipelineStats", "int", "pipelineId"); from_json_key( j, - "taskInstanceIdLeastSignificantBits", - p.taskInstanceIdLeastSignificantBits, - "TaskStatus", - "int64_t", - "taskInstanceIdLeastSignificantBits"); + "firstStartTime", + p.firstStartTime, + "PipelineStats", + "DateTime", + "firstStartTime"); + from_json_key( + j, + "lastStartTime", + p.lastStartTime, + "PipelineStats", + "DateTime", + "lastStartTime"); + from_json_key( + j, + "lastEndTime", + p.lastEndTime, + "PipelineStats", + "DateTime", + "lastEndTime"); + from_json_key( + j, + "inputPipeline", + p.inputPipeline, + "PipelineStats", + "bool", + "inputPipeline"); from_json_key( j, - "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"); + "outputPipeline", + p.outputPipeline, + "PipelineStats", + "bool", + "outputPipeline"); from_json_key( j, - "completedDriverGroups", - p.completedDriverGroups, - "TaskStatus", - "List", - "completedDriverGroups"); + "totalDrivers", + p.totalDrivers, + "PipelineStats", + "int", + "totalDrivers"); from_json_key( j, - "failures", - p.failures, - "TaskStatus", - "List", - "failures"); + "queuedDrivers", + p.queuedDrivers, + "PipelineStats", + "int", + "queuedDrivers"); from_json_key( j, "queuedPartitionedDrivers", p.queuedPartitionedDrivers, - "TaskStatus", + "PipelineStats", "int", "queuedPartitionedDrivers"); + from_json_key( + j, + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "PipelineStats", + "int64_t", + "queuedPartitionedSplitsWeight"); + from_json_key( + j, + "runningDrivers", + p.runningDrivers, + "PipelineStats", + "int", + "runningDrivers"); from_json_key( j, "runningPartitionedDrivers", p.runningPartitionedDrivers, - "TaskStatus", + "PipelineStats", "int", "runningPartitionedDrivers"); from_json_key( j, - "outputBufferUtilization", - p.outputBufferUtilization, - "TaskStatus", - "double", - "outputBufferUtilization"); + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "PipelineStats", + "int64_t", + "runningPartitionedSplitsWeight"); from_json_key( j, - "outputBufferOverutilized", - p.outputBufferOverutilized, - "TaskStatus", - "bool", - "outputBufferOverutilized"); + "blockedDrivers", + p.blockedDrivers, + "PipelineStats", + "int", + "blockedDrivers"); from_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "TaskStatus", + "completedDrivers", + p.completedDrivers, + "PipelineStats", + "int", + "completedDrivers"); + from_json_key( + j, + "userMemoryReservationInBytes", + p.userMemoryReservationInBytes, + "PipelineStats", "int64_t", - "physicalWrittenDataSizeInBytes"); + "userMemoryReservationInBytes"); from_json_key( j, - "memoryReservationInBytes", - p.memoryReservationInBytes, - "TaskStatus", + "revocableMemoryReservationInBytes", + p.revocableMemoryReservationInBytes, + "PipelineStats", "int64_t", - "memoryReservationInBytes"); + "revocableMemoryReservationInBytes"); from_json_key( j, "systemMemoryReservationInBytes", p.systemMemoryReservationInBytes, - "TaskStatus", + "PipelineStats", "int64_t", "systemMemoryReservationInBytes"); from_json_key( j, - "peakNodeTotalMemoryReservationInBytes", - p.peakNodeTotalMemoryReservationInBytes, - "TaskStatus", - "int64_t", - "peakNodeTotalMemoryReservationInBytes"); + "queuedTime", + p.queuedTime, + "PipelineStats", + "DistributionSnapshot", + "queuedTime"); from_json_key( - j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); + j, + "elapsedTime", + p.elapsedTime, + "PipelineStats", + "DistributionSnapshot", + "elapsedTime"); from_json_key( j, - "fullGcTimeInMillis", - p.fullGcTimeInMillis, - "TaskStatus", + "totalScheduledTimeInNanos", + p.totalScheduledTimeInNanos, + "PipelineStats", "int64_t", - "fullGcTimeInMillis"); + "totalScheduledTimeInNanos"); from_json_key( j, "totalCpuTimeInNanos", p.totalCpuTimeInNanos, - "TaskStatus", + "PipelineStats", "int64_t", "totalCpuTimeInNanos"); from_json_key( j, - "taskAgeInMillis", - p.taskAgeInMillis, - "TaskStatus", + "totalBlockedTimeInNanos", + p.totalBlockedTimeInNanos, + "PipelineStats", "int64_t", - "taskAgeInMillis"); + "totalBlockedTimeInNanos"); from_json_key( j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "TaskStatus", + "fullyBlocked", + p.fullyBlocked, + "PipelineStats", + "bool", + "fullyBlocked"); + from_json_key( + j, + "blockedReasons", + p.blockedReasons, + "PipelineStats", + "List", + "blockedReasons"); + from_json_key( + j, + "totalAllocationInBytes", + p.totalAllocationInBytes, + "PipelineStats", "int64_t", - "queuedPartitionedSplitsWeight"); + "totalAllocationInBytes"); from_json_key( j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "TaskStatus", + "rawInputDataSizeInBytes", + p.rawInputDataSizeInBytes, + "PipelineStats", "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; - getConnectorProtocol(type).to_json(j, p); -} - -void from_json( - const json& j, - std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " ConnectorMetadataUpdateHandle"); - } - getConnectorProtocol(type).from_json(j, p); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const MetadataUpdates& p) { - j = json::object(); - to_json_key( + "rawInputDataSizeInBytes"); + from_json_key( j, - "connectorId", - p.connectorId, - "MetadataUpdates", - "ConnectorId", - "connectorId"); - to_json_key( + "rawInputPositions", + p.rawInputPositions, + "PipelineStats", + "int64_t", + "rawInputPositions"); + from_json_key( j, - "metadataUpdates", - p.metadataUpdates, - "MetadataUpdates", - "List>", - "metadataUpdates"); -} - -void from_json(const json& j, MetadataUpdates& p) { + "processedInputDataSizeInBytes", + p.processedInputDataSizeInBytes, + "PipelineStats", + "int64_t", + "processedInputDataSizeInBytes"); from_json_key( j, - "connectorId", - p.connectorId, - "MetadataUpdates", - "ConnectorId", - "connectorId"); + "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, - "metadataUpdates", - p.metadataUpdates, - "MetadataUpdates", - "List>", - "metadataUpdates"); + "operatorSummaries", + p.operatorSummaries, + "PipelineStats", + "List", + "operatorSummaries"); + from_json_key( + j, "drivers", p.drivers, "PipelineStats", "List", "drivers"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { @@ -9014,1426 +9616,1548 @@ void from_json(const json& j, TaskStats& p) { from_json_key( j, "outputPositions", - p.outputPositions, - "TaskStats", - "int64_t", - "outputPositions"); - from_json_key( - j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "TaskStats", - "int64_t", - "physicalWrittenDataSizeInBytes"); - from_json_key( - j, "fullGcCount", p.fullGcCount, "TaskStats", "int", "fullGcCount"); - from_json_key( - j, - "fullGcTimeInMillis", - p.fullGcTimeInMillis, - "TaskStats", - "int64_t", - "fullGcTimeInMillis"); - from_json_key( - j, - "pipelines", - p.pipelines, - "TaskStats", - "List", - "pipelines"); - from_json_key( - j, - "runtimeStats", - p.runtimeStats, - "TaskStats", - "RuntimeStats", - "runtimeStats"); -} -} // 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, - "lastHeartbeat", - p.lastHeartbeat, - "TaskInfo", - "DateTime", - "lastHeartbeat"); - from_json_key( - j, - "outputBuffers", - p.outputBuffers, - "TaskInfo", - "OutputBufferInfo", - "outputBuffers"); - 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"); - 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 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"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const MemoryPoolInfo& 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, - "reservedRevocableBytes", - p.reservedRevocableBytes, - "MemoryPoolInfo", - "int64_t", - "reservedRevocableBytes"); - to_json_key( - j, - "queryMemoryReservations", - p.queryMemoryReservations, - "MemoryPoolInfo", - "Map", - "queryMemoryReservations"); - to_json_key( - j, - "queryMemoryAllocations", - p.queryMemoryAllocations, - "MemoryPoolInfo", - "Map>", - "queryMemoryAllocations"); - to_json_key( - j, - "queryMemoryRevocableReservations", - p.queryMemoryRevocableReservations, - "MemoryPoolInfo", - "Map", - "queryMemoryRevocableReservations"); -} - -void from_json(const json& j, MemoryPoolInfo& p) { - from_json_key( - j, "maxBytes", p.maxBytes, "MemoryPoolInfo", "int64_t", "maxBytes"); - from_json_key( - j, - "reservedBytes", - p.reservedBytes, - "MemoryPoolInfo", - "int64_t", - "reservedBytes"); - from_json_key( - j, - "reservedRevocableBytes", - p.reservedRevocableBytes, - "MemoryPoolInfo", - "int64_t", - "reservedRevocableBytes"); - from_json_key( - j, - "queryMemoryReservations", - p.queryMemoryReservations, - "MemoryPoolInfo", - "Map", - "queryMemoryReservations"); - from_json_key( - j, - "queryMemoryAllocations", - p.queryMemoryAllocations, - "MemoryPoolInfo", - "Map>", - "queryMemoryAllocations"); - from_json_key( - j, - "queryMemoryRevocableReservations", - p.queryMemoryRevocableReservations, - "MemoryPoolInfo", - "Map", - "queryMemoryRevocableReservations"); -} -} // 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, - "connectorId", - p.connectorId, - "EmptySplit", - "ConnectorId", - "connectorId"); -} - -void from_json(const json& j, EmptySplit& p) { - p._type = j["@type"]; + p.outputPositions, + "TaskStats", + "int64_t", + "outputPositions"); from_json_key( j, - "connectorId", - p.connectorId, - "EmptySplit", - "ConnectorId", - "connectorId"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const NodeVersion& p) { - j = json::object(); - to_json_key(j, "version", p.version, "NodeVersion", "String", "version"); -} - -void from_json(const json& j, NodeVersion& p) { - from_json_key(j, "version", p.version, "NodeVersion", "String", "version"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const ServerInfo& p) { - j = json::object(); - to_json_key( - j, - "nodeVersion", - p.nodeVersion, - "ServerInfo", - "NodeVersion", - "nodeVersion"); - to_json_key( - j, "environment", p.environment, "ServerInfo", "String", "environment"); - to_json_key( - j, "coordinator", p.coordinator, "ServerInfo", "bool", "coordinator"); - to_json_key(j, "starting", p.starting, "ServerInfo", "bool", "starting"); - to_json_key(j, "uptime", p.uptime, "ServerInfo", "Duration", "uptime"); -} - -void from_json(const json& j, ServerInfo& p) { + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "TaskStats", + "int64_t", + "physicalWrittenDataSizeInBytes"); + from_json_key( + j, "fullGcCount", p.fullGcCount, "TaskStats", "int", "fullGcCount"); from_json_key( j, - "nodeVersion", - p.nodeVersion, - "ServerInfo", - "NodeVersion", - "nodeVersion"); + "fullGcTimeInMillis", + p.fullGcTimeInMillis, + "TaskStats", + "int64_t", + "fullGcTimeInMillis"); from_json_key( - j, "environment", p.environment, "ServerInfo", "String", "environment"); + j, + "pipelines", + p.pipelines, + "TaskStats", + "List", + "pipelines"); 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, + "runtimeStats", + p.runtimeStats, + "TaskStats", + "RuntimeStats", + "runtimeStats"); } } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ namespace facebook::presto::protocol { - -void to_json(json& j, const HiveBucketFilter& p) { - j = json::object(); - to_json_key( - j, - "bucketsToKeep", - p.bucketsToKeep, - "HiveBucketFilter", - "List", - "bucketsToKeep"); +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + getConnectorProtocol(type).to_json(j, p); } -void from_json(const json& j, HiveBucketFilter& p) { - from_json_key( - j, - "bucketsToKeep", - p.bucketsToKeep, - "HiveBucketFilter", - "List", - "bucketsToKeep"); +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"); + } + getConnectorProtocol(type).from_json(j, p); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const HiveBucketHandle& p) { +void to_json(json& j, const MetadataUpdates& p) { j = json::object(); to_json_key( j, - "columns", - p.columns, - "HiveBucketHandle", - "List", - "columns"); - to_json_key( - j, - "tableBucketCount", - p.tableBucketCount, - "HiveBucketHandle", - "int", - "tableBucketCount"); + "connectorId", + p.connectorId, + "MetadataUpdates", + "ConnectorId", + "connectorId"); to_json_key( j, - "readBucketCount", - p.readBucketCount, - "HiveBucketHandle", - "int", - "readBucketCount"); + "metadataUpdates", + p.metadataUpdates, + "MetadataUpdates", + "List>", + "metadataUpdates"); } -void from_json(const json& j, HiveBucketHandle& p) { - from_json_key( - j, - "columns", - p.columns, - "HiveBucketHandle", - "List", - "columns"); +void from_json(const json& j, MetadataUpdates& p) { from_json_key( j, - "tableBucketCount", - p.tableBucketCount, - "HiveBucketHandle", - "int", - "tableBucketCount"); + "connectorId", + p.connectorId, + "MetadataUpdates", + "ConnectorId", + "connectorId"); from_json_key( j, - "readBucketCount", - p.readBucketCount, - "HiveBucketHandle", - "int", - "readBucketCount"); + "metadataUpdates", + p.metadataUpdates, + "MetadataUpdates", + "List>", + "metadataUpdates"); } } // 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 TaskInfo& p) { j = json::object(); - j["@type"] = "hive"; - to_json_key( - j, - "schemaTableName", - p.schemaTableName, - "HiveTableLayoutHandle", - "SchemaTableName", - "schemaTableName"); - to_json_key( - j, - "tablePath", - p.tablePath, - "HiveTableLayoutHandle", - "String", - "tablePath"); - to_json_key( - j, - "partitionColumns", - p.partitionColumns, - "HiveTableLayoutHandle", - "List", - "partitionColumns"); - to_json_key( - j, - "dataColumns", - p.dataColumns, - "HiveTableLayoutHandle", - "List", - "dataColumns"); - to_json_key( - j, - "tableParameters", - p.tableParameters, - "HiveTableLayoutHandle", - "Map", - "tableParameters"); - to_json_key( - j, - "domainPredicate", - p.domainPredicate, - "HiveTableLayoutHandle", - "TupleDomain", - "domainPredicate"); - to_json_key( - j, - "remainingPredicate", - p.remainingPredicate, - "HiveTableLayoutHandle", - "RowExpression", - "remainingPredicate"); - to_json_key( - j, - "predicateColumns", - p.predicateColumns, - "HiveTableLayoutHandle", - "Map", - "predicateColumns"); - to_json_key( - j, - "partitionColumnPredicate", - p.partitionColumnPredicate, - "HiveTableLayoutHandle", - "TupleDomain>", - "partitionColumnPredicate"); - to_json_key( - j, - "bucketHandle", - p.bucketHandle, - "HiveTableLayoutHandle", - "HiveBucketHandle", - "bucketHandle"); + to_json_key(j, "taskId", p.taskId, "TaskInfo", "TaskId", "taskId"); to_json_key( - j, - "bucketFilter", - p.bucketFilter, - "HiveTableLayoutHandle", - "HiveBucketFilter", - "bucketFilter"); + j, "taskStatus", p.taskStatus, "TaskInfo", "TaskStatus", "taskStatus"); to_json_key( j, - "pushdownFilterEnabled", - p.pushdownFilterEnabled, - "HiveTableLayoutHandle", - "bool", - "pushdownFilterEnabled"); + "lastHeartbeat", + p.lastHeartbeat, + "TaskInfo", + "DateTime", + "lastHeartbeat"); to_json_key( j, - "layoutString", - p.layoutString, - "HiveTableLayoutHandle", - "String", - "layoutString"); + "outputBuffers", + p.outputBuffers, + "TaskInfo", + "OutputBufferInfo", + "outputBuffers"); to_json_key( j, - "requestedColumns", - p.requestedColumns, - "HiveTableLayoutHandle", - "List", - "requestedColumns"); + "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, - "partialAggregationsPushedDown", - p.partialAggregationsPushedDown, - "HiveTableLayoutHandle", - "bool", - "partialAggregationsPushedDown"); - to_json_key( + "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, - "appendRowNumber", - p.appendRowNumber, - "HiveTableLayoutHandle", - "bool", - "appendRowNumber"); - to_json_key( + "lastHeartbeat", + p.lastHeartbeat, + "TaskInfo", + "DateTime", + "lastHeartbeat"); + from_json_key( j, - "footerStatsUnreliable", - p.footerStatsUnreliable, - "HiveTableLayoutHandle", - "bool", - "footerStatsUnreliable"); + "outputBuffers", + p.outputBuffers, + "TaskInfo", + "OutputBufferInfo", + "outputBuffers"); + 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"); + 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 { +InsertHandle::InsertHandle() noexcept { + _type = "InsertHandle"; } -void from_json(const json& j, HiveTableLayoutHandle& p) { - p._type = j["@type"]; - from_json_key( +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, - "HiveTableLayoutHandle", + "InsertHandle", "SchemaTableName", "schemaTableName"); +} + +void from_json(const json& j, InsertHandle& p) { + p._type = j["@type"]; from_json_key( - j, - "tablePath", - p.tablePath, - "HiveTableLayoutHandle", - "String", - "tablePath"); - from_json_key( - j, - "partitionColumns", - p.partitionColumns, - "HiveTableLayoutHandle", - "List", - "partitionColumns"); + j, "handle", p.handle, "InsertHandle", "InsertTableHandle", "handle"); from_json_key( j, - "dataColumns", - p.dataColumns, - "HiveTableLayoutHandle", - "List", - "dataColumns"); - from_json_key( + "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; + getConnectorProtocol(type).to_json(j, p); +} + +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError( + std::string(e.what()) + + " ConnectorOutputTableHandle ConnectorOutputTableHandle"); + } + getConnectorProtocol(type).from_json(j, p); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const OutputTableHandle& p) { + j = json::object(); + to_json_key( j, - "tableParameters", - p.tableParameters, - "HiveTableLayoutHandle", - "Map", - "tableParameters"); - from_json_key( + "connectorId", + p.connectorId, + "OutputTableHandle", + "ConnectorId", + "connectorId"); + to_json_key( j, - "domainPredicate", - p.domainPredicate, - "HiveTableLayoutHandle", - "TupleDomain", - "domainPredicate"); - from_json_key( + "transactionHandle", + p.transactionHandle, + "OutputTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); + to_json_key( j, - "remainingPredicate", - p.remainingPredicate, - "HiveTableLayoutHandle", - "RowExpression", - "remainingPredicate"); + "connectorHandle", + p.connectorHandle, + "OutputTableHandle", + "ConnectorOutputTableHandle", + "connectorHandle"); +} + +void from_json(const json& j, OutputTableHandle& p) { from_json_key( j, - "predicateColumns", - p.predicateColumns, - "HiveTableLayoutHandle", - "Map", - "predicateColumns"); + "connectorId", + p.connectorId, + "OutputTableHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "partitionColumnPredicate", - p.partitionColumnPredicate, - "HiveTableLayoutHandle", - "TupleDomain>", - "partitionColumnPredicate"); + "transactionHandle", + p.transactionHandle, + "OutputTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); from_json_key( j, - "bucketHandle", - p.bucketHandle, - "HiveTableLayoutHandle", - "HiveBucketHandle", - "bucketHandle"); - from_json_key( + "connectorHandle", + p.connectorHandle, + "OutputTableHandle", + "ConnectorOutputTableHandle", + "connectorHandle"); +} +} // 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, - "bucketFilter", - p.bucketFilter, - "HiveTableLayoutHandle", - "HiveBucketFilter", - "bucketFilter"); + "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, - "pushdownFilterEnabled", - p.pushdownFilterEnabled, - "HiveTableLayoutHandle", - "bool", - "pushdownFilterEnabled"); - from_json_key( + "allocation", + p.allocation, + "MemoryAllocation", + "int64_t", + "allocation"); +} +} // 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; +} +} // 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, - "layoutString", - p.layoutString, - "HiveTableLayoutHandle", - "String", - "layoutString"); - from_json_key( + "sourceLocation", + p.sourceLocation, + "SpecialFormExpression", + "SourceLocation", + "sourceLocation"); + to_json_key(j, "form", p.form, "SpecialFormExpression", "Form", "form"); + to_json_key( j, - "requestedColumns", - p.requestedColumns, - "HiveTableLayoutHandle", - "List", - "requestedColumns"); + "returnType", + p.returnType, + "SpecialFormExpression", + "Type", + "returnType"); + to_json_key( + j, + "arguments", + p.arguments, + "SpecialFormExpression", + "List>", + "arguments"); +} + +void from_json(const json& j, SpecialFormExpression& p) { + p._type = j["@type"]; from_json_key( j, - "partialAggregationsPushedDown", - p.partialAggregationsPushedDown, - "HiveTableLayoutHandle", - "bool", - "partialAggregationsPushedDown"); + "sourceLocation", + p.sourceLocation, + "SpecialFormExpression", + "SourceLocation", + "sourceLocation"); + from_json_key(j, "form", p.form, "SpecialFormExpression", "Form", "form"); from_json_key( j, - "appendRowNumber", - p.appendRowNumber, - "HiveTableLayoutHandle", - "bool", - "appendRowNumber"); + "returnType", + p.returnType, + "SpecialFormExpression", + "Type", + "returnType"); from_json_key( j, - "footerStatsUnreliable", - p.footerStatsUnreliable, - "HiveTableLayoutHandle", - "bool", - "footerStatsUnreliable"); + "arguments", + p.arguments, + "SpecialFormExpression", + "List>", + "arguments"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -OutputNode::OutputNode() noexcept { - _type = ".OutputNode"; +HiveTransactionHandle::HiveTransactionHandle() noexcept { + _type = "hive"; } -void to_json(json& j, const OutputNode& p) { +void to_json(json& j, const HiveTransactionHandle& 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"); + 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 { +SystemColumnHandle::SystemColumnHandle() noexcept { + _type = "$system@system"; +} + +void to_json(json& j, const SystemColumnHandle& p) { + j = json::object(); + j["@type"] = "$system@system"; to_json_key( j, - "columnNames", - p.columnNames, - "OutputNode", - "List", - "columnNames"); + "connectorId", + p.connectorId, + "SystemColumnHandle", + "ConnectorId", + "connectorId"); to_json_key( j, - "outputVariables", - p.outputVariables, - "OutputNode", - "List", - "outputVariables"); + "columnName", + p.columnName, + "SystemColumnHandle", + "String", + "columnName"); } -void from_json(const json& j, OutputNode& p) { +void from_json(const json& j, SystemColumnHandle& 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"); from_json_key( j, - "columnNames", - p.columnNames, - "OutputNode", - "List", - "columnNames"); + "connectorId", + p.connectorId, + "SystemColumnHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "outputVariables", - p.outputVariables, - "OutputNode", - "List", - "outputVariables"); + "columnName", + p.columnName, + "SystemColumnHandle", + "String", + "columnName"); } } // 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 NodeVersion& p) { + j = json::object(); + to_json_key(j, "version", p.version, "NodeVersion", "String", "version"); } -void to_json(json& j, const RowNumberNode& p) { +void from_json(const json& j, NodeVersion& p) { + from_json_key(j, "version", p.version, "NodeVersion", "String", "version"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const GroupingSetDescriptor& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.RowNumberNode"; - to_json_key(j, "id", p.id, "RowNumberNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "RowNumberNode", "PlanNode", "source"); to_json_key( j, - "partitionBy", - p.partitionBy, - "RowNumberNode", + "groupingKeys", + p.groupingKeys, + "GroupingSetDescriptor", "List", - "partitionBy"); - to_json_key( - j, - "rowNumberVariable", - p.rowNumberVariable, - "RowNumberNode", - "VariableReferenceExpression", - "rowNumberVariable"); + "groupingKeys"); to_json_key( j, - "maxRowCountPerPartition", - p.maxRowCountPerPartition, - "RowNumberNode", - "Integer", - "maxRowCountPerPartition"); - to_json_key(j, "partial", p.partial, "RowNumberNode", "bool", "partial"); + "groupingSetCount", + p.groupingSetCount, + "GroupingSetDescriptor", + "int", + "groupingSetCount"); to_json_key( j, - "hashVariable", - p.hashVariable, - "RowNumberNode", - "VariableReferenceExpression", - "hashVariable"); + "globalGroupingSets", + p.globalGroupingSets, + "GroupingSetDescriptor", + "List", + "globalGroupingSets"); } -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, GroupingSetDescriptor& p) { from_json_key( j, - "partitionBy", - p.partitionBy, - "RowNumberNode", + "groupingKeys", + p.groupingKeys, + "GroupingSetDescriptor", "List", - "partitionBy"); - from_json_key( - j, - "rowNumberVariable", - p.rowNumberVariable, - "RowNumberNode", - "VariableReferenceExpression", - "rowNumberVariable"); + "groupingKeys"); from_json_key( j, - "maxRowCountPerPartition", - p.maxRowCountPerPartition, - "RowNumberNode", - "Integer", - "maxRowCountPerPartition"); - from_json_key(j, "partial", p.partial, "RowNumberNode", "bool", "partial"); + "groupingSetCount", + p.groupingSetCount, + "GroupingSetDescriptor", + "int", + "groupingSetCount"); from_json_key( j, - "hashVariable", - p.hashVariable, - "RowNumberNode", - "VariableReferenceExpression", - "hashVariable"); + "globalGroupingSets", + p.globalGroupingSets, + "GroupingSetDescriptor", + "List", + "globalGroupingSets"); } } // 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 { +TpchColumnHandle::TpchColumnHandle() noexcept { + _type = "tpch"; +} -// dependency KeyedSubclass +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, + "columnName", + p.columnName, + "TpchColumnHandle", + "String", + "columnName"); + from_json_key(j, "type", p.type, "TpchColumnHandle", "Type", "type"); +} +} // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -std::string JsonEncodedSubclass::getSubclassKey(const nlohmann::json& j) { - if (j.is_array()) { - // enum is serialized as an array: ["type","instance"] - return j[0]; - } else { - return j["@type"]; - } +// 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 { +// 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 { -void to_json(json& j, const PlanCostEstimate& p) { +void to_json(json& j, const DeleteFile& p) { j = json::object(); - to_json_key(j, "cpuCost", p.cpuCost, "PlanCostEstimate", "double", "cpuCost"); + 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, "maxMemory", p.maxMemory, "PlanCostEstimate", "double", "maxMemory"); + j, "recordCount", p.recordCount, "DeleteFile", "int64_t", "recordCount"); to_json_key( j, - "maxMemoryWhenOutputting", - p.maxMemoryWhenOutputting, - "PlanCostEstimate", - "double", - "maxMemoryWhenOutputting"); + "fileSizeInBytes", + p.fileSizeInBytes, + "DeleteFile", + "int64_t", + "fileSizeInBytes"); to_json_key( j, - "networkCost", - p.networkCost, - "PlanCostEstimate", - "double", - "networkCost"); + "equalityFieldIds", + p.equalityFieldIds, + "DeleteFile", + "List", + "equalityFieldIds"); + to_json_key( + j, + "lowerBounds", + p.lowerBounds, + "DeleteFile", + "Map", + "lowerBounds"); + to_json_key( + j, + "upperBounds", + p.upperBounds, + "DeleteFile", + "Map", + "upperBounds"); } -void from_json(const json& j, PlanCostEstimate& p) { +void from_json(const json& j, DeleteFile& p) { from_json_key( - j, "cpuCost", p.cpuCost, "PlanCostEstimate", "double", "cpuCost"); + 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, "maxMemory", p.maxMemory, "PlanCostEstimate", "double", "maxMemory"); + j, "recordCount", p.recordCount, "DeleteFile", "int64_t", "recordCount"); from_json_key( j, - "maxMemoryWhenOutputting", - p.maxMemoryWhenOutputting, - "PlanCostEstimate", - "double", - "maxMemoryWhenOutputting"); + "fileSizeInBytes", + p.fileSizeInBytes, + "DeleteFile", + "int64_t", + "fileSizeInBytes"); from_json_key( j, - "networkCost", - p.networkCost, - "PlanCostEstimate", - "double", - "networkCost"); + "equalityFieldIds", + p.equalityFieldIds, + "DeleteFile", + "List", + "equalityFieldIds"); + from_json_key( + j, + "lowerBounds", + p.lowerBounds, + "DeleteFile", + "Map", + "lowerBounds"); + from_json_key( + j, + "upperBounds", + p.upperBounds, + "DeleteFile", + "Map", + "upperBounds"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const StatsAndCosts& p) { +void to_json(json& j, const StatisticAggregations& p) { j = json::object(); to_json_key( j, - "stats", - p.stats, - "StatsAndCosts", - "Map", - "stats"); + "outputVariables", + p.outputVariables, + "StatisticAggregations", + "List", + "outputVariables"); + to_json_key( + j, + "aggregations", + p.aggregations, + "StatisticAggregations", + "Map", + "aggregations"); to_json_key( j, - "costs", - p.costs, - "StatsAndCosts", - "Map", - "costs"); + "groupingVariables", + p.groupingVariables, + "StatisticAggregations", + "List", + "groupingVariables"); } -void from_json(const json& j, StatsAndCosts& p) { +void from_json(const json& j, StatisticAggregations& p) { from_json_key( j, - "stats", - p.stats, - "StatsAndCosts", - "Map", - "stats"); + "outputVariables", + p.outputVariables, + "StatisticAggregations", + "List", + "outputVariables"); from_json_key( j, - "costs", - p.costs, - "StatsAndCosts", - "Map", - "costs"); + "aggregations", + p.aggregations, + "StatisticAggregations", + "Map", + "aggregations"); + from_json_key( + j, + "groupingVariables", + p.groupingVariables, + "StatisticAggregations", + "List", + "groupingVariables"); } } // 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 MemoryPoolInfo& p) { j = json::object(); - to_json_key(j, "id", p.id, "PlanFragment", "PlanFragmentId", "id"); - to_json_key(j, "root", p.root, "PlanFragment", "PlanNode", "root"); - to_json_key( - j, - "variables", - p.variables, - "PlanFragment", - "List", - "variables"); to_json_key( - j, - "partitioning", - p.partitioning, - "PlanFragment", - "PartitioningHandle", - "partitioning"); + j, "maxBytes", p.maxBytes, "MemoryPoolInfo", "int64_t", "maxBytes"); to_json_key( j, - "tableScanSchedulingOrder", - p.tableScanSchedulingOrder, - "PlanFragment", - "List", - "tableScanSchedulingOrder"); + "reservedBytes", + p.reservedBytes, + "MemoryPoolInfo", + "int64_t", + "reservedBytes"); to_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "PlanFragment", - "PartitioningScheme", - "partitioningScheme"); + "reservedRevocableBytes", + p.reservedRevocableBytes, + "MemoryPoolInfo", + "int64_t", + "reservedRevocableBytes"); to_json_key( j, - "stageExecutionDescriptor", - p.stageExecutionDescriptor, - "PlanFragment", - "StageExecutionDescriptor", - "stageExecutionDescriptor"); + "queryMemoryReservations", + p.queryMemoryReservations, + "MemoryPoolInfo", + "Map", + "queryMemoryReservations"); to_json_key( j, - "outputTableWriterFragment", - p.outputTableWriterFragment, - "PlanFragment", - "bool", - "outputTableWriterFragment"); + "queryMemoryAllocations", + p.queryMemoryAllocations, + "MemoryPoolInfo", + "Map>", + "queryMemoryAllocations"); to_json_key( j, - "jsonRepresentation", - p.jsonRepresentation, - "PlanFragment", - "String", - "jsonRepresentation"); + "queryMemoryRevocableReservations", + p.queryMemoryRevocableReservations, + "MemoryPoolInfo", + "Map", + "queryMemoryRevocableReservations"); } -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"); +void from_json(const json& j, MemoryPoolInfo& p) { + from_json_key( + j, "maxBytes", p.maxBytes, "MemoryPoolInfo", "int64_t", "maxBytes"); from_json_key( j, - "variables", - p.variables, - "PlanFragment", - "List", - "variables"); + "reservedBytes", + p.reservedBytes, + "MemoryPoolInfo", + "int64_t", + "reservedBytes"); from_json_key( j, - "partitioning", - p.partitioning, - "PlanFragment", - "PartitioningHandle", - "partitioning"); + "reservedRevocableBytes", + p.reservedRevocableBytes, + "MemoryPoolInfo", + "int64_t", + "reservedRevocableBytes"); from_json_key( j, - "tableScanSchedulingOrder", - p.tableScanSchedulingOrder, - "PlanFragment", - "List", - "tableScanSchedulingOrder"); + "queryMemoryReservations", + p.queryMemoryReservations, + "MemoryPoolInfo", + "Map", + "queryMemoryReservations"); from_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "PlanFragment", - "PartitioningScheme", - "partitioningScheme"); + "queryMemoryAllocations", + p.queryMemoryAllocations, + "MemoryPoolInfo", + "Map>", + "queryMemoryAllocations"); from_json_key( j, - "stageExecutionDescriptor", - p.stageExecutionDescriptor, - "PlanFragment", - "StageExecutionDescriptor", - "stageExecutionDescriptor"); + "queryMemoryRevocableReservations", + p.queryMemoryRevocableReservations, + "MemoryPoolInfo", + "Map", + "queryMemoryRevocableReservations"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const MemoryInfo& p) { + j = json::object(); + to_json_key( + j, + "totalNodeMemory", + p.totalNodeMemory, + "MemoryInfo", + "DataSize", + "totalNodeMemory"); + to_json_key( + j, + "pools", + p.pools, + "MemoryInfo", + "Map", + "pools"); +} + +void from_json(const json& j, MemoryInfo& p) { from_json_key( j, - "outputTableWriterFragment", - p.outputTableWriterFragment, - "PlanFragment", - "bool", - "outputTableWriterFragment"); + "totalNodeMemory", + p.totalNodeMemory, + "MemoryInfo", + "DataSize", + "totalNodeMemory"); from_json_key( j, - "jsonRepresentation", - p.jsonRepresentation, - "PlanFragment", - "String", - "jsonRepresentation"); + "pools", + p.pools, + "MemoryInfo", + "Map", + "pools"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -LambdaDefinitionExpression::LambdaDefinitionExpression() noexcept { - _type = "lambda"; -} -void to_json(json& j, const LambdaDefinitionExpression& p) { +void to_json(json& j, const NodeStatus& p) { j = json::object(); - j["@type"] = "lambda"; + to_json_key(j, "nodeId", p.nodeId, "NodeStatus", "String", "nodeId"); + to_json_key( + j, + "nodeVersion", + p.nodeVersion, + "NodeStatus", + "NodeVersion", + "nodeVersion"); + to_json_key( + j, "environment", p.environment, "NodeStatus", "String", "environment"); + to_json_key( + j, "coordinator", p.coordinator, "NodeStatus", "bool", "coordinator"); + to_json_key(j, "uptime", p.uptime, "NodeStatus", "Duration", "uptime"); + to_json_key( + j, + "externalAddress", + p.externalAddress, + "NodeStatus", + "String", + "externalAddress"); to_json_key( j, - "sourceLocation", - p.sourceLocation, - "LambdaDefinitionExpression", - "SourceLocation", - "sourceLocation"); + "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"); to_json_key( j, - "argumentTypes", - p.argumentTypes, - "LambdaDefinitionExpression", - "List", - "argumentTypes"); + "processCpuLoad", + p.processCpuLoad, + "NodeStatus", + "double", + "processCpuLoad"); to_json_key( j, - "arguments", - p.arguments, - "LambdaDefinitionExpression", - "List", - "arguments"); + "systemCpuLoad", + p.systemCpuLoad, + "NodeStatus", + "double", + "systemCpuLoad"); + to_json_key(j, "heapUsed", p.heapUsed, "NodeStatus", "int64_t", "heapUsed"); to_json_key( - j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); + j, + "heapAvailable", + p.heapAvailable, + "NodeStatus", + "int64_t", + "heapAvailable"); + to_json_key( + j, "nonHeapUsed", p.nonHeapUsed, "NodeStatus", "int64_t", "nonHeapUsed"); } -void from_json(const json& j, LambdaDefinitionExpression& p) { - p._type = j["@type"]; +void from_json(const json& j, NodeStatus& p) { + from_json_key(j, "nodeId", p.nodeId, "NodeStatus", "String", "nodeId"); from_json_key( j, - "sourceLocation", - p.sourceLocation, - "LambdaDefinitionExpression", - "SourceLocation", - "sourceLocation"); + "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, - "argumentTypes", - p.argumentTypes, - "LambdaDefinitionExpression", - "List", - "argumentTypes"); + "externalAddress", + p.externalAddress, + "NodeStatus", + "String", + "externalAddress"); from_json_key( j, - "arguments", - p.arguments, - "LambdaDefinitionExpression", - "List", - "arguments"); + "internalAddress", + p.internalAddress, + "NodeStatus", + "String", + "internalAddress"); from_json_key( - j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); + j, "memoryInfo", p.memoryInfo, "NodeStatus", "MemoryInfo", "memoryInfo"); + from_json_key( + j, "processors", p.processors, "NodeStatus", "int", "processors"); + from_json_key( + j, + "processCpuLoad", + p.processCpuLoad, + "NodeStatus", + "double", + "processCpuLoad"); + from_json_key( + j, + "systemCpuLoad", + p.systemCpuLoad, + "NodeStatus", + "double", + "systemCpuLoad"); + from_json_key(j, "heapUsed", p.heapUsed, "NodeStatus", "int64_t", "heapUsed"); + from_json_key( + j, + "heapAvailable", + p.heapAvailable, + "NodeStatus", + "int64_t", + "heapAvailable"); + from_json_key( + j, "nonHeapUsed", p.nonHeapUsed, "NodeStatus", "int64_t", "nonHeapUsed"); } } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// dependency KeyedSubclass + namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair 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; +std::string JsonEncodedSubclass::getSubclassKey(const nlohmann::json& j) { + if (j.is_array()) { + // enum is serialized as an array: ["type","instance"] + return j[0]; + } else { + return j["@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 FileFormat_enum_table[] = +static const std::pair IcebergTableType_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!"); + {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(FileFormat_enum_table), - std::end(FileFormat_enum_table), - [e](const std::pair& ej_pair) -> bool { + 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(FileFormat_enum_table)) + j = ((it != std::end(IcebergTableType_enum_table)) ? it - : std::begin(FileFormat_enum_table)) + : std::begin(IcebergTableType_enum_table)) ->second; } -void from_json(const json& j, FileFormat& e) { - static_assert(std::is_enum::value, "FileFormat must be an enum!"); +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(FileFormat_enum_table), - std::end(FileFormat_enum_table), - [&j](const std::pair& ej_pair) -> bool { + 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(FileFormat_enum_table)) + e = ((it != std::end(IcebergTableType_enum_table)) ? it - : std::begin(FileFormat_enum_table)) + : std::begin(IcebergTableType_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const DeleteFile& p) { +void to_json(json& j, const IcebergTableName& p) { j = json::object(); - 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, - "fileSizeInBytes", - p.fileSizeInBytes, - "DeleteFile", - "int64_t", - "fileSizeInBytes"); to_json_key( - j, - "equalityFieldIds", - p.equalityFieldIds, - "DeleteFile", - "List", - "equalityFieldIds"); + j, "tableName", p.tableName, "IcebergTableName", "String", "tableName"); to_json_key( j, - "lowerBounds", - p.lowerBounds, - "DeleteFile", - "Map", - "lowerBounds"); + "tableType", + p.tableType, + "IcebergTableName", + "IcebergTableType", + "tableType"); to_json_key( - j, - "upperBounds", - p.upperBounds, - "DeleteFile", - "Map", - "upperBounds"); + j, "snapshotId", p.snapshotId, "IcebergTableName", "Long", "snapshotId"); + to_json_key( + j, + "changelogEndSnapshot", + p.changelogEndSnapshot, + "IcebergTableName", + "Long", + "changelogEndSnapshot"); } -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"); +void from_json(const json& j, IcebergTableName& p) { from_json_key( - j, - "fileSizeInBytes", - p.fileSizeInBytes, - "DeleteFile", - "int64_t", - "fileSizeInBytes"); + j, "tableName", p.tableName, "IcebergTableName", "String", "tableName"); from_json_key( j, - "equalityFieldIds", - p.equalityFieldIds, - "DeleteFile", - "List", - "equalityFieldIds"); + "tableType", + p.tableType, + "IcebergTableName", + "IcebergTableType", + "tableType"); from_json_key( - j, - "lowerBounds", - p.lowerBounds, - "DeleteFile", - "Map", - "lowerBounds"); + j, "snapshotId", p.snapshotId, "IcebergTableName", "Long", "snapshotId"); from_json_key( j, - "upperBounds", - p.upperBounds, - "DeleteFile", - "Map", - "upperBounds"); -} -} // 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"); + "changelogEndSnapshot", + p.changelogEndSnapshot, + "IcebergTableName", + "Long", + "changelogEndSnapshot"); } } // 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 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, - "NodeSelectionStrategy must be an enum!"); + std::is_enum::value, "SampleNodeType 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(SampleNodeType_enum_table), + std::end(SampleNodeType_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(SampleNodeType_enum_table)) ? it - : std::begin(NodeSelectionStrategy_enum_table)) + : std::begin(SampleNodeType_enum_table)) ->second; } -void from_json(const json& j, NodeSelectionStrategy& e) { +void from_json(const json& j, SampleNodeType& e) { static_assert( - std::is_enum::value, - "NodeSelectionStrategy must be an enum!"); + std::is_enum::value, "SampleNodeType 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(SampleNodeType_enum_table), + std::end(SampleNodeType_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(SampleNodeType_enum_table)) ? it - : std::begin(NodeSelectionStrategy_enum_table)) + : std::begin(SampleNodeType_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -IcebergSplit::IcebergSplit() noexcept { - _type = "hive-iceberg"; +SampleNode::SampleNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.SampleNode"; } -void to_json(json& j, const IcebergSplit& p) { +void to_json(json& j, const SampleNode& 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"); + 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, "sampleRatio", p.sampleRatio, "SampleNode", "double", "sampleRatio"); to_json_key( j, - "fileFormat", - p.fileFormat, - "IcebergSplit", - "FileFormat", - "fileFormat"); + "sampleType", + p.sampleType, + "SampleNode", + "SampleNodeType", + "sampleType"); +} + +void from_json(const json& j, SampleNode& 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"); + from_json_key( + j, + "sampleType", + p.sampleType, + "SampleNode", + "SampleNodeType", + "sampleType"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +UniformDistributionHistogram::UniformDistributionHistogram() noexcept { + _type = "uniform-distribution"; +} + +void to_json(json& j, const UniformDistributionHistogram& p) { + j = json::object(); + j["@type"] = "uniform-distribution"; to_json_key( j, - "addresses", - p.addresses, - "IcebergSplit", - "List", - "addresses"); + "lowValue", + p.lowValue, + "UniformDistributionHistogram", + "double", + "lowValue"); to_json_key( j, - "partitionKeys", - p.partitionKeys, - "IcebergSplit", - "Map", - "partitionKeys"); + "highValue", + p.highValue, + "UniformDistributionHistogram", + "double", + "highValue"); +} + +void from_json(const json& j, UniformDistributionHistogram& p) { + p._type = j["@type"]; + from_json_key( + j, + "lowValue", + p.lowValue, + "UniformDistributionHistogram", + "double", + "lowValue"); + from_json_key( + j, + "highValue", + p.highValue, + "UniformDistributionHistogram", + "double", + "highValue"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +SystemTableLayoutHandle::SystemTableLayoutHandle() noexcept { + _type = "$system@system"; +} + +void to_json(json& j, const SystemTableLayoutHandle& p) { + j = json::object(); + j["@type"] = "$system@system"; to_json_key( j, - "partitionSpecAsJson", - p.partitionSpecAsJson, - "IcebergSplit", - "String", - "partitionSpecAsJson"); + "connectorId", + p.connectorId, + "SystemTableLayoutHandle", + "ConnectorId", + "connectorId"); to_json_key( j, - "partitionDataJson", - p.partitionDataJson, - "IcebergSplit", - "String", - "partitionDataJson"); + "table", + p.table, + "SystemTableLayoutHandle", + "SystemTableHandle", + "table"); to_json_key( j, - "nodeSelectionStrategy", - p.nodeSelectionStrategy, - "IcebergSplit", - "NodeSelectionStrategy", - "nodeSelectionStrategy"); + "constraint", + p.constraint, + "SystemTableLayoutHandle", + "TupleDomain>", + "constraint"); +} + +void from_json(const json& j, SystemTableLayoutHandle& p) { + p._type = j["@type"]; + from_json_key( + j, + "connectorId", + p.connectorId, + "SystemTableLayoutHandle", + "ConnectorId", + "connectorId"); + from_json_key( + j, + "table", + p.table, + "SystemTableLayoutHandle", + "SystemTableHandle", + "table"); + from_json_key( + j, + "constraint", + p.constraint, + "SystemTableLayoutHandle", + "TupleDomain>", + "constraint"); +} +} // 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( j, - "splitWeight", - p.splitWeight, - "IcebergSplit", - "SplitWeight", - "splitWeight"); + "scaleFactor", + p.scaleFactor, + "TpchTableHandle", + "double", + "scaleFactor"); +} + +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, + "scaleFactor", + p.scaleFactor, + "TpchTableHandle", + "double", + "scaleFactor"); +} +} // 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, "deletes", p.deletes, "IcebergSplit", "List", "deletes"); + j, + "replicateVariables", + p.replicateVariables, + "UnnestNode", + "List", + "replicateVariables"); to_json_key( j, - "changelogSplitInfo", - p.changelogSplitInfo, - "IcebergSplit", - "ChangelogSplitInfo", - "changelogSplitInfo"); + "unnestVariables", + p.unnestVariables, + "UnnestNode", + "Map>", + "unnestVariables"); to_json_key( j, - "dataSequenceNumber", - p.dataSequenceNumber, - "IcebergSplit", - "int64_t", - "dataSequenceNumber"); + "ordinalityVariable", + p.ordinalityVariable, + "UnnestNode", + "VariableReferenceExpression", + "ordinalityVariable"); } -void from_json(const json& j, IcebergSplit& p) { +void from_json(const json& j, UnnestNode& 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, "id", p.id, "UnnestNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "UnnestNode", "PlanNode", "source"); from_json_key( j, - "fileFormat", - p.fileFormat, - "IcebergSplit", - "FileFormat", - "fileFormat"); + "replicateVariables", + p.replicateVariables, + "UnnestNode", + "List", + "replicateVariables"); from_json_key( j, - "addresses", - p.addresses, - "IcebergSplit", - "List", - "addresses"); + "unnestVariables", + p.unnestVariables, + "UnnestNode", + "Map>", + "unnestVariables"); from_json_key( j, - "partitionKeys", - p.partitionKeys, - "IcebergSplit", - "Map", - "partitionKeys"); - from_json_key( + "ordinalityVariable", + p.ordinalityVariable, + "UnnestNode", + "VariableReferenceExpression", + "ordinalityVariable"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +TpchPartitioningHandle::TpchPartitioningHandle() noexcept { + _type = "tpch"; +} + +void to_json(json& j, const TpchPartitioningHandle& p) { + j = json::object(); + j["@type"] = "tpch"; + to_json_key(j, "table", p.table, "TpchPartitioningHandle", "String", "table"); + to_json_key( j, - "partitionSpecAsJson", - p.partitionSpecAsJson, - "IcebergSplit", - "String", - "partitionSpecAsJson"); + "totalRows", + p.totalRows, + "TpchPartitioningHandle", + "int64_t", + "totalRows"); +} + +void from_json(const json& j, TpchPartitioningHandle& p) { + p._type = j["@type"]; from_json_key( - j, - "partitionDataJson", - p.partitionDataJson, - "IcebergSplit", - "String", - "partitionDataJson"); + j, "table", p.table, "TpchPartitioningHandle", "String", "table"); from_json_key( j, - "nodeSelectionStrategy", - p.nodeSelectionStrategy, - "IcebergSplit", - "NodeSelectionStrategy", - "nodeSelectionStrategy"); - from_json_key( + "totalRows", + p.totalRows, + "TpchPartitioningHandle", + "int64_t", + "totalRows"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +HiveTableHandle::HiveTableHandle() noexcept { + _type = "hive"; +} + +void to_json(json& j, const HiveTableHandle& p) { + j = json::object(); + j["@type"] = "hive"; + to_json_key( + j, "schemaName", p.schemaName, "HiveTableHandle", "String", "schemaName"); + to_json_key( + j, "tableName", p.tableName, "HiveTableHandle", "String", "tableName"); + to_json_key( j, - "splitWeight", - p.splitWeight, - "IcebergSplit", - "SplitWeight", - "splitWeight"); + "analyzePartitionValues", + p.analyzePartitionValues, + "HiveTableHandle", + "List>", + "analyzePartitionValues"); +} + +void from_json(const json& j, HiveTableHandle& p) { + p._type = j["@type"]; from_json_key( - j, "deletes", p.deletes, "IcebergSplit", "List", "deletes"); + j, "schemaName", p.schemaName, "HiveTableHandle", "String", "schemaName"); + from_json_key( + j, "tableName", p.tableName, "HiveTableHandle", "String", "tableName"); from_json_key( j, - "changelogSplitInfo", - p.changelogSplitInfo, - "IcebergSplit", - "ChangelogSplitInfo", - "changelogSplitInfo"); + "analyzePartitionValues", + p.analyzePartitionValues, + "HiveTableHandle", + "List>", + "analyzePartitionValues"); +} +} // 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( + j, + "signature", + p.signature, + "BuiltInFunctionHandle", + "Signature", + "signature"); +} + +void from_json(const json& j, BuiltInFunctionHandle& p) { + p._type = j["@type"]; from_json_key( j, - "dataSequenceNumber", - p.dataSequenceNumber, - "IcebergSplit", - "int64_t", - "dataSequenceNumber"); + "signature", + p.signature, + "BuiltInFunctionHandle", + "Signature", + "signature"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -RemoteSplit::RemoteSplit() noexcept { +RemoteTransactionHandle::RemoteTransactionHandle() noexcept { _type = "$remote"; } -void to_json(json& j, const RemoteSplit& p) { +void to_json(json& j, const RemoteTransactionHandle& 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"); + j, "dummy", p.dummy, "RemoteTransactionHandle", "String", "dummy"); } -void from_json(const json& j, RemoteSplit& p) { +void from_json(const json& j, RemoteTransactionHandle& p) { p._type = j["@type"]; from_json_key( - j, "location", p.location, "RemoteSplit", "Location", "location"); + j, "dummy", p.dummy, "RemoteTransactionHandle", "String", "dummy"); +} +} // 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, + "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, - "remoteSourceTaskId", - p.remoteSourceTaskId, - "RemoteSplit", - "TaskId", - "remoteSourceTaskId"); + "idVariable", + p.idVariable, + "AssignUniqueId", + "VariableReferenceExpression", + "idVariable"); } } // namespace facebook::presto::protocol /* @@ -10450,1165 +11174,1289 @@ void from_json(const json& j, RemoteSplit& p) { * 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; - getConnectorProtocol(type).to_json(j, 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"); - } - getConnectorProtocol(type).from_json(j, p); +TableWriterNode::TableWriterNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.TableWriterNode"; } -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -void to_json(json& j, const InsertTableHandle& p) { +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, - "connectorId", - p.connectorId, - "InsertTableHandle", - "ConnectorId", - "connectorId"); + "rowCountVariable", + p.rowCountVariable, + "TableWriterNode", + "VariableReferenceExpression", + "rowCountVariable"); to_json_key( j, - "transactionHandle", - p.transactionHandle, - "InsertTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "fragmentVariable", + p.fragmentVariable, + "TableWriterNode", + "VariableReferenceExpression", + "fragmentVariable"); to_json_key( j, - "connectorHandle", - p.connectorHandle, - "InsertTableHandle", - "ConnectorInsertTableHandle", - "connectorHandle"); -} - -void from_json(const json& j, InsertTableHandle& p) { - from_json_key( + "tableCommitContextVariable", + p.tableCommitContextVariable, + "TableWriterNode", + "VariableReferenceExpression", + "tableCommitContextVariable"); + to_json_key( j, - "connectorId", - p.connectorId, - "InsertTableHandle", - "ConnectorId", - "connectorId"); - from_json_key( + "columns", + p.columns, + "TableWriterNode", + "List", + "columns"); + to_json_key( j, - "transactionHandle", - p.transactionHandle, - "InsertTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); - from_json_key( + "columnNames", + p.columnNames, + "TableWriterNode", + "List", + "columnNames"); + to_json_key( j, - "connectorHandle", - p.connectorHandle, - "InsertTableHandle", - "ConnectorInsertTableHandle", - "connectorHandle"); -} -} // 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"; + "notNullColumnVariables", + p.notNullColumnVariables, + "TableWriterNode", + "List", + "notNullColumnVariables"); to_json_key( - j, "handle", p.handle, "InsertHandle", "InsertTableHandle", "handle"); + j, + "partitioningScheme", + p.partitioningScheme, + "TableWriterNode", + "PartitioningScheme", + "partitioningScheme"); to_json_key( j, - "schemaTableName", - p.schemaTableName, - "InsertHandle", - "SchemaTableName", - "schemaTableName"); + "preferredShufflePartitioningScheme", + p.preferredShufflePartitioningScheme, + "TableWriterNode", + "PartitioningScheme", + "preferredShufflePartitioningScheme"); + to_json_key( + j, + "statisticsAggregation", + p.statisticsAggregation, + "TableWriterNode", + "StatisticAggregations", + "statisticsAggregation"); } -void from_json(const json& j, InsertHandle& p) { +void from_json(const json& j, TableWriterNode& p) { p._type = j["@type"]; + from_json_key(j, "id", p.id, "TableWriterNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "TableWriterNode", "PlanNode", "source"); from_json_key( - j, "handle", p.handle, "InsertHandle", "InsertTableHandle", "handle"); + j, + "rowCountVariable", + p.rowCountVariable, + "TableWriterNode", + "VariableReferenceExpression", + "rowCountVariable"); from_json_key( j, - "schemaTableName", - p.schemaTableName, - "InsertHandle", - "SchemaTableName", - "schemaTableName"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const MemoryInfo& p) { - j = json::object(); - to_json_key( + "fragmentVariable", + p.fragmentVariable, + "TableWriterNode", + "VariableReferenceExpression", + "fragmentVariable"); + from_json_key( j, - "totalNodeMemory", - p.totalNodeMemory, - "MemoryInfo", - "DataSize", - "totalNodeMemory"); - to_json_key( + "tableCommitContextVariable", + p.tableCommitContextVariable, + "TableWriterNode", + "VariableReferenceExpression", + "tableCommitContextVariable"); + from_json_key( j, - "pools", - p.pools, - "MemoryInfo", - "Map", - "pools"); -} - -void from_json(const json& j, MemoryInfo& p) { + "columns", + p.columns, + "TableWriterNode", + "List", + "columns"); from_json_key( j, - "totalNodeMemory", - p.totalNodeMemory, - "MemoryInfo", - "DataSize", - "totalNodeMemory"); + "columnNames", + p.columnNames, + "TableWriterNode", + "List", + "columnNames"); from_json_key( j, - "pools", - p.pools, - "MemoryInfo", - "Map", - "pools"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair PrestoTableType_enum_table[] = - { // NOLINT: cert-err58-cpp - {PrestoTableType::MANAGED_TABLE, "MANAGED_TABLE"}, - {PrestoTableType::EXTERNAL_TABLE, "EXTERNAL_TABLE"}, - {PrestoTableType::VIRTUAL_VIEW, "VIRTUAL_VIEW"}, - {PrestoTableType::MATERIALIZED_VIEW, "MATERIALIZED_VIEW"}, - {PrestoTableType::TEMPORARY_TABLE, "TEMPORARY_TABLE"}, - {PrestoTableType::OTHER, "OTHER"}}; -void to_json(json& j, const PrestoTableType& e) { - static_assert( - std::is_enum::value, "PrestoTableType must be an enum!"); - const auto* it = std::find_if( - std::begin(PrestoTableType_enum_table), - std::end(PrestoTableType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(PrestoTableType_enum_table)) - ? it - : std::begin(PrestoTableType_enum_table)) - ->second; -} -void from_json(const json& j, PrestoTableType& e) { - static_assert( - std::is_enum::value, "PrestoTableType must be an enum!"); - const auto* it = std::find_if( - std::begin(PrestoTableType_enum_table), - std::end(PrestoTableType_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(PrestoTableType_enum_table)) - ? it - : std::begin(PrestoTableType_enum_table)) - ->first; + "notNullColumnVariables", + p.notNullColumnVariables, + "TableWriterNode", + "List", + "notNullColumnVariables"); + from_json_key( + j, + "partitioningScheme", + p.partitioningScheme, + "TableWriterNode", + "PartitioningScheme", + "partitioningScheme"); + from_json_key( + j, + "preferredShufflePartitioningScheme", + p.preferredShufflePartitioningScheme, + "TableWriterNode", + "PartitioningScheme", + "preferredShufflePartitioningScheme"); + from_json_key( + j, + "statisticsAggregation", + p.statisticsAggregation, + "TableWriterNode", + "StatisticAggregations", + "statisticsAggregation"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +IcebergTableHandle::IcebergTableHandle() noexcept { + _type = "hive-iceberg"; +} -void to_json(json& j, const Table& p) { +void to_json(json& j, const IcebergTableHandle& p) { j = json::object(); + j["@type"] = "hive-iceberg"; 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"); + j, + "schemaName", + p.schemaName, + "IcebergTableHandle", + "String", + "schemaName"); to_json_key( - j, "tableType", p.tableType, "Table", "PrestoTableType", "tableType"); - to_json_key(j, "storage", p.storage, "Table", "Storage", "storage"); + j, + "icebergTableName", + p.icebergTableName, + "IcebergTableHandle", + "IcebergTableName", + "icebergTableName"); to_json_key( - j, "dataColumns", p.dataColumns, "Table", "List", "dataColumns"); + j, + "snapshotSpecified", + p.snapshotSpecified, + "IcebergTableHandle", + "bool", + "snapshotSpecified"); to_json_key( j, - "partitionColumns", - p.partitionColumns, - "Table", - "List", - "partitionColumns"); + "predicate", + p.predicate, + "IcebergTableHandle", + "TupleDomain", + "predicate"); to_json_key( j, - "parameters", - p.parameters, - "Table", + "outputPath", + p.outputPath, + "IcebergTableHandle", + "String", + "outputPath"); + to_json_key( + j, + "storageProperties", + p.storageProperties, + "IcebergTableHandle", "Map", - "parameters"); + "storageProperties"); to_json_key( j, - "viewOriginalText", - p.viewOriginalText, - "Table", + "tableSchemaJson", + p.tableSchemaJson, + "IcebergTableHandle", "String", - "viewOriginalText"); + "tableSchemaJson"); to_json_key( j, - "viewExpandedText", - p.viewExpandedText, - "Table", - "String", - "viewExpandedText"); + "partitionFieldIds", + p.partitionFieldIds, + "IcebergTableHandle", + "List", + "partitionFieldIds"); + to_json_key( + j, + "equalityFieldIds", + p.equalityFieldIds, + "IcebergTableHandle", + "List", + "equalityFieldIds"); } -void from_json(const json& j, Table& p) { +void from_json(const json& j, IcebergTableHandle& p) { + p._type = j["@type"]; from_json_key( - j, "databaseName", p.databaseName, "Table", "String", "databaseName"); - from_json_key(j, "tableName", p.tableName, "Table", "String", "tableName"); - from_json_key(j, "owner", p.owner, "Table", "String", "owner"); + j, + "schemaName", + p.schemaName, + "IcebergTableHandle", + "String", + "schemaName"); from_json_key( - j, "tableType", p.tableType, "Table", "PrestoTableType", "tableType"); - from_json_key(j, "storage", p.storage, "Table", "Storage", "storage"); + j, + "icebergTableName", + p.icebergTableName, + "IcebergTableHandle", + "IcebergTableName", + "icebergTableName"); from_json_key( - j, "dataColumns", p.dataColumns, "Table", "List", "dataColumns"); + j, + "snapshotSpecified", + p.snapshotSpecified, + "IcebergTableHandle", + "bool", + "snapshotSpecified"); from_json_key( j, - "partitionColumns", - p.partitionColumns, - "Table", - "List", - "partitionColumns"); + "predicate", + p.predicate, + "IcebergTableHandle", + "TupleDomain", + "predicate"); from_json_key( j, - "parameters", - p.parameters, - "Table", + "outputPath", + p.outputPath, + "IcebergTableHandle", + "String", + "outputPath"); + from_json_key( + j, + "storageProperties", + p.storageProperties, + "IcebergTableHandle", "Map", - "parameters"); + "storageProperties"); from_json_key( j, - "viewOriginalText", - p.viewOriginalText, - "Table", + "tableSchemaJson", + p.tableSchemaJson, + "IcebergTableHandle", "String", - "viewOriginalText"); + "tableSchemaJson"); from_json_key( j, - "viewExpandedText", - p.viewExpandedText, - "Table", - "String", - "viewExpandedText"); + "partitionFieldIds", + p.partitionFieldIds, + "IcebergTableHandle", + "List", + "partitionFieldIds"); + from_json_key( + j, + "equalityFieldIds", + p.equalityFieldIds, + "IcebergTableHandle", + "List", + "equalityFieldIds"); } } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const HivePageSinkMetadata& p) { - j = json::object(); - to_json_key( - j, - "schemaTableName", - p.schemaTableName, - "HivePageSinkMetadata", - "SchemaTableName", - "schemaTableName"); - to_json_key(j, "table", p.table, "HivePageSinkMetadata", "Table", "table"); +// 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, HivePageSinkMetadata& p) { - from_json_key( - j, - "schemaTableName", - p.schemaTableName, - "HivePageSinkMetadata", - "SchemaTableName", - "schemaTableName"); - from_json_key(j, "table", p.table, "HivePageSinkMetadata", "Table", "table"); +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 BatchTaskUpdateRequest& p) { +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( + j, + "groupingSets", + p.groupingSets, + "AggregationNode", + "GroupingSetDescriptor", + "groupingSets"); to_json_key( j, - "taskUpdateRequest", - p.taskUpdateRequest, - "BatchTaskUpdateRequest", - "TaskUpdateRequest", - "taskUpdateRequest"); + "preGroupedVariables", + p.preGroupedVariables, + "AggregationNode", + "List", + "preGroupedVariables"); + to_json_key( + j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); to_json_key( j, - "shuffleWriteInfo", - p.shuffleWriteInfo, - "BatchTaskUpdateRequest", - "String", - "shuffleWriteInfo"); + "hashVariable", + p.hashVariable, + "AggregationNode", + "VariableReferenceExpression", + "hashVariable"); to_json_key( j, - "broadcastBasePath", - p.broadcastBasePath, - "BatchTaskUpdateRequest", - "String", - "broadcastBasePath"); + "groupIdVariable", + p.groupIdVariable, + "AggregationNode", + "VariableReferenceExpression", + "groupIdVariable"); + to_json_key( + j, + "aggregationId", + p.aggregationId, + "AggregationNode", + "Integer", + "aggregationId"); } -void from_json(const json& j, BatchTaskUpdateRequest& p) { +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"); from_json_key( j, - "taskUpdateRequest", - p.taskUpdateRequest, - "BatchTaskUpdateRequest", - "TaskUpdateRequest", - "taskUpdateRequest"); + "aggregations", + p.aggregations, + "AggregationNode", + "Map", + "aggregations"); from_json_key( j, - "shuffleWriteInfo", - p.shuffleWriteInfo, - "BatchTaskUpdateRequest", - "String", - "shuffleWriteInfo"); + "groupingSets", + p.groupingSets, + "AggregationNode", + "GroupingSetDescriptor", + "groupingSets"); from_json_key( j, - "broadcastBasePath", - p.broadcastBasePath, - "BatchTaskUpdateRequest", - "String", - "broadcastBasePath"); -} -} // 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( + "preGroupedVariables", + p.preGroupedVariables, + "AggregationNode", + "List", + "preGroupedVariables"); + from_json_key( + j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); + from_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"); + "hashVariable", + p.hashVariable, + "AggregationNode", + "VariableReferenceExpression", + "hashVariable"); from_json_key( j, - "schemaTableName", - p.schemaTableName, - "UpdateHandle", - "SchemaTableName", - "schemaTableName"); + "groupIdVariable", + p.groupIdVariable, + "AggregationNode", + "VariableReferenceExpression", + "groupIdVariable"); + from_json_key( + j, + "aggregationId", + p.aggregationId, + "AggregationNode", + "Integer", + "aggregationId"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -EnforceSingleRowNode::EnforceSingleRowNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode"; +ConstantExpression::ConstantExpression() noexcept { + _type = "constant"; } -void to_json(json& j, const EnforceSingleRowNode& p) { +void to_json(json& j, const ConstantExpression& 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"] = "constant"; to_json_key( - j, "source", p.source, "EnforceSingleRowNode", "PlanNode", "source"); + j, + "valueBlock", + p.valueBlock, + "ConstantExpression", + "Block", + "valueBlock"); + to_json_key(j, "type", p.type, "ConstantExpression", "Type", "type"); } -void from_json(const json& j, EnforceSingleRowNode& p) { +void from_json(const json& j, ConstantExpression& 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"); + 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 { -DeleteHandle::DeleteHandle() noexcept { - _type = "DeleteHandle"; +SqlFunctionHandle::SqlFunctionHandle() noexcept { + _type = "json_file"; } -void to_json(json& j, const DeleteHandle& p) { +void to_json(json& j, const SqlFunctionHandle& p) { j = json::object(); - j["@type"] = "DeleteHandle"; - to_json_key(j, "handle", p.handle, "DeleteHandle", "TableHandle", "handle"); + j["@type"] = "json_file"; to_json_key( j, - "schemaTableName", - p.schemaTableName, - "DeleteHandle", - "SchemaTableName", - "schemaTableName"); + "functionId", + p.functionId, + "SqlFunctionHandle", + "SqlFunctionId", + "functionId"); + to_json_key( + j, "version", p.version, "SqlFunctionHandle", "String", "version"); } -void from_json(const json& j, DeleteHandle& p) { +void from_json(const json& j, SqlFunctionHandle& p) { p._type = j["@type"]; - from_json_key(j, "handle", p.handle, "DeleteHandle", "TableHandle", "handle"); from_json_key( j, - "schemaTableName", - p.schemaTableName, - "DeleteHandle", - "SchemaTableName", - "schemaTableName"); + "functionId", + p.functionId, + "SqlFunctionHandle", + "SqlFunctionId", + "functionId"); + from_json_key( + j, "version", p.version, "SqlFunctionHandle", "String", "version"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const DwrfEncryptionMetadata& p) { +void to_json(json& j, const ServerInfo& p) { j = json::object(); to_json_key( j, - "fieldToKeyData", - p.fieldToKeyData, - "DwrfEncryptionMetadata", - "Map", - "fieldToKeyData"); - to_json_key( - j, - "extraMetadata", - p.extraMetadata, - "DwrfEncryptionMetadata", - "Map", - "extraMetadata"); + "nodeVersion", + p.nodeVersion, + "ServerInfo", + "NodeVersion", + "nodeVersion"); to_json_key( - j, - "encryptionAlgorithm", - p.encryptionAlgorithm, - "DwrfEncryptionMetadata", - "String", - "encryptionAlgorithm"); + j, "environment", p.environment, "ServerInfo", "String", "environment"); to_json_key( - j, - "encryptionProvider", - p.encryptionProvider, - "DwrfEncryptionMetadata", - "String", - "encryptionProvider"); + 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, DwrfEncryptionMetadata& p) { - from_json_key( - j, - "fieldToKeyData", - p.fieldToKeyData, - "DwrfEncryptionMetadata", - "Map", - "fieldToKeyData"); +void from_json(const json& j, ServerInfo& p) { from_json_key( j, - "extraMetadata", - p.extraMetadata, - "DwrfEncryptionMetadata", - "Map", - "extraMetadata"); + "nodeVersion", + p.nodeVersion, + "ServerInfo", + "NodeVersion", + "nodeVersion"); from_json_key( - j, - "encryptionAlgorithm", - p.encryptionAlgorithm, - "DwrfEncryptionMetadata", - "String", - "encryptionAlgorithm"); + j, "environment", p.environment, "ServerInfo", "String", "environment"); from_json_key( - j, - "encryptionProvider", - p.encryptionProvider, - "DwrfEncryptionMetadata", - "String", - "encryptionProvider"); + 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 { +TopNRowNumberNode::TopNRowNumberNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.TopNRowNumberNode"; +} -void to_json(json& j, const EncryptionInformation& 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, - "dwrfEncryptionMetadata", - p.dwrfEncryptionMetadata, - "EncryptionInformation", - "DwrfEncryptionMetadata", - "dwrfEncryptionMetadata"); -} - -void from_json(const json& j, EncryptionInformation& p) { - from_json_key( + "specification", + p.specification, + "TopNRowNumberNode", + "Specification", + "specification"); + to_json_key( j, - "dwrfEncryptionMetadata", - p.dwrfEncryptionMetadata, - "EncryptionInformation", - "DwrfEncryptionMetadata", - "dwrfEncryptionMetadata"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const TableToPartitionMapping& p) { - j = json::object(); + "rowNumberVariable", + p.rowNumberVariable, + "TopNRowNumberNode", + "VariableReferenceExpression", + "rowNumberVariable"); to_json_key( j, - "tableToPartitionColumns", - p.tableToPartitionColumns, - "TableToPartitionMapping", - "Map", - "tableToPartitionColumns"); + "maxRowCountPerPartition", + p.maxRowCountPerPartition, + "TopNRowNumberNode", + "int", + "maxRowCountPerPartition"); + to_json_key(j, "partial", p.partial, "TopNRowNumberNode", "bool", "partial"); to_json_key( j, - "partitionSchemaDifference", - p.partitionSchemaDifference, - "TableToPartitionMapping", - "Map", - "partitionSchemaDifference"); + "hashVariable", + p.hashVariable, + "TopNRowNumberNode", + "VariableReferenceExpression", + "hashVariable"); } -void from_json(const json& j, TableToPartitionMapping& 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, "source", p.source, "TopNRowNumberNode", "PlanNode", "source"); from_json_key( j, - "tableToPartitionColumns", - p.tableToPartitionColumns, - "TableToPartitionMapping", - "Map", - "tableToPartitionColumns"); + "specification", + p.specification, + "TopNRowNumberNode", + "Specification", + "specification"); from_json_key( j, - "partitionSchemaDifference", - p.partitionSchemaDifference, - "TableToPartitionMapping", - "Map", - "partitionSchemaDifference"); + "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"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveSplit::HiveSplit() noexcept { +HiveMetadataUpdateHandle::HiveMetadataUpdateHandle() noexcept { _type = "hive"; } -void to_json(json& j, const HiveSplit& p) { +void to_json(json& j, const HiveMetadataUpdateHandle& 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"); + j, + "requestId", + p.requestId, + "HiveMetadataUpdateHandle", + "UUID", + "requestId"); + to_json_key( + j, + "schemaTableName", + p.schemaTableName, + "HiveMetadataUpdateHandle", + "SchemaTableName", + "schemaTableName"); to_json_key( j, "partitionName", p.partitionName, - "HiveSplit", + "HiveMetadataUpdateHandle", "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"); - to_json_key( - j, - "tableToPartitionMapping", - p.tableToPartitionMapping, - "HiveSplit", - "TableToPartitionMapping", - "tableToPartitionMapping"); - to_json_key( + "fileName", + p.fileName, + "HiveMetadataUpdateHandle", + "String", + "fileName"); +} + +void from_json(const json& j, HiveMetadataUpdateHandle& p) { + p._type = j["@type"]; + from_json_key( j, - "bucketConversion", - p.bucketConversion, - "HiveSplit", - "BucketConversion", - "bucketConversion"); - to_json_key( + "requestId", + p.requestId, + "HiveMetadataUpdateHandle", + "UUID", + "requestId"); + from_json_key( j, - "s3SelectPushdownEnabled", - p.s3SelectPushdownEnabled, - "HiveSplit", - "bool", - "s3SelectPushdownEnabled"); - to_json_key( + "schemaTableName", + p.schemaTableName, + "HiveMetadataUpdateHandle", + "SchemaTableName", + "schemaTableName"); + from_json_key( j, - "cacheQuota", - p.cacheQuota, - "HiveSplit", - "CacheQuotaRequirement", - "cacheQuota"); - to_json_key( + "partitionName", + p.partitionName, + "HiveMetadataUpdateHandle", + "String", + "partitionName"); + from_json_key( j, - "encryptionMetadata", - p.encryptionMetadata, - "HiveSplit", - "EncryptionInformation", - "encryptionMetadata"); + "fileName", + p.fileName, + "HiveMetadataUpdateHandle", + "String", + "fileName"); +} +} // 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"; to_json_key( j, - "redundantColumnDomains", - p.redundantColumnDomains, - "HiveSplit", - "List>", - "redundantColumnDomains"); + "sourceLocation", + p.sourceLocation, + "LambdaDefinitionExpression", + "SourceLocation", + "sourceLocation"); to_json_key( j, - "splitWeight", - p.splitWeight, - "HiveSplit", - "SplitWeight", - "splitWeight"); + "argumentTypes", + p.argumentTypes, + "LambdaDefinitionExpression", + "List", + "argumentTypes"); to_json_key( j, - "rowIdPartitionComponent", - p.rowIdPartitionComponent, - "HiveSplit", - "String", - "rowIdPartitionComponent"); + "arguments", + p.arguments, + "LambdaDefinitionExpression", + "List", + "arguments"); + to_json_key( + j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); } -void from_json(const json& j, HiveSplit& p) { +void from_json(const json& j, LambdaDefinitionExpression& 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"); + "sourceLocation", + p.sourceLocation, + "LambdaDefinitionExpression", + "SourceLocation", + "sourceLocation"); from_json_key( j, - "nodeSelectionStrategy", - p.nodeSelectionStrategy, - "HiveSplit", - "NodeSelectionStrategy", - "nodeSelectionStrategy"); + "argumentTypes", + p.argumentTypes, + "LambdaDefinitionExpression", + "List", + "argumentTypes"); from_json_key( j, - "partitionDataColumnCount", - p.partitionDataColumnCount, - "HiveSplit", - "int", - "partitionDataColumnCount"); + "arguments", + p.arguments, + "LambdaDefinitionExpression", + "List", + "arguments"); from_json_key( + j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +GroupIdNode::GroupIdNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.GroupIdNode"; +} + +void to_json(json& j, const GroupIdNode& p) { + 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, - "tableToPartitionMapping", - p.tableToPartitionMapping, - "HiveSplit", - "TableToPartitionMapping", - "tableToPartitionMapping"); - from_json_key( + "groupingSets", + p.groupingSets, + "GroupIdNode", + "List>", + "groupingSets"); + to_json_key( j, - "bucketConversion", - p.bucketConversion, - "HiveSplit", - "BucketConversion", - "bucketConversion"); - from_json_key( + "groupingColumns", + p.groupingColumns, + "GroupIdNode", + "Map", + "groupingColumns"); + to_json_key( j, - "s3SelectPushdownEnabled", - p.s3SelectPushdownEnabled, - "HiveSplit", - "bool", - "s3SelectPushdownEnabled"); - from_json_key( + "aggregationArguments", + p.aggregationArguments, + "GroupIdNode", + "List", + "aggregationArguments"); + to_json_key( j, - "cacheQuota", - p.cacheQuota, - "HiveSplit", - "CacheQuotaRequirement", - "cacheQuota"); + "groupIdVariable", + p.groupIdVariable, + "GroupIdNode", + "VariableReferenceExpression", + "groupIdVariable"); +} + +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, - "encryptionMetadata", - p.encryptionMetadata, - "HiveSplit", - "EncryptionInformation", - "encryptionMetadata"); + "groupingSets", + p.groupingSets, + "GroupIdNode", + "List>", + "groupingSets"); from_json_key( j, - "redundantColumnDomains", - p.redundantColumnDomains, - "HiveSplit", - "List>", - "redundantColumnDomains"); + "groupingColumns", + p.groupingColumns, + "GroupIdNode", + "Map", + "groupingColumns"); from_json_key( j, - "splitWeight", - p.splitWeight, - "HiveSplit", - "SplitWeight", - "splitWeight"); + "aggregationArguments", + p.aggregationArguments, + "GroupIdNode", + "List", + "aggregationArguments"); from_json_key( j, - "rowIdPartitionComponent", - p.rowIdPartitionComponent, - "HiveSplit", - "String", - "rowIdPartitionComponent"); + "groupIdVariable", + p.groupIdVariable, + "GroupIdNode", + "VariableReferenceExpression", + "groupIdVariable"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() +TpchTableLayoutHandle::TpchTableLayoutHandle() noexcept { + _type = "tpch"; +} -// 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 to_json(json& j, const TpchTableLayoutHandle& p) { + j = json::object(); + j["@type"] = "tpch"; + to_json_key( + j, "table", p.table, "TpchTableLayoutHandle", "TpchTableHandle", "table"); + to_json_key( + j, + "predicate", + p.predicate, + "TpchTableLayoutHandle", + "TupleDomain>", + "predicate"); } -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; + +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, + "predicate", + p.predicate, + "TpchTableLayoutHandle", + "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 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 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, - "ExchangeNodeType must be an enum!"); + std::is_enum::value, "LimitNodeStep 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(LimitNodeStep_enum_table), + std::end(LimitNodeStep_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(LimitNodeStep_enum_table)) ? it - : std::begin(ExchangeNodeType_enum_table)) + : std::begin(LimitNodeStep_enum_table)) ->second; } -void from_json(const json& j, ExchangeNodeType& e) { +void from_json(const json& j, LimitNodeStep& e) { static_assert( - std::is_enum::value, - "ExchangeNodeType must be an enum!"); + std::is_enum::value, "LimitNodeStep 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(LimitNodeStep_enum_table), + std::end(LimitNodeStep_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(LimitNodeStep_enum_table)) ? it - : std::begin(ExchangeNodeType_enum_table)) + : std::begin(LimitNodeStep_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -ExchangeNode::ExchangeNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.ExchangeNode"; +LimitNode::LimitNode() noexcept { + _type = ".LimitNode"; } -void to_json(json& j, const ExchangeNode& p) { +void to_json(json& j, const LimitNode& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.ExchangeNode"; - to_json_key(j, "id", p.id, "ExchangeNode", "PlanNodeId", "id"); - to_json_key(j, "type", p.type, "ExchangeNode", "ExchangeNodeType", "type"); - to_json_key( - j, "scope", p.scope, "ExchangeNode", "ExchangeNodeScope", "scope"); - to_json_key( - j, - "partitioningScheme", - p.partitioningScheme, - "ExchangeNode", - "PartitioningScheme", - "partitioningScheme"); - to_json_key( - j, - "sources", - p.sources, - "ExchangeNode", - "List>", - "sources"); - to_json_key( - j, - "inputs", - p.inputs, - "ExchangeNode", - "List>", - "inputs"); - to_json_key( - j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "ExchangeNode", - "bool", - "ensureSourceOrdering"); - to_json_key( - j, - "orderingScheme", - p.orderingScheme, - "ExchangeNode", - "OrderingScheme", - "orderingScheme"); + 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, ExchangeNode& p) { +void from_json(const json& j, LimitNode& 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, - "partitioningScheme", - p.partitioningScheme, - "ExchangeNode", - "PartitioningScheme", - "partitioningScheme"); - from_json_key( - j, - "sources", - p.sources, - "ExchangeNode", - "List>", - "sources"); - from_json_key( - j, - "inputs", - p.inputs, - "ExchangeNode", - "List>", - "inputs"); - from_json_key( - j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "ExchangeNode", - "bool", - "ensureSourceOrdering"); - from_json_key( - j, - "orderingScheme", - p.orderingScheme, - "ExchangeNode", - "OrderingScheme", - "orderingScheme"); + from_json_key(j, "id", p.id, "LimitNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "LimitNode", "PlanNode", "source"); + from_json_key(j, "count", p.count, "LimitNode", "int64_t", "count"); + from_json_key(j, "step", p.step, "LimitNode", "LimitNodeStep", "step"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -UnnestNode::UnnestNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.UnnestNode"; -} -void to_json(json& j, const UnnestNode& p) { +void to_json(json& j, const EquiJoinClause& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.UnnestNode"; - to_json_key(j, "id", p.id, "UnnestNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "UnnestNode", "PlanNode", "source"); to_json_key( j, - "replicateVariables", - p.replicateVariables, - "UnnestNode", - "List", - "replicateVariables"); - to_json_key( - j, - "unnestVariables", - p.unnestVariables, - "UnnestNode", - "Map>", - "unnestVariables"); + "left", + p.left, + "EquiJoinClause", + "VariableReferenceExpression", + "left"); to_json_key( j, - "ordinalityVariable", - p.ordinalityVariable, - "UnnestNode", + "right", + p.right, + "EquiJoinClause", "VariableReferenceExpression", - "ordinalityVariable"); + "right"); } -void from_json(const json& j, UnnestNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "UnnestNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "UnnestNode", "PlanNode", "source"); - from_json_key( - j, - "replicateVariables", - p.replicateVariables, - "UnnestNode", - "List", - "replicateVariables"); +void from_json(const json& j, EquiJoinClause& p) { from_json_key( j, - "unnestVariables", - p.unnestVariables, - "UnnestNode", - "Map>", - "unnestVariables"); + "left", + p.left, + "EquiJoinClause", + "VariableReferenceExpression", + "left"); from_json_key( j, - "ordinalityVariable", - p.ordinalityVariable, - "UnnestNode", + "right", + p.right, + "EquiJoinClause", "VariableReferenceExpression", - "ordinalityVariable"); + "right"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair 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 + 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, "SampleNodeType must be an enum!"); + std::is_enum::value, + "JoinDistributionType 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(JoinDistributionType_enum_table), + std::end(JoinDistributionType_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(JoinDistributionType_enum_table)) ? it - : std::begin(SampleNodeType_enum_table)) + : std::begin(JoinDistributionType_enum_table)) ->second; } -void from_json(const json& j, SampleNodeType& e) { +void from_json(const json& j, JoinDistributionType& e) { static_assert( - std::is_enum::value, "SampleNodeType must be an enum!"); + std::is_enum::value, + "JoinDistributionType 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(JoinDistributionType_enum_table), + std::end(JoinDistributionType_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)) + ->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 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(SampleNodeType_enum_table)) - ? it - : std::begin(SampleNodeType_enum_table)) + e = ((it != std::end(JoinType_enum_table)) ? it + : std::begin(JoinType_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -SampleNode::SampleNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.SampleNode"; +JoinNode::JoinNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.JoinNode"; } -void to_json(json& j, const SampleNode& p) { +void to_json(json& j, const JoinNode& 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"] = "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, "sampleRatio", p.sampleRatio, "SampleNode", "double", "sampleRatio"); + j, + "criteria", + p.criteria, + "JoinNode", + "List", + "criteria"); to_json_key( j, - "sampleType", - p.sampleType, - "SampleNode", - "SampleNodeType", - "sampleType"); + "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"); } -void from_json(const json& j, SampleNode& p) { +void from_json(const json& j, JoinNode& 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, "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, "sampleRatio", p.sampleRatio, "SampleNode", "double", "sampleRatio"); + j, + "criteria", + p.criteria, + "JoinNode", + "List", + "criteria"); from_json_key( j, - "sampleType", - p.sampleType, - "SampleNode", - "SampleNodeType", - "sampleType"); + "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 { -HiveTableHandle::HiveTableHandle() noexcept { - _type = "hive"; +MergeJoinNode::MergeJoinNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.MergeJoinNode"; } -void to_json(json& j, const HiveTableHandle& p) { +void to_json(json& j, const MergeJoinNode& p) { j = json::object(); - j["@type"] = "hive"; + 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, "schemaName", p.schemaName, "HiveTableHandle", "String", "schemaName"); + j, + "criteria", + p.criteria, + "MergeJoinNode", + "List", + "criteria"); to_json_key( - j, "tableName", p.tableName, "HiveTableHandle", "String", "tableName"); + j, + "outputVariables", + p.outputVariables, + "MergeJoinNode", + "List", + "outputVariables"); to_json_key( j, - "analyzePartitionValues", - p.analyzePartitionValues, - "HiveTableHandle", - "List>", - "analyzePartitionValues"); + "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, HiveTableHandle& p) { +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, "schemaName", p.schemaName, "HiveTableHandle", "String", "schemaName"); + j, + "criteria", + p.criteria, + "MergeJoinNode", + "List", + "criteria"); from_json_key( - j, "tableName", p.tableName, "HiveTableHandle", "String", "tableName"); + j, + "outputVariables", + p.outputVariables, + "MergeJoinNode", + "List", + "outputVariables"); from_json_key( j, - "analyzePartitionValues", - p.analyzePartitionValues, - "HiveTableHandle", - "List>", - "analyzePartitionValues"); + "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 { -SqlFunctionHandle::SqlFunctionHandle() noexcept { - _type = "json_file"; +ValuesNode::ValuesNode() noexcept { + _type = ".ValuesNode"; } -void to_json(json& j, const SqlFunctionHandle& p) { +void to_json(json& j, const ValuesNode& p) { j = json::object(); - j["@type"] = "json_file"; + j["@type"] = ".ValuesNode"; + to_json_key( + j, "location", p.location, "ValuesNode", "SourceLocation", "location"); + to_json_key(j, "id", p.id, "ValuesNode", "PlanNodeId", "id"); to_json_key( j, - "functionId", - p.functionId, - "SqlFunctionHandle", - "SqlFunctionId", - "functionId"); + "outputVariables", + p.outputVariables, + "ValuesNode", + "List", + "outputVariables"); + to_json_key( + j, + "rows", + p.rows, + "ValuesNode", + "List>>", + "rows"); to_json_key( - j, "version", p.version, "SqlFunctionHandle", "String", "version"); + j, + "valuesNodeLabel", + p.valuesNodeLabel, + "ValuesNode", + "String", + "valuesNodeLabel"); } -void from_json(const json& j, SqlFunctionHandle& p) { +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, - "functionId", - p.functionId, - "SqlFunctionHandle", - "SqlFunctionId", - "functionId"); + "outputVariables", + p.outputVariables, + "ValuesNode", + "List", + "outputVariables"); from_json_key( - j, "version", p.version, "SqlFunctionHandle", "String", "version"); + j, + "rows", + p.rows, + "ValuesNode", + "List>>", + "rows"); + from_json_key( + j, + "valuesNodeLabel", + p.valuesNodeLabel, + "ValuesNode", + "String", + "valuesNodeLabel"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { @@ -11665,1983 +12513,1405 @@ void from_json(const json& j, UpdateTarget& p) { } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -BuiltInFunctionHandle::BuiltInFunctionHandle() noexcept { - _type = "$static"; +FilterNode::FilterNode() noexcept { + _type = ".FilterNode"; } -void to_json(json& j, const BuiltInFunctionHandle& p) { +void to_json(json& j, const FilterNode& p) { j = json::object(); - j["@type"] = "$static"; + 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, - "signature", - p.signature, - "BuiltInFunctionHandle", - "Signature", - "signature"); + j, "predicate", p.predicate, "FilterNode", "RowExpression", "predicate"); } -void from_json(const json& j, BuiltInFunctionHandle& p) { +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, - "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 - SystemPartitioning_enum_table[] = { // NOLINT: cert-err58-cpp - {SystemPartitioning::SINGLE, "SINGLE"}, - {SystemPartitioning::FIXED, "FIXED"}, - {SystemPartitioning::SOURCE, "SOURCE"}, - {SystemPartitioning::SCALED, "SCALED"}, - {SystemPartitioning::COORDINATOR_ONLY, "COORDINATOR_ONLY"}, - {SystemPartitioning::ARBITRARY, "ARBITRARY"}}; -void to_json(json& j, const SystemPartitioning& e) { - static_assert( - std::is_enum::value, - "SystemPartitioning must be an enum!"); - const auto* it = std::find_if( - std::begin(SystemPartitioning_enum_table), - std::end(SystemPartitioning_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(SystemPartitioning_enum_table)) - ? it - : std::begin(SystemPartitioning_enum_table)) - ->second; -} -void from_json(const json& j, SystemPartitioning& e) { - static_assert( - std::is_enum::value, - "SystemPartitioning must be an enum!"); - const auto* it = std::find_if( - std::begin(SystemPartitioning_enum_table), - std::end(SystemPartitioning_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(SystemPartitioning_enum_table)) - ? it - : std::begin(SystemPartitioning_enum_table)) - ->first; + j, "predicate", p.predicate, "FilterNode", "RowExpression", "predicate"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair - 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 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, - "SystemPartitionFunction must be an enum!"); + std::is_enum::value, + "DistributionType 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 { + 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(SystemPartitionFunction_enum_table)) + j = ((it != std::end(DistributionType_enum_table)) ? it - : std::begin(SystemPartitionFunction_enum_table)) + : std::begin(DistributionType_enum_table)) ->second; } -void from_json(const json& j, SystemPartitionFunction& e) { +void from_json(const json& j, DistributionType& e) { static_assert( - std::is_enum::value, - "SystemPartitionFunction must be an enum!"); + std::is_enum::value, + "DistributionType 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 { + 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(SystemPartitionFunction_enum_table)) + e = ((it != std::end(DistributionType_enum_table)) ? it - : std::begin(SystemPartitionFunction_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -SystemPartitioningHandle::SystemPartitioningHandle() noexcept { - _type = "$remote"; -} - -void to_json(json& j, const SystemPartitioningHandle& p) { - j = json::object(); - j["@type"] = "$remote"; - to_json_key( - j, - "partitioning", - p.partitioning, - "SystemPartitioningHandle", - "SystemPartitioning", - "partitioning"); - to_json_key( - j, - "function", - p.function, - "SystemPartitioningHandle", - "SystemPartitionFunction", - "function"); -} - -void from_json(const json& j, SystemPartitioningHandle& p) { - p._type = j["@type"]; - from_json_key( - j, - "partitioning", - p.partitioning, - "SystemPartitioningHandle", - "SystemPartitioning", - "partitioning"); - from_json_key( - j, - "function", - p.function, - "SystemPartitioningHandle", - "SystemPartitionFunction", - "function"); -} -} // 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"]; - 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 { -TpchTableLayoutHandle::TpchTableLayoutHandle() noexcept { - _type = "tpch"; -} - -void to_json(json& j, const TpchTableLayoutHandle& p) { - j = json::object(); - j["@type"] = "tpch"; - to_json_key( - j, "table", p.table, "TpchTableLayoutHandle", "TpchTableHandle", "table"); - to_json_key( - j, - "predicate", - p.predicate, - "TpchTableLayoutHandle", - "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, - "predicate", - p.predicate, - "TpchTableLayoutHandle", - "TupleDomain>", - "predicate"); + : std::begin(DistributionType_enum_table)) + ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TableWriterMergeNode::TableWriterMergeNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.TableWriterMergeNode"; +SemiJoinNode::SemiJoinNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.SemiJoinNode"; } -void to_json(json& j, const TableWriterMergeNode& p) { +void to_json(json& j, const SemiJoinNode& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.TableWriterMergeNode"; - to_json_key(j, "id", p.id, "TableWriterMergeNode", "PlanNodeId", "id"); + 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, "source", p.source, "TableWriterMergeNode", "PlanNode", "source"); + j, + "filteringSource", + p.filteringSource, + "SemiJoinNode", + "PlanNode", + "filteringSource"); to_json_key( j, - "rowCountVariable", - p.rowCountVariable, - "TableWriterMergeNode", + "sourceJoinVariable", + p.sourceJoinVariable, + "SemiJoinNode", "VariableReferenceExpression", - "rowCountVariable"); + "sourceJoinVariable"); to_json_key( j, - "fragmentVariable", - p.fragmentVariable, - "TableWriterMergeNode", + "filteringSourceJoinVariable", + p.filteringSourceJoinVariable, + "SemiJoinNode", "VariableReferenceExpression", - "fragmentVariable"); + "filteringSourceJoinVariable"); to_json_key( j, - "tableCommitContextVariable", - p.tableCommitContextVariable, - "TableWriterMergeNode", + "semiJoinOutput", + p.semiJoinOutput, + "SemiJoinNode", "VariableReferenceExpression", - "tableCommitContextVariable"); + "semiJoinOutput"); to_json_key( j, - "statisticsAggregation", - p.statisticsAggregation, - "TableWriterMergeNode", - "StatisticAggregations", - "statisticsAggregation"); + "sourceHashVariable", + p.sourceHashVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "sourceHashVariable"); + to_json_key( + j, + "filteringSourceHashVariable", + p.filteringSourceHashVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "filteringSourceHashVariable"); + to_json_key( + j, + "distributionType", + p.distributionType, + "SemiJoinNode", + "DistributionType", + "distributionType"); + to_json_key( + j, + "dynamicFilters", + p.dynamicFilters, + "SemiJoinNode", + "Map", + "dynamicFilters"); } -void from_json(const json& j, TableWriterMergeNode& p) { +void from_json(const json& j, SemiJoinNode& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "TableWriterMergeNode", "PlanNodeId", "id"); + from_json_key(j, "id", p.id, "SemiJoinNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "SemiJoinNode", "PlanNode", "source"); from_json_key( - j, "source", p.source, "TableWriterMergeNode", "PlanNode", "source"); + j, + "filteringSource", + p.filteringSource, + "SemiJoinNode", + "PlanNode", + "filteringSource"); from_json_key( j, - "rowCountVariable", - p.rowCountVariable, - "TableWriterMergeNode", + "sourceJoinVariable", + p.sourceJoinVariable, + "SemiJoinNode", "VariableReferenceExpression", - "rowCountVariable"); + "sourceJoinVariable"); from_json_key( j, - "fragmentVariable", - p.fragmentVariable, - "TableWriterMergeNode", + "filteringSourceJoinVariable", + p.filteringSourceJoinVariable, + "SemiJoinNode", "VariableReferenceExpression", - "fragmentVariable"); + "filteringSourceJoinVariable"); from_json_key( j, - "tableCommitContextVariable", - p.tableCommitContextVariable, - "TableWriterMergeNode", + "semiJoinOutput", + p.semiJoinOutput, + "SemiJoinNode", "VariableReferenceExpression", - "tableCommitContextVariable"); + "semiJoinOutput"); from_json_key( j, - "statisticsAggregation", - p.statisticsAggregation, - "TableWriterMergeNode", - "StatisticAggregations", - "statisticsAggregation"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -TpchPartitioningHandle::TpchPartitioningHandle() noexcept { - _type = "tpch"; -} - -void to_json(json& j, const TpchPartitioningHandle& p) { - j = json::object(); - j["@type"] = "tpch"; - to_json_key(j, "table", p.table, "TpchPartitioningHandle", "String", "table"); - to_json_key( + "sourceHashVariable", + p.sourceHashVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "sourceHashVariable"); + from_json_key( j, - "totalRows", - p.totalRows, - "TpchPartitioningHandle", - "int64_t", - "totalRows"); -} - -void from_json(const json& j, TpchPartitioningHandle& p) { - p._type = j["@type"]; + "filteringSourceHashVariable", + p.filteringSourceHashVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "filteringSourceHashVariable"); from_json_key( - j, "table", p.table, "TpchPartitioningHandle", "String", "table"); + j, + "distributionType", + p.distributionType, + "SemiJoinNode", + "DistributionType", + "distributionType"); from_json_key( j, - "totalRows", - p.totalRows, - "TpchPartitioningHandle", - "int64_t", - "totalRows"); + "dynamicFilters", + p.dynamicFilters, + "SemiJoinNode", + "Map", + "dynamicFilters"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveMetadataUpdateHandle::HiveMetadataUpdateHandle() noexcept { - _type = "hive"; -} +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const HiveMetadataUpdateHandle& 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"); +// 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, 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, 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 { -TopNRowNumberNode::TopNRowNumberNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.TopNRowNumberNode"; + +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 to_json(json& j, const TopNRowNumberNode& p) { +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 { +TableWriterMergeNode::TableWriterMergeNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.TableWriterMergeNode"; +} + +void to_json(json& j, const TableWriterMergeNode& 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"); + j["@type"] = "com.facebook.presto.sql.planner.plan.TableWriterMergeNode"; + to_json_key(j, "id", p.id, "TableWriterMergeNode", "PlanNodeId", "id"); to_json_key( - j, - "specification", - p.specification, - "TopNRowNumberNode", - "Specification", - "specification"); + j, "source", p.source, "TableWriterMergeNode", "PlanNode", "source"); to_json_key( j, - "rowNumberVariable", - p.rowNumberVariable, - "TopNRowNumberNode", + "rowCountVariable", + p.rowCountVariable, + "TableWriterMergeNode", "VariableReferenceExpression", - "rowNumberVariable"); + "rowCountVariable"); to_json_key( j, - "maxRowCountPerPartition", - p.maxRowCountPerPartition, - "TopNRowNumberNode", - "int", - "maxRowCountPerPartition"); - to_json_key(j, "partial", p.partial, "TopNRowNumberNode", "bool", "partial"); + "fragmentVariable", + p.fragmentVariable, + "TableWriterMergeNode", + "VariableReferenceExpression", + "fragmentVariable"); to_json_key( j, - "hashVariable", - p.hashVariable, - "TopNRowNumberNode", + "tableCommitContextVariable", + p.tableCommitContextVariable, + "TableWriterMergeNode", "VariableReferenceExpression", - "hashVariable"); + "tableCommitContextVariable"); + to_json_key( + j, + "statisticsAggregation", + p.statisticsAggregation, + "TableWriterMergeNode", + "StatisticAggregations", + "statisticsAggregation"); } -void from_json(const json& j, TopNRowNumberNode& p) { +void from_json(const json& j, TableWriterMergeNode& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "TopNRowNumberNode", "PlanNodeId", "id"); + from_json_key(j, "id", p.id, "TableWriterMergeNode", "PlanNodeId", "id"); from_json_key( - j, "source", p.source, "TopNRowNumberNode", "PlanNode", "source"); + j, "source", p.source, "TableWriterMergeNode", "PlanNode", "source"); from_json_key( j, - "specification", - p.specification, - "TopNRowNumberNode", - "Specification", - "specification"); + "rowCountVariable", + p.rowCountVariable, + "TableWriterMergeNode", + "VariableReferenceExpression", + "rowCountVariable"); from_json_key( j, - "rowNumberVariable", - p.rowNumberVariable, - "TopNRowNumberNode", + "fragmentVariable", + p.fragmentVariable, + "TableWriterMergeNode", "VariableReferenceExpression", - "rowNumberVariable"); + "fragmentVariable"); from_json_key( j, - "maxRowCountPerPartition", - p.maxRowCountPerPartition, - "TopNRowNumberNode", - "int", - "maxRowCountPerPartition"); - from_json_key( - j, "partial", p.partial, "TopNRowNumberNode", "bool", "partial"); + "tableCommitContextVariable", + p.tableCommitContextVariable, + "TableWriterMergeNode", + "VariableReferenceExpression", + "tableCommitContextVariable"); from_json_key( j, - "hashVariable", - p.hashVariable, - "TopNRowNumberNode", - "VariableReferenceExpression", - "hashVariable"); + "statisticsAggregation", + p.statisticsAggregation, + "TableWriterMergeNode", + "StatisticAggregations", + "statisticsAggregation"); } } // 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!"); +static const std::pair + SystemPartitioning_enum_table[] = { // NOLINT: cert-err58-cpp + {SystemPartitioning::SINGLE, "SINGLE"}, + {SystemPartitioning::FIXED, "FIXED"}, + {SystemPartitioning::SOURCE, "SOURCE"}, + {SystemPartitioning::SCALED, "SCALED"}, + {SystemPartitioning::COORDINATOR_ONLY, "COORDINATOR_ONLY"}, + {SystemPartitioning::ARBITRARY, "ARBITRARY"}}; +void to_json(json& j, const SystemPartitioning& e) { + static_assert( + std::is_enum::value, + "SystemPartitioning must be an enum!"); const auto* it = std::find_if( - std::begin(Step_enum_table), - std::end(Step_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(SystemPartitioning_enum_table), + std::end(SystemPartitioning_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(Step_enum_table)) ? it : std::begin(Step_enum_table)) + j = ((it != std::end(SystemPartitioning_enum_table)) + ? it + : std::begin(SystemPartitioning_enum_table)) ->second; } -void from_json(const json& j, Step& e) { - static_assert(std::is_enum::value, "Step must be an enum!"); +void from_json(const json& j, SystemPartitioning& e) { + static_assert( + std::is_enum::value, + "SystemPartitioning must be an enum!"); const auto* it = std::find_if( - std::begin(Step_enum_table), - std::end(Step_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(SystemPartitioning_enum_table), + std::end(SystemPartitioning_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(Step_enum_table)) ? it : std::begin(Step_enum_table)) + e = ((it != std::end(SystemPartitioning_enum_table)) + ? it + : std::begin(SystemPartitioning_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, - "orderingScheme", - p.orderingScheme, - "TopNNode", - "OrderingScheme", - "orderingScheme"); - to_json_key(j, "step", p.step, "TopNNode", "Step", "step"); -} - -void from_json(const json& j, TopNNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "TopNNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "TopNNode", "PlanNode", "source"); - from_json_key(j, "count", p.count, "TopNNode", "int64_t", "count"); - from_json_key( - j, - "orderingScheme", - p.orderingScheme, - "TopNNode", - "OrderingScheme", - "orderingScheme"); - from_json_key(j, "step", p.step, "TopNNode", "Step", "step"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { // 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!"); +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(Form_enum_table), - std::end(Form_enum_table), - [e](const std::pair& ej_pair) -> bool { + 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(Form_enum_table)) ? it : std::begin(Form_enum_table)) + j = ((it != std::end(SystemPartitionFunction_enum_table)) + ? it + : std::begin(SystemPartitionFunction_enum_table)) ->second; } -void from_json(const json& j, Form& e) { - static_assert(std::is_enum::value, "Form must be an enum!"); +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(Form_enum_table), - std::end(Form_enum_table), - [&j](const std::pair& ej_pair) -> bool { + 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(Form_enum_table)) ? it : std::begin(Form_enum_table)) + e = ((it != std::end(SystemPartitionFunction_enum_table)) + ? it + : std::begin(SystemPartitionFunction_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -SpecialFormExpression::SpecialFormExpression() noexcept { - _type = "special"; +SystemPartitioningHandle::SystemPartitioningHandle() noexcept { + _type = "$remote"; } -void to_json(json& j, const SpecialFormExpression& p) { +void to_json(json& j, const SystemPartitioningHandle& 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"] = "$remote"; to_json_key( j, - "returnType", - p.returnType, - "SpecialFormExpression", - "Type", - "returnType"); + "partitioning", + p.partitioning, + "SystemPartitioningHandle", + "SystemPartitioning", + "partitioning"); to_json_key( j, - "arguments", - p.arguments, - "SpecialFormExpression", - "List>", - "arguments"); + "function", + p.function, + "SystemPartitioningHandle", + "SystemPartitionFunction", + "function"); } -void from_json(const json& j, SpecialFormExpression& p) { +void from_json(const json& j, SystemPartitioningHandle& 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"); + "partitioning", + p.partitioning, + "SystemPartitioningHandle", + "SystemPartitioning", + "partitioning"); from_json_key( j, - "arguments", - p.arguments, - "SpecialFormExpression", - "List>", - "arguments"); + "function", + p.function, + "SystemPartitioningHandle", + "SystemPartitionFunction", + "function"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -RemoteSourceNode::RemoteSourceNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.RemoteSourceNode"; +// 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 RemoteSourceNode& p) { +void to_json(json& j, const TopNNode& 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, - "sourceFragmentIds", - p.sourceFragmentIds, - "RemoteSourceNode", - "List", - "sourceFragmentIds"); - to_json_key( - j, - "outputVariables", - p.outputVariables, - "RemoteSourceNode", - "List", - "outputVariables"); - to_json_key( - j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "RemoteSourceNode", - "bool", - "ensureSourceOrdering"); + j["@type"] = ".TopNNode"; + to_json_key(j, "id", p.id, "TopNNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "TopNNode", "PlanNode", "source"); + to_json_key(j, "count", p.count, "TopNNode", "int64_t", "count"); to_json_key( j, "orderingScheme", p.orderingScheme, - "RemoteSourceNode", + "TopNNode", "OrderingScheme", "orderingScheme"); - to_json_key( - j, - "exchangeType", - p.exchangeType, - "RemoteSourceNode", - "ExchangeNodeType", - "exchangeType"); + to_json_key(j, "step", p.step, "TopNNode", "Step", "step"); } -void from_json(const json& j, RemoteSourceNode& p) { +void from_json(const json& j, TopNNode& 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"); - from_json_key( - j, - "outputVariables", - p.outputVariables, - "RemoteSourceNode", - "List", - "outputVariables"); - from_json_key( - j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "RemoteSourceNode", - "bool", - "ensureSourceOrdering"); + from_json_key(j, "id", p.id, "TopNNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "TopNNode", "PlanNode", "source"); + from_json_key(j, "count", p.count, "TopNNode", "int64_t", "count"); from_json_key( j, "orderingScheme", p.orderingScheme, - "RemoteSourceNode", + "TopNNode", "OrderingScheme", "orderingScheme"); - from_json_key( - j, - "exchangeType", - p.exchangeType, - "RemoteSourceNode", - "ExchangeNodeType", - "exchangeType"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -HivePartitioningHandle::HivePartitioningHandle() noexcept { - _type = "hive"; -} - -void to_json(json& j, const HivePartitioningHandle& p) { - j = json::object(); - j["@type"] = "hive"; - to_json_key( - j, - "bucketCount", - p.bucketCount, - "HivePartitioningHandle", - "int", - "bucketCount"); - to_json_key( - j, - "maxCompatibleBucketCount", - p.maxCompatibleBucketCount, - "HivePartitioningHandle", - "int", - "maxCompatibleBucketCount"); - to_json_key( - j, - "bucketFunctionType", - p.bucketFunctionType, - "HivePartitioningHandle", - "BucketFunctionType", - "bucketFunctionType"); - to_json_key( - j, - "hiveTypes", - p.hiveTypes, - "HivePartitioningHandle", - "List", - "hiveTypes"); - to_json_key( - j, "types", p.types, "HivePartitioningHandle", "List", "types"); -} - -void from_json(const json& j, HivePartitioningHandle& p) { - p._type = j["@type"]; - from_json_key( - j, - "bucketCount", - p.bucketCount, - "HivePartitioningHandle", - "int", - "bucketCount"); - from_json_key( - j, - "maxCompatibleBucketCount", - p.maxCompatibleBucketCount, - "HivePartitioningHandle", - "int", - "maxCompatibleBucketCount"); - 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"); + from_json_key(j, "step", p.step, "TopNNode", "Step", "step"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -SortedRangeSet::SortedRangeSet() noexcept { - _type = "sortable"; +AllOrNoneValueSet::AllOrNoneValueSet() noexcept { + _type = "allOrNone"; } -void to_json(json& j, const SortedRangeSet& p) { +void to_json(json& j, const AllOrNoneValueSet& 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"); + 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, SortedRangeSet& p) { +void from_json(const json& j, AllOrNoneValueSet& p) { p._type = j["@type"]; - from_json_key(j, "type", p.type, "SortedRangeSet", "Type", "type"); - from_json_key( - j, "ranges", p.ranges, "SortedRangeSet", "List", "ranges"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair 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; + 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 { -SemiJoinNode::SemiJoinNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.SemiJoinNode"; +TpchSplit::TpchSplit() noexcept { + _type = "tpch"; } -void to_json(json& j, const SemiJoinNode& p) { +void to_json(json& j, const TpchSplit& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.SemiJoinNode"; - to_json_key(j, "id", p.id, "SemiJoinNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "SemiJoinNode", "PlanNode", "source"); - to_json_key( - j, - "filteringSource", - p.filteringSource, - "SemiJoinNode", - "PlanNode", - "filteringSource"); - to_json_key( - j, - "sourceJoinVariable", - p.sourceJoinVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "sourceJoinVariable"); - to_json_key( - j, - "filteringSourceJoinVariable", - p.filteringSourceJoinVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "filteringSourceJoinVariable"); - to_json_key( - j, - "semiJoinOutput", - p.semiJoinOutput, - "SemiJoinNode", - "VariableReferenceExpression", - "semiJoinOutput"); - to_json_key( - j, - "sourceHashVariable", - p.sourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "sourceHashVariable"); + j["@type"] = "tpch"; to_json_key( j, - "filteringSourceHashVariable", - p.filteringSourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "filteringSourceHashVariable"); + "tableHandle", + p.tableHandle, + "TpchSplit", + "TpchTableHandle", + "tableHandle"); + to_json_key(j, "partNumber", p.partNumber, "TpchSplit", "int", "partNumber"); + to_json_key(j, "totalParts", p.totalParts, "TpchSplit", "int", "totalParts"); to_json_key( j, - "distributionType", - p.distributionType, - "SemiJoinNode", - "DistributionType", - "distributionType"); + "addresses", + p.addresses, + "TpchSplit", + "List", + "addresses"); to_json_key( j, - "dynamicFilters", - p.dynamicFilters, - "SemiJoinNode", - "Map", - "dynamicFilters"); + "predicate", + p.predicate, + "TpchSplit", + "TupleDomain>", + "predicate"); } -void from_json(const json& j, SemiJoinNode& p) { +void from_json(const json& j, TpchSplit& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "SemiJoinNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "SemiJoinNode", "PlanNode", "source"); - from_json_key( - j, - "filteringSource", - p.filteringSource, - "SemiJoinNode", - "PlanNode", - "filteringSource"); - from_json_key( - j, - "sourceJoinVariable", - p.sourceJoinVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "sourceJoinVariable"); - from_json_key( - j, - "filteringSourceJoinVariable", - p.filteringSourceJoinVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "filteringSourceJoinVariable"); from_json_key( j, - "semiJoinOutput", - p.semiJoinOutput, - "SemiJoinNode", - "VariableReferenceExpression", - "semiJoinOutput"); + "tableHandle", + p.tableHandle, + "TpchSplit", + "TpchTableHandle", + "tableHandle"); from_json_key( - j, - "sourceHashVariable", - p.sourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "sourceHashVariable"); + j, "partNumber", p.partNumber, "TpchSplit", "int", "partNumber"); from_json_key( - j, - "filteringSourceHashVariable", - p.filteringSourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "filteringSourceHashVariable"); + j, "totalParts", p.totalParts, "TpchSplit", "int", "totalParts"); from_json_key( j, - "distributionType", - p.distributionType, - "SemiJoinNode", - "DistributionType", - "distributionType"); + "addresses", + p.addresses, + "TpchSplit", + "List", + "addresses"); from_json_key( j, - "dynamicFilters", - p.dynamicFilters, - "SemiJoinNode", - "Map", - "dynamicFilters"); -} -} // 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; + "predicate", + p.predicate, + "TpchSplit", + "TupleDomain>", + "predicate"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -AggregationNode::AggregationNode() noexcept { - _type = ".AggregationNode"; +HiveInsertTableHandle::HiveInsertTableHandle() noexcept { + _type = "hive"; } -void to_json(json& j, const AggregationNode& p) { +void to_json(json& j, const HiveInsertTableHandle& 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"); + j["@type"] = "hive"; + to_json_key( + j, + "schemaName", + p.schemaName, + "HiveInsertTableHandle", + "String", + "schemaName"); to_json_key( j, - "aggregations", - p.aggregations, - "AggregationNode", - "Map", - "aggregations"); + "tableName", + p.tableName, + "HiveInsertTableHandle", + "String", + "tableName"); to_json_key( j, - "groupingSets", - p.groupingSets, - "AggregationNode", - "GroupingSetDescriptor", - "groupingSets"); + "inputColumns", + p.inputColumns, + "HiveInsertTableHandle", + "List", + "inputColumns"); to_json_key( j, - "preGroupedVariables", - p.preGroupedVariables, - "AggregationNode", - "List", - "preGroupedVariables"); + "pageSinkMetadata", + p.pageSinkMetadata, + "HiveInsertTableHandle", + "HivePageSinkMetadata", + "pageSinkMetadata"); to_json_key( - j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); + j, + "locationHandle", + p.locationHandle, + "HiveInsertTableHandle", + "LocationHandle", + "locationHandle"); to_json_key( j, - "hashVariable", - p.hashVariable, - "AggregationNode", - "VariableReferenceExpression", - "hashVariable"); + "bucketProperty", + p.bucketProperty, + "HiveInsertTableHandle", + "HiveBucketProperty", + "bucketProperty"); to_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "AggregationNode", - "VariableReferenceExpression", - "groupIdVariable"); + "preferredOrderingColumns", + p.preferredOrderingColumns, + "HiveInsertTableHandle", + "List", + "preferredOrderingColumns"); to_json_key( j, - "aggregationId", - p.aggregationId, - "AggregationNode", - "Integer", - "aggregationId"); + "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, AggregationNode& p) { +void from_json(const json& j, HiveInsertTableHandle& 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"); from_json_key( j, - "aggregations", - p.aggregations, - "AggregationNode", - "Map", - "aggregations"); + "schemaName", + p.schemaName, + "HiveInsertTableHandle", + "String", + "schemaName"); from_json_key( j, - "groupingSets", - p.groupingSets, - "AggregationNode", - "GroupingSetDescriptor", - "groupingSets"); + "tableName", + p.tableName, + "HiveInsertTableHandle", + "String", + "tableName"); from_json_key( j, - "preGroupedVariables", - p.preGroupedVariables, - "AggregationNode", - "List", - "preGroupedVariables"); + "inputColumns", + p.inputColumns, + "HiveInsertTableHandle", + "List", + "inputColumns"); from_json_key( - j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); + j, + "pageSinkMetadata", + p.pageSinkMetadata, + "HiveInsertTableHandle", + "HivePageSinkMetadata", + "pageSinkMetadata"); from_json_key( j, - "hashVariable", - p.hashVariable, - "AggregationNode", - "VariableReferenceExpression", - "hashVariable"); + "locationHandle", + p.locationHandle, + "HiveInsertTableHandle", + "LocationHandle", + "locationHandle"); from_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "AggregationNode", - "VariableReferenceExpression", - "groupIdVariable"); + "bucketProperty", + p.bucketProperty, + "HiveInsertTableHandle", + "HiveBucketProperty", + "bucketProperty"); from_json_key( j, - "aggregationId", - p.aggregationId, - "AggregationNode", - "Integer", - "aggregationId"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -SortNode::SortNode() noexcept { - _type = ".SortNode"; -} - -void to_json(json& j, const SortNode& p) { - j = json::object(); - j["@type"] = ".SortNode"; - to_json_key(j, "id", p.id, "SortNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "SortNode", "PlanNode", "source"); - to_json_key( + "preferredOrderingColumns", + p.preferredOrderingColumns, + "HiveInsertTableHandle", + "List", + "preferredOrderingColumns"); + from_json_key( j, - "orderingScheme", - p.orderingScheme, - "SortNode", - "OrderingScheme", - "orderingScheme"); - to_json_key(j, "isPartial", p.isPartial, "SortNode", "bool", "isPartial"); -} - -void from_json(const json& j, SortNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "SortNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "SortNode", "PlanNode", "source"); + "tableStorageFormat", + p.tableStorageFormat, + "HiveInsertTableHandle", + "HiveStorageFormat", + "tableStorageFormat"); 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 { -// 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 { -ProjectNode::ProjectNode() noexcept { - _type = ".ProjectNode"; -} - -void to_json(json& j, const ProjectNode& p) { - j = json::object(); - j["@type"] = ".ProjectNode"; - to_json_key(j, "id", p.id, "ProjectNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "ProjectNode", "PlanNode", "source"); - to_json_key( + "partitionStorageFormat", + p.partitionStorageFormat, + "HiveInsertTableHandle", + "HiveStorageFormat", + "partitionStorageFormat"); + 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"); + "actualStorageFormat", + p.actualStorageFormat, + "HiveInsertTableHandle", + "HiveStorageFormat", + "actualStorageFormat"); from_json_key( j, - "assignments", - p.assignments, - "ProjectNode", - "Assignments", - "assignments"); + "compressionCodec", + p.compressionCodec, + "HiveInsertTableHandle", + "HiveCompressionCodec", + "compressionCodec"); from_json_key( - j, "locality", p.locality, "ProjectNode", "Locality", "locality"); + j, + "encryptionInformation", + p.encryptionInformation, + "HiveInsertTableHandle", + "EncryptionInformation", + "encryptionInformation"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +IcebergTableLayoutHandle::IcebergTableLayoutHandle() noexcept { + _type = "hive-iceberg"; +} -void to_json(json& j, const NodeStatus& p) { +void to_json(json& j, const IcebergTableLayoutHandle& p) { j = json::object(); - to_json_key(j, "nodeId", p.nodeId, "NodeStatus", "String", "nodeId"); + j["@type"] = "hive-iceberg"; to_json_key( j, - "nodeVersion", - p.nodeVersion, - "NodeStatus", - "NodeVersion", - "nodeVersion"); - to_json_key( - j, "environment", p.environment, "NodeStatus", "String", "environment"); + "partitionColumns", + p.partitionColumns, + "IcebergTableLayoutHandle", + "List", + "partitionColumns"); to_json_key( - j, "coordinator", p.coordinator, "NodeStatus", "bool", "coordinator"); - to_json_key(j, "uptime", p.uptime, "NodeStatus", "Duration", "uptime"); + j, + "dataColumns", + p.dataColumns, + "IcebergTableLayoutHandle", + "List", + "dataColumns"); to_json_key( j, - "externalAddress", - p.externalAddress, - "NodeStatus", - "String", - "externalAddress"); + "domainPredicate", + p.domainPredicate, + "IcebergTableLayoutHandle", + "TupleDomain", + "domainPredicate"); to_json_key( j, - "internalAddress", - p.internalAddress, - "NodeStatus", - "String", - "internalAddress"); + "remainingPredicate", + p.remainingPredicate, + "IcebergTableLayoutHandle", + "RowExpression", + "remainingPredicate"); to_json_key( - j, "memoryInfo", p.memoryInfo, "NodeStatus", "MemoryInfo", "memoryInfo"); - to_json_key(j, "processors", p.processors, "NodeStatus", "int", "processors"); + j, + "predicateColumns", + p.predicateColumns, + "IcebergTableLayoutHandle", + "Map", + "predicateColumns"); to_json_key( j, - "processCpuLoad", - p.processCpuLoad, - "NodeStatus", - "double", - "processCpuLoad"); + "requestedColumns", + p.requestedColumns, + "IcebergTableLayoutHandle", + "List", + "requestedColumns"); to_json_key( j, - "systemCpuLoad", - p.systemCpuLoad, - "NodeStatus", - "double", - "systemCpuLoad"); - to_json_key(j, "heapUsed", p.heapUsed, "NodeStatus", "int64_t", "heapUsed"); + "pushdownFilterEnabled", + p.pushdownFilterEnabled, + "IcebergTableLayoutHandle", + "bool", + "pushdownFilterEnabled"); to_json_key( j, - "heapAvailable", - p.heapAvailable, - "NodeStatus", - "int64_t", - "heapAvailable"); + "partitionColumnPredicate", + p.partitionColumnPredicate, + "IcebergTableLayoutHandle", + "TupleDomain>", + "partitionColumnPredicate"); to_json_key( - j, "nonHeapUsed", p.nonHeapUsed, "NodeStatus", "int64_t", "nonHeapUsed"); + j, + "table", + p.table, + "IcebergTableLayoutHandle", + "IcebergTableHandle", + "table"); } -void from_json(const json& j, NodeStatus& p) { - from_json_key(j, "nodeId", p.nodeId, "NodeStatus", "String", "nodeId"); +void from_json(const json& j, IcebergTableLayoutHandle& p) { + p._type = j["@type"]; 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"); + "partitionColumns", + p.partitionColumns, + "IcebergTableLayoutHandle", + "List", + "partitionColumns"); from_json_key( j, - "externalAddress", - p.externalAddress, - "NodeStatus", - "String", - "externalAddress"); + "dataColumns", + p.dataColumns, + "IcebergTableLayoutHandle", + "List", + "dataColumns"); from_json_key( j, - "internalAddress", - p.internalAddress, - "NodeStatus", - "String", - "internalAddress"); + "domainPredicate", + p.domainPredicate, + "IcebergTableLayoutHandle", + "TupleDomain", + "domainPredicate"); from_json_key( - j, "memoryInfo", p.memoryInfo, "NodeStatus", "MemoryInfo", "memoryInfo"); + j, + "remainingPredicate", + p.remainingPredicate, + "IcebergTableLayoutHandle", + "RowExpression", + "remainingPredicate"); from_json_key( - j, "processors", p.processors, "NodeStatus", "int", "processors"); + j, + "predicateColumns", + p.predicateColumns, + "IcebergTableLayoutHandle", + "Map", + "predicateColumns"); from_json_key( j, - "processCpuLoad", - p.processCpuLoad, - "NodeStatus", - "double", - "processCpuLoad"); + "requestedColumns", + p.requestedColumns, + "IcebergTableLayoutHandle", + "List", + "requestedColumns"); from_json_key( j, - "systemCpuLoad", - p.systemCpuLoad, - "NodeStatus", - "double", - "systemCpuLoad"); - from_json_key(j, "heapUsed", p.heapUsed, "NodeStatus", "int64_t", "heapUsed"); + "pushdownFilterEnabled", + p.pushdownFilterEnabled, + "IcebergTableLayoutHandle", + "bool", + "pushdownFilterEnabled"); from_json_key( j, - "heapAvailable", - p.heapAvailable, - "NodeStatus", - "int64_t", - "heapAvailable"); + "partitionColumnPredicate", + p.partitionColumnPredicate, + "IcebergTableLayoutHandle", + "TupleDomain>", + "partitionColumnPredicate"); from_json_key( - j, "nonHeapUsed", p.nonHeapUsed, "NodeStatus", "int64_t", "nonHeapUsed"); + j, + "table", + p.table, + "IcebergTableLayoutHandle", + "IcebergTableHandle", + "table"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TableScanNode::TableScanNode() noexcept { - _type = ".TableScanNode"; +RemoteSourceNode::RemoteSourceNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.RemoteSourceNode"; } -void to_json(json& j, const TableScanNode& p) { +void to_json(json& j, const RemoteSourceNode& 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"] = "com.facebook.presto.sql.planner.plan.RemoteSourceNode"; + to_json_key(j, "id", p.id, "RemoteSourceNode", "PlanNodeId", "id"); + to_json_key( + j, + "sourceFragmentIds", + p.sourceFragmentIds, + "RemoteSourceNode", + "List", + "sourceFragmentIds"); to_json_key( j, "outputVariables", p.outputVariables, - "TableScanNode", + "RemoteSourceNode", "List", "outputVariables"); to_json_key( j, - "assignments", - p.assignments, - "TableScanNode", - "Map>", - "assignments"); + "ensureSourceOrdering", + p.ensureSourceOrdering, + "RemoteSourceNode", + "bool", + "ensureSourceOrdering"); + to_json_key( + j, + "orderingScheme", + p.orderingScheme, + "RemoteSourceNode", + "OrderingScheme", + "orderingScheme"); + to_json_key( + j, + "exchangeType", + p.exchangeType, + "RemoteSourceNode", + "ExchangeNodeType", + "exchangeType"); } -void from_json(const json& j, TableScanNode& p) { +void from_json(const json& j, RemoteSourceNode& 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, "RemoteSourceNode", "PlanNodeId", "id"); + from_json_key( + j, + "sourceFragmentIds", + p.sourceFragmentIds, + "RemoteSourceNode", + "List", + "sourceFragmentIds"); from_json_key( j, "outputVariables", p.outputVariables, - "TableScanNode", + "RemoteSourceNode", "List", "outputVariables"); from_json_key( j, - "assignments", - p.assignments, - "TableScanNode", - "Map>", - "assignments"); + "ensureSourceOrdering", + p.ensureSourceOrdering, + "RemoteSourceNode", + "bool", + "ensureSourceOrdering"); + from_json_key( + j, + "orderingScheme", + p.orderingScheme, + "RemoteSourceNode", + "OrderingScheme", + "orderingScheme"); + from_json_key( + j, + "exchangeType", + p.exchangeType, + "RemoteSourceNode", + "ExchangeNodeType", + "exchangeType"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +CreateHandle::CreateHandle() noexcept { + _type = "CreateHandle"; +} -void to_json(json& j, const RefreshMaterializedViewHandle& p) { +void to_json(json& j, const CreateHandle& p) { j = json::object(); + j["@type"] = "CreateHandle"; to_json_key( - j, - "handle", - p.handle, - "RefreshMaterializedViewHandle", - "InsertTableHandle", - "handle"); + j, "handle", p.handle, "CreateHandle", "OutputTableHandle", "handle"); to_json_key( j, "schemaTableName", p.schemaTableName, - "RefreshMaterializedViewHandle", + "CreateHandle", "SchemaTableName", "schemaTableName"); } -void from_json(const json& j, RefreshMaterializedViewHandle& p) { +void from_json(const json& j, CreateHandle& p) { + p._type = j["@type"]; from_json_key( - j, - "handle", - p.handle, - "RefreshMaterializedViewHandle", - "InsertTableHandle", - "handle"); + j, "handle", p.handle, "CreateHandle", "OutputTableHandle", "handle"); from_json_key( j, "schemaTableName", p.schemaTableName, - "RefreshMaterializedViewHandle", + "CreateHandle", "SchemaTableName", "schemaTableName"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -AssignUniqueId::AssignUniqueId() noexcept { - _type = "com.facebook.presto.sql.planner.plan.AssignUniqueId"; +EnforceSingleRowNode::EnforceSingleRowNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode"; } -void to_json(json& j, const AssignUniqueId& p) { +void to_json(json& j, const EnforceSingleRowNode& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.AssignUniqueId"; - to_json_key(j, "id", p.id, "AssignUniqueId", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "AssignUniqueId", "PlanNode", "source"); + j["@type"] = "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode"; + to_json_key(j, "id", p.id, "EnforceSingleRowNode", "PlanNodeId", "id"); to_json_key( - j, - "idVariable", - p.idVariable, - "AssignUniqueId", - "VariableReferenceExpression", - "idVariable"); + j, "source", p.source, "EnforceSingleRowNode", "PlanNode", "source"); } -void from_json(const json& j, AssignUniqueId& p) { +void from_json(const json& j, EnforceSingleRowNode& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "AssignUniqueId", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "AssignUniqueId", "PlanNode", "source"); + from_json_key(j, "id", p.id, "EnforceSingleRowNode", "PlanNodeId", "id"); from_json_key( - j, - "idVariable", - p.idVariable, - "AssignUniqueId", - "VariableReferenceExpression", - "idVariable"); -} -} // 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() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair - HiveCompressionCodec_enum_table[] = { // NOLINT: cert-err58-cpp - {HiveCompressionCodec::NONE, "NONE"}, - {HiveCompressionCodec::SNAPPY, "SNAPPY"}, - {HiveCompressionCodec::GZIP, "GZIP"}, - {HiveCompressionCodec::LZ4, "LZ4"}, - {HiveCompressionCodec::ZSTD, "ZSTD"}}; -void to_json(json& j, const HiveCompressionCodec& e) { - static_assert( - std::is_enum::value, - "HiveCompressionCodec must be an enum!"); - const auto* it = std::find_if( - std::begin(HiveCompressionCodec_enum_table), - std::end(HiveCompressionCodec_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(HiveCompressionCodec_enum_table)) - ? it - : std::begin(HiveCompressionCodec_enum_table)) - ->second; -} -void from_json(const json& j, HiveCompressionCodec& e) { - static_assert( - std::is_enum::value, - "HiveCompressionCodec must be an enum!"); - const auto* it = std::find_if( - std::begin(HiveCompressionCodec_enum_table), - std::end(HiveCompressionCodec_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(HiveCompressionCodec_enum_table)) - ? it - : std::begin(HiveCompressionCodec_enum_table)) - ->first; + j, "source", p.source, "EnforceSingleRowNode", "PlanNode", "source"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveOutputTableHandle::HiveOutputTableHandle() noexcept { - _type = "hive"; -} -void to_json(json& j, const HiveOutputTableHandle& p) { +void to_json(json& j, const BatchTaskUpdateRequest& p) { j = json::object(); - j["@type"] = "hive"; to_json_key( j, - "schemaName", - p.schemaName, - "HiveOutputTableHandle", - "String", - "schemaName"); + "taskUpdateRequest", + p.taskUpdateRequest, + "BatchTaskUpdateRequest", + "TaskUpdateRequest", + "taskUpdateRequest"); to_json_key( j, - "tableName", - p.tableName, - "HiveOutputTableHandle", + "shuffleWriteInfo", + p.shuffleWriteInfo, + "BatchTaskUpdateRequest", "String", - "tableName"); + "shuffleWriteInfo"); to_json_key( j, - "inputColumns", - p.inputColumns, - "HiveOutputTableHandle", - "List", - "inputColumns"); - to_json_key( + "broadcastBasePath", + p.broadcastBasePath, + "BatchTaskUpdateRequest", + "String", + "broadcastBasePath"); +} + +void from_json(const json& j, BatchTaskUpdateRequest& p) { + from_json_key( + j, + "taskUpdateRequest", + p.taskUpdateRequest, + "BatchTaskUpdateRequest", + "TaskUpdateRequest", + "taskUpdateRequest"); + from_json_key( j, - "pageSinkMetadata", - p.pageSinkMetadata, - "HiveOutputTableHandle", - "HivePageSinkMetadata", - "pageSinkMetadata"); - to_json_key( + "shuffleWriteInfo", + p.shuffleWriteInfo, + "BatchTaskUpdateRequest", + "String", + "shuffleWriteInfo"); + from_json_key( j, - "locationHandle", - p.locationHandle, - "HiveOutputTableHandle", - "LocationHandle", - "locationHandle"); + "broadcastBasePath", + p.broadcastBasePath, + "BatchTaskUpdateRequest", + "String", + "broadcastBasePath"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +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, - "tableStorageFormat", - p.tableStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "tableStorageFormat"); + "fileFormat", + p.fileFormat, + "IcebergSplit", + "FileFormat", + "fileFormat"); to_json_key( j, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); + "addresses", + p.addresses, + "IcebergSplit", + "List", + "addresses"); to_json_key( j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); + "partitionKeys", + p.partitionKeys, + "IcebergSplit", + "Map", + "partitionKeys"); to_json_key( j, - "compressionCodec", - p.compressionCodec, - "HiveOutputTableHandle", - "HiveCompressionCodec", - "compressionCodec"); + "partitionSpecAsJson", + p.partitionSpecAsJson, + "IcebergSplit", + "String", + "partitionSpecAsJson"); to_json_key( j, - "partitionedBy", - p.partitionedBy, - "HiveOutputTableHandle", - "List", - "partitionedBy"); + "partitionDataJson", + p.partitionDataJson, + "IcebergSplit", + "String", + "partitionDataJson"); to_json_key( j, - "bucketProperty", - p.bucketProperty, - "HiveOutputTableHandle", - "HiveBucketProperty", - "bucketProperty"); + "nodeSelectionStrategy", + p.nodeSelectionStrategy, + "IcebergSplit", + "NodeSelectionStrategy", + "nodeSelectionStrategy"); to_json_key( j, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveOutputTableHandle", - "List", - "preferredOrderingColumns"); + "splitWeight", + p.splitWeight, + "IcebergSplit", + "SplitWeight", + "splitWeight"); to_json_key( - j, - "tableOwner", - p.tableOwner, - "HiveOutputTableHandle", - "String", - "tableOwner"); + j, "deletes", p.deletes, "IcebergSplit", "List", "deletes"); to_json_key( j, - "additionalTableParameters", - p.additionalTableParameters, - "HiveOutputTableHandle", - "Map", - "additionalTableParameters"); + "changelogSplitInfo", + p.changelogSplitInfo, + "IcebergSplit", + "ChangelogSplitInfo", + "changelogSplitInfo"); to_json_key( j, - "encryptionInformation", - p.encryptionInformation, - "HiveOutputTableHandle", - "EncryptionInformation", - "encryptionInformation"); + "dataSequenceNumber", + p.dataSequenceNumber, + "IcebergSplit", + "int64_t", + "dataSequenceNumber"); } -void from_json(const json& j, HiveOutputTableHandle& p) { +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, - "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"); + "fileFormat", + p.fileFormat, + "IcebergSplit", + "FileFormat", + "fileFormat"); from_json_key( j, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); + "addresses", + p.addresses, + "IcebergSplit", + "List", + "addresses"); from_json_key( j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); + "partitionKeys", + p.partitionKeys, + "IcebergSplit", + "Map", + "partitionKeys"); from_json_key( j, - "compressionCodec", - p.compressionCodec, - "HiveOutputTableHandle", - "HiveCompressionCodec", - "compressionCodec"); + "partitionSpecAsJson", + p.partitionSpecAsJson, + "IcebergSplit", + "String", + "partitionSpecAsJson"); from_json_key( j, - "partitionedBy", - p.partitionedBy, - "HiveOutputTableHandle", - "List", - "partitionedBy"); + "partitionDataJson", + p.partitionDataJson, + "IcebergSplit", + "String", + "partitionDataJson"); from_json_key( j, - "bucketProperty", - p.bucketProperty, - "HiveOutputTableHandle", - "HiveBucketProperty", - "bucketProperty"); + "nodeSelectionStrategy", + p.nodeSelectionStrategy, + "IcebergSplit", + "NodeSelectionStrategy", + "nodeSelectionStrategy"); from_json_key( j, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveOutputTableHandle", - "List", - "preferredOrderingColumns"); + "splitWeight", + p.splitWeight, + "IcebergSplit", + "SplitWeight", + "splitWeight"); from_json_key( - j, - "tableOwner", - p.tableOwner, - "HiveOutputTableHandle", - "String", - "tableOwner"); + j, "deletes", p.deletes, "IcebergSplit", "List", "deletes"); from_json_key( j, - "additionalTableParameters", - p.additionalTableParameters, - "HiveOutputTableHandle", - "Map", - "additionalTableParameters"); + "changelogSplitInfo", + p.changelogSplitInfo, + "IcebergSplit", + "ChangelogSplitInfo", + "changelogSplitInfo"); from_json_key( j, - "encryptionInformation", - p.encryptionInformation, - "HiveOutputTableHandle", - "EncryptionInformation", - "encryptionInformation"); + "dataSequenceNumber", + p.dataSequenceNumber, + "IcebergSplit", + "int64_t", + "dataSequenceNumber"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveInsertTableHandle::HiveInsertTableHandle() noexcept { +HivePartitioningHandle::HivePartitioningHandle() noexcept { _type = "hive"; } -void to_json(json& j, const HiveInsertTableHandle& p) { +void to_json(json& j, const HivePartitioningHandle& p) { j = json::object(); j["@type"] = "hive"; to_json_key( j, - "schemaName", - p.schemaName, - "HiveInsertTableHandle", - "String", - "schemaName"); - to_json_key( - j, - "tableName", - p.tableName, - "HiveInsertTableHandle", - "String", - "tableName"); - to_json_key( - j, - "inputColumns", - p.inputColumns, - "HiveInsertTableHandle", - "List", - "inputColumns"); - to_json_key( - j, - "pageSinkMetadata", - p.pageSinkMetadata, - "HiveInsertTableHandle", - "HivePageSinkMetadata", - "pageSinkMetadata"); - to_json_key( - j, - "locationHandle", - p.locationHandle, - "HiveInsertTableHandle", - "LocationHandle", - "locationHandle"); - to_json_key( - j, - "bucketProperty", - p.bucketProperty, - "HiveInsertTableHandle", - "HiveBucketProperty", - "bucketProperty"); - to_json_key( - j, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveInsertTableHandle", - "List", - "preferredOrderingColumns"); - to_json_key( - j, - "tableStorageFormat", - p.tableStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "tableStorageFormat"); + "bucketCount", + p.bucketCount, + "HivePartitioningHandle", + "int", + "bucketCount"); to_json_key( j, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); + "maxCompatibleBucketCount", + p.maxCompatibleBucketCount, + "HivePartitioningHandle", + "int", + "maxCompatibleBucketCount"); to_json_key( j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); + "bucketFunctionType", + p.bucketFunctionType, + "HivePartitioningHandle", + "BucketFunctionType", + "bucketFunctionType"); to_json_key( j, - "compressionCodec", - p.compressionCodec, - "HiveInsertTableHandle", - "HiveCompressionCodec", - "compressionCodec"); + "hiveTypes", + p.hiveTypes, + "HivePartitioningHandle", + "List", + "hiveTypes"); to_json_key( - j, - "encryptionInformation", - p.encryptionInformation, - "HiveInsertTableHandle", - "EncryptionInformation", - "encryptionInformation"); + j, "types", p.types, "HivePartitioningHandle", "List", "types"); } -void from_json(const json& j, HiveInsertTableHandle& p) { +void from_json(const json& j, HivePartitioningHandle& p) { p._type = j["@type"]; from_json_key( j, - "schemaName", - p.schemaName, - "HiveInsertTableHandle", - "String", - "schemaName"); + "bucketCount", + p.bucketCount, + "HivePartitioningHandle", + "int", + "bucketCount"); from_json_key( j, - "tableName", - p.tableName, - "HiveInsertTableHandle", - "String", - "tableName"); + "maxCompatibleBucketCount", + p.maxCompatibleBucketCount, + "HivePartitioningHandle", + "int", + "maxCompatibleBucketCount"); from_json_key( j, - "inputColumns", - p.inputColumns, - "HiveInsertTableHandle", - "List", - "inputColumns"); + "bucketFunctionType", + p.bucketFunctionType, + "HivePartitioningHandle", + "BucketFunctionType", + "bucketFunctionType"); from_json_key( j, - "pageSinkMetadata", - p.pageSinkMetadata, - "HiveInsertTableHandle", - "HivePageSinkMetadata", - "pageSinkMetadata"); + "hiveTypes", + p.hiveTypes, + "HivePartitioningHandle", + "List", + "hiveTypes"); from_json_key( + j, "types", p.types, "HivePartitioningHandle", "List", "types"); +} +} // 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, - "locationHandle", - p.locationHandle, - "HiveInsertTableHandle", - "LocationHandle", - "locationHandle"); - from_json_key( + "distinctVariables", + p.distinctVariables, + "DistinctLimitNode", + "List", + "distinctVariables"); + to_json_key( j, - "bucketProperty", - p.bucketProperty, - "HiveInsertTableHandle", - "HiveBucketProperty", - "bucketProperty"); - from_json_key( + "hashVariable", + p.hashVariable, + "DistinctLimitNode", + "VariableReferenceExpression", + "hashVariable"); + to_json_key( j, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveInsertTableHandle", - "List", - "preferredOrderingColumns"); + "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, - "tableStorageFormat", - p.tableStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "tableStorageFormat"); + j, "source", p.source, "DistinctLimitNode", "PlanNode", "source"); + from_json_key(j, "limit", p.limit, "DistinctLimitNode", "int64_t", "limit"); from_json_key( - j, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); + j, "partial", p.partial, "DistinctLimitNode", "bool", "partial"); from_json_key( j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); + "distinctVariables", + p.distinctVariables, + "DistinctLimitNode", + "List", + "distinctVariables"); from_json_key( j, - "compressionCodec", - p.compressionCodec, - "HiveInsertTableHandle", - "HiveCompressionCodec", - "compressionCodec"); + "hashVariable", + p.hashVariable, + "DistinctLimitNode", + "VariableReferenceExpression", + "hashVariable"); from_json_key( j, - "encryptionInformation", - p.encryptionInformation, - "HiveInsertTableHandle", - "EncryptionInformation", - "encryptionInformation"); + "timeoutMillis", + p.timeoutMillis, + "DistinctLimitNode", + "int", + "timeoutMillis"); } } // 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 52e24fa51a16..cd0bf998531c 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.h +++ b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.h @@ -254,6 +254,13 @@ struct adl_serializer> { // Forward declaration of all abstract types // namespace facebook::presto::protocol { +struct PlanNode : public JsonEncodedSubclass { + PlanNodeId id = {}; +}; +void to_json(json& j, const std::shared_ptr& p); +void from_json(const json& j, std::shared_ptr& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { struct ConnectorTransactionHandle : public JsonEncodedSubclass {}; void to_json(json& j, const std::shared_ptr& p); void from_json(const json& j, std::shared_ptr& p); @@ -264,19 +271,13 @@ 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); +struct ColumnHandle : public JsonEncodedSubclass { + virtual bool operator<(const ColumnHandle& /* o */) const { + throw std::runtime_error("missing operator<() in {class_name} subclass"); + } +}; +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 {}; @@ -291,25 +292,24 @@ void to_json(json& j, const std::shared_ptr& p); void from_json(const json& j, std::shared_ptr& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct PlanNode : public JsonEncodedSubclass { - PlanNodeId id = {}; -}; -void to_json(json& j, const std::shared_ptr& p); -void from_json(const json& j, std::shared_ptr& p); -} // 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 ColumnHandle : public JsonEncodedSubclass { - virtual bool operator<(const ColumnHandle& /* o */) const { - throw std::runtime_error("missing operator<() in {class_name} subclass"); - } -}; -void to_json(json& j, const std::shared_ptr& p); -void from_json(const json& j, std::shared_ptr& p); +struct ConnectorHistogram : 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 ConnectorPartitioningHandle : public JsonEncodedSubclass {}; @@ -317,14 +317,14 @@ 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 ConnectorHistogram : 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 ConnectorOutputTableHandle : public JsonEncodedSubclass {}; -void to_json(json& j, const std::shared_ptr& p); -void from_json(const json& j, std::shared_ptr& p); +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 ConnectorMetadataUpdateHandle : public JsonEncodedSubclass {}; @@ -334,60 +334,18 @@ void from_json( 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 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 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 Parameter { - String name = {}; - TypeSignature type = {}; -}; -void to_json(json& j, const Parameter& p); -void from_json(const json& j, Parameter& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class 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 { -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 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 { -struct RoutineCharacteristics { - std::shared_ptr language = {}; - std::shared_ptr determinism = {}; - std::shared_ptr nullCallClause = {}; +struct SourceLocation { + int line = {}; + int column = {}; }; -void to_json(json& j, const RoutineCharacteristics& p); -void from_json(const json& j, RoutineCharacteristics& p); +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"); @@ -404,55 +362,73 @@ void from_json(const json& j, RoutineCharacteristics& p); */ namespace facebook::presto::protocol { -struct TypeVariableConstraint { - String name = {}; - bool comparableRequired = {}; - bool orderableRequired = {}; - String variadicBound = {}; - bool nonDecimalNumericRequired = {}; - String boundedBy = {}; +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 TypeVariableConstraint& p); -void from_json(const json& j, TypeVariableConstraint& 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 namespace facebook::presto::protocol { -struct LongVariableConstraint { - String name = {}; - String expression = {}; +struct MarkDistinctNode : public PlanNode { + std::shared_ptr source = {}; + VariableReferenceExpression markerVariable = {}; + List distinctVariables = {}; + std::shared_ptr hashVariable = {}; + + MarkDistinctNode() noexcept; }; -void to_json(json& j, const LongVariableConstraint& p); -void from_json(const json& j, LongVariableConstraint& p); +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 FunctionKind { SCALAR, AGGREGATE, WINDOW }; -extern void to_json(json& j, const FunctionKind& e); -extern void from_json(const json& j, FunctionKind& e); +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 Signature { - QualifiedObjectName name = {}; - FunctionKind kind = {}; - List typeVariableConstraints = {}; - List longVariableConstraints = {}; - TypeSignature returnType = {}; - List argumentTypes = {}; - bool variableArity = {}; +struct Ordering { + VariableReferenceExpression variable = {}; + SortOrder sortOrder = {}; }; -void to_json(json& j, const Signature& p); -void from_json(const json& j, Signature& p); +void to_json(json& j, const Ordering& p); +void from_json(const json& j, Ordering& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct SqlInvokedFunction { - List parameters = {}; - String description = {}; - RoutineCharacteristics routineCharacteristics = {}; - String body = {}; - Signature signature = {}; - SqlFunctionId functionId = {}; +struct OrderingScheme { + List orderBy = {}; }; -void to_json(json& j, const SqlInvokedFunction& p); -void from_json(const json& j, SqlInvokedFunction& p); +void to_json(json& j, const OrderingScheme& p); +void from_json(const json& j, OrderingScheme& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -468,13 +444,38 @@ void from_json(const json& j, SqlInvokedFunction& p); * limitations under the License. */ +// TpchTransactionHandle is special since +// the corresponding class in Java is an enum. + namespace facebook::presto::protocol { -std::ostream& operator<<(std::ostream& os, const Duration& d); +struct TpchTransactionHandle : public ConnectorTransactionHandle { + String instance = {}; +}; -void to_json(json& j, const Duration& p); -void from_json(const json& j, Duration& p); +void to_json(json& j, const TpchTransactionHandle& p); +void from_json(const json& j, TpchTransactionHandle& p); + +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct SystemTransactionHandle : public ConnectorTransactionHandle { + ConnectorId connectorId = {}; + TransactionId transactionId = {}; + std::shared_ptr connectorTransactionHandle = {}; + + SystemTransactionHandle() noexcept; +}; +void to_json(json& j, const SystemTransactionHandle& p); +void from_json(const json& j, SystemTransactionHandle& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct Specification { + List partitionBy = {}; + std::shared_ptr orderingScheme = {}; +}; +void to_json(json& j, const Specification& p); +void from_json(const json& j, Specification& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -491,105 +492,17 @@ void from_json(const json& j, Duration& p); */ namespace facebook::presto::protocol { -std::ostream& operator<<(std::ostream& os, const DataSize& d); +struct Lifespan { + bool isgroup = false; + long groupid = 0; -void to_json(nlohmann::json& j, const DataSize& p); -void from_json(const nlohmann::json& j, DataSize& p); - -} // 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 = {}; -}; -void to_json(json& j, const ResourceEstimates& p); -void from_json(const json& j, ResourceEstimates& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct 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 SessionRepresentation& p); -void from_json(const json& j, SessionRepresentation& 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 Lifespan { - bool isgroup = false; - long groupid = 0; - - bool operator<(const Lifespan& o) const { - return groupid < o.groupid; - } -}; - -void to_json(json& j, const Lifespan& p); -void from_json(const json& j, Lifespan& p); - -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -// TpchTransactionHandle is special since -// the corresponding class in Java is an enum. - -namespace facebook::presto::protocol { - -struct TpchTransactionHandle : public ConnectorTransactionHandle { - String instance = {}; -}; - -void to_json(json& j, const TpchTransactionHandle& p); - -void from_json(const json& j, TpchTransactionHandle& p); + bool operator<(const Lifespan& o) const { + return groupid < o.groupid; + } +}; + +void to_json(json& j, const Lifespan& p); +void from_json(const json& j, Lifespan& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { @@ -650,75 +563,6 @@ void to_json(json& j, const TaskSource& p); void from_json(const json& j, TaskSource& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TableHandle { - ConnectorId connectorId = {}; - std::shared_ptr connectorHandle = {}; - std::shared_ptr transaction = {}; - std::shared_ptr connectorTableLayout = {}; -}; -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 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 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 TableWriteInfo { - std::shared_ptr writerTarget = {}; - std::shared_ptr analyzeTableHandle = {}; - std::shared_ptr deleteScanInfo = {}; -}; -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 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 TaskUpdateRequest { - SessionRepresentation session = {}; - Map extraCredentials = {}; - std::shared_ptr fragment = {}; - List sources = {}; - OutputBuffers outputIds = {}; - std::shared_ptr tableWriteInfo = {}; -}; -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 Order { ASCENDING, DESCENDING }; extern void to_json(json& j, const Order& e); extern void from_json(const json& j, Order& e); @@ -747,69 +591,6 @@ struct HiveBucketProperty { 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 StorageFormat { - String serDe = {}; - String inputFormat = {}; - String outputFormat = {}; -}; -void to_json(json& j, const StorageFormat& p); -void from_json(const json& j, StorageFormat& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct Storage { - StorageFormat storageFormat = {}; - String location = {}; - std::shared_ptr bucketProperty = {}; - bool skewed = {}; - Map serdeParameters = {}; - Map parameters = {}; -}; -void to_json(json& j, const Storage& p); -void from_json(const json& j, Storage& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class 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 { -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 { -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 ColumnIdentity& p); -void from_json(const json& j, ColumnIdentity& 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"); * you may not use this file except in compliance with the License. @@ -824,65 +605,53 @@ extern void from_json(const json& j, ColumnType& e); * 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; +using HostAddress = std::string; - bool operator<(const ColumnHandle& o) const override { - return columnIdentity.name < - dynamic_cast(o).columnIdentity.name; - } +} // 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 IcebergColumnHandle& p); -void from_json(const json& j, IcebergColumnHandle& 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 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); +enum class ColumnType { PARTITION_KEY, REGULAR, SYNTHESIZED, AGGREGATED }; +extern void to_json(json& j, const ColumnType& e); +extern void from_json(const json& j, ColumnType& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct 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 = {}; +struct CallExpression : public RowExpression { + String displayName = {}; + std::shared_ptr functionHandle = {}; + Type returnType = {}; + List> arguments = {}; + + CallExpression() noexcept; }; -void to_json(json& j, const DistributionSnapshot& p); -void from_json(const json& j, DistributionSnapshot& p); +void to_json(json& j, const CallExpression& p); +void from_json(const json& j, CallExpression& 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 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 /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -897,135 +666,79 @@ extern void from_json(const json& j, BlockedReason& e); * See the License for the specific language governing permissions and * limitations under the License. */ + +// HiveColumnHandle is special since it needs an implementation of +// operator<(). + namespace facebook::presto::protocol { -struct OperatorInfo {}; -void to_json(json& j, const OperatorInfo& p); -void from_json(const json& j, OperatorInfo& p); + +struct HiveColumnHandle : public ColumnHandle { + String name = {}; + HiveType hiveType = {}; + TypeSignature typeSignature = {}; + int hiveColumnIndex = {}; + ColumnType columnType = {}; + std::shared_ptr comment = {}; + List requiredSubfields = {}; + std::shared_ptr partialAggregation = {}; + + HiveColumnHandle() noexcept; + + 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 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 BucketConversion { + int tableBucketCount = {}; + int partitionBucketCount = {}; + List bucketColumnHandles = {}; }; -void to_json(json& j, const OperatorStats& p); -void from_json(const json& j, OperatorStats& 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 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 StorageFormat { + String serDe = {}; + String inputFormat = {}; + String outputFormat = {}; }; -void to_json(json& j, const DriverStats& p); -void from_json(const json& j, DriverStats& p); +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 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 Storage { + StorageFormat storageFormat = {}; + String location = {}; + std::shared_ptr bucketProperty = {}; + bool skewed = {}; + Map serdeParameters = {}; + Map parameters = {}; }; -void to_json(json& j, const PipelineStats& p); -void from_json(const json& j, PipelineStats& p); +void to_json(json& j, const Storage& p); +void from_json(const json& j, Storage& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class Bound { BELOW, EXACTLY, ABOVE }; -extern void to_json(json& j, const Bound& e); -extern void from_json(const json& j, Bound& e); +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 NodeSelectionStrategy { + HARD_AFFINITY, + SOFT_AFFINITY, + NO_PREFERENCE +}; +extern void to_json(json& j, const NodeSelectionStrategy& e); +extern void from_json(const json& j, NodeSelectionStrategy& e); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -1040,33 +753,30 @@ extern void from_json(const json& j, Bound& e); * See the License for the specific language governing permissions and * limitations under the License. */ + namespace facebook::presto::protocol { -struct Block { - std::string data; -}; +struct Column { + String name; + String type; -void to_json(json& j, const Block& p); + Column() = default; + explicit Column(const String& str) { + name = str; + } +}; -void from_json(const json& j, Block& 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 Marker { - Type type = {}; - std::shared_ptr valueBlock = {}; - Bound bound = {}; +struct TableToPartitionMapping { + std::shared_ptr> tableToPartitionColumns = {}; + Map partitionSchemaDifference = {}; }; -void to_json(json& j, const Marker& p); -void from_json(const json& j, Marker& 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); +void to_json(json& j, const TableToPartitionMapping& p); +void from_json(const json& j, TableToPartitionMapping& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -1083,157 +793,128 @@ void from_json(const json& j, SourceLocation& p); */ 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; - } -}; - -void to_json(json& j, const VariableReferenceExpression& p); -void from_json(const json& j, VariableReferenceExpression& p); +std::ostream& operator<<(std::ostream& os, const DataSize& d); -std::string json_map_key( - const facebook::presto::protocol::VariableReferenceExpression& p); +void to_json(nlohmann::json& j, const DataSize& p); +void from_json(const nlohmann::json& j, DataSize& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class SortOrder { - ASC_NULLS_FIRST, - ASC_NULLS_LAST, - DESC_NULLS_FIRST, - DESC_NULLS_LAST +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 = {}; }; -extern void to_json(json& j, const SortOrder& e); -extern void from_json(const json& j, SortOrder& e); +void to_json(json& j, const CacheQuotaRequirement& p); +void from_json(const json& j, CacheQuotaRequirement& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Ordering { - VariableReferenceExpression variable = {}; - SortOrder sortOrder = {}; +struct DwrfEncryptionMetadata { + Map fieldToKeyData = {}; + Map extraMetadata = {}; + String encryptionAlgorithm = {}; + String encryptionProvider = {}; }; -void to_json(json& j, const Ordering& p); -void from_json(const json& j, Ordering& p); +void to_json(json& j, const DwrfEncryptionMetadata& p); +void from_json(const json& j, DwrfEncryptionMetadata& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct OrderingScheme { - List orderBy = {}; +struct EncryptionInformation { + std::shared_ptr dwrfEncryptionMetadata = {}; }; -void to_json(json& j, const OrderingScheme& p); -void from_json(const json& j, OrderingScheme& 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 { -struct CallExpression : public RowExpression { - String displayName = {}; - std::shared_ptr functionHandle = {}; - Type returnType = {}; - List> arguments = {}; +struct HiveSplit : public ConnectorSplit { + HiveFileSplit fileSplit = {}; + String database = {}; + String table = {}; + String partitionName = {}; + Storage storage = {}; + List partitionKeys = {}; + List addresses = {}; + std::shared_ptr readBucketNumber = {}; + std::shared_ptr tableBucketNumber = {}; + NodeSelectionStrategy nodeSelectionStrategy = {}; + int partitionDataColumnCount = {}; + TableToPartitionMapping tableToPartitionMapping = {}; + std::shared_ptr bucketConversion = {}; + bool s3SelectPushdownEnabled = {}; + CacheQuotaRequirement cacheQuota = {}; + std::shared_ptr encryptionMetadata = {}; + List> redundantColumnDomains = {}; + SplitWeight splitWeight = {}; + std::shared_ptr rowIdPartitionComponent = {}; - CallExpression() noexcept; + HiveSplit() noexcept; }; -void to_json(json& j, const CallExpression& p); -void from_json(const json& j, CallExpression& p); +void to_json(json& j, const HiveSplit& p); +void from_json(const json& j, HiveSplit& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct 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 -/* - * 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. - */ - -// HiveColumnHandle is special since it needs an implementation of -// operator<(). - namespace facebook::presto::protocol { - -struct HiveColumnHandle : public ColumnHandle { - String name = {}; - HiveType hiveType = {}; - TypeSignature typeSignature = {}; - int hiveColumnIndex = {}; - ColumnType columnType = {}; - std::shared_ptr comment = {}; - List requiredSubfields = {}; - std::shared_ptr partialAggregation = {}; - - HiveColumnHandle() noexcept; - - bool operator<(const ColumnHandle& o) const override { - return name < dynamic_cast(o).name; - } +struct PageBufferInfo { + int partition = {}; + int64_t bufferedPages = {}; + int64_t bufferedBytes = {}; + int64_t rowsAdded = {}; + int64_t pagesAdded = {}; }; - -void to_json(json& j, const HiveColumnHandle& p); -void from_json(const json& j, HiveColumnHandle& 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 BucketConversion { - int tableBucketCount = {}; - int partitionBucketCount = {}; - List bucketColumnHandles = {}; +struct BufferInfo { + OutputBufferId bufferId = {}; + bool finished = {}; + int bufferedPages = {}; + int64_t pagesSent = {}; + PageBufferInfo pageBufferInfo = {}; }; -void to_json(json& j, const BucketConversion& p); -void from_json(const json& j, BucketConversion& 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 TpchTableHandle : public ConnectorTableHandle { - String tableName = {}; - double scaleFactor = {}; - - TpchTableHandle() noexcept; +struct OutputBufferInfo { + String type = {}; + BufferState state = {}; + bool canAddBuffers = {}; + bool canAddPages = {}; + int64_t totalBufferedBytes = {}; + int64_t totalBufferedPages = {}; + int64_t totalRowsSent = {}; + int64_t totalPagesSent = {}; + List buffers = {}; }; -void to_json(json& j, const TpchTableHandle& p); -void from_json(const json& j, TpchTableHandle& p); +void to_json(json& j, const OutputBufferInfo& p); +void from_json(const json& j, OutputBufferInfo& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct MarkDistinctNode : public PlanNode { - std::shared_ptr source = {}; - VariableReferenceExpression markerVariable = {}; - List distinctVariables = {}; - std::shared_ptr hashVariable = {}; +struct SystemTableHandle : public ConnectorTableHandle { + ConnectorId connectorId = {}; + String schemaName = {}; + String tableName = {}; - MarkDistinctNode() noexcept; + SystemTableHandle() noexcept; }; -void to_json(json& j, const MarkDistinctNode& p); -void from_json(const json& j, MarkDistinctNode& p); +void to_json(json& j, const SystemTableHandle& p); +void from_json(const json& j, SystemTableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct Domain { @@ -1392,6 +1073,42 @@ struct adl_serializer< }; } // namespace nlohmann +namespace facebook::presto::protocol { +struct SystemSplit : public ConnectorSplit { + ConnectorId connectorId = {}; + SystemTableHandle tableHandle = {}; + List addresses = {}; + TupleDomain> constraint = {}; + + SystemSplit() noexcept; +}; +void to_json(json& j, const SystemSplit& p); +void from_json(const json& j, SystemSplit& 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 Block { + std::string data; +}; + +void to_json(json& j, const Block& p); + +void from_json(const json& j, Block& 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. @@ -1429,87 +1146,40 @@ void to_json(json& j, const EquatableValueSet& p); void from_json(const json& j, EquatableValueSet& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Specification { - List partitionBy = {}; - std::shared_ptr orderingScheme = {}; +struct SchemaTableName { + String schema = {}; + String table = {}; }; -void to_json(json& j, const Specification& p); -void from_json(const json& j, Specification& 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 WindowType { - RANGE, - ROWS, - GROUPS, -}; -extern void to_json(json& j, const WindowType& e); -extern void from_json(const json& j, WindowType& e); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class BoundType { - UNBOUNDED_PRECEDING, - PRECEDING, - CURRENT_ROW, - FOLLOWING, - UNBOUNDED_FOLLOWING -}; -extern void to_json(json& j, const BoundType& e); -extern void from_json(const json& j, BoundType& e); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct Frame { - WindowType type = {}; - BoundType startType = {}; - std::shared_ptr startValue = {}; - std::shared_ptr - sortKeyCoercedForFrameStartComparison = {}; - BoundType endType = {}; - std::shared_ptr endValue = {}; - std::shared_ptr - sortKeyCoercedForFrameEndComparison = {}; - std::shared_ptr originalStartValue = {}; - std::shared_ptr originalEndValue = {}; -}; -void to_json(json& j, const Frame& p); -void from_json(const json& j, Frame& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct Function { - CallExpression functionCall = {}; - Frame frame = {}; - bool ignoreNulls = {}; -}; -void to_json(json& j, const Function& p); -void from_json(const json& j, Function& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct WindowNode : public PlanNode { - std::shared_ptr sourceLocation = {}; - - std::shared_ptr source = {}; - Specification specification = {}; - Map windowFunctions = {}; - std::shared_ptr hashVariable = {}; - List prePartitionedInputs = {}; - int preSortedOrderPrefix = {}; - - WindowNode() noexcept; +enum class PrestoTableType { + MANAGED_TABLE, + EXTERNAL_TABLE, + VIRTUAL_VIEW, + MATERIALIZED_VIEW, + TEMPORARY_TABLE, + OTHER }; -void to_json(json& j, const WindowNode& p); -void from_json(const json& j, WindowNode& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class JoinType { INNER, LEFT, RIGHT, FULL }; -extern void to_json(json& j, const JoinType& e); -extern void from_json(const json& j, JoinType& e); +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 EquiJoinClause { - VariableReferenceExpression left = {}; - VariableReferenceExpression right = {}; +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 EquiJoinClause& p); -void from_json(const json& j, EquiJoinClause& p); +void to_json(json& j, const Table& p); +void from_json(const json& j, Table& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -1524,79 +1194,45 @@ void from_json(const json& j, EquiJoinClause& p); * See the License for the specific language governing permissions and * limitations under the License. */ -namespace facebook::presto::protocol { -struct MergeJoinNode : public PlanNode { - MergeJoinNode() noexcept; - PlanNodeId id = {}; - 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 = {}; -}; -void to_json(json& j, const MergeJoinNode& p); -void from_json(const json& j, MergeJoinNode& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct 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 ValuesNode : public PlanNode { - std::shared_ptr location = {}; - List outputVariables = {}; - List>> rows = {}; - std::shared_ptr valuesNodeLabel = {}; +// dependency Table +// dependency SchemaTableName - 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 { -struct GroupIdNode : public PlanNode { - std::shared_ptr source = {}; - List> groupingSets = {}; - Map - groupingColumns = {}; - List aggregationArguments = {}; - VariableReferenceExpression groupIdVariable = {}; - GroupIdNode() noexcept; +struct HivePageSinkMetadata { + SchemaTableName schemaTableName = {}; + std::shared_ptr table = {}; + // TODO Add modifiedPartitions }; -void to_json(json& j, const GroupIdNode& p); -void from_json(const json& j, GroupIdNode& p); +void to_json(json& j, const HivePageSinkMetadata& p); +void from_json(const json& j, HivePageSinkMetadata& p); + } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HiveTransactionHandle : public ConnectorTransactionHandle { - UUID uuid = {}; - - HiveTransactionHandle() noexcept; -}; -void to_json(json& j, const HiveTransactionHandle& p); -void from_json(const json& j, HiveTransactionHandle& p); +enum class 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 CacheQuotaScope { GLOBAL, SCHEMA, TABLE, PARTITION }; -extern void to_json(json& j, const CacheQuotaScope& e); -extern void from_json(const json& j, CacheQuotaScope& e); +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 CacheQuotaRequirement { - CacheQuotaScope cacheQuotaScope = {}; - std::shared_ptr quota = {}; +struct LocationHandle { + String targetPath = {}; + String writePath = {}; + std::shared_ptr tempPath = {}; + TableType tableType = {}; + WriteMode writeMode = {}; }; -void to_json(json& j, const CacheQuotaRequirement& p); -void from_json(const json& j, CacheQuotaRequirement& p); +void to_json(json& j, const LocationHandle& p); +void from_json(const json& j, LocationHandle& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -1614,72 +1250,134 @@ void from_json(const json& j, CacheQuotaRequirement& p); namespace facebook::presto::protocol { -using HostAddress = std::string; +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 { -struct TpchSplit : public ConnectorSplit { - TpchTableHandle tableHandle = {}; - int partNumber = {}; - int totalParts = {}; - List addresses = {}; - TupleDomain> predicate = {}; - - TpchSplit() noexcept; -}; -void to_json(json& j, const TpchSplit& p); -void from_json(const json& j, TpchSplit& p); +enum class HiveCompressionCodec { NONE, SNAPPY, GZIP, LZ4, ZSTD }; +extern void to_json(json& j, const HiveCompressionCodec& e); +extern void from_json(const json& j, HiveCompressionCodec& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Range { - Marker low = {}; - Marker high = {}; +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 Range& p); -void from_json(const json& j, Range& p); +void to_json(json& j, const HiveOutputTableHandle& p); +void from_json(const json& j, HiveOutputTableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class IcebergTableType { - DATA, - HISTORY, - SNAPSHOTS, - MANIFESTS, - PARTITIONS, - FILES, - PROPERTIES, - CHANGELOG, - EQUALITY_DELETES, - DATA_WITHOUT_EQUALITY_DELETES +struct PartialAggregationStatsEstimate { + double inputBytes = {}; + double outputBytes = {}; + double inputRowCount = {}; + double outputRowCount = {}; }; -extern void to_json(json& j, const IcebergTableType& e); -extern void from_json(const json& j, IcebergTableType& e); +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 IcebergTableName { - String tableName = {}; - IcebergTableType tableType = {}; - std::shared_ptr snapshotId = {}; - std::shared_ptr changelogEndSnapshot = {}; +struct VariableStatsEstimate { + double lowValue = {}; + double highValue = {}; + double nullsFraction = {}; + double averageRowSize = {}; + double distinctValuesCount = {}; + std::shared_ptr histogram = {}; }; -void to_json(json& j, const IcebergTableName& p); -void from_json(const json& j, IcebergTableName& p); +void to_json(json& j, const VariableStatsEstimate& p); +void from_json(const json& j, VariableStatsEstimate& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct IcebergTableHandle : public ConnectorTableHandle { - String schemaName = {}; - IcebergTableName icebergTableName = {}; - bool snapshotSpecified = {}; - TupleDomain predicate = {}; - std::shared_ptr outputPath = {}; - std::shared_ptr> storageProperties = {}; - std::shared_ptr tableSchemaJson = {}; - std::shared_ptr> partitionFieldIds = {}; - std::shared_ptr> equalityFieldIds = {}; +struct JoinNodeStatsEstimate { + double nullJoinBuildKeyCount = {}; + double joinBuildKeyCount = {}; + double nullJoinProbeKeyCount = {}; + double joinProbeKeyCount = {}; +}; +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 TableWriterNodeStatsEstimate { + double taskCountIfScaledWriter = {}; +}; +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 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); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct EmptySplit : public ConnectorSplit { + ConnectorId connectorId = {}; - IcebergTableHandle() noexcept; + EmptySplit() noexcept; }; -void to_json(json& j, const IcebergTableHandle& p); -void from_json(const json& j, IcebergTableHandle& 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 { +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 { +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 ColumnIdentity& p); +void from_json(const json& j, ColumnIdentity& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -1695,65 +1393,119 @@ void from_json(const json& j, IcebergTableHandle& p); * limitations under the License. */ +// IcebergColumnHandle is special since it needs an implementation of +// operator<(). + namespace facebook::presto::protocol { -struct Column { - String name; - String type; +struct IcebergColumnHandle : public ColumnHandle { + ColumnIdentity columnIdentity = {}; + Type type = {}; + std::shared_ptr comment = {}; + ColumnType columnType = {}; + List requiredSubfields = {}; - Column() = default; - explicit Column(const String& str) { - name = str; + IcebergColumnHandle() noexcept; + + bool operator<(const ColumnHandle& o) const override { + return columnIdentity.name < + dynamic_cast(o).columnIdentity.name; } }; -void to_json(json& j, const Column& p); -void from_json(const json& j, Column& 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 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 = {}; - - IcebergTableLayoutHandle() noexcept; +struct ChangelogSplitInfo { + ChangelogOperation operation = {}; + int64_t ordinal = {}; + int64_t snapshotId = {}; + List icebergColumns = {}; }; -void to_json(json& j, const IcebergTableLayoutHandle& p); -void from_json(const json& j, IcebergTableLayoutHandle& p); +void to_json(json& j, const ChangelogSplitInfo& p); +void from_json(const json& j, ChangelogSplitInfo& 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 FilterNode : public PlanNode { - std::shared_ptr source = {}; - std::shared_ptr predicate = {}; - FilterNode() noexcept; +struct TypeVariableConstraint { + String name = {}; + bool comparableRequired = {}; + bool orderableRequired = {}; + String variadicBound = {}; + bool nonDecimalNumericRequired = {}; + String boundedBy = {}; }; -void to_json(json& j, const FilterNode& p); -void from_json(const json& j, FilterNode& p); +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 StatisticAggregations { - List outputVariables = {}; - Map aggregations = {}; - List groupingVariables = {}; +enum class FunctionKind { SCALAR, AGGREGATE, WINDOW }; +extern void to_json(json& j, const FunctionKind& e); +extern void from_json(const json& j, FunctionKind& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct LongVariableConstraint { + String name = {}; + String expression = {}; }; -void to_json(json& j, const StatisticAggregations& p); -void from_json(const json& j, StatisticAggregations& 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 Assignments { - Map> assignments = - {}; +struct Signature { + QualifiedObjectName name = {}; + FunctionKind kind = {}; + List typeVariableConstraints = {}; + List longVariableConstraints = {}; + TypeSignature returnType = {}; + List argumentTypes = {}; + bool variableArity = {}; }; -void to_json(json& j, const Assignments& p); -void from_json(const json& j, Assignments& 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 TableHandle { + ConnectorId connectorId = {}; + std::shared_ptr connectorHandle = {}; + std::shared_ptr transaction = {}; + std::shared_ptr connectorTableLayout = {}; +}; +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 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 { +enum class ExchangeNodeType { + GATHER, + REPARTITION, + REPLICATE, +}; +extern void to_json(json& j, const ExchangeNodeType& e); +extern void from_json(const json& j, ExchangeNodeType& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct PartitioningHandle { @@ -1783,195 +1535,144 @@ struct PartitioningScheme { void to_json(json& j, const PartitioningScheme& p); void from_json(const json& j, PartitioningScheme& 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 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 = {}; - - 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 { -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 ExchangeNodeScope { LOCAL, REMOTE_STREAMING, REMOTE_MATERIALIZED }; +extern void to_json(json& j, const ExchangeNodeScope& e); +extern void from_json(const json& j, ExchangeNodeScope& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -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 ExchangeNode : public PlanNode { + ExchangeNodeType type = {}; + ExchangeNodeScope scope = {}; + PartitioningScheme partitioningScheme = {}; + List> sources = {}; + List> inputs = {}; + bool ensureSourceOrdering = {}; + std::shared_ptr orderingScheme = {}; - JoinNode() noexcept; + ExchangeNode() noexcept; }; -void to_json(json& j, const JoinNode& p); -void from_json(const json& j, JoinNode& p); +void to_json(json& j, const ExchangeNode& p); +void from_json(const json& j, ExchangeNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TableWriterNodeStatsEstimate { - double taskCountIfScaledWriter = {}; +struct DeleteHandle : public ExecutionWriterTarget { + TableHandle handle = {}; + SchemaTableName schemaTableName = {}; + + DeleteHandle() noexcept; }; -void to_json(json& j, const TableWriterNodeStatsEstimate& p); -void from_json(const json& j, TableWriterNodeStatsEstimate& p); +void to_json(json& j, const DeleteHandle& p); +void from_json(const json& j, DeleteHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct PartialAggregationStatsEstimate { - double inputBytes = {}; - double outputBytes = {}; - double inputRowCount = {}; - double outputRowCount = {}; +struct HiveBucketHandle { + List columns = {}; + int tableBucketCount = {}; + int readBucketCount = {}; }; -void to_json(json& j, const PartialAggregationStatsEstimate& p); -void from_json(const json& j, PartialAggregationStatsEstimate& p); +void to_json(json& j, const HiveBucketHandle& p); +void from_json(const json& j, HiveBucketHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct JoinNodeStatsEstimate { - double nullJoinBuildKeyCount = {}; - double joinBuildKeyCount = {}; - double nullJoinProbeKeyCount = {}; - double joinProbeKeyCount = {}; +struct Assignments { + Map> assignments = + {}; }; -void to_json(json& j, const JoinNodeStatsEstimate& p); -void from_json(const json& j, JoinNodeStatsEstimate& p); +void to_json(json& j, const Assignments& p); +void from_json(const json& j, Assignments& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct VariableStatsEstimate { - double lowValue = {}; - double highValue = {}; - double nullsFraction = {}; - double averageRowSize = {}; - double distinctValuesCount = {}; - std::shared_ptr histogram = {}; +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; }; -void to_json(json& j, const VariableStatsEstimate& p); -void from_json(const json& j, VariableStatsEstimate& p); +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 PlanNodeStatsEstimate { - double outputRowCount = {}; - double totalSize = {}; - bool confident = {}; - Map variableStatistics = - {}; - JoinNodeStatsEstimate joinNodeStatsEstimate = {}; - TableWriterNodeStatsEstimate tableWriterNodeStatsEstimate = {}; - PartialAggregationStatsEstimate partialAggregationStatsEstimate = {}; +struct HiveBucketFilter { + List bucketsToKeep = {}; }; -void to_json(json& j, const PlanNodeStatsEstimate& p); -void from_json(const json& j, PlanNodeStatsEstimate& p); +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 DistinctLimitNode : public PlanNode { - std::shared_ptr source = {}; - int64_t limit = {}; - bool partial = {}; - List distinctVariables = {}; - std::shared_ptr hashVariable = {}; - int timeoutMillis = {}; +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 = {}; - DistinctLimitNode() noexcept; + HiveTableLayoutHandle() noexcept; }; -void to_json(json& j, const DistinctLimitNode& p); -void from_json(const json& j, DistinctLimitNode& p); +void to_json(json& j, const HiveTableLayoutHandle& p); +void from_json(const json& j, HiveTableLayoutHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct SchemaTableName { - String schema = {}; - String table = {}; -}; -void to_json(json& j, const SchemaTableName& p); -void from_json(const json& j, SchemaTableName& p); +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 OutputTableHandle { - ConnectorId connectorId = {}; - std::shared_ptr transactionHandle = {}; - std::shared_ptr connectorHandle = {}; +struct SelectedRole { + SelectedRoleType type = {}; + std::shared_ptr role = {}; }; -void to_json(json& j, const OutputTableHandle& p); -void from_json(const json& j, OutputTableHandle& p); +void to_json(json& j, const SelectedRole& p); +void from_json(const json& j, SelectedRole& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct CreateHandle : public ExecutionWriterTarget { - OutputTableHandle handle = {}; - SchemaTableName schemaTableName = {}; - - CreateHandle() noexcept; +struct Location { + String location = {}; }; -void to_json(json& j, const CreateHandle& p); -void from_json(const json& j, CreateHandle& p); +void to_json(json& j, const Location& p); +void from_json(const json& j, Location& 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 it needs an implementation of -// operator<(). - namespace facebook::presto::protocol { -struct TpchColumnHandle : public ColumnHandle { - String columnName = {}; - Type type = {}; - - TpchColumnHandle() noexcept; +struct RemoteSplit : public ConnectorSplit { + Location location = {}; + TaskId remoteSourceTaskId = {}; - bool operator<(const ColumnHandle& o) const override { - return columnName < dynamic_cast(o).columnName; - } + RemoteSplit() noexcept; }; -void to_json(json& j, const TpchColumnHandle& p); -void from_json(const json& j, TpchColumnHandle& 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 GroupingSetDescriptor { - List groupingKeys = {}; - int groupingSetCount = {}; - List globalGroupingSets = {}; +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 GroupingSetDescriptor& p); -void from_json(const json& j, GroupingSetDescriptor& 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 StageExecutionStrategy { @@ -1993,419 +1694,475 @@ void to_json(json& j, const StageExecutionDescriptor& p); void from_json(const json& j, StageExecutionDescriptor& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Location { - String location = {}; +struct PlanCostEstimate { + double cpuCost = {}; + double maxMemory = {}; + double maxMemoryWhenOutputting = {}; + double networkCost = {}; }; -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 TableType { NEW, EXISTING, TEMPORARY }; -extern void to_json(json& j, const TableType& e); -extern void from_json(const json& j, TableType& e); +void to_json(json& j, const PlanCostEstimate& p); +void from_json(const json& j, PlanCostEstimate& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class WriteMode { - STAGE_AND_MOVE_TO_TARGET_DIRECTORY, - DIRECT_TO_TARGET_NEW_DIRECTORY, - DIRECT_TO_TARGET_EXISTING_DIRECTORY +struct StatsAndCosts { + Map stats = {}; + Map costs = {}; }; -extern void to_json(json& j, const WriteMode& e); -extern void from_json(const json& j, WriteMode& e); +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"); + * 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 LocationHandle { - String targetPath = {}; - String writePath = {}; - std::shared_ptr tempPath = {}; - TableType tableType = {}; - WriteMode writeMode = {}; +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 LocationHandle& p); -void from_json(const json& j, LocationHandle& p); +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 AllOrNoneValueSet : public ValueSet { - Type type = {}; - bool all = {}; +struct OutputNode : public PlanNode { + std::shared_ptr source = {}; + List columnNames = {}; + List outputVariables = {}; - AllOrNoneValueSet() noexcept; + OutputNode() noexcept; }; -void to_json(json& j, const AllOrNoneValueSet& p); -void from_json(const json& j, AllOrNoneValueSet& 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 HiveFileSplit { - String path = {}; - int64_t start = {}; - int64_t length = {}; - int64_t fileSize = {}; - int64_t fileModifiedTime = {}; - std::shared_ptr extraFileInfo = {}; - Map customSplitInfo = {}; +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 +namespace facebook::presto::protocol { +struct Marker { + Type type = {}; + std::shared_ptr valueBlock = {}; + Bound bound = {}; }; -void to_json(json& j, const HiveFileSplit& p); -void from_json(const json& j, HiveFileSplit& p); +void to_json(json& j, const Marker& p); +void from_json(const json& j, Marker& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct UniformDistributionHistogram : public ConnectorHistogram { - double lowValue = {}; - double highValue = {}; - - UniformDistributionHistogram() noexcept; +struct Range { + Marker low = {}; + Marker high = {}; }; -void to_json(json& j, const UniformDistributionHistogram& p); -void from_json(const json& j, UniformDistributionHistogram& 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 { -enum class LimitNodeStep { PARTIAL, FINAL }; -extern void to_json(json& j, const LimitNodeStep& e); -extern void from_json(const json& j, LimitNodeStep& e); +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 LimitNode : public PlanNode { - std::shared_ptr source = {}; - int64_t count = {}; - LimitNodeStep step = {}; +struct TableScanNode : public PlanNode { + TableHandle table = {}; + List outputVariables = {}; + Map> assignments = + {}; - LimitNode() noexcept; + TableScanNode() noexcept; }; -void to_json(json& j, const LimitNode& p); -void from_json(const json& j, LimitNode& p); +void to_json(json& j, const TableScanNode& p); +void from_json(const json& j, TableScanNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct PageBufferInfo { - int partition = {}; - int64_t bufferedPages = {}; - int64_t bufferedBytes = {}; - int64_t rowsAdded = {}; - int64_t pagesAdded = {}; +struct InsertTableHandle { + ConnectorId connectorId = {}; + std::shared_ptr transactionHandle = {}; + std::shared_ptr connectorHandle = {}; }; -void to_json(json& j, const PageBufferInfo& p); -void from_json(const json& j, PageBufferInfo& p); +void to_json(json& j, const InsertTableHandle& p); +void from_json(const json& j, InsertTableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct BufferInfo { - OutputBufferId bufferId = {}; - bool finished = {}; - int bufferedPages = {}; - int64_t pagesSent = {}; - PageBufferInfo pageBufferInfo = {}; +struct RefreshMaterializedViewHandle { + InsertTableHandle handle = {}; + SchemaTableName schemaTableName = {}; }; -void to_json(json& j, const BufferInfo& p); -void from_json(const json& j, BufferInfo& 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 BufferState { - OPEN, - NO_MORE_BUFFERS, - NO_MORE_PAGES, - FLUSHING, - FINISHED, - FAILED +enum class WindowType { + RANGE, + ROWS, + GROUPS, }; -extern void to_json(json& j, const BufferState& e); -extern void from_json(const json& j, BufferState& e); +extern void to_json(json& j, const WindowType& e); +extern void from_json(const json& j, WindowType& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct 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 BoundType { + UNBOUNDED_PRECEDING, + PRECEDING, + CURRENT_ROW, + FOLLOWING, + UNBOUNDED_FOLLOWING }; -void to_json(json& j, const OutputBufferInfo& p); -void from_json(const json& j, OutputBufferInfo& p); +extern void to_json(json& j, const BoundType& e); +extern void from_json(const json& j, BoundType& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -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 Frame { + WindowType type = {}; + BoundType startType = {}; + std::shared_ptr startValue = {}; + std::shared_ptr + sortKeyCoercedForFrameStartComparison = {}; + BoundType endType = {}; + std::shared_ptr endValue = {}; + std::shared_ptr + sortKeyCoercedForFrameEndComparison = {}; + std::shared_ptr originalStartValue = {}; + std::shared_ptr originalEndValue = {}; +}; +void to_json(json& j, const Frame& p); +void from_json(const json& j, Frame& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class ErrorCause { - UNKNOWN, - LOW_PARTITION_COUNT, - EXCEEDS_BROADCAST_MEMORY_LIMIT +struct Function { + CallExpression functionCall = {}; + Frame frame = {}; + bool ignoreNulls = {}; }; -extern void to_json(json& j, const ErrorCause& e); -extern void from_json(const json& j, ErrorCause& e); +void to_json(json& j, const Function& p); +void from_json(const json& j, Function& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class ErrorType { - USER_ERROR, - INTERNAL_ERROR, - INSUFFICIENT_RESOURCES, - EXTERNAL +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; }; -extern void to_json(json& j, const ErrorType& e); -extern void from_json(const json& j, ErrorType& e); +void to_json(json& j, const WindowNode& p); +void from_json(const json& j, WindowNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ErrorCode { - int code = {}; - String name = {}; - ErrorType type = {}; - bool retriable = {}; +struct DeleteScanInfo { + PlanNodeId id = {}; + TableHandle tableHandle = {}; }; -void to_json(json& j, const ErrorCode& p); -void from_json(const json& j, ErrorCode& p); +void to_json(json& j, const DeleteScanInfo& p); +void from_json(const json& j, DeleteScanInfo& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ErrorLocation { - int lineNumber = {}; - int columnNumber = {}; +struct AnalyzeTableHandle { + ConnectorId connectorId = {}; + std::shared_ptr transactionHandle = {}; + std::shared_ptr connectorHandle = {}; }; -void to_json(json& j, const ErrorLocation& p); -void from_json(const json& j, ErrorLocation& p); +void to_json(json& j, const AnalyzeTableHandle& p); +void from_json(const json& j, AnalyzeTableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ExecutionFailureInfo { - String type = {}; - String message = {}; - std::shared_ptr cause = {}; - List suppressed = {}; - List stack = {}; - ErrorLocation errorLocation = {}; - ErrorCode errorCode = {}; - HostAddress remoteHost = {}; - ErrorCause errorCause = {}; +struct TableWriteInfo { + std::shared_ptr writerTarget = {}; + std::shared_ptr analyzeTableHandle = {}; + std::shared_ptr deleteScanInfo = {}; }; -void to_json(json& j, const ExecutionFailureInfo& p); -void from_json(const json& j, ExecutionFailureInfo& p); +void to_json(json& j, const TableWriteInfo& p); +void from_json(const json& j, TableWriteInfo& 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 TaskStatus { - int64_t taskInstanceIdLeastSignificantBits = {}; - int64_t taskInstanceIdMostSignificantBits = {}; - int64_t version = {}; - TaskState state = {}; - URI self = {}; - List completedDriverGroups = {}; - List failures = {}; - int queuedPartitionedDrivers = {}; - int runningPartitionedDrivers = {}; - double outputBufferUtilization = {}; - bool outputBufferOverutilized = {}; - int64_t physicalWrittenDataSizeInBytes = {}; - int64_t memoryReservationInBytes = {}; - int64_t systemMemoryReservationInBytes = {}; - int64_t peakNodeTotalMemoryReservationInBytes = {}; - int64_t fullGcCount = {}; - int64_t fullGcTimeInMillis = {}; - int64_t totalCpuTimeInNanos = {}; - int64_t taskAgeInMillis = {}; - int64_t queuedPartitionedSplitsWeight = {}; - int64_t runningPartitionedSplitsWeight = {}; -}; -void to_json(json& j, const TaskStatus& p); -void from_json(const json& j, TaskStatus& p); + +std::ostream& operator<<(std::ostream& os, const Duration& d); + +void to_json(json& j, const Duration& p); +void from_json(const json& j, Duration& p); + } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct MetadataUpdates { - ConnectorId connectorId = {}; - List> metadataUpdates = {}; +struct ResourceEstimates { + std::shared_ptr executionTime = {}; + std::shared_ptr cpuTime = {}; + std::shared_ptr peakMemory = {}; + std::shared_ptr peakTaskMemory = {}; }; -void to_json(json& j, const MetadataUpdates& p); -void from_json(const json& j, MetadataUpdates& p); +void to_json(json& j, const ResourceEstimates& p); +void from_json(const json& j, ResourceEstimates& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct 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 = {}; +struct Parameter { + String name = {}; + TypeSignature type = {}; }; -void to_json(json& j, const TaskStats& p); -void from_json(const json& j, TaskStats& 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 TaskInfo { - TaskId taskId = {}; - TaskStatus taskStatus = {}; - DateTime lastHeartbeat = {}; - OutputBufferInfo outputBuffers = {}; - List noMoreSplits = {}; - TaskStats stats = {}; - bool needsPlan = {}; - MetadataUpdates metadataUpdates = {}; - String nodeId = {}; +enum class Determinism { + DETERMINISTIC, + NOT_DETERMINISTIC, }; -void to_json(json& j, const TaskInfo& p); -void from_json(const json& j, TaskInfo& 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 MemoryAllocation { - String tag = {}; - int64_t allocation = {}; -}; -void to_json(json& j, const MemoryAllocation& p); -void from_json(const json& j, MemoryAllocation& p); +enum class NullCallClause { RETURNS_NULL_ON_NULL_INPUT, CALLED_ON_NULL_INPUT }; +extern void to_json(json& j, const NullCallClause& e); +extern void from_json(const json& j, NullCallClause& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct MemoryPoolInfo { - int64_t maxBytes = {}; - int64_t reservedBytes = {}; - int64_t reservedRevocableBytes = {}; - Map queryMemoryReservations = {}; - Map> queryMemoryAllocations = {}; - Map queryMemoryRevocableReservations = {}; +struct Language { + String language = {}; }; -void to_json(json& j, const MemoryPoolInfo& p); -void from_json(const json& j, MemoryPoolInfo& p); +void to_json(json& j, const Language& p); +void from_json(const json& j, Language& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct EmptySplit : public ConnectorSplit { - ConnectorId connectorId = {}; - - EmptySplit() noexcept; +struct RoutineCharacteristics { + std::shared_ptr language = {}; + std::shared_ptr determinism = {}; + std::shared_ptr nullCallClause = {}; }; -void to_json(json& j, const EmptySplit& p); -void from_json(const json& j, EmptySplit& 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 { -struct NodeVersion { - String version = {}; +struct SqlInvokedFunction { + List parameters = {}; + String description = {}; + RoutineCharacteristics routineCharacteristics = {}; + String body = {}; + Signature signature = {}; + SqlFunctionId functionId = {}; }; -void to_json(json& j, const NodeVersion& p); -void from_json(const json& j, NodeVersion& p); +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 ServerInfo { - NodeVersion nodeVersion = {}; - String environment = {}; - bool coordinator = {}; - bool starting = {}; - std::shared_ptr uptime = {}; +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 ServerInfo& p); -void from_json(const json& j, ServerInfo& 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 HiveBucketFilter { - List bucketsToKeep = {}; +enum class BufferType { + PARTITIONED, + BROADCAST, + ARBITRARY, + DISCARDING, + SPOOLING }; -void to_json(json& j, const HiveBucketFilter& p); -void from_json(const json& j, HiveBucketFilter& p); +extern void to_json(json& j, const BufferType& e); +extern void from_json(const json& j, BufferType& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HiveBucketHandle { - List columns = {}; - int tableBucketCount = {}; - int readBucketCount = {}; +struct OutputBuffers { + BufferType type = {}; + int64_t version = {}; + bool noMoreBufferIds = {}; + Map buffers = {}; }; -void to_json(json& j, const HiveBucketHandle& p); -void from_json(const json& j, HiveBucketHandle& p); +void to_json(json& j, const OutputBuffers& p); +void from_json(const json& j, OutputBuffers& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -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; +struct TaskUpdateRequest { + SessionRepresentation session = {}; + Map extraCredentials = {}; + std::shared_ptr fragment = {}; + List sources = {}; + OutputBuffers outputIds = {}; + std::shared_ptr tableWriteInfo = {}; }; -void to_json(json& j, const HiveTableLayoutHandle& p); -void from_json(const json& j, HiveTableLayoutHandle& p); +void to_json(json& j, const TaskUpdateRequest& p); +void from_json(const json& j, TaskUpdateRequest& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct OutputNode : public PlanNode { +struct SortNode : public PlanNode { std::shared_ptr source = {}; - List columnNames = {}; - List outputVariables = {}; + OrderingScheme orderingScheme = {}; + bool isPartial = {}; - OutputNode() noexcept; + SortNode() noexcept; }; -void to_json(json& j, const OutputNode& p); -void from_json(const json& j, OutputNode& 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 RowNumberNode : public PlanNode { - std::shared_ptr source = {}; - List partitionBy = {}; - VariableReferenceExpression rowNumberVariable = {}; - std::shared_ptr maxRowCountPerPartition = {}; - bool partial = {}; - std::shared_ptr hashVariable = {}; - - RowNumberNode() noexcept; +enum class ErrorCause { + UNKNOWN, + LOW_PARTITION_COUNT, + EXCEEDS_BROADCAST_MEMORY_LIMIT }; -void to_json(json& j, const RowNumberNode& p); -void from_json(const json& j, RowNumberNode& 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 { -struct PlanCostEstimate { - double cpuCost = {}; - double maxMemory = {}; - double maxMemoryWhenOutputting = {}; - double networkCost = {}; +struct ErrorLocation { + int lineNumber = {}; + int columnNumber = {}; }; -void to_json(json& j, const PlanCostEstimate& p); -void from_json(const json& j, PlanCostEstimate& p); +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 StatsAndCosts { - Map stats = {}; - Map costs = {}; +enum class ErrorType { + USER_ERROR, + INTERNAL_ERROR, + INSUFFICIENT_RESOURCES, + EXTERNAL }; -void to_json(json& j, const StatsAndCosts& p); -void from_json(const json& j, StatsAndCosts& 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 ErrorCode { + int code = {}; + String name = {}; + ErrorType type = {}; + bool retriable = {}; +}; +void to_json(json& j, const ErrorCode& p); +void from_json(const json& j, ErrorCode& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct ExecutionFailureInfo { + String type = {}; + String message = {}; + std::shared_ptr cause = {}; + List suppressed = {}; + List stack = {}; + ErrorLocation errorLocation = {}; + ErrorCode errorCode = {}; + HostAddress remoteHost = {}; + ErrorCause errorCause = {}; +}; +void to_json(json& j, const ExecutionFailureInfo& p); +void from_json(const json& j, ExecutionFailureInfo& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +enum class TaskState { PLANNED, RUNNING, FINISHED, CANCELED, ABORTED, FAILED }; +extern void to_json(json& j, const TaskState& e); +extern void from_json(const json& j, TaskState& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct TaskStatus { + int64_t taskInstanceIdLeastSignificantBits = {}; + int64_t taskInstanceIdMostSignificantBits = {}; + int64_t version = {}; + TaskState state = {}; + URI self = {}; + List completedDriverGroups = {}; + List failures = {}; + int queuedPartitionedDrivers = {}; + int runningPartitionedDrivers = {}; + double outputBufferUtilization = {}; + bool outputBufferOverutilized = {}; + int64_t physicalWrittenDataSizeInBytes = {}; + int64_t memoryReservationInBytes = {}; + int64_t systemMemoryReservationInBytes = {}; + int64_t peakNodeTotalMemoryReservationInBytes = {}; + int64_t fullGcCount = {}; + int64_t fullGcTimeInMillis = {}; + int64_t totalCpuTimeInNanos = {}; + int64_t taskAgeInMillis = {}; + int64_t queuedPartitionedSplitsWeight = {}; + int64_t runningPartitionedSplitsWeight = {}; +}; +void to_json(json& j, const TaskStatus& p); +void from_json(const json& j, TaskStatus& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +enum class 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 +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 = {}; +}; +void to_json(json& j, const DistributionSnapshot& p); +void from_json(const json& j, DistributionSnapshot& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -2420,113 +2177,200 @@ void from_json(const json& j, StatsAndCosts& p); * See the License for the specific language governing permissions and * limitations under the License. */ - -namespace facebook::presto::protocol { -struct PlanFragment { - PlanFragmentId id = {}; - std::shared_ptr root = {}; - List variables = {}; - PartitioningHandle partitioning = {}; - List tableScanSchedulingOrder = {}; - PartitioningScheme partitioningScheme = {}; - StageExecutionDescriptor stageExecutionDescriptor = {}; - bool outputTableWriterFragment = {}; - std::shared_ptr jsonRepresentation = {}; -}; -void to_json(json& j, const PlanFragment& p); -void from_json(const json& j, PlanFragment& p); -} // namespace facebook::presto::protocol namespace facebook::presto::protocol { -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 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 { -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); +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 DeleteFile { - FileContent content = {}; - String path = {}; - FileFormat format = {}; - int64_t recordCount = {}; - int64_t fileSizeInBytes = {}; - List equalityFieldIds = {}; - Map lowerBounds = {}; - Map upperBounds = {}; +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 DeleteFile& p); -void from_json(const json& j, DeleteFile& p); +void to_json(json& j, const OperatorStats& p); +void from_json(const json& j, OperatorStats& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HivePartitionKey { - String name = {}; - std::shared_ptr value = {}; +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 HivePartitionKey& p); -void from_json(const json& j, HivePartitionKey& 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 { -enum class NodeSelectionStrategy { - HARD_AFFINITY, - SOFT_AFFINITY, - NO_PREFERENCE +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 = {}; }; -extern void to_json(json& j, const NodeSelectionStrategy& e); -extern void from_json(const json& j, NodeSelectionStrategy& e); +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 IcebergSplit : public ConnectorSplit { - String path = {}; - int64_t start = {}; - int64_t length = {}; - FileFormat fileFormat = {}; - List addresses = {}; - Map partitionKeys = {}; - String partitionSpecAsJson = {}; - std::shared_ptr partitionDataJson = {}; - NodeSelectionStrategy nodeSelectionStrategy = {}; - SplitWeight splitWeight = {}; - List deletes = {}; - std::shared_ptr changelogSplitInfo = {}; - int64_t dataSequenceNumber = {}; - - IcebergSplit() noexcept; +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 IcebergSplit& p); -void from_json(const json& j, IcebergSplit& 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 RemoteSplit : public ConnectorSplit { - Location location = {}; - TaskId remoteSourceTaskId = {}; - - RemoteSplit() noexcept; +struct MetadataUpdates { + ConnectorId connectorId = {}; + List> metadataUpdates = {}; }; -void to_json(json& j, const RemoteSplit& p); -void from_json(const json& j, RemoteSplit& 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 InsertTableHandle { - ConnectorId connectorId = {}; - std::shared_ptr transactionHandle = {}; - std::shared_ptr connectorHandle = {}; +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 InsertTableHandle& p); -void from_json(const json& j, InsertTableHandle& p); +void to_json(json& j, const TaskInfo& p); +void from_json(const json& j, TaskInfo& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct InsertHandle : public ExecutionWriterTarget { @@ -2539,40 +2383,85 @@ 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 MemoryInfo { - DataSize totalNodeMemory = {}; - Map pools = {}; +struct OutputTableHandle { + ConnectorId connectorId = {}; + std::shared_ptr transactionHandle = {}; + std::shared_ptr connectorHandle = {}; }; -void to_json(json& j, const MemoryInfo& p); -void from_json(const json& j, MemoryInfo& p); +void to_json(json& j, const OutputTableHandle& p); +void from_json(const json& j, OutputTableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class PrestoTableType { - MANAGED_TABLE, - EXTERNAL_TABLE, - VIRTUAL_VIEW, - MATERIALIZED_VIEW, - TEMPORARY_TABLE, - OTHER +struct MemoryAllocation { + String tag = {}; + int64_t allocation = {}; }; -extern void to_json(json& j, const PrestoTableType& e); -extern void from_json(const json& j, PrestoTableType& e); +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 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 = {}; +enum class Form { + IF, + NULL_IF, + SWITCH, + WHEN, + IS_NULL, + COALESCE, + IN, + AND, + OR, + DEREFERENCE, + ROW_CONSTRUCTOR, + BIND }; -void to_json(json& j, const Table& p); -void from_json(const json& j, Table& p); +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 = {}; + + 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 HiveTransactionHandle : public ConnectorTransactionHandle { + UUID uuid = {}; + + HiveTransactionHandle() noexcept; +}; +void to_json(json& j, const HiveTransactionHandle& p); +void from_json(const json& j, HiveTransactionHandle& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct SystemColumnHandle : public ColumnHandle { + ConnectorId connectorId = {}; + String columnName = {}; + + SystemColumnHandle() noexcept; +}; +void to_json(json& j, const SystemColumnHandle& p); +void from_json(const json& j, SystemColumnHandle& 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 GroupingSetDescriptor { + List groupingKeys = {}; + int groupingSetCount = {}; + List globalGroupingSets = {}; +}; +void to_json(json& j, const GroupingSetDescriptor& p); +void from_json(const json& j, GroupingSetDescriptor& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -2588,136 +2477,168 @@ void from_json(const json& j, Table& p); * limitations under the License. */ -// dependency Table -// dependency SchemaTableName +// TpchColumnHandle is special since it needs an implementation of +// operator<(). namespace facebook::presto::protocol { +struct TpchColumnHandle : public ColumnHandle { + String columnName = {}; + Type type = {}; -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); + 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 namespace facebook::presto::protocol { -struct BatchTaskUpdateRequest { - TaskUpdateRequest taskUpdateRequest = {}; - std::shared_ptr shuffleWriteInfo = {}; - std::shared_ptr broadcastBasePath = {}; -}; -void to_json(json& j, const BatchTaskUpdateRequest& p); -void from_json(const json& j, BatchTaskUpdateRequest& p); +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 UpdateHandle { - TableHandle handle = {}; - SchemaTableName schemaTableName = {}; +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 { +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 UpdateHandle& p); -void from_json(const json& j, UpdateHandle& p); +void to_json(json& j, const DeleteFile& p); +void from_json(const json& j, DeleteFile& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct EnforceSingleRowNode : public PlanNode { - std::shared_ptr source = {}; - - EnforceSingleRowNode() noexcept; +struct StatisticAggregations { + List outputVariables = {}; + Map aggregations = {}; + List groupingVariables = {}; }; -void to_json(json& j, const EnforceSingleRowNode& p); -void from_json(const json& j, EnforceSingleRowNode& p); +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 DeleteHandle : public ExecutionWriterTarget { - TableHandle handle = {}; - SchemaTableName schemaTableName = {}; - - DeleteHandle() noexcept; +struct MemoryPoolInfo { + int64_t maxBytes = {}; + int64_t reservedBytes = {}; + int64_t reservedRevocableBytes = {}; + Map queryMemoryReservations = {}; + Map> queryMemoryAllocations = {}; + Map queryMemoryRevocableReservations = {}; }; -void to_json(json& j, const DeleteHandle& p); -void from_json(const json& j, DeleteHandle& 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 { -struct DwrfEncryptionMetadata { - Map fieldToKeyData = {}; - Map extraMetadata = {}; - String encryptionAlgorithm = {}; - String encryptionProvider = {}; +struct MemoryInfo { + DataSize totalNodeMemory = {}; + Map pools = {}; }; -void to_json(json& j, const DwrfEncryptionMetadata& p); -void from_json(const json& j, DwrfEncryptionMetadata& 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 EncryptionInformation { - std::shared_ptr dwrfEncryptionMetadata = {}; +struct NodeStatus { + String nodeId = {}; + NodeVersion nodeVersion = {}; + String environment = {}; + bool coordinator = {}; + Duration uptime = {}; + String externalAddress = {}; + String internalAddress = {}; + MemoryInfo memoryInfo = {}; + int processors = {}; + double processCpuLoad = {}; + double systemCpuLoad = {}; + int64_t heapUsed = {}; + int64_t heapAvailable = {}; + int64_t nonHeapUsed = {}; }; -void to_json(json& j, const EncryptionInformation& p); -void from_json(const json& j, EncryptionInformation& p); +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 TableToPartitionMapping { - std::shared_ptr> tableToPartitionColumns = {}; - Map partitionSchemaDifference = {}; +enum class IcebergTableType { + DATA, + HISTORY, + SNAPSHOTS, + MANIFESTS, + PARTITIONS, + FILES, + PROPERTIES, + CHANGELOG, + EQUALITY_DELETES, + DATA_WITHOUT_EQUALITY_DELETES }; -void to_json(json& j, const TableToPartitionMapping& p); -void from_json(const json& j, TableToPartitionMapping& 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 HiveSplit : public ConnectorSplit { - HiveFileSplit fileSplit = {}; - String database = {}; - String table = {}; - String partitionName = {}; - Storage storage = {}; - List partitionKeys = {}; - List addresses = {}; - std::shared_ptr readBucketNumber = {}; - std::shared_ptr tableBucketNumber = {}; - NodeSelectionStrategy nodeSelectionStrategy = {}; - int partitionDataColumnCount = {}; - TableToPartitionMapping tableToPartitionMapping = {}; - std::shared_ptr bucketConversion = {}; - bool s3SelectPushdownEnabled = {}; - CacheQuotaRequirement cacheQuota = {}; - std::shared_ptr encryptionMetadata = {}; - List> redundantColumnDomains = {}; - SplitWeight splitWeight = {}; - std::shared_ptr rowIdPartitionComponent = {}; +struct IcebergTableName { + String tableName = {}; + IcebergTableType tableType = {}; + std::shared_ptr snapshotId = {}; + std::shared_ptr changelogEndSnapshot = {}; +}; +void to_json(json& j, const IcebergTableName& p); +void from_json(const json& j, IcebergTableName& 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 = {}; - HiveSplit() noexcept; + SampleNode() noexcept; }; -void to_json(json& j, const HiveSplit& p); -void from_json(const json& j, HiveSplit& 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 { -enum class ExchangeNodeScope { LOCAL, REMOTE_STREAMING, REMOTE_MATERIALIZED }; -extern void to_json(json& j, const ExchangeNodeScope& e); -extern void from_json(const json& j, ExchangeNodeScope& e); +struct UniformDistributionHistogram : public ConnectorHistogram { + double lowValue = {}; + double highValue = {}; + + UniformDistributionHistogram() noexcept; +}; +void to_json(json& j, const UniformDistributionHistogram& p); +void from_json(const json& j, UniformDistributionHistogram& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class ExchangeNodeType { - GATHER, - REPARTITION, - REPLICATE, +struct SystemTableLayoutHandle : public ConnectorTableLayoutHandle { + ConnectorId connectorId = {}; + SystemTableHandle table = {}; + TupleDomain> constraint = {}; + + SystemTableLayoutHandle() noexcept; }; -extern void to_json(json& j, const ExchangeNodeType& e); -extern void from_json(const json& j, ExchangeNodeType& e); +void to_json(json& j, const SystemTableLayoutHandle& p); +void from_json(const json& j, SystemTableLayoutHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ExchangeNode : public PlanNode { - ExchangeNodeType type = {}; - ExchangeNodeScope scope = {}; - PartitioningScheme partitioningScheme = {}; - List> sources = {}; - List> inputs = {}; - bool ensureSourceOrdering = {}; - std::shared_ptr orderingScheme = {}; +struct TpchTableHandle : public ConnectorTableHandle { + String tableName = {}; + double scaleFactor = {}; - ExchangeNode() noexcept; + TpchTableHandle() noexcept; }; -void to_json(json& j, const ExchangeNode& p); -void from_json(const json& j, ExchangeNode& 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 UnnestNode : public PlanNode { @@ -2733,20 +2654,14 @@ 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 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 = {}; +struct TpchPartitioningHandle : public ConnectorPartitioningHandle { + String table = {}; + int64_t totalRows = {}; - SampleNode() noexcept; + TpchPartitioningHandle() noexcept; }; -void to_json(json& j, const SampleNode& p); -void from_json(const json& j, SampleNode& 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 HiveTableHandle : public ConnectorTableHandle { @@ -2760,6 +2675,115 @@ void to_json(json& j, const HiveTableHandle& p); void from_json(const json& j, HiveTableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +struct BuiltInFunctionHandle : public FunctionHandle { + Signature signature = {}; + + BuiltInFunctionHandle() noexcept; +}; +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 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 AssignUniqueId : public PlanNode { + std::shared_ptr source = {}; + VariableReferenceExpression idVariable = {}; + + AssignUniqueId() noexcept; +}; +void to_json(json& j, const AssignUniqueId& p); +void from_json(const json& j, AssignUniqueId& p); +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +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 = {}; + + 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 IcebergTableHandle : public ConnectorTableHandle { + String schemaName = {}; + IcebergTableName icebergTableName = {}; + bool snapshotSpecified = {}; + TupleDomain predicate = {}; + std::shared_ptr outputPath = {}; + std::shared_ptr> storageProperties = {}; + std::shared_ptr tableSchemaJson = {}; + std::shared_ptr> partitionFieldIds = {}; + std::shared_ptr> equalityFieldIds = {}; + + IcebergTableHandle() noexcept; +}; +void to_json(json& j, const IcebergTableHandle& p); +void from_json(const json& j, IcebergTableHandle& 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 { +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 SqlFunctionHandle : public FunctionHandle { SqlFunctionId functionId = {}; String version = {}; @@ -2770,66 +2794,66 @@ 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 UpdateTarget { - TableHandle handle = {}; - SchemaTableName schemaTableName = {}; - List updatedColumns = {}; - List> updatedColumnHandles = {}; +struct ServerInfo { + NodeVersion nodeVersion = {}; + String environment = {}; + bool coordinator = {}; + bool starting = {}; + std::shared_ptr uptime = {}; }; -void to_json(json& j, const UpdateTarget& p); -void from_json(const json& j, UpdateTarget& p); +void to_json(json& j, const ServerInfo& p); +void from_json(const json& j, ServerInfo& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct BuiltInFunctionHandle : public FunctionHandle { - Signature signature = {}; +struct TopNRowNumberNode : public PlanNode { + std::shared_ptr source = {}; + Specification specification = {}; + VariableReferenceExpression rowNumberVariable = {}; + int maxRowCountPerPartition = {}; + bool partial = {}; + std::shared_ptr hashVariable = {}; - BuiltInFunctionHandle() noexcept; -}; -void to_json(json& j, const BuiltInFunctionHandle& p); -void from_json(const json& j, BuiltInFunctionHandle& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class SystemPartitioning { - SINGLE, - FIXED, - SOURCE, - SCALED, - COORDINATOR_ONLY, - ARBITRARY + TopNRowNumberNode() 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 TopNRowNumberNode& p); +void from_json(const json& j, TopNRowNumberNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class SystemPartitionFunction { - SINGLE, - HASH, - ROUND_ROBIN, - BROADCAST, - UNKNOWN +struct HiveMetadataUpdateHandle : public ConnectorMetadataUpdateHandle { + UUID requestId = {}; + SchemaTableName schemaTableName = {}; + std::shared_ptr partitionName = {}; + std::shared_ptr fileName = {}; + + HiveMetadataUpdateHandle() noexcept; }; -extern void to_json(json& j, const SystemPartitionFunction& e); -extern void from_json(const json& j, SystemPartitionFunction& e); +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 SystemPartitioningHandle : public ConnectorPartitioningHandle { - SystemPartitioning partitioning = {}; - SystemPartitionFunction function = {}; +struct LambdaDefinitionExpression : public RowExpression { + List argumentTypes = {}; + List arguments = {}; + std::shared_ptr body = {}; - SystemPartitioningHandle() noexcept; + LambdaDefinitionExpression() noexcept; }; -void to_json(json& j, const SystemPartitioningHandle& p); -void from_json(const json& j, SystemPartitioningHandle& p); +void to_json(json& j, const LambdaDefinitionExpression& p); +void from_json(const json& j, LambdaDefinitionExpression& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ConstantExpression : public RowExpression { - Block valueBlock = {}; - Type type = {}; +struct GroupIdNode : public PlanNode { + std::shared_ptr source = {}; + List> groupingSets = {}; + Map + groupingColumns = {}; + List aggregationArguments = {}; + VariableReferenceExpression groupIdVariable = {}; - ConstantExpression() noexcept; + GroupIdNode() noexcept; }; -void to_json(json& j, const ConstantExpression& p); -void from_json(const json& j, ConstantExpression& p); +void to_json(json& j, const GroupIdNode& p); +void from_json(const json& j, GroupIdNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct TpchTableLayoutHandle : public ConnectorTableLayoutHandle { @@ -2842,135 +2866,118 @@ void to_json(json& j, const TpchTableLayoutHandle& p); void from_json(const json& j, TpchTableLayoutHandle& 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); +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 TpchPartitioningHandle : public ConnectorPartitioningHandle { - String table = {}; - int64_t totalRows = {}; +struct LimitNode : public PlanNode { + std::shared_ptr source = {}; + int64_t count = {}; + LimitNodeStep step = {}; - TpchPartitioningHandle() noexcept; + LimitNode() noexcept; }; -void to_json(json& j, const TpchPartitioningHandle& p); -void from_json(const json& j, TpchPartitioningHandle& p); +void to_json(json& j, const LimitNode& p); +void from_json(const json& j, LimitNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HiveMetadataUpdateHandle : public ConnectorMetadataUpdateHandle { - UUID requestId = {}; - SchemaTableName schemaTableName = {}; - std::shared_ptr partitionName = {}; - std::shared_ptr fileName = {}; - - HiveMetadataUpdateHandle() noexcept; +struct EquiJoinClause { + VariableReferenceExpression left = {}; + VariableReferenceExpression right = {}; }; -void to_json(json& j, const HiveMetadataUpdateHandle& p); -void from_json(const json& j, HiveMetadataUpdateHandle& p); +void to_json(json& j, const EquiJoinClause& p); +void from_json(const json& j, EquiJoinClause& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct 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 TopNRowNumberNode& p); -void from_json(const json& j, TopNRowNumberNode& p); +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 { -enum class Step { SINGLE, PARTIAL, FINAL }; -extern void to_json(json& j, const Step& e); -extern void from_json(const json& j, Step& e); +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 TopNNode : public PlanNode { - std::shared_ptr source = {}; - int64_t count = {}; - OrderingScheme orderingScheme = {}; - Step step = {}; +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 = {}; - TopNNode() noexcept; + JoinNode() noexcept; }; -void to_json(json& j, const TopNNode& p); -void from_json(const json& j, TopNNode& p); +void to_json(json& j, const JoinNode& p); +void from_json(const json& j, JoinNode& p); } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * 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 Form { - IF, - NULL_IF, - SWITCH, - WHEN, - IS_NULL, - COALESCE, - IN, - AND, - OR, - DEREFERENCE, - ROW_CONSTRUCTOR, - BIND +struct MergeJoinNode : public PlanNode { + MergeJoinNode() noexcept; + PlanNodeId id = {}; + 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 = {}; }; -extern void to_json(json& j, const Form& e); -extern void from_json(const json& j, Form& e); +void to_json(json& j, const MergeJoinNode& p); +void from_json(const json& j, MergeJoinNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct SpecialFormExpression : public RowExpression { - Form form = {}; - Type returnType = {}; - List> arguments = {}; +struct ValuesNode : public PlanNode { + std::shared_ptr location = {}; - 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 RemoteSourceNode : public PlanNode { - List sourceFragmentIds = {}; List outputVariables = {}; - bool ensureSourceOrdering = {}; - std::shared_ptr orderingScheme = {}; - ExchangeNodeType exchangeType = {}; + List>> rows = {}; + std::shared_ptr valuesNodeLabel = {}; - RemoteSourceNode() noexcept; + ValuesNode() noexcept; }; -void to_json(json& j, const RemoteSourceNode& p); -void from_json(const json& j, RemoteSourceNode& p); +void to_json(json& j, const ValuesNode& p); +void from_json(const json& j, ValuesNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HivePartitioningHandle : public ConnectorPartitioningHandle { - int bucketCount = {}; - std::shared_ptr maxCompatibleBucketCount = {}; - BucketFunctionType bucketFunctionType = {}; - std::shared_ptr> hiveTypes = {}; - std::shared_ptr> types = {}; - - HivePartitioningHandle() noexcept; +struct UpdateTarget { + TableHandle handle = {}; + SchemaTableName schemaTableName = {}; + List updatedColumns = {}; + List> updatedColumnHandles = {}; }; -void to_json(json& j, const HivePartitioningHandle& p); -void from_json(const json& j, HivePartitioningHandle& p); +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 SortedRangeSet : public ValueSet { - Type type = {}; - List ranges = {}; +struct FilterNode : public PlanNode { + std::shared_ptr source = {}; + std::shared_ptr predicate = {}; - SortedRangeSet() noexcept; + FilterNode() noexcept; }; -void to_json(json& j, const SortedRangeSet& p); -void from_json(const json& j, SortedRangeSet& 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 { enum class DistributionType { PARTITIONED, REPLICATED }; @@ -2995,185 +3002,233 @@ 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); +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 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 +namespace facebook::presto::protocol { +enum class SystemPartitioning { + SINGLE, + FIXED, + SOURCE, + SCALED, + COORDINATOR_ONLY, + ARBITRARY +}; +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 { +enum class SystemPartitionFunction { + SINGLE, + HASH, + ROUND_ROBIN, + BROADCAST, + UNKNOWN +}; +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 SystemPartitioningHandle : public ConnectorPartitioningHandle { + SystemPartitioning partitioning = {}; + SystemPartitionFunction function = {}; + + SystemPartitioningHandle() noexcept; +}; +void to_json(json& j, const SystemPartitioningHandle& p); +void from_json(const json& j, SystemPartitioningHandle& 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); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct AggregationNode : public PlanNode { +struct TopNNode : 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 = {}; + int64_t count = {}; + OrderingScheme orderingScheme = {}; + Step step = {}; - AggregationNode() noexcept; + TopNNode() noexcept; }; -void to_json(json& j, const AggregationNode& p); -void from_json(const json& j, AggregationNode& 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 SortNode : public PlanNode { - std::shared_ptr source = {}; - OrderingScheme orderingScheme = {}; - bool isPartial = {}; +struct AllOrNoneValueSet : public ValueSet { + Type type = {}; + bool all = {}; - SortNode() noexcept; + AllOrNoneValueSet() noexcept; }; -void to_json(json& j, const SortNode& p); -void from_json(const json& j, SortNode& 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 Locality { UNKNOWN, LOCAL, REMOTE }; -extern void to_json(json& j, const Locality& e); -extern void from_json(const json& j, Locality& e); +struct TpchSplit : public ConnectorSplit { + TpchTableHandle tableHandle = {}; + int partNumber = {}; + int totalParts = {}; + List addresses = {}; + TupleDomain> predicate = {}; + + 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 ProjectNode : public PlanNode { - std::shared_ptr source = {}; - Assignments assignments = {}; - Locality locality = {}; +struct HiveInsertTableHandle : public ConnectorInsertTableHandle { + String schemaName = {}; + String tableName = {}; + List inputColumns = {}; + HivePageSinkMetadata pageSinkMetadata = {}; + LocationHandle locationHandle = {}; + std::shared_ptr bucketProperty = {}; + List preferredOrderingColumns = {}; + HiveStorageFormat tableStorageFormat = {}; + HiveStorageFormat partitionStorageFormat = {}; + HiveStorageFormat actualStorageFormat = {}; + HiveCompressionCodec compressionCodec = {}; + std::shared_ptr encryptionInformation = {}; - ProjectNode() noexcept; + HiveInsertTableHandle() noexcept; }; -void to_json(json& j, const ProjectNode& p); -void from_json(const json& j, ProjectNode& p); +void to_json(json& j, const HiveInsertTableHandle& p); +void from_json(const json& j, HiveInsertTableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct 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 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 = {}; + + IcebergTableLayoutHandle() noexcept; }; -void to_json(json& j, const NodeStatus& p); -void from_json(const json& j, NodeStatus& 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 RemoteSourceNode : public PlanNode { + List sourceFragmentIds = {}; List outputVariables = {}; - Map> assignments = - {}; + bool ensureSourceOrdering = {}; + std::shared_ptr orderingScheme = {}; + ExchangeNodeType exchangeType = {}; - TableScanNode() noexcept; + RemoteSourceNode() noexcept; }; -void to_json(json& j, const TableScanNode& p); -void from_json(const json& j, TableScanNode& p); +void to_json(json& j, const RemoteSourceNode& p); +void from_json(const json& j, RemoteSourceNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct RefreshMaterializedViewHandle { - InsertTableHandle handle = {}; +struct CreateHandle : public ExecutionWriterTarget { + OutputTableHandle handle = {}; SchemaTableName schemaTableName = {}; + + CreateHandle() noexcept; }; -void to_json(json& j, const RefreshMaterializedViewHandle& p); -void from_json(const json& j, RefreshMaterializedViewHandle& 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 AssignUniqueId : public PlanNode { +struct EnforceSingleRowNode : public PlanNode { std::shared_ptr source = {}; - VariableReferenceExpression idVariable = {}; - AssignUniqueId() noexcept; + EnforceSingleRowNode() noexcept; }; -void to_json(json& j, const AssignUniqueId& p); -void from_json(const json& j, AssignUniqueId& p); +void to_json(json& j, const EnforceSingleRowNode& p); +void from_json(const json& j, EnforceSingleRowNode& 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 BatchTaskUpdateRequest { + TaskUpdateRequest taskUpdateRequest = {}; + std::shared_ptr shuffleWriteInfo = {}; + std::shared_ptr broadcastBasePath = {}; }; - -void to_json(json& j, const HiveStorageFormat& p); -void from_json(const json& j, HiveStorageFormat& 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 HiveCompressionCodec { NONE, SNAPPY, GZIP, LZ4, ZSTD }; -extern void to_json(json& j, const HiveCompressionCodec& e); -extern void from_json(const json& j, HiveCompressionCodec& e); +struct IcebergSplit : public ConnectorSplit { + String path = {}; + int64_t start = {}; + int64_t length = {}; + FileFormat fileFormat = {}; + List addresses = {}; + Map partitionKeys = {}; + String partitionSpecAsJson = {}; + std::shared_ptr partitionDataJson = {}; + NodeSelectionStrategy nodeSelectionStrategy = {}; + SplitWeight splitWeight = {}; + List deletes = {}; + std::shared_ptr changelogSplitInfo = {}; + int64_t dataSequenceNumber = {}; + + IcebergSplit() noexcept; +}; +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 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 HivePartitioningHandle : public ConnectorPartitioningHandle { + int bucketCount = {}; + std::shared_ptr maxCompatibleBucketCount = {}; + BucketFunctionType bucketFunctionType = {}; + std::shared_ptr> hiveTypes = {}; + std::shared_ptr> types = {}; - HiveOutputTableHandle() noexcept; + HivePartitioningHandle() noexcept; }; -void to_json(json& j, const HiveOutputTableHandle& p); -void from_json(const json& j, HiveOutputTableHandle& 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 HiveInsertTableHandle : public ConnectorInsertTableHandle { - String schemaName = {}; - String tableName = {}; - List inputColumns = {}; - HivePageSinkMetadata pageSinkMetadata = {}; - LocationHandle locationHandle = {}; - std::shared_ptr bucketProperty = {}; - List preferredOrderingColumns = {}; - HiveStorageFormat tableStorageFormat = {}; - HiveStorageFormat partitionStorageFormat = {}; - HiveStorageFormat actualStorageFormat = {}; - HiveCompressionCodec compressionCodec = {}; - std::shared_ptr encryptionInformation = {}; +struct DistinctLimitNode : public PlanNode { + std::shared_ptr source = {}; + int64_t limit = {}; + bool partial = {}; + List distinctVariables = {}; + std::shared_ptr hashVariable = {}; + int timeoutMillis = {}; - HiveInsertTableHandle() noexcept; + DistinctLimitNode() noexcept; }; -void to_json(json& j, const HiveInsertTableHandle& p); -void from_json(const json& j, HiveInsertTableHandle& p); +void to_json(json& j, const DistinctLimitNode& p); +void from_json(const json& j, DistinctLimitNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { enum class NodeState { ACTIVE, INACTIVE, SHUTTING_DOWN }; 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 9baede9c11b5..c924369d3946 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.yml +++ b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.yml @@ -52,6 +52,7 @@ AbstractClasses: - { name: HiveColumnHandle, key: hive } - { name: IcebergColumnHandle, key: hive-iceberg } - { name: TpchColumnHandle, key: tpch } + - { name: SystemColumnHandle, key: $system@system } ConnectorPartitioningHandle: super: JsonEncodedSubclass @@ -66,6 +67,7 @@ AbstractClasses: - { name: HiveTableHandle, key: hive } - { name: IcebergTableHandle, key: hive-iceberg } - { name: TpchTableHandle, key: tpch } + - { name: SystemTableHandle, key: $system@system } ConnectorOutputTableHandle: super: JsonEncodedSubclass @@ -82,6 +84,7 @@ AbstractClasses: subclasses: - { name: HiveTransactionHandle, key: hive } - { name: RemoteTransactionHandle, key: $remote } + - { name: SystemTransactionHandle, key: $system@system } ConnectorTableLayoutHandle: super: JsonEncodedSubclass @@ -89,6 +92,7 @@ AbstractClasses: - { name: HiveTableLayoutHandle, key: hive } - { name: IcebergTableLayoutHandle, key: hive-iceberg } - { name: TpchTableLayoutHandle, key: tpch } + - { name: SystemTableLayoutHandle, key: $system@system } ConnectorMetadataUpdateHandle: super: JsonEncodedSubclass @@ -103,6 +107,7 @@ AbstractClasses: - { name: TpchSplit, key: tpch } - { name: RemoteSplit, key: $remote } - { name: EmptySplit, key: $empty } + - { name: SystemSplit, key: $system@system } ConnectorHistogram: super: JsonEncodedSubclass @@ -345,4 +350,9 @@ 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 \ No newline at end of file + - presto-spi/src/main/java/com/facebook/presto/spi/plan/EquiJoinClause.java + - presto-main/src/main/java/com/facebook/presto/connector/system/SystemSplit.java + - presto-main/src/main/java/com/facebook/presto/connector/system/SystemTableHandle.java + - presto-main/src/main/java/com/facebook/presto/connector/system/SystemColumnHandle.java + - presto-main/src/main/java/com/facebook/presto/connector/system/SystemTableLayoutHandle.java + - presto-main/src/main/java/com/facebook/presto/connector/system/SystemTransactionHandle.java \ No newline at end of file diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeSystemQueries.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeSystemQueries.java index 7280598f65d9..a3fea4bf1d06 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeSystemQueries.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeSystemQueries.java @@ -13,16 +13,33 @@ */ package com.facebook.presto.nativeworker; +import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.tests.AbstractTestQueryFramework; import org.testng.annotations.Test; +import java.util.Collections; + import static com.facebook.airlift.testing.Assertions.assertGreaterThanOrEqual; import static com.facebook.presto.nativeworker.PrestoNativeQueryRunnerUtils.getNativeQueryRunnerParameters; +import static com.facebook.presto.spi.plan.AggregationNode.Step.SINGLE; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.aggregation; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anyTree; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.exchange; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.tableScan; +import static com.facebook.presto.sql.planner.plan.ExchangeNode.Scope.LOCAL; +import static com.facebook.presto.sql.planner.plan.ExchangeNode.Scope.REMOTE_STREAMING; +import static com.facebook.presto.sql.planner.plan.ExchangeNode.Type.GATHER; import static org.testng.Assert.assertEquals; public abstract class AbstractTestNativeSystemQueries extends AbstractTestQueryFramework { + @Override + protected FeaturesConfig createFeaturesConfig() + { + return new FeaturesConfig().setNativeExecutionEnabled(true); + } + @Test public void testNodes() { @@ -33,8 +50,27 @@ public void testNodes() @Test public void testTasks() { - assertQueryFails("select * from system.runtime.tasks", - ".*system not registered.*"); + // This query has a single task on the co-ordinator, and at least one on a Native worker. + // So limit 2 should always return correctly. + assertQueryResultCount("select * from system.runtime.tasks limit 2", 2); + + // This query performs an aggregation on tasks table. + // There are 2 special planning rules for system tables : + // i) Partial aggregations are disabled for system tables + // (as aggregations are not consistent across co-ordinator and native workers) + // ii) A remote gather exchange is added after the system table TableScanNode so that the partitioning + // is made consistent with the rest of the tables. + String aggregation = "select sum(output_bytes) from system.runtime.tasks"; + assertQuerySucceeds(aggregation); + assertPlan( + aggregation, + anyTree( + aggregation( + Collections.emptyMap(), + SINGLE, + exchange(LOCAL, GATHER, + exchange(REMOTE_STREAMING, GATHER, + tableScan("tasks")))))); } @Test