diff --git a/presto-native-execution/presto_cpp/main/PrestoServer.cpp b/presto-native-execution/presto_cpp/main/PrestoServer.cpp index d8788b2de132e..eec870ed97394 100644 --- a/presto-native-execution/presto_cpp/main/PrestoServer.cpp +++ b/presto-native-execution/presto_cpp/main/PrestoServer.cpp @@ -30,6 +30,8 @@ #include "presto_cpp/main/http/HttpServer.h" #include "presto_cpp/main/http/filters/AccessLogFilter.h" #include "presto_cpp/main/http/filters/StatsFilter.h" +#include "presto_cpp/main/operators/BroadcastExchangeSource.h" +#include "presto_cpp/main/operators/BroadcastWrite.h" #include "presto_cpp/main/operators/LocalPersistentShuffle.h" #include "presto_cpp/main/operators/PartitionAndSerialize.h" #include "presto_cpp/main/operators/ShuffleInterface.h" @@ -290,6 +292,10 @@ void PrestoServer::run() { facebook::velox::exec::ExchangeSource::registerFactory( operators::UnsafeRowExchangeSource::createExchangeSource); + // Batch broadcast exchange source. + facebook::velox::exec::ExchangeSource::registerFactory( + operators::BroadcastExchangeSource::createExchangeSource); + pool_ = velox::memory::addDefaultLeafMemoryPool(); taskManager_ = std::make_unique(); @@ -672,6 +678,12 @@ void PrestoServer::registerCustomOperators() { std::make_unique()); facebook::velox::exec::Operator::registerOperator( std::make_unique()); + + // Todo - Split Presto & Presto-on-Spark server into different classes + // which will allow server specific operator registration. + facebook::velox::exec::Operator::registerOperator( + std::make_unique< + facebook::presto::operators::BroadcastWriteTranslator>()); } void PrestoServer::registerFunctions() { diff --git a/presto-native-execution/presto_cpp/main/TaskResource.cpp b/presto-native-execution/presto_cpp/main/TaskResource.cpp index 98ed2a5f68ec1..aeb6a0977eff4 100644 --- a/presto-native-execution/presto_cpp/main/TaskResource.cpp +++ b/presto-native-execution/presto_cpp/main/TaskResource.cpp @@ -238,6 +238,7 @@ proxygen::RequestHandler* TaskResource::createOrUpdateBatchTask( protocol::PlanFragment prestoPlan = json::parse(fragment); auto serializedShuffleWriteInfo = batchUpdateRequest.shuffleWriteInfo; + auto broadcastBasePath = batchUpdateRequest.broadcastBasePath; auto shuffleName = SystemConfig::instance()->shuffleName(); if (serializedShuffleWriteInfo) { VELOX_USER_CHECK( @@ -253,6 +254,7 @@ proxygen::RequestHandler* TaskResource::createOrUpdateBatchTask( VeloxBatchQueryPlanConverter converter( shuffleName, std::move(serializedShuffleWriteInfo), + std::move(broadcastBasePath), queryCtx.get(), pool_); auto planFragment = converter.toVeloxQueryPlan( diff --git a/presto-native-execution/presto_cpp/main/operators/BroadcastExchangeSource.cpp b/presto-native-execution/presto_cpp/main/operators/BroadcastExchangeSource.cpp index d988b9bcb21ec..76e2a7664e918 100644 --- a/presto-native-execution/presto_cpp/main/operators/BroadcastExchangeSource.cpp +++ b/presto-native-execution/presto_cpp/main/operators/BroadcastExchangeSource.cpp @@ -77,21 +77,22 @@ BroadcastExchangeSource::createExchangeSource( return nullptr; } - std::unique_ptr broadcastInfo; + std::unique_ptr broadcastFileInfo; try { - broadcastInfo = BroadcastInfo::deserialize(broadcastInfoJson.value()); + broadcastFileInfo = + BroadcastFileInfo::deserialize(broadcastInfoJson.value()); } catch (const VeloxException& e) { throw; } catch (const std::exception& e) { VELOX_USER_FAIL("BroadcastInfo deserialization failed: {}", e.what()); } - auto fileSystemBroadcast = BroadcastFactory(broadcastInfo->basePath_); + auto fileSystemBroadcast = BroadcastFactory(broadcastFileInfo->filePath_); return std::make_unique( uri.host(), destination, std::move(queue), - fileSystemBroadcast.createReader(broadcastInfo->fileInfos_, pool), + fileSystemBroadcast.createReader(std::move(broadcastFileInfo), pool), pool); } }; // namespace facebook::presto::operators \ No newline at end of file diff --git a/presto-native-execution/presto_cpp/main/operators/BroadcastFactory.cpp b/presto-native-execution/presto_cpp/main/operators/BroadcastFactory.cpp index 1b0840e8b257b..a3d7b4e22615b 100644 --- a/presto-native-execution/presto_cpp/main/operators/BroadcastFactory.cpp +++ b/presto-native-execution/presto_cpp/main/operators/BroadcastFactory.cpp @@ -49,32 +49,22 @@ std::unique_ptr BroadcastFactory::createWriter( } std::shared_ptr BroadcastFactory::createReader( - const std::vector fileInfos, + std::unique_ptr fileInfo, velox::memory::MemoryPool* pool) { - auto broadcastFileReader = std::make_shared( - std::move(fileInfos), fileSystem_, pool); + auto broadcastFileReader = + std::make_shared(fileInfo, fileSystem_, pool); return broadcastFileReader; } // static -std::unique_ptr BroadcastInfo::deserialize( +std::unique_ptr BroadcastFileInfo::deserialize( const std::string& info) { const auto root = nlohmann::json::parse(info); - std::vector broadcastFileInfos; - - for (auto& fileInfo : root["fileInfos"]) { - BroadcastFileInfo broadcastFileInfo; - fileInfo.at("filePath").get_to(broadcastFileInfo.filePath_); - broadcastFileInfos.emplace_back(broadcastFileInfo); - } - return std::make_unique(root["basePath"], broadcastFileInfos); + auto broadcastFileInfo = std::make_unique(); + root.at("filePath").get_to(broadcastFileInfo->filePath_); + return broadcastFileInfo; } -BroadcastInfo::BroadcastInfo( - std::string basePath, - std::vector fileInfos) - : basePath_(basePath), fileInfos_(fileInfos) {} - BroadcastFileWriter::BroadcastFileWriter( std::string_view filename, const RowTypePtr& inputType, @@ -142,18 +132,17 @@ void BroadcastFileWriter::write(const RowVectorPtr& rowVector) { } BroadcastFileReader::BroadcastFileReader( - std::vector broadcastFileInfos, + std::unique_ptr& broadcastFileInfo, std::shared_ptr fileSystem, velox::memory::MemoryPool* pool) - : broadcastFileInfos_(broadcastFileInfos), + : broadcastFileInfo_(std::move(broadcastFileInfo)), fileSystem_(fileSystem), - readfileIndex_(0), - numFiles_(0), + hasData_(true), numBytes_(0), pool_(pool) {} bool BroadcastFileReader::hasNext() { - return readfileIndex_ < broadcastFileInfos_.size(); + return hasData_; } velox::BufferPtr BroadcastFileReader::next() { @@ -161,20 +150,16 @@ velox::BufferPtr BroadcastFileReader::next() { return nullptr; } - auto readFile = fileSystem_->openFileForRead( - broadcastFileInfos_[readfileIndex_].filePath_); + auto readFile = fileSystem_->openFileForRead(broadcastFileInfo_->filePath_); auto buffer = AlignedBuffer::allocate(readFile->size(), pool_, 0); readFile->pread(0, readFile->size(), buffer->asMutable()); - ++readfileIndex_; - ++numFiles_; numBytes_ += readFile->size(); + hasData_ = false; return buffer; } folly::F14FastMap BroadcastFileReader::stats() { - return { - {"broadcastExchangeSource.numFiles", numFiles_}, - {"broadcastExchangeSource.numBytes", numBytes_}}; + return {{"broadcastExchangeSource.numBytes", numBytes_}}; } } // namespace facebook::presto::operators diff --git a/presto-native-execution/presto_cpp/main/operators/BroadcastFactory.h b/presto-native-execution/presto_cpp/main/operators/BroadcastFactory.h index 921fe612bd17e..4bd67d7608f7b 100644 --- a/presto-native-execution/presto_cpp/main/operators/BroadcastFactory.h +++ b/presto-native-execution/presto_cpp/main/operators/BroadcastFactory.h @@ -22,21 +22,18 @@ using namespace facebook::velox; namespace facebook::presto::operators { -/// Struct for single broadcast file info. +/// Struct for single broadcast file info.] +// This struct is a 1:1 strict API mapping to +// presto-spark-base/src/main/java/com/facebook/presto/spark/execution/BroadcastFileInfo.java +// Please refrain from making changes to this API class. If any changes have to +// be made to this struct, one should make sure to make corresponding changes in +// the above Java classes and its corresponding serde functionalities. struct BroadcastFileInfo { std::string filePath_; // TODO: Add additional stats including checksum, num rows, size. -}; - -/// Struct for broadcastInfo in split location. -struct BroadcastInfo { - /// Deserializes JSON string representing BroadcastInfo. - static std::unique_ptr deserialize(const std::string& info); - - BroadcastInfo(std::string basePath, std::vector fileInfos); - std::string basePath_; - std::vector fileInfos_; + static std::unique_ptr deserialize( + const std::string& info); }; /// Writes broadcast data to a file. @@ -81,7 +78,7 @@ class BroadcastFileWriter { class BroadcastFileReader { public: BroadcastFileReader( - std::vector broadcastFileInfos, + std::unique_ptr& broadcastFileInfo, std::shared_ptr fileSystem, velox::memory::MemoryPool* pool); @@ -93,14 +90,13 @@ class BroadcastFileReader { /// Read next block of data. velox::BufferPtr next(); - /// Reader stats - number of files, number of bytes. + /// Reader stats - number of bytes. folly::F14FastMap stats(); private: - std::vector broadcastFileInfos_; + std::unique_ptr broadcastFileInfo_; std::shared_ptr fileSystem_; - int32_t readfileIndex_; - int64_t numFiles_; + bool hasData_; int64_t numBytes_; velox::memory::MemoryPool* pool_; }; @@ -117,7 +113,7 @@ class BroadcastFactory { const RowTypePtr& inputType); std::shared_ptr createReader( - const std::vector fileInfos, + const std::unique_ptr fileInfo, velox::memory::MemoryPool* pool); private: diff --git a/presto-native-execution/presto_cpp/main/operators/tests/BroadcastTest.cpp b/presto-native-execution/presto_cpp/main/operators/tests/BroadcastTest.cpp index b1406abf3f87e..f3ac3faaa6283 100644 --- a/presto-native-execution/presto_cpp/main/operators/tests/BroadcastTest.cpp +++ b/presto-native-execution/presto_cpp/main/operators/tests/BroadcastTest.cpp @@ -100,22 +100,22 @@ class BroadcastTest : public exec::test::OperatorTestBase { fmt::format(kBroadcastFileInfoFormat, broadcastFilePath)); } - bool noMoreSplits = false; + uint8_t splitIndex = 0; // Read back result using BroadcastExchangeSource. return readCursor(broadcastReadParams, [&](auto* task) { - if (noMoreSplits) { + if (splitIndex >= broadcastFilePaths.size()) { + task->noMoreSplits("0"); return; } auto split = exec::Split( std::make_shared(fmt::format( - "batch://task?broadcastInfo={{\"basePath\": \"{}\", \"fileInfos\":[{}]}}", - basePath, - boost::algorithm::join(fileInfos, ","))), + "batch://task?broadcastInfo={}", + fmt::format( + kBroadcastFileInfoFormat, broadcastFilePaths[splitIndex]))), -1); task->addSplit("0", std::move(split)); - task->noMoreSplits("0"); - noMoreSplits = true; + ++splitIndex; }); } diff --git a/presto-native-execution/presto_cpp/main/types/PrestoToVeloxQueryPlan.cpp b/presto-native-execution/presto_cpp/main/types/PrestoToVeloxQueryPlan.cpp index 99914fdda8f4d..e163afe75bc2b 100644 --- a/presto-native-execution/presto_cpp/main/types/PrestoToVeloxQueryPlan.cpp +++ b/presto-native-execution/presto_cpp/main/types/PrestoToVeloxQueryPlan.cpp @@ -2700,9 +2700,25 @@ velox::core::PlanFragment VeloxBatchQueryPlanConverter::toVeloxQueryPlan( VELOX_USER_CHECK_NOT_NULL( partitionedOutputNode, "PartitionedOutputNode is required"); - VELOX_USER_CHECK( - !partitionedOutputNode->isBroadcast(), - "Broadcast shuffle is not supported"); + if (partitionedOutputNode->isBroadcast()) { + VELOX_USER_CHECK_NOT_NULL( + broadcastBasePath_, "broadcastBasePath is required"); + // TODO - Use original plan node with root node and aggregate operator + // stats for additional nodes. + auto broadcastWriteNode = std::make_shared( + "broadcast-write", + *broadcastBasePath_, + core::LocalPartitionNode::gather( + "broadcast-write-gather", + std::vector{partitionedOutputNode->sources()})); + + planFragment.planNode = core::PartitionedOutputNode::broadcast( + "partitioned-output", + 1, + broadcastWriteNode->outputType(), + {broadcastWriteNode}); + return planFragment; + } // If the serializedShuffleWriteInfo is not nullptr, it means this fragment // ends with a shuffle stage. We convert the PartitionedOutputNode to a @@ -2746,6 +2762,11 @@ velox::core::PlanNodePtr VeloxBatchQueryPlanConverter::toVeloxQueryPlan( const std::shared_ptr& /* tableWriteInfo */, const protocol::TaskId& taskId) { auto rowType = toRowType(node->outputVariables); + // Broadcast exchange source. + if (node->exchangeType == protocol::ExchangeNodeType::REPLICATE) { + return std::make_shared(node->id, rowType); + } + // Partitioned shuffle exchange source. return std::make_shared(node->id, rowType); } diff --git a/presto-native-execution/presto_cpp/main/types/PrestoToVeloxQueryPlan.h b/presto-native-execution/presto_cpp/main/types/PrestoToVeloxQueryPlan.h index 036b90f374eb2..9811a8bca75d9 100644 --- a/presto-native-execution/presto_cpp/main/types/PrestoToVeloxQueryPlan.h +++ b/presto-native-execution/presto_cpp/main/types/PrestoToVeloxQueryPlan.h @@ -230,11 +230,13 @@ class VeloxBatchQueryPlanConverter : public VeloxQueryPlanConverterBase { VeloxBatchQueryPlanConverter( const std::string& shuffleName, std::shared_ptr&& serializedShuffleWriteInfo, + std::shared_ptr&& broadcastBasePath, velox::core::QueryCtx* queryCtx, velox::memory::MemoryPool* pool) : VeloxQueryPlanConverterBase(queryCtx, pool), shuffleName_(shuffleName), - serializedShuffleWriteInfo_(std::move(serializedShuffleWriteInfo)) {} + serializedShuffleWriteInfo_(std::move(serializedShuffleWriteInfo)), + broadcastBasePath_(std::move(broadcastBasePath)) {} velox::core::PlanFragment toVeloxQueryPlan( const protocol::PlanFragment& fragment, @@ -252,6 +254,7 @@ class VeloxBatchQueryPlanConverter : public VeloxQueryPlanConverterBase { private: const std::string shuffleName_; const std::shared_ptr serializedShuffleWriteInfo_; + const std::shared_ptr broadcastBasePath_; }; void registerPrestoPlanNodeSerDe(); diff --git a/presto-native-execution/presto_cpp/main/types/tests/PlanConverterTest.cpp b/presto-native-execution/presto_cpp/main/types/tests/PlanConverterTest.cpp index a8767f52e5add..02e3f72195d80 100644 --- a/presto-native-execution/presto_cpp/main/types/tests/PlanConverterTest.cpp +++ b/presto-native-execution/presto_cpp/main/types/tests/PlanConverterTest.cpp @@ -75,7 +75,8 @@ std::shared_ptr assertToVeloxQueryPlan( std::shared_ptr assertToBatchVeloxQueryPlan( const std::string& fileName, const std::string& shuffleName, - std::shared_ptr&& serializedShuffleWriteInfo) { + std::shared_ptr&& serializedShuffleWriteInfo, + std::shared_ptr&& broadcastBasePath) { const std::string fragment = slurp(getDataPath(fileName)); protocol::PlanFragment prestoPlan = json::parse(fragment); @@ -84,6 +85,7 @@ std::shared_ptr assertToBatchVeloxQueryPlan( VeloxBatchQueryPlanConverter converter( shuffleName, std::move(serializedShuffleWriteInfo), + std::move(broadcastBasePath), queryCtx.get(), pool.get()); return converter @@ -165,7 +167,8 @@ TEST_F(PlanConverterTest, batchPlanConversion) { " \"numPartitions\": {}\n" "}}", exec::test::TempDirectoryPath::create()->path, - 10))); + 10)), + std::make_shared("/tmp")); auto shuffleWrite = std::dynamic_pointer_cast(root); @@ -187,7 +190,8 @@ TEST_F(PlanConverterTest, batchPlanConversion) { auto curNode = assertToBatchVeloxQueryPlan( "FinalAgg.json", std::string(operators::LocalPersistentShuffleFactory::kShuffleName), - nullptr); + nullptr, + std::make_shared("/tmp")); std::shared_ptr shuffleReadNode; while (!curNode->sources().empty()) { 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 ed63f748cfab7..c5e6988d45d31 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.cpp +++ b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.cpp @@ -55,881 +55,300 @@ std::string json_map_key(const VariableReferenceExpression& p) { } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (type == ".AggregationNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.GroupIdNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".DistinctLimitNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.ExchangeNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".FilterNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.JoinNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".LimitNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".MarkDistinctNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.SortNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".OutputNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".ProjectNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.RowNumberNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.RemoteSourceNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.SemiJoinNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".TableScanNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.TableWriterNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.TableWriterMergeNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".TopNNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.TopNRowNumberNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.UnnestNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".ValuesNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.AssignUniqueId") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.MergeJoinNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.WindowNode") { - j = *std::static_pointer_cast(p); - return; - } - - throw TypeError(type + " no abstract type PlanNode "); -} - -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " PlanNode PlanNode"); - } - - if (type == ".AggregationNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.GroupIdNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == ".DistinctLimitNode") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.ExchangeNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == ".FilterNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.JoinNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == ".LimitNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == ".MarkDistinctNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.SortNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == ".OutputNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == ".ProjectNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.RowNumberNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.RemoteSourceNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.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, "line", p.line, "SourceLocation", "int", "line"); - to_json_key(j, "column", p.column, "SourceLocation", "int", "column"); -} - -void from_json(const json& j, SourceLocation& p) { - from_json_key(j, "line", p.line, "SourceLocation", "int", "line"); - from_json_key(j, "column", p.column, "SourceLocation", "int", "column"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -VariableReferenceExpression::VariableReferenceExpression() noexcept { - _type = "variable"; -} - -void to_json(json& j, const VariableReferenceExpression& p) { - j = json::object(); - j["@type"] = "variable"; - 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"); -} - -void from_json(const json& j, VariableReferenceExpression& p) { - p._type = j["@type"]; - from_json_key( - j, - "sourceLocation", - p.sourceLocation, - "VariableReferenceExpression", - "SourceLocation", - "sourceLocation"); - from_json_key( - j, "name", p.name, "VariableReferenceExpression", "String", "name"); - from_json_key( - j, "type", p.type, "VariableReferenceExpression", "Type", "type"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -RowNumberNode::RowNumberNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.RowNumberNode"; -} - -void to_json(json& j, const RowNumberNode& p) { - j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.RowNumberNode"; - to_json_key(j, "id", p.id, "RowNumberNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "RowNumberNode", "PlanNode", "source"); - to_json_key( - j, - "partitionBy", - p.partitionBy, - "RowNumberNode", - "List", - "partitionBy"); - to_json_key( - j, - "rowNumberVariable", - p.rowNumberVariable, - "RowNumberNode", - "VariableReferenceExpression", - "rowNumberVariable"); - to_json_key( - j, - "maxRowCountPerPartition", - p.maxRowCountPerPartition, - "RowNumberNode", - "Integer", - "maxRowCountPerPartition"); - to_json_key(j, "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 -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { - -void to_json(json& j, const Block& p) { - j = p.data; -} - -void from_json(const json& j, Block& p) { - p.data = std::string(j); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const ValueEntry& p) { - j = json::object(); - to_json_key(j, "type", p.type, "ValueEntry", "Type", "type"); - to_json_key(j, "block", p.block, "ValueEntry", "Block", "block"); -} - -void from_json(const json& j, ValueEntry& p) { - from_json_key(j, "type", p.type, "ValueEntry", "Type", "type"); - from_json_key(j, "block", p.block, "ValueEntry", "Block", "block"); +TpchPartitioningHandle::TpchPartitioningHandle() noexcept { + _type = "tpch"; } -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -void to_json(json& j, const EquiJoinClause& p) { +void to_json(json& j, const TpchPartitioningHandle& p) { j = json::object(); - to_json_key( - j, - "left", - p.left, - "EquiJoinClause", - "VariableReferenceExpression", - "left"); - to_json_key( - j, - "right", - p.right, - "EquiJoinClause", - "VariableReferenceExpression", - "right"); + j["@type"] = "tpch"; + to_json_key(j, "table", p.table, "TpchPartitioningHandle", "String", "table"); + to_json_key( + j, + "totalRows", + p.totalRows, + "TpchPartitioningHandle", + "int64_t", + "totalRows"); } -void from_json(const json& j, EquiJoinClause& p) { +void from_json(const json& j, TpchPartitioningHandle& p) { + p._type = j["@type"]; from_json_key( - j, - "left", - p.left, - "EquiJoinClause", - "VariableReferenceExpression", - "left"); + j, "table", p.table, "TpchPartitioningHandle", "String", "table"); from_json_key( j, - "right", - p.right, - "EquiJoinClause", - "VariableReferenceExpression", - "right"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair JoinNodeType_enum_table[] = - { // NOLINT: cert-err58-cpp - {JoinNodeType::INNER, "INNER"}, - {JoinNodeType::LEFT, "LEFT"}, - {JoinNodeType::RIGHT, "RIGHT"}, - {JoinNodeType::FULL, "FULL"}}; -void to_json(json& j, const JoinNodeType& e) { - static_assert( - std::is_enum::value, "JoinNodeType must be an enum!"); - const auto* it = std::find_if( - std::begin(JoinNodeType_enum_table), - std::end(JoinNodeType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(JoinNodeType_enum_table)) - ? it - : std::begin(JoinNodeType_enum_table)) - ->second; -} -void from_json(const json& j, JoinNodeType& e) { - static_assert( - std::is_enum::value, "JoinNodeType must be an enum!"); - const auto* it = std::find_if( - std::begin(JoinNodeType_enum_table), - std::end(JoinNodeType_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(JoinNodeType_enum_table)) - ? it - : std::begin(JoinNodeType_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// 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; + "totalRows", + p.totalRows, + "TpchPartitioningHandle", + "int64_t", + "totalRows"); } } // 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 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, 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 "); +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 { -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 VariableStatsEstimate& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.JoinNode"; - to_json_key(j, "id", p.id, "JoinNode", "PlanNodeId", "id"); - to_json_key(j, "type", p.type, "JoinNode", "JoinNodeType", "type"); - to_json_key(j, "left", p.left, "JoinNode", "PlanNode", "left"); - to_json_key(j, "right", p.right, "JoinNode", "PlanNode", "right"); - to_json_key( - j, - "criteria", - p.criteria, - "JoinNode", - "List", - "criteria"); - to_json_key( - j, - "outputVariables", - p.outputVariables, - "JoinNode", - "List", - "outputVariables"); - to_json_key( - j, - "filter", - p.filter, - "JoinNode", - "std::shared_ptr", - "filter"); - to_json_key( - j, - "leftHashVariable", - p.leftHashVariable, - "JoinNode", - "VariableReferenceExpression", - "leftHashVariable"); to_json_key( - j, - "rightHashVariable", - p.rightHashVariable, - "JoinNode", - "VariableReferenceExpression", - "rightHashVariable"); + j, "lowValue", p.lowValue, "VariableStatsEstimate", "double", "lowValue"); to_json_key( j, - "distributionType", - p.distributionType, - "JoinNode", - "DistributionType", - "distributionType"); + "highValue", + p.highValue, + "VariableStatsEstimate", + "double", + "highValue"); to_json_key( j, - "dynamicFilters", - p.dynamicFilters, - "JoinNode", - "Map", - "dynamicFilters"); -} - -void from_json(const json& j, JoinNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "JoinNode", "PlanNodeId", "id"); - from_json_key(j, "type", p.type, "JoinNode", "JoinNodeType", "type"); - from_json_key(j, "left", p.left, "JoinNode", "PlanNode", "left"); - from_json_key(j, "right", p.right, "JoinNode", "PlanNode", "right"); - from_json_key( + "nullsFraction", + p.nullsFraction, + "VariableStatsEstimate", + "double", + "nullsFraction"); + to_json_key( j, - "criteria", - p.criteria, - "JoinNode", - "List", - "criteria"); - from_json_key( + "averageRowSize", + p.averageRowSize, + "VariableStatsEstimate", + "double", + "averageRowSize"); + to_json_key( j, - "outputVariables", - p.outputVariables, - "JoinNode", - "List", - "outputVariables"); + "distinctValuesCount", + p.distinctValuesCount, + "VariableStatsEstimate", + "double", + "distinctValuesCount"); +} + +void from_json(const json& j, VariableStatsEstimate& p) { from_json_key( - j, - "filter", - p.filter, - "JoinNode", - "std::shared_ptr", - "filter"); + j, "lowValue", p.lowValue, "VariableStatsEstimate", "double", "lowValue"); from_json_key( j, - "leftHashVariable", - p.leftHashVariable, - "JoinNode", - "VariableReferenceExpression", - "leftHashVariable"); + "highValue", + p.highValue, + "VariableStatsEstimate", + "double", + "highValue"); from_json_key( j, - "rightHashVariable", - p.rightHashVariable, - "JoinNode", - "VariableReferenceExpression", - "rightHashVariable"); + "nullsFraction", + p.nullsFraction, + "VariableStatsEstimate", + "double", + "nullsFraction"); from_json_key( j, - "distributionType", - p.distributionType, - "JoinNode", - "DistributionType", - "distributionType"); + "averageRowSize", + p.averageRowSize, + "VariableStatsEstimate", + "double", + "averageRowSize"); from_json_key( j, - "dynamicFilters", - p.dynamicFilters, - "JoinNode", - "Map", - "dynamicFilters"); + "distinctValuesCount", + p.distinctValuesCount, + "VariableStatsEstimate", + "double", + "distinctValuesCount"); } } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ namespace facebook::presto::protocol { -void to_json(json& j, const Lifespan& p) { - if (p.isgroup) { - j = "Group" + std::to_string(p.groupid); - } else { - j = "TaskWide"; - } +void to_json(json& j, const SourceLocation& p) { + j = json::object(); + to_json_key(j, "line", p.line, "SourceLocation", "int", "line"); + to_json_key(j, "column", p.column, "SourceLocation", "int", "column"); } -void from_json(const json& j, Lifespan& p) { - String lifespan = j; - - if (lifespan == "TaskWide") { - p.isgroup = false; - p.groupid = 0; - } else { - if (lifespan != "Group") { - // fail... - } - p.isgroup = true; - p.groupid = std::stoi(lifespan.substr(strlen("Group"))); - } +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 -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (type == "$remote") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "$empty") { - j = *std::static_pointer_cast(p); - return; - } - if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); - return; - } +VariableReferenceExpression::VariableReferenceExpression() noexcept { + _type = "variable"; +} - if (getConnectorKey(type) == "tpch") { - j = *std::static_pointer_cast(p); - return; - } +void to_json(json& j, const VariableReferenceExpression& p) { + j = json::object(); + j["@type"] = "variable"; + 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"); +} - throw TypeError(type + " no abstract type ConnectorSplit"); +void from_json(const json& j, VariableReferenceExpression& p) { + p._type = j["@type"]; + from_json_key( + j, + "sourceLocation", + p.sourceLocation, + "VariableReferenceExpression", + "SourceLocation", + "sourceLocation"); + from_json_key( + j, "name", p.name, "VariableReferenceExpression", "String", "name"); + from_json_key( + j, "type", p.type, "VariableReferenceExpression", "Type", "type"); } +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " ConnectorSplit"); - } +void to_json(json& j, const PlanNodeStatsEstimate& p) { + j = json::object(); + 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"); +} - if (type == "$remote") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - if (type == "$empty") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - if (getConnectorKey(type) == "hive") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } +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"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +TpchTableHandle::TpchTableHandle() noexcept { + _type = "tpch"; +} - if (getConnectorKey(type) == "tpch") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } +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, + "scaleFactor", + p.scaleFactor, + "TpchTableHandle", + "double", + "scaleFactor"); +} - throw TypeError(type + " no abstract type ConnectorSplit"); +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 { -void to_json(json& j, const SplitContext& p) { +void to_json(json& j, const DistributionSnapshot& p) { j = json::object(); - to_json_key(j, "cacheable", p.cacheable, "SplitContext", "bool", "cacheable"); + 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, SplitContext& p) { +void from_json(const json& j, DistributionSnapshot& p) { from_json_key( - j, "cacheable", p.cacheable, "SplitContext", "bool", "cacheable"); + 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 /* @@ -1038,1480 +457,1321 @@ void from_json(const json& j, std::shared_ptr& p) { throw TypeError(type + " no abstract type ConnectorTransactionHandle"); } } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ namespace facebook::presto::protocol { - -void to_json(json& j, const Split& p) { - j = json::object(); - to_json_key( - j, "connectorId", p.connectorId, "Split", "ConnectorId", "connectorId"); - to_json_key( - j, - "transactionHandle", - p.transactionHandle, - "Split", - "ConnectorTransactionHandle", - "transactionHandle"); - to_json_key( - j, - "connectorSplit", - p.connectorSplit, - "Split", - "ConnectorSplit", - "connectorSplit"); - to_json_key(j, "lifespan", p.lifespan, "Split", "Lifespan", "lifespan"); - to_json_key( - j, - "splitContext", - p.splitContext, - "Split", - "SplitContext", - "splitContext"); -} - -void from_json(const json& j, Split& p) { - from_json_key( - j, "connectorId", p.connectorId, "Split", "ConnectorId", "connectorId"); - from_json_key( - j, - "transactionHandle", - p.transactionHandle, - "Split", - "ConnectorTransactionHandle", - "transactionHandle"); - from_json_key( - j, - "connectorSplit", - p.connectorSplit, - "Split", - "ConnectorSplit", - "connectorSplit"); - from_json_key(j, "lifespan", p.lifespan, "Split", "Lifespan", "lifespan"); - from_json_key( - j, - "splitContext", - p.splitContext, - "Split", - "SplitContext", - "splitContext"); -} -} // 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 { -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 == "$static") { - j = *std::static_pointer_cast(p); + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); return; } - if (type == "json_file") { - j = *std::static_pointer_cast(p); + + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); return; } - throw TypeError(type + " no abstract type FunctionHandle "); + throw TypeError(type + " no abstract type ConnectorTableHandle"); } -void from_json(const json& j, std::shared_ptr& p) { +void from_json(const json& j, std::shared_ptr& p) { String type; try { type = p->getSubclassKey(j); } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " FunctionHandle FunctionHandle"); + throw ParseError( + std::string(e.what()) + " ConnectorTableHandle ConnectorTableHandle"); } - 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(); + if (getConnectorKey(type) == "hive") { + auto k = std::make_shared(); j.get_to(*k); - p = std::static_pointer_cast(k); + p = k; return; - } - - throw TypeError(type + " no abstract type FunctionHandle "); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -CallExpression::CallExpression() noexcept { - _type = "call"; -} - -void to_json(json& j, const CallExpression& p) { - j = json::object(); - j["@type"] = "call"; - to_json_key( - j, - "sourceLocation", - p.sourceLocation, - "CallExpression", - "SourceLocation", - "sourceLocation"); - to_json_key( - j, - "displayName", - p.displayName, - "CallExpression", - "String", - "displayName"); - to_json_key( - j, - "functionHandle", - p.functionHandle, - "CallExpression", - "FunctionHandle", - "functionHandle"); - to_json_key( - j, "returnType", p.returnType, "CallExpression", "Type", "returnType"); - to_json_key( - j, - "arguments", - p.arguments, - "CallExpression", - "List>", - "arguments"); -} - -void from_json(const json& j, CallExpression& p) { - p._type = j["@type"]; - from_json_key( - j, - "sourceLocation", - p.sourceLocation, - "CallExpression", - "SourceLocation", - "sourceLocation"); - from_json_key( - j, - "displayName", - p.displayName, - "CallExpression", - "String", - "displayName"); - from_json_key( - j, - "functionHandle", - p.functionHandle, - "CallExpression", - "FunctionHandle", - "functionHandle"); - from_json_key( - j, "returnType", p.returnType, "CallExpression", "Type", "returnType"); - from_json_key( - j, - "arguments", - p.arguments, - "CallExpression", - "List>", - "arguments"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair SortOrder_enum_table[] = - { // NOLINT: cert-err58-cpp - {SortOrder::ASC_NULLS_FIRST, "ASC_NULLS_FIRST"}, - {SortOrder::ASC_NULLS_LAST, "ASC_NULLS_LAST"}, - {SortOrder::DESC_NULLS_FIRST, "DESC_NULLS_FIRST"}, - {SortOrder::DESC_NULLS_LAST, "DESC_NULLS_LAST"}}; -void to_json(json& j, const SortOrder& e) { - static_assert(std::is_enum::value, "SortOrder must be an enum!"); - const auto* it = std::find_if( - std::begin(SortOrder_enum_table), - std::end(SortOrder_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(SortOrder_enum_table)) - ? it - : std::begin(SortOrder_enum_table)) - ->second; -} -void from_json(const json& j, SortOrder& e) { - static_assert(std::is_enum::value, "SortOrder must be an enum!"); - const auto* it = std::find_if( - std::begin(SortOrder_enum_table), - std::end(SortOrder_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(SortOrder_enum_table)) - ? it - : std::begin(SortOrder_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Ordering& p) { - j = json::object(); - to_json_key( - j, - "variable", - p.variable, - "Ordering", - "VariableReferenceExpression", - "variable"); - to_json_key( - j, "sortOrder", p.sortOrder, "Ordering", "SortOrder", "sortOrder"); -} - -void from_json(const json& j, Ordering& p) { - from_json_key( - j, - "variable", - p.variable, - "Ordering", - "VariableReferenceExpression", - "variable"); - from_json_key( - j, "sortOrder", p.sortOrder, "Ordering", "SortOrder", "sortOrder"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const OrderingScheme& p) { - j = json::object(); - to_json_key( - j, "orderBy", p.orderBy, "OrderingScheme", "List", "orderBy"); -} - -void from_json(const json& j, OrderingScheme& p) { - from_json_key( - j, "orderBy", p.orderBy, "OrderingScheme", "List", "orderBy"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Aggregation& 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, - "arguments", - p.arguments, - "Aggregation", - "List>", - "arguments"); -} - -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"); - from_json_key( - j, - "arguments", - p.arguments, - "Aggregation", - "List>", - "arguments"); + } + + if (getConnectorKey(type) == "tpch") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + + throw TypeError(type + " no abstract type ConnectorTableHandle"); } } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair ColumnType_enum_table[] = - { // NOLINT: cert-err58-cpp - {ColumnType::PARTITION_KEY, "PARTITION_KEY"}, - {ColumnType::REGULAR, "REGULAR"}, - {ColumnType::SYNTHESIZED, "SYNTHESIZED"}, - {ColumnType::AGGREGATED, "AGGREGATED"}}; -void to_json(json& j, const ColumnType& e) { - static_assert(std::is_enum::value, "ColumnType must be an enum!"); - const auto* it = std::find_if( - std::begin(ColumnType_enum_table), - std::end(ColumnType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(ColumnType_enum_table)) - ? it - : std::begin(ColumnType_enum_table)) - ->second; + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); + return; + } + + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ConnectorTableLayoutHandle"); } -void from_json(const json& j, ColumnType& e) { - static_assert(std::is_enum::value, "ColumnType must be an enum!"); - const auto* it = std::find_if( - std::begin(ColumnType_enum_table), - std::end(ColumnType_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(ColumnType_enum_table)) - ? it - : std::begin(ColumnType_enum_table)) - ->first; + +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError( + std::string(e.what()) + + " ConnectorTableLayoutHandle ConnectorTableLayoutHandle"); + } + + if (getConnectorKey(type) == "hive") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + + if (getConnectorKey(type) == "tpch") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + + throw TypeError(type + " no abstract type ConnectorTableLayoutHandle"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveColumnHandle::HiveColumnHandle() noexcept { - _type = "hive"; -} -void to_json(json& j, const HiveColumnHandle& p) { +void to_json(json& j, const TableHandle& p) { j = json::object(); - j["@type"] = "hive"; - to_json_key(j, "name", p.name, "HiveColumnHandle", "String", "name"); - to_json_key( - j, "hiveType", p.hiveType, "HiveColumnHandle", "HiveType", "hiveType"); - to_json_key( - j, - "typeSignature", - p.typeSignature, - "HiveColumnHandle", - "TypeSignature", - "typeSignature"); to_json_key( j, - "hiveColumnIndex", - p.hiveColumnIndex, - "HiveColumnHandle", - "int", - "hiveColumnIndex"); + "connectorId", + p.connectorId, + "TableHandle", + "ConnectorId", + "connectorId"); to_json_key( j, - "columnType", - p.columnType, - "HiveColumnHandle", - "ColumnType", - "columnType"); - to_json_key(j, "comment", p.comment, "HiveColumnHandle", "String", "comment"); + "connectorHandle", + p.connectorHandle, + "TableHandle", + "ConnectorTableHandle", + "connectorHandle"); to_json_key( j, - "requiredSubfields", - p.requiredSubfields, - "HiveColumnHandle", - "List", - "requiredSubfields"); + "transaction", + p.transaction, + "TableHandle", + "ConnectorTransactionHandle", + "transaction"); to_json_key( j, - "partialAggregation", - p.partialAggregation, - "HiveColumnHandle", - "Aggregation", - "partialAggregation"); + "connectorTableLayout", + p.connectorTableLayout, + "TableHandle", + "ConnectorTableLayoutHandle", + "connectorTableLayout"); } -void from_json(const json& j, HiveColumnHandle& p) { - p._type = j["@type"]; - from_json_key(j, "name", p.name, "HiveColumnHandle", "String", "name"); - from_json_key( - j, "hiveType", p.hiveType, "HiveColumnHandle", "HiveType", "hiveType"); - from_json_key( - j, - "typeSignature", - p.typeSignature, - "HiveColumnHandle", - "TypeSignature", - "typeSignature"); +void from_json(const json& j, TableHandle& p) { from_json_key( j, - "hiveColumnIndex", - p.hiveColumnIndex, - "HiveColumnHandle", - "int", - "hiveColumnIndex"); + "connectorId", + p.connectorId, + "TableHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "columnType", - p.columnType, - "HiveColumnHandle", - "ColumnType", - "columnType"); - from_json_key( - j, "comment", p.comment, "HiveColumnHandle", "String", "comment"); + "connectorHandle", + p.connectorHandle, + "TableHandle", + "ConnectorTableHandle", + "connectorHandle"); from_json_key( j, - "requiredSubfields", - p.requiredSubfields, - "HiveColumnHandle", - "List", - "requiredSubfields"); + "transaction", + p.transaction, + "TableHandle", + "ConnectorTransactionHandle", + "transaction"); from_json_key( j, - "partialAggregation", - p.partialAggregation, - "HiveColumnHandle", - "Aggregation", - "partialAggregation"); + "connectorTableLayout", + p.connectorTableLayout, + "TableHandle", + "ConnectorTableLayoutHandle", + "connectorTableLayout"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const BucketConversion& p) { +void to_json(json& j, const SchemaTableName& p) { j = json::object(); - to_json_key( - j, - "tableBucketCount", - p.tableBucketCount, - "BucketConversion", - "int", - "tableBucketCount"); - to_json_key( - j, - "partitionBucketCount", - p.partitionBucketCount, - "BucketConversion", - "int", - "partitionBucketCount"); - to_json_key( - j, - "bucketColumnHandles", - p.bucketColumnHandles, - "BucketConversion", - "List", - "bucketColumnHandles"); + 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, BucketConversion& p) { - from_json_key( - j, - "tableBucketCount", - p.tableBucketCount, - "BucketConversion", - "int", - "tableBucketCount"); - from_json_key( - j, - "partitionBucketCount", - p.partitionBucketCount, - "BucketConversion", - "int", - "partitionBucketCount"); - from_json_key( - j, - "bucketColumnHandles", - p.bucketColumnHandles, - "BucketConversion", - "List", - "bucketColumnHandles"); +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 { -TpchTableHandle::TpchTableHandle() noexcept { - _type = "tpch"; +DeleteHandle::DeleteHandle() noexcept { + _type = "DeleteHandle"; } -void to_json(json& j, const TpchTableHandle& p) { +void to_json(json& j, const DeleteHandle& p) { j = json::object(); - j["@type"] = "tpch"; - to_json_key( - j, "tableName", p.tableName, "TpchTableHandle", "String", "tableName"); + j["@type"] = "DeleteHandle"; + to_json_key(j, "handle", p.handle, "DeleteHandle", "TableHandle", "handle"); to_json_key( j, - "scaleFactor", - p.scaleFactor, - "TpchTableHandle", - "double", - "scaleFactor"); + "schemaTableName", + p.schemaTableName, + "DeleteHandle", + "SchemaTableName", + "schemaTableName"); } -void from_json(const json& j, TpchTableHandle& p) { +void from_json(const json& j, DeleteHandle& p) { p._type = j["@type"]; - from_json_key( - j, "tableName", p.tableName, "TpchTableHandle", "String", "tableName"); + from_json_key(j, "handle", p.handle, "DeleteHandle", "TableHandle", "handle"); from_json_key( j, - "scaleFactor", - p.scaleFactor, - "TpchTableHandle", - "double", - "scaleFactor"); + "schemaTableName", + p.schemaTableName, + "DeleteHandle", + "SchemaTableName", + "schemaTableName"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +RemoteSplit::RemoteSplit() noexcept { + _type = "$remote"; +} -void to_json(json& j, const ScheduledSplit& p) { +void to_json(json& j, const RemoteSplit& p) { j = json::object(); - to_json_key( - j, "sequenceId", p.sequenceId, "ScheduledSplit", "int64_t", "sequenceId"); + j["@type"] = "$remote"; + to_json_key(j, "location", p.location, "RemoteSplit", "Location", "location"); to_json_key( j, - "planNodeId", - p.planNodeId, - "ScheduledSplit", - "PlanNodeId", - "planNodeId"); - to_json_key(j, "split", p.split, "ScheduledSplit", "Split", "split"); + "remoteSourceTaskId", + p.remoteSourceTaskId, + "RemoteSplit", + "TaskId", + "remoteSourceTaskId"); } -void from_json(const json& j, ScheduledSplit& p) { +void from_json(const json& j, RemoteSplit& p) { + p._type = j["@type"]; from_json_key( - j, "sequenceId", p.sequenceId, "ScheduledSplit", "int64_t", "sequenceId"); + j, "location", p.location, "RemoteSplit", "Location", "location"); from_json_key( j, - "planNodeId", - p.planNodeId, - "ScheduledSplit", - "PlanNodeId", - "planNodeId"); - from_json_key(j, "split", p.split, "ScheduledSplit", "Split", "split"); + "remoteSourceTaskId", + p.remoteSourceTaskId, + "RemoteSplit", + "TaskId", + "remoteSourceTaskId"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const TaskSource& p) { - j = json::object(); - to_json_key( - j, "planNodeId", p.planNodeId, "TaskSource", "PlanNodeId", "planNodeId"); - to_json_key( - j, "splits", p.splits, "TaskSource", "List", "splits"); - to_json_key( - j, - "noMoreSplitsForLifespan", - p.noMoreSplitsForLifespan, - "TaskSource", - "List", - "noMoreSplitsForLifespan"); - to_json_key( - j, "noMoreSplits", p.noMoreSplits, "TaskSource", "bool", "noMoreSplits"); +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair + AggregationNodeStep_enum_table[] = { // NOLINT: cert-err58-cpp + {AggregationNodeStep::PARTIAL, "PARTIAL"}, + {AggregationNodeStep::FINAL, "FINAL"}, + {AggregationNodeStep::INTERMEDIATE, "INTERMEDIATE"}, + {AggregationNodeStep::SINGLE, "SINGLE"}}; +void to_json(json& j, const AggregationNodeStep& e) { + static_assert( + std::is_enum::value, + "AggregationNodeStep must be an enum!"); + const auto* it = std::find_if( + std::begin(AggregationNodeStep_enum_table), + std::end(AggregationNodeStep_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(AggregationNodeStep_enum_table)) + ? it + : std::begin(AggregationNodeStep_enum_table)) + ->second; +} +void from_json(const json& j, AggregationNodeStep& e) { + static_assert( + std::is_enum::value, + "AggregationNodeStep must be an enum!"); + const auto* it = std::find_if( + std::begin(AggregationNodeStep_enum_table), + std::end(AggregationNodeStep_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(AggregationNodeStep_enum_table)) + ? it + : std::begin(AggregationNodeStep_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +void to_json(json& j, const 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, TaskSource& p) { - from_json_key( - j, "planNodeId", p.planNodeId, "TaskSource", "PlanNodeId", "planNodeId"); - from_json_key( - j, "splits", p.splits, "TaskSource", "List", "splits"); - from_json_key( - j, - "noMoreSplitsForLifespan", - p.noMoreSplitsForLifespan, - "TaskSource", - "List", - "noMoreSplitsForLifespan"); - from_json_key( - j, "noMoreSplits", p.noMoreSplits, "TaskSource", "bool", "noMoreSplits"); +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 { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + + 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; + } -// 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; + throw TypeError(type + " no abstract type FunctionHandle "); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HivePartitioningHandle::HivePartitioningHandle() noexcept { - _type = "hive"; +CallExpression::CallExpression() noexcept { + _type = "call"; } -void to_json(json& j, const HivePartitioningHandle& p) { +void to_json(json& j, const CallExpression& p) { j = json::object(); - j["@type"] = "hive"; + j["@type"] = "call"; to_json_key( j, - "bucketCount", - p.bucketCount, - "HivePartitioningHandle", - "int", - "bucketCount"); + "sourceLocation", + p.sourceLocation, + "CallExpression", + "SourceLocation", + "sourceLocation"); to_json_key( j, - "maxCompatibleBucketCount", - p.maxCompatibleBucketCount, - "HivePartitioningHandle", - "int", - "maxCompatibleBucketCount"); + "displayName", + p.displayName, + "CallExpression", + "String", + "displayName"); to_json_key( j, - "bucketFunctionType", - p.bucketFunctionType, - "HivePartitioningHandle", - "BucketFunctionType", - "bucketFunctionType"); + "functionHandle", + p.functionHandle, + "CallExpression", + "FunctionHandle", + "functionHandle"); to_json_key( - j, - "hiveTypes", - p.hiveTypes, - "HivePartitioningHandle", - "List", - "hiveTypes"); + j, "returnType", p.returnType, "CallExpression", "Type", "returnType"); to_json_key( - j, "types", p.types, "HivePartitioningHandle", "List", "types"); + j, + "arguments", + p.arguments, + "CallExpression", + "List>", + "arguments"); } -void from_json(const json& j, HivePartitioningHandle& p) { +void from_json(const json& j, CallExpression& p) { p._type = j["@type"]; from_json_key( j, - "bucketCount", - p.bucketCount, - "HivePartitioningHandle", - "int", - "bucketCount"); + "sourceLocation", + p.sourceLocation, + "CallExpression", + "SourceLocation", + "sourceLocation"); from_json_key( j, - "maxCompatibleBucketCount", - p.maxCompatibleBucketCount, - "HivePartitioningHandle", - "int", - "maxCompatibleBucketCount"); + "displayName", + p.displayName, + "CallExpression", + "String", + "displayName"); from_json_key( j, - "bucketFunctionType", - p.bucketFunctionType, - "HivePartitioningHandle", - "BucketFunctionType", - "bucketFunctionType"); + "functionHandle", + p.functionHandle, + "CallExpression", + "FunctionHandle", + "functionHandle"); from_json_key( - j, - "hiveTypes", - p.hiveTypes, - "HivePartitioningHandle", - "List", - "hiveTypes"); + j, "returnType", p.returnType, "CallExpression", "Type", "returnType"); from_json_key( - j, "types", p.types, "HivePartitioningHandle", "List", "types"); + j, + "arguments", + p.arguments, + "CallExpression", + "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 Order_enum_table[] = +static const std::pair SortOrder_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!"); + {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(Order_enum_table), - std::end(Order_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(Order_enum_table)) ? it : std::begin(Order_enum_table)) + j = ((it != std::end(SortOrder_enum_table)) + ? it + : std::begin(SortOrder_enum_table)) ->second; } -void from_json(const json& j, Order& e) { - static_assert(std::is_enum::value, "Order 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(Order_enum_table), - std::end(Order_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(Order_enum_table)) ? it : std::begin(Order_enum_table)) + e = ((it != std::end(SortOrder_enum_table)) + ? it + : std::begin(SortOrder_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const SortingColumn& p) { +void to_json(json& j, const Ordering& p) { j = json::object(); to_json_key( - j, "columnName", p.columnName, "SortingColumn", "String", "columnName"); - to_json_key(j, "order", p.order, "SortingColumn", "Order", "order"); + j, + "variable", + p.variable, + "Ordering", + "VariableReferenceExpression", + "variable"); + to_json_key( + j, "sortOrder", p.sortOrder, "Ordering", "SortOrder", "sortOrder"); } -void from_json(const json& j, SortingColumn& p) { +void from_json(const json& j, Ordering& 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 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; + j, + "variable", + p.variable, + "Ordering", + "VariableReferenceExpression", + "variable"); + from_json_key( + j, "sortOrder", p.sortOrder, "Ordering", "SortOrder", "sortOrder"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -// 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 to_json(json& j, const OrderingScheme& p) { + j = json::object(); + to_json_key( + j, "orderBy", p.orderBy, "OrderingScheme", "List", "orderBy"); } -void from_json(const json& j, 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; + +void from_json(const json& j, OrderingScheme& p) { + from_json_key( + j, "orderBy", p.orderBy, "OrderingScheme", "List", "orderBy"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const LocationHandle& 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, "targetPath", p.targetPath, "LocationHandle", "String", "targetPath"); + j, + "filter", + p.filter, + "Aggregation", + "std::shared_ptr", + "filter"); to_json_key( - j, "writePath", p.writePath, "LocationHandle", "String", "writePath"); + j, "orderBy", p.orderBy, "Aggregation", "OrderingScheme", "orderBy"); + to_json_key(j, "distinct", p.distinct, "Aggregation", "bool", "distinct"); to_json_key( - j, "tempPath", p.tempPath, "LocationHandle", "String", "tempPath"); + j, "mask", p.mask, "Aggregation", "VariableReferenceExpression", "mask"); to_json_key( - j, "tableType", p.tableType, "LocationHandle", "TableType", "tableType"); + j, + "functionHandle", + p.functionHandle, + "Aggregation", + "FunctionHandle", + "functionHandle"); to_json_key( - j, "writeMode", p.writeMode, "LocationHandle", "WriteMode", "writeMode"); + j, + "arguments", + p.arguments, + "Aggregation", + "List>", + "arguments"); +} + +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"); + from_json_key( + j, + "arguments", + p.arguments, + "Aggregation", + "List>", + "arguments"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + + if (type == ".AggregationNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.GroupIdNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".DistinctLimitNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.ExchangeNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".FilterNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.JoinNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".LimitNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".MarkDistinctNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.SortNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".OutputNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".ProjectNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.RowNumberNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.RemoteSourceNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.SemiJoinNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".TableScanNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.TableWriterNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.TableWriterMergeNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".TopNNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.TopNRowNumberNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.UnnestNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".ValuesNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.AssignUniqueId") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.MergeJoinNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.WindowNode") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type PlanNode "); } -void from_json(const json& j, 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"); +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError(std::string(e.what()) + " PlanNode PlanNode"); + } + + if (type == ".AggregationNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.GroupIdNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == ".DistinctLimitNode") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.ExchangeNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == ".FilterNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.JoinNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == ".LimitNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == ".MarkDistinctNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.SortNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == ".OutputNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == ".ProjectNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.RowNumberNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.RemoteSourceNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.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 DwrfEncryptionMetadata& p) { +void to_json(json& j, const GroupingSetDescriptor& p) { j = json::object(); to_json_key( j, - "fieldToKeyData", - p.fieldToKeyData, - "DwrfEncryptionMetadata", - "Map", - "fieldToKeyData"); - to_json_key( - j, - "extraMetadata", - p.extraMetadata, - "DwrfEncryptionMetadata", - "Map", - "extraMetadata"); + "groupingKeys", + p.groupingKeys, + "GroupingSetDescriptor", + "List", + "groupingKeys"); to_json_key( j, - "encryptionAlgorithm", - p.encryptionAlgorithm, - "DwrfEncryptionMetadata", - "String", - "encryptionAlgorithm"); + "groupingSetCount", + p.groupingSetCount, + "GroupingSetDescriptor", + "int", + "groupingSetCount"); to_json_key( j, - "encryptionProvider", - p.encryptionProvider, - "DwrfEncryptionMetadata", - "String", - "encryptionProvider"); + "globalGroupingSets", + p.globalGroupingSets, + "GroupingSetDescriptor", + "List", + "globalGroupingSets"); } -void from_json(const json& j, DwrfEncryptionMetadata& p) { - from_json_key( - j, - "fieldToKeyData", - p.fieldToKeyData, - "DwrfEncryptionMetadata", - "Map", - "fieldToKeyData"); +void from_json(const json& j, GroupingSetDescriptor& p) { from_json_key( j, - "extraMetadata", - p.extraMetadata, - "DwrfEncryptionMetadata", - "Map", - "extraMetadata"); + "groupingKeys", + p.groupingKeys, + "GroupingSetDescriptor", + "List", + "groupingKeys"); from_json_key( j, - "encryptionAlgorithm", - p.encryptionAlgorithm, - "DwrfEncryptionMetadata", - "String", - "encryptionAlgorithm"); + "groupingSetCount", + p.groupingSetCount, + "GroupingSetDescriptor", + "int", + "groupingSetCount"); from_json_key( j, - "encryptionProvider", - p.encryptionProvider, - "DwrfEncryptionMetadata", - "String", - "encryptionProvider"); + "globalGroupingSets", + p.globalGroupingSets, + "GroupingSetDescriptor", + "List", + "globalGroupingSets"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +AggregationNode::AggregationNode() noexcept { + _type = ".AggregationNode"; +} -void to_json(json& j, const EncryptionInformation& 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, - "dwrfEncryptionMetadata", - p.dwrfEncryptionMetadata, - "EncryptionInformation", - "DwrfEncryptionMetadata", - "dwrfEncryptionMetadata"); -} - -void from_json(const json& j, EncryptionInformation& p) { - from_json_key( + "aggregations", + p.aggregations, + "AggregationNode", + "Map", + "aggregations"); + to_json_key( j, - "dwrfEncryptionMetadata", - p.dwrfEncryptionMetadata, - "EncryptionInformation", - "DwrfEncryptionMetadata", - "dwrfEncryptionMetadata"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const SchemaTableName& p) { - j = json::object(); - to_json_key(j, "schema", p.schema, "SchemaTableName", "String", "schema"); - to_json_key(j, "table", p.table, "SchemaTableName", "String", "table"); -} - -void from_json(const json& j, SchemaTableName& p) { - from_json_key(j, "schema", p.schema, "SchemaTableName", "String", "schema"); - from_json_key(j, "table", p.table, "SchemaTableName", "String", "table"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Column& p) { - j = json::object(); - to_json_key(j, "name", p.name, "Column", "String", "name"); - to_json_key(j, "type", p.type, "Column", "String", "type"); -} - -void from_json(const json& j, Column& p) { - from_json_key(j, "name", p.name, "Column", "String", "name"); - from_json_key(j, "type", p.type, "Column", "String", "type"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const HiveBucketProperty& p) { - j = json::object(); + "groupingSets", + p.groupingSets, + "AggregationNode", + "GroupingSetDescriptor", + "groupingSets"); to_json_key( j, - "bucketedBy", - p.bucketedBy, - "HiveBucketProperty", - "List", - "bucketedBy"); + "preGroupedVariables", + p.preGroupedVariables, + "AggregationNode", + "List", + "preGroupedVariables"); to_json_key( - j, - "bucketCount", - p.bucketCount, - "HiveBucketProperty", - "int", - "bucketCount"); + j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); to_json_key( j, - "sortedBy", - p.sortedBy, - "HiveBucketProperty", - "List", - "sortedBy"); + "hashVariable", + p.hashVariable, + "AggregationNode", + "VariableReferenceExpression", + "hashVariable"); to_json_key( j, - "bucketFunctionType", - p.bucketFunctionType, - "HiveBucketProperty", - "BucketFunctionType", - "bucketFunctionType"); - to_json_key(j, "types", p.types, "HiveBucketProperty", "List", "types"); + "groupIdVariable", + p.groupIdVariable, + "AggregationNode", + "VariableReferenceExpression", + "groupIdVariable"); } -void from_json(const json& j, HiveBucketProperty& 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, - "bucketedBy", - p.bucketedBy, - "HiveBucketProperty", - "List", - "bucketedBy"); + "aggregations", + p.aggregations, + "AggregationNode", + "Map", + "aggregations"); from_json_key( j, - "bucketCount", - p.bucketCount, - "HiveBucketProperty", - "int", - "bucketCount"); + "groupingSets", + p.groupingSets, + "AggregationNode", + "GroupingSetDescriptor", + "groupingSets"); from_json_key( j, - "sortedBy", - p.sortedBy, - "HiveBucketProperty", - "List", - "sortedBy"); + "preGroupedVariables", + p.preGroupedVariables, + "AggregationNode", + "List", + "preGroupedVariables"); + from_json_key( + j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); from_json_key( j, - "bucketFunctionType", - p.bucketFunctionType, - "HiveBucketProperty", - "BucketFunctionType", - "bucketFunctionType"); + "hashVariable", + p.hashVariable, + "AggregationNode", + "VariableReferenceExpression", + "hashVariable"); from_json_key( - j, "types", p.types, "HiveBucketProperty", "List", "types"); + j, + "groupIdVariable", + p.groupIdVariable, + "AggregationNode", + "VariableReferenceExpression", + "groupIdVariable"); } } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ namespace facebook::presto::protocol { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; -void to_json(json& j, const StorageFormat& p) { - j = json::object(); - to_json_key(j, "serDe", p.serDe, "StorageFormat", "String", "serDe"); - to_json_key( - j, - "inputFormat", - p.inputFormat, - "StorageFormat", - "String", - "inputFormat"); - to_json_key( - j, - "outputFormat", - p.outputFormat, - "StorageFormat", - "String", - "outputFormat"); + if (type == "$remote") { + j = *std::static_pointer_cast(p); + return; + } + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); + return; + } + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ConnectorPartitioningHandle"); } -void from_json(const json& j, StorageFormat& p) { - from_json_key(j, "serDe", p.serDe, "StorageFormat", "String", "serDe"); - from_json_key( - j, - "inputFormat", - p.inputFormat, - "StorageFormat", - "String", - "inputFormat"); - from_json_key( - j, - "outputFormat", - p.outputFormat, - "StorageFormat", - "String", - "outputFormat"); +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError(std::string(e.what()) + " ConnectorPartitioningHandle"); + } + + if (type == "$remote") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + if (getConnectorKey(type) == "hive") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + if (getConnectorKey(type) == "tpch") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + + throw TypeError(type + " no abstract type ConnectorPartitioningHandle"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Storage& p) { +void to_json(json& j, const PartitioningHandle& p) { j = json::object(); to_json_key( j, - "storageFormat", - p.storageFormat, - "Storage", - "StorageFormat", - "storageFormat"); - to_json_key(j, "location", p.location, "Storage", "String", "location"); - to_json_key( - j, - "bucketProperty", - p.bucketProperty, - "Storage", - "HiveBucketProperty", - "bucketProperty"); - to_json_key(j, "skewed", p.skewed, "Storage", "bool", "skewed"); + "connectorId", + p.connectorId, + "PartitioningHandle", + "ConnectorId", + "connectorId"); to_json_key( j, - "serdeParameters", - p.serdeParameters, - "Storage", - "Map", - "serdeParameters"); + "transactionHandle", + p.transactionHandle, + "PartitioningHandle", + "ConnectorTransactionHandle", + "transactionHandle"); to_json_key( j, - "parameters", - p.parameters, - "Storage", - "Map", - "parameters"); + "connectorHandle", + p.connectorHandle, + "PartitioningHandle", + "ConnectorPartitioningHandle", + "connectorHandle"); } -void from_json(const json& j, Storage& p) { - from_json_key( - j, - "storageFormat", - p.storageFormat, - "Storage", - "StorageFormat", - "storageFormat"); - from_json_key(j, "location", p.location, "Storage", "String", "location"); +void from_json(const json& j, PartitioningHandle& p) { from_json_key( j, - "bucketProperty", - p.bucketProperty, - "Storage", - "HiveBucketProperty", - "bucketProperty"); - from_json_key(j, "skewed", p.skewed, "Storage", "bool", "skewed"); + "connectorId", + p.connectorId, + "PartitioningHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "serdeParameters", - p.serdeParameters, - "Storage", - "Map", - "serdeParameters"); + "transactionHandle", + p.transactionHandle, + "PartitioningHandle", + "ConnectorTransactionHandle", + "transactionHandle"); from_json_key( j, - "parameters", - p.parameters, - "Storage", - "Map", - "parameters"); + "connectorHandle", + p.connectorHandle, + "PartitioningHandle", + "ConnectorPartitioningHandle", + "connectorHandle"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair PrestoTableType_enum_table[] = - { // NOLINT: cert-err58-cpp - {PrestoTableType::MANAGED_TABLE, "MANAGED_TABLE"}, - {PrestoTableType::EXTERNAL_TABLE, "EXTERNAL_TABLE"}, - {PrestoTableType::VIRTUAL_VIEW, "VIRTUAL_VIEW"}, - {PrestoTableType::MATERIALIZED_VIEW, "MATERIALIZED_VIEW"}, - {PrestoTableType::TEMPORARY_TABLE, "TEMPORARY_TABLE"}, - {PrestoTableType::OTHER, "OTHER"}}; -void to_json(json& j, const PrestoTableType& e) { - static_assert( - std::is_enum::value, "PrestoTableType must be an enum!"); - const auto* it = std::find_if( - std::begin(PrestoTableType_enum_table), - std::end(PrestoTableType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(PrestoTableType_enum_table)) - ? it - : std::begin(PrestoTableType_enum_table)) - ->second; +void to_json(json& j, const 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, PrestoTableType& e) { - static_assert( - std::is_enum::value, "PrestoTableType must be an enum!"); - const auto* it = std::find_if( - std::begin(PrestoTableType_enum_table), - std::end(PrestoTableType_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(PrestoTableType_enum_table)) - ? it - : std::begin(PrestoTableType_enum_table)) - ->first; + +void from_json(const json& j, 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 Table& p) { +void to_json(json& j, const PartitioningScheme& p) { j = json::object(); to_json_key( - j, "databaseName", p.databaseName, "Table", "String", "databaseName"); - to_json_key(j, "tableName", p.tableName, "Table", "String", "tableName"); - to_json_key(j, "owner", p.owner, "Table", "String", "owner"); - to_json_key( - j, "tableType", p.tableType, "Table", "PrestoTableType", "tableType"); - to_json_key(j, "storage", p.storage, "Table", "Storage", "storage"); - to_json_key( - j, "dataColumns", p.dataColumns, "Table", "List", "dataColumns"); + j, + "partitioning", + p.partitioning, + "PartitioningScheme", + "Partitioning", + "partitioning"); to_json_key( j, - "partitionColumns", - p.partitionColumns, - "Table", - "List", - "partitionColumns"); + "outputLayout", + p.outputLayout, + "PartitioningScheme", + "List", + "outputLayout"); to_json_key( j, - "parameters", - p.parameters, - "Table", - "Map", - "parameters"); + "hashColumn", + p.hashColumn, + "PartitioningScheme", + "VariableReferenceExpression", + "hashColumn"); to_json_key( j, - "viewOriginalText", - p.viewOriginalText, - "Table", - "String", - "viewOriginalText"); + "replicateNullsAndAny", + p.replicateNullsAndAny, + "PartitioningScheme", + "bool", + "replicateNullsAndAny"); to_json_key( j, - "viewExpandedText", - p.viewExpandedText, - "Table", - "String", - "viewExpandedText"); + "bucketToPartition", + p.bucketToPartition, + "PartitioningScheme", + "List", + "bucketToPartition"); } -void from_json(const json& j, Table& p) { - from_json_key( - j, "databaseName", p.databaseName, "Table", "String", "databaseName"); - from_json_key(j, "tableName", p.tableName, "Table", "String", "tableName"); - from_json_key(j, "owner", p.owner, "Table", "String", "owner"); - from_json_key( - j, "tableType", p.tableType, "Table", "PrestoTableType", "tableType"); - from_json_key(j, "storage", p.storage, "Table", "Storage", "storage"); - from_json_key( - j, "dataColumns", p.dataColumns, "Table", "List", "dataColumns"); +void from_json(const json& j, PartitioningScheme& p) { from_json_key( j, - "partitionColumns", - p.partitionColumns, - "Table", - "List", - "partitionColumns"); + "partitioning", + p.partitioning, + "PartitioningScheme", + "Partitioning", + "partitioning"); from_json_key( j, - "parameters", - p.parameters, - "Table", - "Map", - "parameters"); + "outputLayout", + p.outputLayout, + "PartitioningScheme", + "List", + "outputLayout"); from_json_key( j, - "viewOriginalText", - p.viewOriginalText, - "Table", - "String", - "viewOriginalText"); + "hashColumn", + p.hashColumn, + "PartitioningScheme", + "VariableReferenceExpression", + "hashColumn"); from_json_key( j, - "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, - "schemaTableName", - p.schemaTableName, - "HivePageSinkMetadata", - "SchemaTableName", - "schemaTableName"); - to_json_key(j, "table", p.table, "HivePageSinkMetadata", "Table", "table"); -} - -void from_json(const json& j, HivePageSinkMetadata& p) { + "replicateNullsAndAny", + p.replicateNullsAndAny, + "PartitioningScheme", + "bool", + "replicateNullsAndAny"); from_json_key( j, - "schemaTableName", - p.schemaTableName, - "HivePageSinkMetadata", - "SchemaTableName", - "schemaTableName"); - from_json_key(j, "table", p.table, "HivePageSinkMetadata", "Table", "table"); + "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 { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -static const std::pair HiveStorageFormat_enum_table[] = +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair ExchangeNodeScope_enum_table[] = { // NOLINT: cert-err58-cpp - {HiveStorageFormat::ORC, "ORC"}, - {HiveStorageFormat::DWRF, "DWRF"}, - {HiveStorageFormat::PARQUET, "PARQUET"}, - {HiveStorageFormat::AVRO, "AVRO"}, - {HiveStorageFormat::RCBINARY, "RCBINARY"}, - {HiveStorageFormat::RCTEXT, "RCTEXT"}, - {HiveStorageFormat::SEQUENCEFILE, "SEQUENCEFILE"}, - {HiveStorageFormat::JSON, "JSON"}, - {HiveStorageFormat::TEXTFILE, "TEXTFILE"}, - {HiveStorageFormat::CSV, "CSV"}, - {HiveStorageFormat::PAGEFILE, "PAGEFILE"}}; - -void to_json(json& j, const HiveStorageFormat& p) { + {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, - "HiveStorageFormat must be an enum!"); + std::is_enum::value, + "ExchangeNodeScope 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; + 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(HiveStorageFormat_enum_table)) + j = ((it != std::end(ExchangeNodeScope_enum_table)) ? it - : std::begin(HiveStorageFormat_enum_table)) + : std::begin(ExchangeNodeScope_enum_table)) ->second; } - -void from_json(const json& j, HiveStorageFormat& e) { +void from_json(const json& j, ExchangeNodeScope& e) { static_assert( - std::is_enum::value, - "HiveStorageFormat must be an enum!"); + std::is_enum::value, + "ExchangeNodeScope 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 { + 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(HiveStorageFormat_enum_table)) + e = ((it != std::end(ExchangeNodeScope_enum_table)) ? it - : std::begin(HiveStorageFormat_enum_table)) + : std::begin(ExchangeNodeScope_enum_table)) ->first; } } // namespace facebook::presto::protocol @@ -2519,346 +1779,434 @@ 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 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, - "HiveCompressionCodec must be an enum!"); + std::is_enum::value, + "ExchangeNodeType 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 { + 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(HiveCompressionCodec_enum_table)) + j = ((it != std::end(ExchangeNodeType_enum_table)) ? it - : std::begin(HiveCompressionCodec_enum_table)) + : std::begin(ExchangeNodeType_enum_table)) ->second; } -void from_json(const json& j, HiveCompressionCodec& e) { +void from_json(const json& j, ExchangeNodeType& e) { static_assert( - std::is_enum::value, - "HiveCompressionCodec must be an enum!"); + std::is_enum::value, + "ExchangeNodeType 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 { + 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(HiveCompressionCodec_enum_table)) + e = ((it != std::end(ExchangeNodeType_enum_table)) ? it - : std::begin(HiveCompressionCodec_enum_table)) + : std::begin(ExchangeNodeType_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveOutputTableHandle::HiveOutputTableHandle() noexcept { - _type = "hive"; +ExchangeNode::ExchangeNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.ExchangeNode"; } -void to_json(json& j, const HiveOutputTableHandle& p) { +void to_json(json& j, const ExchangeNode& p) { j = json::object(); - j["@type"] = "hive"; - to_json_key( - j, - "schemaName", - p.schemaName, - "HiveOutputTableHandle", - "String", - "schemaName"); - to_json_key( - j, - "tableName", - p.tableName, - "HiveOutputTableHandle", - "String", - "tableName"); - to_json_key( - j, - "inputColumns", - p.inputColumns, - "HiveOutputTableHandle", - "List", - "inputColumns"); - to_json_key( - j, - "pageSinkMetadata", - p.pageSinkMetadata, - "HiveOutputTableHandle", - "HivePageSinkMetadata", - "pageSinkMetadata"); - to_json_key( - j, - "locationHandle", - p.locationHandle, - "HiveOutputTableHandle", - "LocationHandle", - "locationHandle"); - to_json_key( - j, - "tableStorageFormat", - p.tableStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "tableStorageFormat"); - to_json_key( - j, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); - to_json_key( - j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); - to_json_key( - j, - "compressionCodec", - p.compressionCodec, - "HiveOutputTableHandle", - "HiveCompressionCodec", - "compressionCodec"); + 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, - "partitionedBy", - p.partitionedBy, - "HiveOutputTableHandle", - "List", - "partitionedBy"); + j, "scope", p.scope, "ExchangeNode", "ExchangeNodeScope", "scope"); to_json_key( j, - "bucketProperty", - p.bucketProperty, - "HiveOutputTableHandle", - "HiveBucketProperty", - "bucketProperty"); + "partitioningScheme", + p.partitioningScheme, + "ExchangeNode", + "PartitioningScheme", + "partitioningScheme"); to_json_key( j, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveOutputTableHandle", - "List", - "preferredOrderingColumns"); + "sources", + p.sources, + "ExchangeNode", + "List>", + "sources"); to_json_key( j, - "tableOwner", - p.tableOwner, - "HiveOutputTableHandle", - "String", - "tableOwner"); + "inputs", + p.inputs, + "ExchangeNode", + "List>", + "inputs"); to_json_key( j, - "additionalTableParameters", - p.additionalTableParameters, - "HiveOutputTableHandle", - "Map", - "additionalTableParameters"); + "ensureSourceOrdering", + p.ensureSourceOrdering, + "ExchangeNode", + "bool", + "ensureSourceOrdering"); to_json_key( j, - "encryptionInformation", - p.encryptionInformation, - "HiveOutputTableHandle", - "EncryptionInformation", - "encryptionInformation"); + "orderingScheme", + p.orderingScheme, + "ExchangeNode", + "OrderingScheme", + "orderingScheme"); } -void from_json(const json& j, HiveOutputTableHandle& p) { +void from_json(const json& j, ExchangeNode& p) { p._type = j["@type"]; + from_json_key(j, "id", p.id, "ExchangeNode", "PlanNodeId", "id"); + from_json_key(j, "type", p.type, "ExchangeNode", "ExchangeNodeType", "type"); from_json_key( - j, - "schemaName", - p.schemaName, - "HiveOutputTableHandle", - "String", - "schemaName"); + j, "scope", p.scope, "ExchangeNode", "ExchangeNodeScope", "scope"); from_json_key( j, - "tableName", - p.tableName, - "HiveOutputTableHandle", - "String", - "tableName"); + "partitioningScheme", + p.partitioningScheme, + "ExchangeNode", + "PartitioningScheme", + "partitioningScheme"); from_json_key( j, - "inputColumns", - p.inputColumns, - "HiveOutputTableHandle", - "List", - "inputColumns"); + "sources", + p.sources, + "ExchangeNode", + "List>", + "sources"); from_json_key( j, - "pageSinkMetadata", - p.pageSinkMetadata, - "HiveOutputTableHandle", - "HivePageSinkMetadata", - "pageSinkMetadata"); + "inputs", + p.inputs, + "ExchangeNode", + "List>", + "inputs"); from_json_key( j, - "locationHandle", - p.locationHandle, - "HiveOutputTableHandle", - "LocationHandle", - "locationHandle"); + "ensureSourceOrdering", + p.ensureSourceOrdering, + "ExchangeNode", + "bool", + "ensureSourceOrdering"); from_json_key( j, - "tableStorageFormat", - p.tableStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "tableStorageFormat"); - from_json_key( + "orderingScheme", + p.orderingScheme, + "ExchangeNode", + "OrderingScheme", + "orderingScheme"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +RemoteSourceNode::RemoteSourceNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.RemoteSourceNode"; +} + +void to_json(json& j, const RemoteSourceNode& p) { + j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.RemoteSourceNode"; + to_json_key(j, "id", p.id, "RemoteSourceNode", "PlanNodeId", "id"); + to_json_key( + j, + "sourceFragmentIds", + p.sourceFragmentIds, + "RemoteSourceNode", + "List", + "sourceFragmentIds"); + to_json_key( j, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); - from_json_key( + "outputVariables", + p.outputVariables, + "RemoteSourceNode", + "List", + "outputVariables"); + to_json_key( j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); - from_json_key( + "ensureSourceOrdering", + p.ensureSourceOrdering, + "RemoteSourceNode", + "bool", + "ensureSourceOrdering"); + to_json_key( j, - "compressionCodec", - p.compressionCodec, - "HiveOutputTableHandle", - "HiveCompressionCodec", - "compressionCodec"); - from_json_key( + "orderingScheme", + p.orderingScheme, + "RemoteSourceNode", + "OrderingScheme", + "orderingScheme"); + to_json_key( j, - "partitionedBy", - p.partitionedBy, - "HiveOutputTableHandle", - "List", - "partitionedBy"); + "exchangeType", + p.exchangeType, + "RemoteSourceNode", + "ExchangeNodeType", + "exchangeType"); +} + +void from_json(const json& j, RemoteSourceNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "RemoteSourceNode", "PlanNodeId", "id"); from_json_key( j, - "bucketProperty", - p.bucketProperty, - "HiveOutputTableHandle", - "HiveBucketProperty", - "bucketProperty"); + "sourceFragmentIds", + p.sourceFragmentIds, + "RemoteSourceNode", + "List", + "sourceFragmentIds"); from_json_key( j, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveOutputTableHandle", - "List", - "preferredOrderingColumns"); + "outputVariables", + p.outputVariables, + "RemoteSourceNode", + "List", + "outputVariables"); from_json_key( j, - "tableOwner", - p.tableOwner, - "HiveOutputTableHandle", - "String", - "tableOwner"); + "ensureSourceOrdering", + p.ensureSourceOrdering, + "RemoteSourceNode", + "bool", + "ensureSourceOrdering"); from_json_key( j, - "additionalTableParameters", - p.additionalTableParameters, - "HiveOutputTableHandle", - "Map", - "additionalTableParameters"); + "orderingScheme", + p.orderingScheme, + "RemoteSourceNode", + "OrderingScheme", + "orderingScheme"); from_json_key( j, - "encryptionInformation", - p.encryptionInformation, - "HiveOutputTableHandle", - "EncryptionInformation", - "encryptionInformation"); + "exchangeType", + p.exchangeType, + "RemoteSourceNode", + "ExchangeNodeType", + "exchangeType"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const HivePartitionKey& p) { +void to_json(json& j, const SplitContext& p) { j = json::object(); - to_json_key(j, "name", p.name, "HivePartitionKey", "String", "name"); - to_json_key(j, "value", p.value, "HivePartitionKey", "String", "value"); + to_json_key(j, "cacheable", p.cacheable, "SplitContext", "bool", "cacheable"); } -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"); +void from_json(const json& j, SplitContext& p) { + from_json_key( + j, "cacheable", p.cacheable, "SplitContext", "bool", "cacheable"); } } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ namespace facebook::presto::protocol { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; -void to_json(json& j, const PlanCostEstimate& p) { + if (type == "$remote") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "$empty") { + j = *std::static_pointer_cast(p); + return; + } + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); + return; + } + + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ConnectorSplit"); +} + +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError(std::string(e.what()) + " ConnectorSplit"); + } + + if (type == "$remote") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + if (type == "$empty") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + if (getConnectorKey(type) == "hive") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + + if (getConnectorKey(type) == "tpch") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + + throw TypeError(type + " no abstract type ConnectorSplit"); +} +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +namespace facebook::presto::protocol { + +void to_json(json& j, const Lifespan& p) { + if (p.isgroup) { + j = "Group" + std::to_string(p.groupid); + } else { + j = "TaskWide"; + } +} + +void from_json(const json& j, Lifespan& p) { + String lifespan = j; + + if (lifespan == "TaskWide") { + p.isgroup = false; + p.groupid = 0; + } else { + if (lifespan != "Group") { + // fail... + } + p.isgroup = true; + p.groupid = std::stoi(lifespan.substr(strlen("Group"))); + } +} + +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const Split& p) { j = json::object(); - to_json_key(j, "cpuCost", p.cpuCost, "PlanCostEstimate", "double", "cpuCost"); to_json_key( - j, "maxMemory", p.maxMemory, "PlanCostEstimate", "double", "maxMemory"); + j, "connectorId", p.connectorId, "Split", "ConnectorId", "connectorId"); + to_json_key( + j, + "transactionHandle", + p.transactionHandle, + "Split", + "ConnectorTransactionHandle", + "transactionHandle"); to_json_key( j, - "maxMemoryWhenOutputting", - p.maxMemoryWhenOutputting, - "PlanCostEstimate", - "double", - "maxMemoryWhenOutputting"); + "connectorSplit", + p.connectorSplit, + "Split", + "ConnectorSplit", + "connectorSplit"); + to_json_key(j, "lifespan", p.lifespan, "Split", "Lifespan", "lifespan"); to_json_key( j, - "networkCost", - p.networkCost, - "PlanCostEstimate", - "double", - "networkCost"); + "splitContext", + p.splitContext, + "Split", + "SplitContext", + "splitContext"); } -void from_json(const json& j, PlanCostEstimate& p) { +void from_json(const json& j, Split& p) { from_json_key( - j, "cpuCost", p.cpuCost, "PlanCostEstimate", "double", "cpuCost"); + j, "connectorId", p.connectorId, "Split", "ConnectorId", "connectorId"); from_json_key( - j, "maxMemory", p.maxMemory, "PlanCostEstimate", "double", "maxMemory"); + j, + "transactionHandle", + p.transactionHandle, + "Split", + "ConnectorTransactionHandle", + "transactionHandle"); from_json_key( j, - "maxMemoryWhenOutputting", - p.maxMemoryWhenOutputting, - "PlanCostEstimate", - "double", - "maxMemoryWhenOutputting"); + "connectorSplit", + p.connectorSplit, + "Split", + "ConnectorSplit", + "connectorSplit"); + from_json_key(j, "lifespan", p.lifespan, "Split", "Lifespan", "lifespan"); from_json_key( j, - "networkCost", - p.networkCost, - "PlanCostEstimate", - "double", - "networkCost"); + "splitContext", + p.splitContext, + "Split", + "SplitContext", + "splitContext"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Assignments& p) { +void to_json(json& j, const ScheduledSplit& p) { j = json::object(); + to_json_key( + j, "sequenceId", p.sequenceId, "ScheduledSplit", "int64_t", "sequenceId"); to_json_key( j, - "assignments", - p.assignments, - "Assignments", - "Map>", - "assignments"); + "planNodeId", + p.planNodeId, + "ScheduledSplit", + "PlanNodeId", + "planNodeId"); + to_json_key(j, "split", p.split, "ScheduledSplit", "Split", "split"); } -void from_json(const json& j, Assignments& p) { +void from_json(const json& j, ScheduledSplit& p) { + from_json_key( + j, "sequenceId", p.sequenceId, "ScheduledSplit", "int64_t", "sequenceId"); from_json_key( j, - "assignments", - p.assignments, - "Assignments", - "Map>", - "assignments"); + "planNodeId", + p.planNodeId, + "ScheduledSplit", + "PlanNodeId", + "planNodeId"); + from_json_key(j, "split", p.split, "ScheduledSplit", "Split", "split"); } } // namespace facebook::presto::protocol /* @@ -2875,944 +2223,753 @@ void from_json(const json& j, Assignments& 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; - - if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); - return; - } - - if (getConnectorKey(type) == "tpch") { - j = *std::static_pointer_cast(p); - return; - } - - throw TypeError(type + " no abstract type ConnectorTableHandle"); -} - -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError( - std::string(e.what()) + " ConnectorTableHandle ConnectorTableHandle"); - } - - if (getConnectorKey(type) == "hive") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - - if (getConnectorKey(type) == "tpch") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - - throw TypeError(type + " no abstract type ConnectorTableHandle"); +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 AnalyzeTableHandle& p) { +void to_json(json& j, const StatisticAggregations& p) { j = json::object(); to_json_key( j, - "connectorId", - p.connectorId, - "AnalyzeTableHandle", - "ConnectorId", - "connectorId"); + "outputVariables", + p.outputVariables, + "StatisticAggregations", + "std::vector", + "outputVariables"); to_json_key( j, - "transactionHandle", - p.transactionHandle, - "AnalyzeTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "aggregations", + p.aggregations, + "StatisticAggregations", + "Map", + "aggregations"); to_json_key( j, - "connectorHandle", - p.connectorHandle, - "AnalyzeTableHandle", - "ConnectorTableHandle", - "connectorHandle"); + "groupingVariables", + p.groupingVariables, + "StatisticAggregations", + "std::vector", + "groupingVariables"); } -void from_json(const json& j, AnalyzeTableHandle& p) { +void from_json(const json& j, StatisticAggregations& p) { from_json_key( j, - "connectorId", - p.connectorId, - "AnalyzeTableHandle", - "ConnectorId", - "connectorId"); + "outputVariables", + p.outputVariables, + "StatisticAggregations", + "std::vector", + "outputVariables"); from_json_key( j, - "transactionHandle", - p.transactionHandle, - "AnalyzeTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "aggregations", + p.aggregations, + "StatisticAggregations", + "Map", + "aggregations"); from_json_key( j, - "connectorHandle", - p.connectorHandle, - "AnalyzeTableHandle", - "ConnectorTableHandle", - "connectorHandle"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -GroupIdNode::GroupIdNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.GroupIdNode"; + "groupingVariables", + p.groupingVariables, + "StatisticAggregations", + "std::vector", + "groupingVariables"); } -void to_json(json& j, const GroupIdNode& p) { +void to_json(json& j, const TableWriterNode& 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.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, - "groupingSets", - p.groupingSets, - "GroupIdNode", - "List>", - "groupingSets"); + "rowCountVariable", + p.rowCountVariable, + "TableWriterNode", + "VariableReferenceExpression", + "rowCountVariable"); + to_json_key( + j, + "fragmentVariable", + p.fragmentVariable, + "TableWriterNode", + "VariableReferenceExpression", + "fragmentVariable"); + to_json_key( + j, + "tableCommitContextVariable", + p.tableCommitContextVariable, + "TableWriterNode", + "VariableReferenceExpression", + "tableCommitContextVariable"); + to_json_key( + j, + "columns", + p.columns, + "TableWriterNode", + "List", + "columns"); + to_json_key( + j, + "columnNames", + p.columnNames, + "TableWriterNode", + "List", + "columnNames"); + to_json_key( + j, + "notNullColumnVariables", + p.notNullColumnVariables, + "TableWriterNode", + "List", + "notNullColumnVariables"); to_json_key( j, - "groupingColumns", - p.groupingColumns, - "GroupIdNode", - "Map", - "groupingColumns"); + "partitioningScheme", + p.partitioningScheme, + "TableWriterNode", + "PartitioningScheme", + "partitioningScheme"); to_json_key( j, - "aggregationArguments", - p.aggregationArguments, - "GroupIdNode", - "List", - "aggregationArguments"); + "preferredShufflePartitioningScheme", + p.preferredShufflePartitioningScheme, + "TableWriterNode", + "PartitioningScheme", + "preferredShufflePartitioningScheme"); to_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "GroupIdNode", - "VariableReferenceExpression", - "groupIdVariable"); + "statisticsAggregation", + p.statisticsAggregation, + "TableWriterNode", + "StatisticAggregations", + "statisticsAggregation"); } -void from_json(const json& j, GroupIdNode& p) { +void from_json(const json& j, TableWriterNode& 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, "id", p.id, "TableWriterNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "TableWriterNode", "PlanNode", "source"); from_json_key( j, - "groupingSets", - p.groupingSets, - "GroupIdNode", - "List>", - "groupingSets"); + "rowCountVariable", + p.rowCountVariable, + "TableWriterNode", + "VariableReferenceExpression", + "rowCountVariable"); from_json_key( j, - "groupingColumns", - p.groupingColumns, - "GroupIdNode", - "Map", - "groupingColumns"); + "fragmentVariable", + p.fragmentVariable, + "TableWriterNode", + "VariableReferenceExpression", + "fragmentVariable"); from_json_key( j, - "aggregationArguments", - p.aggregationArguments, - "GroupIdNode", + "tableCommitContextVariable", + p.tableCommitContextVariable, + "TableWriterNode", + "VariableReferenceExpression", + "tableCommitContextVariable"); + from_json_key( + j, + "columns", + p.columns, + "TableWriterNode", "List", - "aggregationArguments"); + "columns"); from_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "GroupIdNode", - "VariableReferenceExpression", - "groupIdVariable"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair 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 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() - -// 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; + "columnNames", + p.columnNames, + "TableWriterNode", + "List", + "columnNames"); + from_json_key( + j, + "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 +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ namespace facebook::presto::protocol { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; -void to_json(json& j, const Language& p) { - j = json::object(); - to_json_key(j, "language", p.language, "Language", "String", "language"); + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ConnectorInsertTableHandle "); } -void from_json(const json& j, Language& p) { - from_json_key(j, "language", p.language, "Language", "String", "language"); +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError( + std::string(e.what()) + + " ConnectorInsertTableHandle ConnectorInsertTableHandle"); + } + + if (getConnectorKey(type) == "hive") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + + throw TypeError(type + " no abstract type ConnectorInsertTableHandle "); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const RoutineCharacteristics& p) { +void to_json(json& j, const InsertTableHandle& p) { j = json::object(); to_json_key( j, - "language", - p.language, - "RoutineCharacteristics", - "Language", - "language"); + "connectorId", + p.connectorId, + "InsertTableHandle", + "ConnectorId", + "connectorId"); to_json_key( j, - "determinism", - p.determinism, - "RoutineCharacteristics", - "Determinism", - "determinism"); + "transactionHandle", + p.transactionHandle, + "InsertTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); to_json_key( j, - "nullCallClause", - p.nullCallClause, - "RoutineCharacteristics", - "NullCallClause", - "nullCallClause"); + "connectorHandle", + p.connectorHandle, + "InsertTableHandle", + "ConnectorInsertTableHandle", + "connectorHandle"); } -void from_json(const json& j, RoutineCharacteristics& p) { +void from_json(const json& j, InsertTableHandle& p) { from_json_key( j, - "language", - p.language, - "RoutineCharacteristics", - "Language", - "language"); + "connectorId", + p.connectorId, + "InsertTableHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "determinism", - p.determinism, - "RoutineCharacteristics", - "Determinism", - "determinism"); + "transactionHandle", + p.transactionHandle, + "InsertTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); from_json_key( j, - "nullCallClause", - p.nullCallClause, - "RoutineCharacteristics", - "NullCallClause", - "nullCallClause"); + "connectorHandle", + p.connectorHandle, + "InsertTableHandle", + "ConnectorInsertTableHandle", + "connectorHandle"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const TypeVariableConstraint& p) { - j = json::object(); - to_json_key(j, "name", p.name, "TypeVariableConstraint", "String", "name"); - to_json_key( - j, - "comparableRequired", - p.comparableRequired, - "TypeVariableConstraint", - "bool", - "comparableRequired"); - to_json_key( - j, - "orderableRequired", - p.orderableRequired, - "TypeVariableConstraint", - "bool", - "orderableRequired"); - to_json_key( - j, - "variadicBound", - p.variadicBound, - "TypeVariableConstraint", - "String", - "variadicBound"); - to_json_key( - j, - "nonDecimalNumericRequired", - p.nonDecimalNumericRequired, - "TypeVariableConstraint", - "bool", - "nonDecimalNumericRequired"); +// 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, TypeVariableConstraint& p) { - from_json_key(j, "name", p.name, "TypeVariableConstraint", "String", "name"); - from_json_key( - j, - "comparableRequired", - p.comparableRequired, - "TypeVariableConstraint", - "bool", - "comparableRequired"); - from_json_key( - j, - "orderableRequired", - p.orderableRequired, - "TypeVariableConstraint", - "bool", - "orderableRequired"); - from_json_key( - j, - "variadicBound", - p.variadicBound, - "TypeVariableConstraint", - "String", - "variadicBound"); - from_json_key( - j, - "nonDecimalNumericRequired", - p.nonDecimalNumericRequired, - "TypeVariableConstraint", - "bool", - "nonDecimalNumericRequired"); +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 LongVariableConstraint& p) { +void to_json(json& j, const SortingColumn& p) { j = json::object(); - to_json_key(j, "name", p.name, "LongVariableConstraint", "String", "name"); to_json_key( - j, - "expression", - p.expression, - "LongVariableConstraint", - "String", - "expression"); + j, "columnName", p.columnName, "SortingColumn", "String", "columnName"); + to_json_key(j, "order", p.order, "SortingColumn", "Order", "order"); } -void from_json(const json& j, LongVariableConstraint& p) { - from_json_key(j, "name", p.name, "LongVariableConstraint", "String", "name"); +void from_json(const json& j, SortingColumn& p) { from_json_key( - j, - "expression", - p.expression, - "LongVariableConstraint", - "String", - "expression"); + 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 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 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, "FunctionKind must be an enum!"); + std::is_enum::value, + "BucketFunctionType 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(BucketFunctionType_enum_table), + std::end(BucketFunctionType_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(BucketFunctionType_enum_table)) ? it - : std::begin(FunctionKind_enum_table)) + : std::begin(BucketFunctionType_enum_table)) ->second; } -void from_json(const json& j, FunctionKind& e) { +void from_json(const json& j, BucketFunctionType& e) { static_assert( - std::is_enum::value, "FunctionKind must be an enum!"); + std::is_enum::value, + "BucketFunctionType 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(BucketFunctionType_enum_table), + std::end(BucketFunctionType_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(BucketFunctionType_enum_table)) ? it - : std::begin(FunctionKind_enum_table)) + : std::begin(BucketFunctionType_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 HiveBucketProperty& 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"); + "bucketedBy", + p.bucketedBy, + "HiveBucketProperty", + "List", + "bucketedBy"); to_json_key( j, - "returnType", - p.returnType, - "Signature", - "TypeSignature", - "returnType"); + "bucketCount", + p.bucketCount, + "HiveBucketProperty", + "int", + "bucketCount"); to_json_key( j, - "argumentTypes", - p.argumentTypes, - "Signature", - "List", - "argumentTypes"); + "sortedBy", + p.sortedBy, + "HiveBucketProperty", + "List", + "sortedBy"); to_json_key( j, - "variableArity", - p.variableArity, - "Signature", - "bool", - "variableArity"); + "bucketFunctionType", + p.bucketFunctionType, + "HiveBucketProperty", + "BucketFunctionType", + "bucketFunctionType"); + to_json_key(j, "types", p.types, "HiveBucketProperty", "List", "types"); } -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"); +void from_json(const json& j, HiveBucketProperty& p) { from_json_key( j, - "typeVariableConstraints", - p.typeVariableConstraints, - "Signature", - "List", - "typeVariableConstraints"); + "bucketedBy", + p.bucketedBy, + "HiveBucketProperty", + "List", + "bucketedBy"); from_json_key( j, - "longVariableConstraints", - p.longVariableConstraints, - "Signature", - "List", - "longVariableConstraints"); + "bucketCount", + p.bucketCount, + "HiveBucketProperty", + "int", + "bucketCount"); from_json_key( j, - "returnType", - p.returnType, - "Signature", - "TypeSignature", - "returnType"); + "sortedBy", + p.sortedBy, + "HiveBucketProperty", + "List", + "sortedBy"); from_json_key( j, - "argumentTypes", - p.argumentTypes, - "Signature", - "List", - "argumentTypes"); + "bucketFunctionType", + p.bucketFunctionType, + "HiveBucketProperty", + "BucketFunctionType", + "bucketFunctionType"); from_json_key( - j, - "variableArity", - p.variableArity, - "Signature", - "bool", - "variableArity"); + j, "types", p.types, "HiveBucketProperty", "List", "types"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const SqlInvokedFunction& p) { +void to_json(json& j, const StorageFormat& p) { j = json::object(); + to_json_key(j, "serDe", p.serDe, "StorageFormat", "String", "serDe"); to_json_key( j, - "parameters", - p.parameters, - "SqlInvokedFunction", - "List", - "parameters"); + "inputFormat", + p.inputFormat, + "StorageFormat", + "String", + "inputFormat"); to_json_key( j, - "description", - p.description, - "SqlInvokedFunction", + "outputFormat", + p.outputFormat, + "StorageFormat", "String", - "description"); + "outputFormat"); +} + +void from_json(const json& j, StorageFormat& p) { + from_json_key(j, "serDe", p.serDe, "StorageFormat", "String", "serDe"); + from_json_key( + j, + "inputFormat", + p.inputFormat, + "StorageFormat", + "String", + "inputFormat"); + from_json_key( + j, + "outputFormat", + p.outputFormat, + "StorageFormat", + "String", + "outputFormat"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const Storage& p) { + j = json::object(); to_json_key( j, - "routineCharacteristics", - p.routineCharacteristics, - "SqlInvokedFunction", - "RoutineCharacteristics", - "routineCharacteristics"); - to_json_key(j, "body", p.body, "SqlInvokedFunction", "String", "body"); + "storageFormat", + p.storageFormat, + "Storage", + "StorageFormat", + "storageFormat"); + to_json_key(j, "location", p.location, "Storage", "String", "location"); to_json_key( j, - "signature", - p.signature, - "SqlInvokedFunction", - "Signature", - "signature"); + "bucketProperty", + p.bucketProperty, + "Storage", + "HiveBucketProperty", + "bucketProperty"); + to_json_key(j, "skewed", p.skewed, "Storage", "bool", "skewed"); to_json_key( j, - "functionId", - p.functionId, - "SqlInvokedFunction", - "SqlFunctionId", - "functionId"); + "serdeParameters", + p.serdeParameters, + "Storage", + "Map", + "serdeParameters"); + to_json_key( + j, + "parameters", + p.parameters, + "Storage", + "Map", + "parameters"); +} + +void from_json(const json& j, Storage& p) { + from_json_key( + j, + "storageFormat", + p.storageFormat, + "Storage", + "StorageFormat", + "storageFormat"); + from_json_key(j, "location", p.location, "Storage", "String", "location"); + from_json_key( + j, + "bucketProperty", + p.bucketProperty, + "Storage", + "HiveBucketProperty", + "bucketProperty"); + from_json_key(j, "skewed", p.skewed, "Storage", "bool", "skewed"); + from_json_key( + j, + "serdeParameters", + p.serdeParameters, + "Storage", + "Map", + "serdeParameters"); + from_json_key( + j, + "parameters", + p.parameters, + "Storage", + "Map", + "parameters"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair CacheQuotaScope_enum_table[] = + { // NOLINT: cert-err58-cpp + {CacheQuotaScope::GLOBAL, "GLOBAL"}, + {CacheQuotaScope::SCHEMA, "SCHEMA"}, + {CacheQuotaScope::TABLE, "TABLE"}, + {CacheQuotaScope::PARTITION, "PARTITION"}}; +void to_json(json& j, const CacheQuotaScope& e) { + static_assert( + std::is_enum::value, "CacheQuotaScope must be an enum!"); + const auto* it = std::find_if( + std::begin(CacheQuotaScope_enum_table), + std::end(CacheQuotaScope_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(CacheQuotaScope_enum_table)) + ? it + : std::begin(CacheQuotaScope_enum_table)) + ->second; +} +void from_json(const json& j, CacheQuotaScope& e) { + static_assert( + std::is_enum::value, "CacheQuotaScope must be an enum!"); + const auto* it = std::find_if( + std::begin(CacheQuotaScope_enum_table), + std::end(CacheQuotaScope_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(CacheQuotaScope_enum_table)) + ? it + : std::begin(CacheQuotaScope_enum_table)) + ->first; } +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ -void from_json(const json& j, SqlInvokedFunction& p) { - from_json_key( - j, - "parameters", - p.parameters, - "SqlInvokedFunction", - "List", - "parameters"); - from_json_key( - j, - "description", - p.description, - "SqlInvokedFunction", - "String", - "description"); - from_json_key( - j, - "routineCharacteristics", - p.routineCharacteristics, - "SqlInvokedFunction", - "RoutineCharacteristics", - "routineCharacteristics"); - from_json_key(j, "body", p.body, "SqlInvokedFunction", "String", "body"); - from_json_key( - j, - "signature", - p.signature, - "SqlInvokedFunction", - "Signature", - "signature"); - from_json_key( - j, - "functionId", - p.functionId, - "SqlInvokedFunction", - "SqlFunctionId", - "functionId"); +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 { -void to_json(json& j, const HiveBucketHandle& p) { +void to_json(json& j, const CacheQuotaRequirement& p) { j = json::object(); to_json_key( j, - "columns", - p.columns, - "HiveBucketHandle", - "List", - "columns"); - to_json_key( - j, - "tableBucketCount", - p.tableBucketCount, - "HiveBucketHandle", - "int", - "tableBucketCount"); + "cacheQuotaScope", + p.cacheQuotaScope, + "CacheQuotaRequirement", + "CacheQuotaScope", + "cacheQuotaScope"); to_json_key( - j, - "readBucketCount", - p.readBucketCount, - "HiveBucketHandle", - "int", - "readBucketCount"); + j, "quota", p.quota, "CacheQuotaRequirement", "DataSize", "quota"); } -void from_json(const json& j, HiveBucketHandle& p) { - from_json_key( - j, - "columns", - p.columns, - "HiveBucketHandle", - "List", - "columns"); +void from_json(const json& j, CacheQuotaRequirement& p) { from_json_key( j, - "tableBucketCount", - p.tableBucketCount, - "HiveBucketHandle", - "int", - "tableBucketCount"); + "cacheQuotaScope", + p.cacheQuotaScope, + "CacheQuotaRequirement", + "CacheQuotaScope", + "cacheQuotaScope"); from_json_key( - j, - "readBucketCount", - p.readBucketCount, - "HiveBucketHandle", - "int", - "readBucketCount"); + j, "quota", p.quota, "CacheQuotaRequirement", "DataSize", "quota"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const VariableStatsEstimate& p) { +void to_json(json& j, const HivePartitionKey& p) { j = json::object(); - to_json_key( - j, "lowValue", p.lowValue, "VariableStatsEstimate", "double", "lowValue"); - to_json_key( - j, - "highValue", - p.highValue, - "VariableStatsEstimate", - "double", - "highValue"); - to_json_key( - j, - "nullsFraction", - p.nullsFraction, - "VariableStatsEstimate", - "double", - "nullsFraction"); - to_json_key( - j, - "averageRowSize", - p.averageRowSize, - "VariableStatsEstimate", - "double", - "averageRowSize"); - to_json_key( - j, - "distinctValuesCount", - p.distinctValuesCount, - "VariableStatsEstimate", - "double", - "distinctValuesCount"); + 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, VariableStatsEstimate& p) { - from_json_key( - j, "lowValue", p.lowValue, "VariableStatsEstimate", "double", "lowValue"); - from_json_key( - j, - "highValue", - p.highValue, - "VariableStatsEstimate", - "double", - "highValue"); - from_json_key( - j, - "nullsFraction", - p.nullsFraction, - "VariableStatsEstimate", - "double", - "nullsFraction"); - from_json_key( - j, - "averageRowSize", - p.averageRowSize, - "VariableStatsEstimate", - "double", - "averageRowSize"); - from_json_key( - j, - "distinctValuesCount", - p.distinctValuesCount, - "VariableStatsEstimate", - "double", - "distinctValuesCount"); +void from_json(const json& j, HivePartitionKey& p) { + from_json_key(j, "name", p.name, "HivePartitionKey", "String", "name"); + from_json_key(j, "value", p.value, "HivePartitionKey", "String", "value"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const PlanNodeStatsEstimate& p) { +void to_json(json& j, const DwrfEncryptionMetadata& p) { j = json::object(); to_json_key( j, - "outputRowCount", - p.outputRowCount, - "PlanNodeStatsEstimate", - "double", - "outputRowCount"); + "fieldToKeyData", + p.fieldToKeyData, + "DwrfEncryptionMetadata", + "Map", + "fieldToKeyData"); to_json_key( j, - "totalSize", - p.totalSize, - "PlanNodeStatsEstimate", - "double", - "totalSize"); + "extraMetadata", + p.extraMetadata, + "DwrfEncryptionMetadata", + "Map", + "extraMetadata"); to_json_key( j, - "confident", - p.confident, - "PlanNodeStatsEstimate", - "bool", - "confident"); + "encryptionAlgorithm", + p.encryptionAlgorithm, + "DwrfEncryptionMetadata", + "String", + "encryptionAlgorithm"); to_json_key( j, - "variableStatistics", - p.variableStatistics, - "PlanNodeStatsEstimate", - "Map", - "variableStatistics"); + "encryptionProvider", + p.encryptionProvider, + "DwrfEncryptionMetadata", + "String", + "encryptionProvider"); } -void from_json(const json& j, PlanNodeStatsEstimate& p) { +void from_json(const json& j, DwrfEncryptionMetadata& p) { from_json_key( j, - "outputRowCount", - p.outputRowCount, - "PlanNodeStatsEstimate", - "double", - "outputRowCount"); + "fieldToKeyData", + p.fieldToKeyData, + "DwrfEncryptionMetadata", + "Map", + "fieldToKeyData"); from_json_key( j, - "totalSize", - p.totalSize, - "PlanNodeStatsEstimate", - "double", - "totalSize"); + "extraMetadata", + p.extraMetadata, + "DwrfEncryptionMetadata", + "Map", + "extraMetadata"); from_json_key( j, - "confident", - p.confident, - "PlanNodeStatsEstimate", - "bool", - "confident"); + "encryptionAlgorithm", + p.encryptionAlgorithm, + "DwrfEncryptionMetadata", + "String", + "encryptionAlgorithm"); from_json_key( j, - "variableStatistics", - p.variableStatistics, - "PlanNodeStatsEstimate", - "Map", - "variableStatistics"); + "encryptionProvider", + p.encryptionProvider, + "DwrfEncryptionMetadata", + "String", + "encryptionProvider"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair ExchangeNodeType_enum_table[] = { - // NOLINT: cert-err58-cpp - {ExchangeNodeType::GATHER, "GATHER"}, - {ExchangeNodeType::REPARTITION, "REPARTITION"}, - {ExchangeNodeType::REPLICATE, "REPLICATE"}, -}; -void to_json(json& j, const ExchangeNodeType& e) { - static_assert( - std::is_enum::value, - "ExchangeNodeType must be an enum!"); - const auto* it = std::find_if( - std::begin(ExchangeNodeType_enum_table), - std::end(ExchangeNodeType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(ExchangeNodeType_enum_table)) - ? it - : std::begin(ExchangeNodeType_enum_table)) - ->second; -} -void from_json(const json& j, ExchangeNodeType& e) { - static_assert( - std::is_enum::value, - "ExchangeNodeType must be an enum!"); - const auto* it = std::find_if( - std::begin(ExchangeNodeType_enum_table), - std::end(ExchangeNodeType_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(ExchangeNodeType_enum_table)) - ? it - : std::begin(ExchangeNodeType_enum_table)) - ->first; +void to_json(json& j, const EncryptionInformation& p) { + j = json::object(); + to_json_key( + j, + "dwrfEncryptionMetadata", + p.dwrfEncryptionMetadata, + "EncryptionInformation", + "DwrfEncryptionMetadata", + "dwrfEncryptionMetadata"); } -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair 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; +void from_json(const json& j, EncryptionInformation& p) { + from_json_key( + j, + "dwrfEncryptionMetadata", + p.dwrfEncryptionMetadata, + "EncryptionInformation", + "DwrfEncryptionMetadata", + "dwrfEncryptionMetadata"); } } // namespace facebook::presto::protocol /* @@ -3829,2433 +2986,2445 @@ void from_json(const json& j, ExchangeNodeScope& e) { * limitations under the License. */ namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { +void to_json(json& j, const std::shared_ptr& p) { if (p == nullptr) { return; } String type = p->_type; - if (type == "$remote") { - j = *std::static_pointer_cast(p); - return; - } if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); + j = *std::static_pointer_cast(p); return; } + if (getConnectorKey(type) == "tpch") { - j = *std::static_pointer_cast(p); + j = *std::static_pointer_cast(p); return; } - throw TypeError(type + " no abstract type ConnectorPartitioningHandle"); + throw TypeError(type + " no abstract type ColumnHandle "); } -void from_json(const json& j, std::shared_ptr& p) { +void from_json(const json& j, std::shared_ptr& p) { String type; try { type = p->getSubclassKey(j); } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " ConnectorPartitioningHandle"); + throw ParseError(std::string(e.what()) + " ColumnHandle ColumnHandle"); } - if (type == "$remote") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } if (getConnectorKey(type) == "hive") { - auto k = std::make_shared(); + std::shared_ptr k = std::make_shared(); j.get_to(*k); - p = k; + p = std::static_pointer_cast(k); return; } + if (getConnectorKey(type) == "tpch") { - auto k = std::make_shared(); + std::shared_ptr k = std::make_shared(); j.get_to(*k); - p = k; + p = std::static_pointer_cast(k); return; } - throw TypeError(type + " no abstract type ConnectorPartitioningHandle"); + throw TypeError(type + " no abstract type ColumnHandle "); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair ColumnType_enum_table[] = + { // NOLINT: cert-err58-cpp + {ColumnType::PARTITION_KEY, "PARTITION_KEY"}, + {ColumnType::REGULAR, "REGULAR"}, + {ColumnType::SYNTHESIZED, "SYNTHESIZED"}, + {ColumnType::AGGREGATED, "AGGREGATED"}}; +void to_json(json& j, const ColumnType& e) { + static_assert(std::is_enum::value, "ColumnType must be an enum!"); + const auto* it = std::find_if( + std::begin(ColumnType_enum_table), + std::end(ColumnType_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(ColumnType_enum_table)) + ? it + : std::begin(ColumnType_enum_table)) + ->second; +} +void from_json(const json& j, ColumnType& e) { + static_assert(std::is_enum::value, "ColumnType must be an enum!"); + const auto* it = std::find_if( + std::begin(ColumnType_enum_table), + std::end(ColumnType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(ColumnType_enum_table)) + ? it + : std::begin(ColumnType_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +HiveColumnHandle::HiveColumnHandle() noexcept { + _type = "hive"; +} + +void to_json(json& j, const HiveColumnHandle& p) { + j = json::object(); + j["@type"] = "hive"; + to_json_key(j, "name", p.name, "HiveColumnHandle", "String", "name"); + to_json_key( + j, "hiveType", p.hiveType, "HiveColumnHandle", "HiveType", "hiveType"); + to_json_key( + j, + "typeSignature", + p.typeSignature, + "HiveColumnHandle", + "TypeSignature", + "typeSignature"); + to_json_key( + j, + "hiveColumnIndex", + p.hiveColumnIndex, + "HiveColumnHandle", + "int", + "hiveColumnIndex"); + to_json_key( + j, + "columnType", + p.columnType, + "HiveColumnHandle", + "ColumnType", + "columnType"); + to_json_key(j, "comment", p.comment, "HiveColumnHandle", "String", "comment"); + to_json_key( + j, + "requiredSubfields", + p.requiredSubfields, + "HiveColumnHandle", + "List", + "requiredSubfields"); + to_json_key( + j, + "partialAggregation", + p.partialAggregation, + "HiveColumnHandle", + "Aggregation", + "partialAggregation"); +} + +void from_json(const json& j, HiveColumnHandle& p) { + p._type = j["@type"]; + from_json_key(j, "name", p.name, "HiveColumnHandle", "String", "name"); + from_json_key( + j, "hiveType", p.hiveType, "HiveColumnHandle", "HiveType", "hiveType"); + from_json_key( + j, + "typeSignature", + p.typeSignature, + "HiveColumnHandle", + "TypeSignature", + "typeSignature"); + from_json_key( + j, + "hiveColumnIndex", + p.hiveColumnIndex, + "HiveColumnHandle", + "int", + "hiveColumnIndex"); + from_json_key( + j, + "columnType", + p.columnType, + "HiveColumnHandle", + "ColumnType", + "columnType"); + from_json_key( + j, "comment", p.comment, "HiveColumnHandle", "String", "comment"); + 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 PartitioningHandle& p) { +void to_json(json& j, const BucketConversion& p) { j = json::object(); to_json_key( j, - "connectorId", - p.connectorId, - "PartitioningHandle", - "ConnectorId", - "connectorId"); + "tableBucketCount", + p.tableBucketCount, + "BucketConversion", + "int", + "tableBucketCount"); to_json_key( j, - "transactionHandle", - p.transactionHandle, - "PartitioningHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "partitionBucketCount", + p.partitionBucketCount, + "BucketConversion", + "int", + "partitionBucketCount"); to_json_key( j, - "connectorHandle", - p.connectorHandle, - "PartitioningHandle", - "ConnectorPartitioningHandle", - "connectorHandle"); + "bucketColumnHandles", + p.bucketColumnHandles, + "BucketConversion", + "List", + "bucketColumnHandles"); } -void from_json(const json& j, PartitioningHandle& p) { +void from_json(const json& j, BucketConversion& p) { from_json_key( j, - "connectorId", - p.connectorId, - "PartitioningHandle", - "ConnectorId", - "connectorId"); + "tableBucketCount", + p.tableBucketCount, + "BucketConversion", + "int", + "tableBucketCount"); from_json_key( j, - "transactionHandle", - p.transactionHandle, - "PartitioningHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "partitionBucketCount", + p.partitionBucketCount, + "BucketConversion", + "int", + "partitionBucketCount"); from_json_key( j, - "connectorHandle", - p.connectorHandle, - "PartitioningHandle", - "ConnectorPartitioningHandle", - "connectorHandle"); + "bucketColumnHandles", + p.bucketColumnHandles, + "BucketConversion", + "List", + "bucketColumnHandles"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Partitioning& p) { +void to_json(json& j, const Column& p) { + j = json::object(); + to_json_key(j, "name", p.name, "Column", "String", "name"); + to_json_key(j, "type", p.type, "Column", "String", "type"); +} + +void from_json(const json& j, Column& p) { + from_json_key(j, "name", p.name, "Column", "String", "name"); + from_json_key(j, "type", p.type, "Column", "String", "type"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const TableToPartitionMapping& p) { j = json::object(); to_json_key( - j, "handle", p.handle, "Partitioning", "PartitioningHandle", "handle"); + j, + "tableToPartitionColumns", + p.tableToPartitionColumns, + "TableToPartitionMapping", + "Map", + "tableToPartitionColumns"); to_json_key( j, - "arguments", - p.arguments, - "Partitioning", - "List>", - "arguments"); + "partitionSchemaDifference", + p.partitionSchemaDifference, + "TableToPartitionMapping", + "Map", + "partitionSchemaDifference"); } -void from_json(const json& j, Partitioning& p) { +void from_json(const json& j, TableToPartitionMapping& p) { from_json_key( - j, "handle", p.handle, "Partitioning", "PartitioningHandle", "handle"); + j, + "tableToPartitionColumns", + p.tableToPartitionColumns, + "TableToPartitionMapping", + "Map", + "tableToPartitionColumns"); from_json_key( j, - "arguments", - p.arguments, - "Partitioning", - "List>", - "arguments"); + "partitionSchemaDifference", + p.partitionSchemaDifference, + "TableToPartitionMapping", + "Map", + "partitionSchemaDifference"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const PartitioningScheme& p) { +// 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_assert( + std::is_enum::value, + "NodeSelectionStrategy 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 { + return ej_pair.first == e; + }); + j = ((it != std::end(NodeSelectionStrategy_enum_table)) + ? it + : std::begin(NodeSelectionStrategy_enum_table)) + ->second; +} +void from_json(const json& j, NodeSelectionStrategy& e) { + static_assert( + std::is_enum::value, + "NodeSelectionStrategy must be an enum!"); + const auto* it = std::find_if( + std::begin(NodeSelectionStrategy_enum_table), + std::end(NodeSelectionStrategy_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(NodeSelectionStrategy_enum_table)) + ? it + : std::begin(NodeSelectionStrategy_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const HiveFileSplit& p) { j = json::object(); + to_json_key(j, "path", p.path, "HiveFileSplit", "String", "path"); + to_json_key(j, "start", p.start, "HiveFileSplit", "int64_t", "start"); + to_json_key(j, "length", p.length, "HiveFileSplit", "int64_t", "length"); to_json_key( - j, - "partitioning", - p.partitioning, - "PartitioningScheme", - "Partitioning", - "partitioning"); - to_json_key( - j, - "outputLayout", - p.outputLayout, - "PartitioningScheme", - "List", - "outputLayout"); + j, "fileSize", p.fileSize, "HiveFileSplit", "int64_t", "fileSize"); to_json_key( j, - "hashColumn", - p.hashColumn, - "PartitioningScheme", - "VariableReferenceExpression", - "hashColumn"); + "fileModifiedTime", + p.fileModifiedTime, + "HiveFileSplit", + "int64_t", + "fileModifiedTime"); to_json_key( j, - "replicateNullsAndAny", - p.replicateNullsAndAny, - "PartitioningScheme", - "bool", - "replicateNullsAndAny"); + "extraFileInfo", + p.extraFileInfo, + "HiveFileSplit", + "String", + "extraFileInfo"); to_json_key( j, - "bucketToPartition", - p.bucketToPartition, - "PartitioningScheme", - "List", - "bucketToPartition"); + "customSplitInfo", + p.customSplitInfo, + "HiveFileSplit", + "Map", + "customSplitInfo"); } -void from_json(const json& j, PartitioningScheme& p) { - from_json_key( - j, - "partitioning", - p.partitioning, - "PartitioningScheme", - "Partitioning", - "partitioning"); +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, - "outputLayout", - p.outputLayout, - "PartitioningScheme", - "List", - "outputLayout"); + j, "fileSize", p.fileSize, "HiveFileSplit", "int64_t", "fileSize"); from_json_key( j, - "hashColumn", - p.hashColumn, - "PartitioningScheme", - "VariableReferenceExpression", - "hashColumn"); + "fileModifiedTime", + p.fileModifiedTime, + "HiveFileSplit", + "int64_t", + "fileModifiedTime"); from_json_key( j, - "replicateNullsAndAny", - p.replicateNullsAndAny, - "PartitioningScheme", - "bool", - "replicateNullsAndAny"); + "extraFileInfo", + p.extraFileInfo, + "HiveFileSplit", + "String", + "extraFileInfo"); from_json_key( j, - "bucketToPartition", - p.bucketToPartition, - "PartitioningScheme", - "List", - "bucketToPartition"); + "customSplitInfo", + p.customSplitInfo, + "HiveFileSplit", + "Map", + "customSplitInfo"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -ExchangeNode::ExchangeNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.ExchangeNode"; +HiveSplit::HiveSplit() noexcept { + _type = "hive"; } -void to_json(json& j, const ExchangeNode& p) { +void to_json(json& j, const HiveSplit& 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"); + j["@type"] = "hive"; to_json_key( - j, "scope", p.scope, "ExchangeNode", "ExchangeNodeScope", "scope"); + 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, - "partitioningScheme", - p.partitioningScheme, - "ExchangeNode", - "PartitioningScheme", - "partitioningScheme"); + "partitionName", + p.partitionName, + "HiveSplit", + "String", + "partitionName"); + to_json_key(j, "storage", p.storage, "HiveSplit", "Storage", "storage"); to_json_key( j, - "sources", - p.sources, - "ExchangeNode", - "List>", - "sources"); + "partitionKeys", + p.partitionKeys, + "HiveSplit", + "List", + "partitionKeys"); to_json_key( j, - "inputs", - p.inputs, - "ExchangeNode", - "List>", - "inputs"); + "addresses", + p.addresses, + "HiveSplit", + "List", + "addresses"); to_json_key( j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "ExchangeNode", - "bool", - "ensureSourceOrdering"); + "readBucketNumber", + p.readBucketNumber, + "HiveSplit", + "int", + "readBucketNumber"); 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( + "tableBucketNumber", + p.tableBucketNumber, + "HiveSplit", + "int", + "tableBucketNumber"); + to_json_key( j, - "sources", - p.sources, - "ExchangeNode", - "List>", - "sources"); - from_json_key( + "nodeSelectionStrategy", + p.nodeSelectionStrategy, + "HiveSplit", + "NodeSelectionStrategy", + "nodeSelectionStrategy"); + to_json_key( j, - "inputs", - p.inputs, - "ExchangeNode", - "List>", - "inputs"); - from_json_key( + "partitionDataColumnCount", + p.partitionDataColumnCount, + "HiveSplit", + "int", + "partitionDataColumnCount"); + to_json_key( j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "ExchangeNode", - "bool", - "ensureSourceOrdering"); - from_json_key( + "tableToPartitionMapping", + p.tableToPartitionMapping, + "HiveSplit", + "TableToPartitionMapping", + "tableToPartitionMapping"); + to_json_key( j, - "orderingScheme", - p.orderingScheme, - "ExchangeNode", - "OrderingScheme", - "orderingScheme"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -RemoteSourceNode::RemoteSourceNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.RemoteSourceNode"; -} - -void to_json(json& j, const RemoteSourceNode& p) { - j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.RemoteSourceNode"; - to_json_key(j, "id", p.id, "RemoteSourceNode", "PlanNodeId", "id"); + "bucketConversion", + p.bucketConversion, + "HiveSplit", + "BucketConversion", + "bucketConversion"); to_json_key( j, - "sourceFragmentIds", - p.sourceFragmentIds, - "RemoteSourceNode", - "List", - "sourceFragmentIds"); + "s3SelectPushdownEnabled", + p.s3SelectPushdownEnabled, + "HiveSplit", + "bool", + "s3SelectPushdownEnabled"); to_json_key( j, - "outputVariables", - p.outputVariables, - "RemoteSourceNode", - "List", - "outputVariables"); + "cacheQuota", + p.cacheQuota, + "HiveSplit", + "CacheQuotaRequirement", + "cacheQuota"); to_json_key( j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "RemoteSourceNode", - "bool", - "ensureSourceOrdering"); + "encryptionMetadata", + p.encryptionMetadata, + "HiveSplit", + "EncryptionInformation", + "encryptionMetadata"); to_json_key( j, - "orderingScheme", - p.orderingScheme, - "RemoteSourceNode", - "OrderingScheme", - "orderingScheme"); + "redundantColumnDomains", + p.redundantColumnDomains, + "HiveSplit", + "List>", + "redundantColumnDomains"); to_json_key( j, - "exchangeType", - p.exchangeType, - "RemoteSourceNode", - "ExchangeNodeType", - "exchangeType"); + "splitWeight", + p.splitWeight, + "HiveSplit", + "SplitWeight", + "splitWeight"); } -void from_json(const json& j, RemoteSourceNode& p) { +void from_json(const json& j, HiveSplit& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "RemoteSourceNode", "PlanNodeId", "id"); + from_json_key( + j, "fileSplit", p.fileSplit, "HiveSplit", "HiveFileSplit", "fileSplit"); + from_json_key(j, "database", p.database, "HiveSplit", "String", "database"); + from_json_key(j, "table", p.table, "HiveSplit", "String", "table"); + from_json_key( + j, + "partitionName", + p.partitionName, + "HiveSplit", + "String", + "partitionName"); + from_json_key(j, "storage", p.storage, "HiveSplit", "Storage", "storage"); + from_json_key( + j, + "partitionKeys", + p.partitionKeys, + "HiveSplit", + "List", + "partitionKeys"); + from_json_key( + j, + "addresses", + p.addresses, + "HiveSplit", + "List", + "addresses"); + from_json_key( + j, + "readBucketNumber", + p.readBucketNumber, + "HiveSplit", + "int", + "readBucketNumber"); + from_json_key( + j, + "tableBucketNumber", + p.tableBucketNumber, + "HiveSplit", + "int", + "tableBucketNumber"); + from_json_key( + j, + "nodeSelectionStrategy", + p.nodeSelectionStrategy, + "HiveSplit", + "NodeSelectionStrategy", + "nodeSelectionStrategy"); from_json_key( j, - "sourceFragmentIds", - p.sourceFragmentIds, - "RemoteSourceNode", - "List", - "sourceFragmentIds"); + "partitionDataColumnCount", + p.partitionDataColumnCount, + "HiveSplit", + "int", + "partitionDataColumnCount"); from_json_key( j, - "outputVariables", - p.outputVariables, - "RemoteSourceNode", - "List", - "outputVariables"); + "tableToPartitionMapping", + p.tableToPartitionMapping, + "HiveSplit", + "TableToPartitionMapping", + "tableToPartitionMapping"); from_json_key( j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "RemoteSourceNode", + "bucketConversion", + p.bucketConversion, + "HiveSplit", + "BucketConversion", + "bucketConversion"); + from_json_key( + j, + "s3SelectPushdownEnabled", + p.s3SelectPushdownEnabled, + "HiveSplit", "bool", - "ensureSourceOrdering"); + "s3SelectPushdownEnabled"); from_json_key( j, - "orderingScheme", - p.orderingScheme, - "RemoteSourceNode", - "OrderingScheme", - "orderingScheme"); + "cacheQuota", + p.cacheQuota, + "HiveSplit", + "CacheQuotaRequirement", + "cacheQuota"); from_json_key( j, - "exchangeType", - p.exchangeType, - "RemoteSourceNode", - "ExchangeNodeType", - "exchangeType"); + "encryptionMetadata", + p.encryptionMetadata, + "HiveSplit", + "EncryptionInformation", + "encryptionMetadata"); + from_json_key( + j, + "redundantColumnDomains", + p.redundantColumnDomains, + "HiveSplit", + "List>", + "redundantColumnDomains"); + from_json_key( + j, + "splitWeight", + p.splitWeight, + "HiveSplit", + "SplitWeight", + "splitWeight"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // 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 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, - "AggregationNodeStep must be an enum!"); + std::is_enum::value, "BlockedReason 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 { + 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(AggregationNodeStep_enum_table)) + j = ((it != std::end(BlockedReason_enum_table)) ? it - : std::begin(AggregationNodeStep_enum_table)) + : std::begin(BlockedReason_enum_table)) ->second; } -void from_json(const json& j, AggregationNodeStep& e) { +void from_json(const json& j, BlockedReason& e) { static_assert( - std::is_enum::value, - "AggregationNodeStep must be an enum!"); + std::is_enum::value, "BlockedReason 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 { + 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(AggregationNodeStep_enum_table)) + e = ((it != std::end(BlockedReason_enum_table)) ? it - : std::begin(AggregationNodeStep_enum_table)) + : std::begin(BlockedReason_enum_table)) ->first; } +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +namespace facebook::presto::protocol { +void to_json(json& j, const OperatorInfo& p) {} +void from_json(const json& j, OperatorInfo& 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 Duration& p) { + j = p.toString(); +} + +void from_json(const json& j, Duration& p) { + p = Duration(std::string(j)); +} + +std::ostream& operator<<(std::ostream& os, const Duration& d) { + return os << d.toString(); +} + } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const GroupingSetDescriptor& 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, + "stageExecutionId", + p.stageExecutionId, + "OperatorStats", + "int", + "stageExecutionId"); + to_json_key( + j, "pipelineId", p.pipelineId, "OperatorStats", "int", "pipelineId"); + to_json_key( + j, "operatorId", p.operatorId, "OperatorStats", "int", "operatorId"); + to_json_key( + j, + "planNodeId", + p.planNodeId, + "OperatorStats", + "PlanNodeId", + "planNodeId"); + to_json_key( + j, + "operatorType", + p.operatorType, + "OperatorStats", + "String", + "operatorType"); + 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, + "rawInputPositions", + p.rawInputPositions, + "OperatorStats", + "int64_t", + "rawInputPositions"); + to_json_key( + j, + "inputDataSize", + p.inputDataSize, + "OperatorStats", + "DataSize", + "inputDataSize"); + to_json_key( + j, + "inputPositions", + p.inputPositions, + "OperatorStats", + "int64_t", + "inputPositions"); + to_json_key( + j, + "sumSquaredInputPositions", + p.sumSquaredInputPositions, + "OperatorStats", + "double", + "sumSquaredInputPositions"); + to_json_key( + j, + "getOutputCalls", + p.getOutputCalls, + "OperatorStats", + "int64_t", + "getOutputCalls"); to_json_key( j, - "groupingKeys", - p.groupingKeys, - "GroupingSetDescriptor", - "List", - "groupingKeys"); + "getOutputWall", + p.getOutputWall, + "OperatorStats", + "Duration", + "getOutputWall"); to_json_key( j, - "groupingSetCount", - p.groupingSetCount, - "GroupingSetDescriptor", - "int", - "groupingSetCount"); + "getOutputCpu", + p.getOutputCpu, + "OperatorStats", + "Duration", + "getOutputCpu"); to_json_key( j, - "globalGroupingSets", - p.globalGroupingSets, - "GroupingSetDescriptor", - "List", - "globalGroupingSets"); -} - -void from_json(const json& j, GroupingSetDescriptor& p) { - from_json_key( + "getOutputAllocation", + p.getOutputAllocation, + "OperatorStats", + "DataSize", + "getOutputAllocation"); + to_json_key( j, - "groupingKeys", - p.groupingKeys, - "GroupingSetDescriptor", - "List", - "groupingKeys"); - from_json_key( + "outputDataSize", + p.outputDataSize, + "OperatorStats", + "DataSize", + "outputDataSize"); + to_json_key( j, - "groupingSetCount", - p.groupingSetCount, - "GroupingSetDescriptor", - "int", - "groupingSetCount"); - from_json_key( + "outputPositions", + p.outputPositions, + "OperatorStats", + "int64_t", + "outputPositions"); + to_json_key( j, - "globalGroupingSets", - p.globalGroupingSets, - "GroupingSetDescriptor", - "List", - "globalGroupingSets"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -AggregationNode::AggregationNode() noexcept { - _type = ".AggregationNode"; -} - -void to_json(json& j, const AggregationNode& p) { - j = json::object(); - j["@type"] = ".AggregationNode"; - to_json_key(j, "id", p.id, "AggregationNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "AggregationNode", "PlanNode", "source"); + "physicalWrittenDataSize", + p.physicalWrittenDataSize, + "OperatorStats", + "DataSize", + "physicalWrittenDataSize"); to_json_key( j, - "aggregations", - p.aggregations, - "AggregationNode", - "Map", - "aggregations"); + "additionalCpu", + p.additionalCpu, + "OperatorStats", + "Duration", + "additionalCpu"); to_json_key( j, - "groupingSets", - p.groupingSets, - "AggregationNode", - "GroupingSetDescriptor", - "groupingSets"); + "blockedWall", + p.blockedWall, + "OperatorStats", + "Duration", + "blockedWall"); to_json_key( j, - "preGroupedVariables", - p.preGroupedVariables, - "AggregationNode", - "List", - "preGroupedVariables"); + "finishCalls", + p.finishCalls, + "OperatorStats", + "int64_t", + "finishCalls"); to_json_key( - j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); + j, "finishWall", p.finishWall, "OperatorStats", "Duration", "finishWall"); to_json_key( - j, - "hashVariable", - p.hashVariable, - "AggregationNode", - "VariableReferenceExpression", - "hashVariable"); + j, "finishCpu", p.finishCpu, "OperatorStats", "Duration", "finishCpu"); to_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "AggregationNode", - "VariableReferenceExpression", - "groupIdVariable"); -} - -void from_json(const json& j, AggregationNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "AggregationNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "AggregationNode", "PlanNode", "source"); - from_json_key( - j, - "aggregations", - p.aggregations, - "AggregationNode", - "Map", - "aggregations"); - from_json_key( - j, - "groupingSets", - p.groupingSets, - "AggregationNode", - "GroupingSetDescriptor", - "groupingSets"); - from_json_key( + "finishAllocation", + p.finishAllocation, + "OperatorStats", + "DataSize", + "finishAllocation"); + to_json_key( j, - "preGroupedVariables", - p.preGroupedVariables, - "AggregationNode", - "List", - "preGroupedVariables"); - from_json_key( - j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); - from_json_key( + "userMemoryReservation", + p.userMemoryReservation, + "OperatorStats", + "DataSize", + "userMemoryReservation"); + to_json_key( j, - "hashVariable", - p.hashVariable, - "AggregationNode", - "VariableReferenceExpression", - "hashVariable"); - from_json_key( + "revocableMemoryReservation", + p.revocableMemoryReservation, + "OperatorStats", + "DataSize", + "revocableMemoryReservation"); + to_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "AggregationNode", - "VariableReferenceExpression", - "groupIdVariable"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair 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 -namespace facebook::presto::protocol { - -void to_json(json& j, const Marker& p) { - j = json::object(); - to_json_key(j, "type", p.type, "Marker", "Type", "type"); - to_json_key(j, "valueBlock", p.valueBlock, "Marker", "Block", "valueBlock"); - to_json_key(j, "bound", p.bound, "Marker", "Bound", "bound"); -} - -void from_json(const json& j, Marker& p) { - from_json_key(j, "type", p.type, "Marker", "Type", "type"); - from_json_key(j, "valueBlock", p.valueBlock, "Marker", "Block", "valueBlock"); - from_json_key(j, "bound", p.bound, "Marker", "Bound", "bound"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const HiveBucketFilter& p) { - j = json::object(); + "systemMemoryReservation", + p.systemMemoryReservation, + "OperatorStats", + "DataSize", + "systemMemoryReservation"); to_json_key( j, - "bucketsToKeep", - p.bucketsToKeep, - "HiveBucketFilter", - "List", - "bucketsToKeep"); -} - -void from_json(const json& j, HiveBucketFilter& p) { - from_json_key( + "peakUserMemoryReservation", + p.peakUserMemoryReservation, + "OperatorStats", + "DataSize", + "peakUserMemoryReservation"); + to_json_key( j, - "bucketsToKeep", - p.bucketsToKeep, - "HiveBucketFilter", - "List", - "bucketsToKeep"); -} -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); - return; - } - - if (getConnectorKey(type) == "tpch") { - j = *std::static_pointer_cast(p); - return; - } - - throw TypeError(type + " no abstract type ConnectorTableLayoutHandle"); -} - -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError( - std::string(e.what()) + - " ConnectorTableLayoutHandle ConnectorTableLayoutHandle"); - } - - if (getConnectorKey(type) == "hive") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - - if (getConnectorKey(type) == "tpch") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - - throw TypeError(type + " no abstract type ConnectorTableLayoutHandle"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const TableHandle& p) { - j = json::object(); + "peakSystemMemoryReservation", + p.peakSystemMemoryReservation, + "OperatorStats", + "DataSize", + "peakSystemMemoryReservation"); to_json_key( j, - "connectorId", - p.connectorId, - "TableHandle", - "ConnectorId", - "connectorId"); + "peakTotalMemoryReservation", + p.peakTotalMemoryReservation, + "OperatorStats", + "DataSize", + "peakTotalMemoryReservation"); to_json_key( j, - "connectorHandle", - p.connectorHandle, - "TableHandle", - "ConnectorTableHandle", - "connectorHandle"); + "spilledDataSize", + p.spilledDataSize, + "OperatorStats", + "DataSize", + "spilledDataSize"); to_json_key( j, - "transaction", - p.transaction, - "TableHandle", - "ConnectorTransactionHandle", - "transaction"); + "blockedReason", + p.blockedReason, + "OperatorStats", + "BlockedReason", + "blockedReason"); + to_json_key(j, "info", p.info, "OperatorStats", "OperatorInfo", "info"); to_json_key( j, - "connectorTableLayout", - p.connectorTableLayout, - "TableHandle", - "ConnectorTableLayoutHandle", - "connectorTableLayout"); + "runtimeStats", + p.runtimeStats, + "OperatorStats", + "RuntimeStats", + "runtimeStats"); } -void from_json(const json& j, TableHandle& p) { +void from_json(const json& j, OperatorStats& p) { + from_json_key(j, "stageId", p.stageId, "OperatorStats", "int", "stageId"); from_json_key( j, - "connectorId", - p.connectorId, - "TableHandle", - "ConnectorId", - "connectorId"); + "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, - "connectorHandle", - p.connectorHandle, - "TableHandle", - "ConnectorTableHandle", - "connectorHandle"); + "planNodeId", + p.planNodeId, + "OperatorStats", + "PlanNodeId", + "planNodeId"); from_json_key( j, - "transaction", - p.transaction, - "TableHandle", - "ConnectorTransactionHandle", - "transaction"); + "operatorType", + p.operatorType, + "OperatorStats", + "String", + "operatorType"); from_json_key( j, - "connectorTableLayout", - p.connectorTableLayout, - "TableHandle", - "ConnectorTableLayoutHandle", - "connectorTableLayout"); -} -} // 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( + "totalDrivers", + p.totalDrivers, + "OperatorStats", + "int64_t", + "totalDrivers"); + from_json_key( j, - "analyzePartitionValues", - p.analyzePartitionValues, - "HiveTableHandle", - "List>", - "analyzePartitionValues"); -} - -void from_json(const json& j, HiveTableHandle& p) { - p._type = j["@type"]; + "addInputCalls", + p.addInputCalls, + "OperatorStats", + "int64_t", + "addInputCalls"); from_json_key( - j, "schemaName", p.schemaName, "HiveTableHandle", "String", "schemaName"); + j, + "addInputWall", + p.addInputWall, + "OperatorStats", + "Duration", + "addInputWall"); from_json_key( - j, "tableName", p.tableName, "HiveTableHandle", "String", "tableName"); + j, + "addInputCpu", + p.addInputCpu, + "OperatorStats", + "Duration", + "addInputCpu"); from_json_key( j, - "analyzePartitionValues", - p.analyzePartitionValues, - "HiveTableHandle", - "List>", - "analyzePartitionValues"); -} -} // 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( + "addInputAllocation", + p.addInputAllocation, + "OperatorStats", + "DataSize", + "addInputAllocation"); + from_json_key( + j, + "rawInputDataSize", + p.rawInputDataSize, + "OperatorStats", + "DataSize", + "rawInputDataSize"); + from_json_key( + j, + "rawInputPositions", + p.rawInputPositions, + "OperatorStats", + "int64_t", + "rawInputPositions"); + from_json_key( + j, + "inputDataSize", + p.inputDataSize, + "OperatorStats", + "DataSize", + "inputDataSize"); + from_json_key( j, - "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"]; + "inputPositions", + p.inputPositions, + "OperatorStats", + "int64_t", + "inputPositions"); 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 { - -void to_json(json& j, const Specification& p) { - j = json::object(); - to_json_key( + "sumSquaredInputPositions", + p.sumSquaredInputPositions, + "OperatorStats", + "double", + "sumSquaredInputPositions"); + from_json_key( j, - "partitionBy", - p.partitionBy, - "Specification", - "List", - "partitionBy"); - to_json_key( + "getOutputCalls", + p.getOutputCalls, + "OperatorStats", + "int64_t", + "getOutputCalls"); + from_json_key( j, - "orderingScheme", - p.orderingScheme, - "Specification", - "OrderingScheme", - "orderingScheme"); -} - -void from_json(const json& j, Specification& p) { + "getOutputWall", + p.getOutputWall, + "OperatorStats", + "Duration", + "getOutputWall"); from_json_key( j, - "partitionBy", - p.partitionBy, - "Specification", - "List", - "partitionBy"); + "getOutputCpu", + p.getOutputCpu, + "OperatorStats", + "Duration", + "getOutputCpu"); from_json_key( j, - "orderingScheme", - p.orderingScheme, - "Specification", - "OrderingScheme", - "orderingScheme"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -TopNRowNumberNode::TopNRowNumberNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.TopNRowNumberNode"; -} - -void to_json(json& j, const 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( + "getOutputAllocation", + p.getOutputAllocation, + "OperatorStats", + "DataSize", + "getOutputAllocation"); + from_json_key( j, - "specification", - p.specification, - "TopNRowNumberNode", - "Specification", - "specification"); - to_json_key( + "outputDataSize", + p.outputDataSize, + "OperatorStats", + "DataSize", + "outputDataSize"); + from_json_key( j, - "rowNumberVariable", - p.rowNumberVariable, - "TopNRowNumberNode", - "VariableReferenceExpression", - "rowNumberVariable"); - to_json_key( + "outputPositions", + p.outputPositions, + "OperatorStats", + "int64_t", + "outputPositions"); + from_json_key( j, - "maxRowCountPerPartition", - p.maxRowCountPerPartition, - "TopNRowNumberNode", - "int", - "maxRowCountPerPartition"); - to_json_key(j, "partial", p.partial, "TopNRowNumberNode", "bool", "partial"); - to_json_key( + "physicalWrittenDataSize", + p.physicalWrittenDataSize, + "OperatorStats", + "DataSize", + "physicalWrittenDataSize"); + from_json_key( j, - "hashVariable", - p.hashVariable, - "TopNRowNumberNode", - "VariableReferenceExpression", - "hashVariable"); -} - -void from_json(const json& j, TopNRowNumberNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "TopNRowNumberNode", "PlanNodeId", "id"); + "additionalCpu", + p.additionalCpu, + "OperatorStats", + "Duration", + "additionalCpu"); from_json_key( - j, "source", p.source, "TopNRowNumberNode", "PlanNode", "source"); + j, + "blockedWall", + p.blockedWall, + "OperatorStats", + "Duration", + "blockedWall"); from_json_key( j, - "specification", - p.specification, - "TopNRowNumberNode", - "Specification", - "specification"); + "finishCalls", + p.finishCalls, + "OperatorStats", + "int64_t", + "finishCalls"); + from_json_key( + j, "finishWall", p.finishWall, "OperatorStats", "Duration", "finishWall"); + from_json_key( + j, "finishCpu", p.finishCpu, "OperatorStats", "Duration", "finishCpu"); from_json_key( j, - "rowNumberVariable", - p.rowNumberVariable, - "TopNRowNumberNode", - "VariableReferenceExpression", - "rowNumberVariable"); + "finishAllocation", + p.finishAllocation, + "OperatorStats", + "DataSize", + "finishAllocation"); from_json_key( j, - "maxRowCountPerPartition", - p.maxRowCountPerPartition, - "TopNRowNumberNode", - "int", - "maxRowCountPerPartition"); + "userMemoryReservation", + p.userMemoryReservation, + "OperatorStats", + "DataSize", + "userMemoryReservation"); from_json_key( - j, "partial", p.partial, "TopNRowNumberNode", "bool", "partial"); + j, + "revocableMemoryReservation", + p.revocableMemoryReservation, + "OperatorStats", + "DataSize", + "revocableMemoryReservation"); from_json_key( j, - "hashVariable", - p.hashVariable, - "TopNRowNumberNode", - "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 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( + "systemMemoryReservation", + p.systemMemoryReservation, + "OperatorStats", + "DataSize", + "systemMemoryReservation"); + from_json_key( + j, + "peakUserMemoryReservation", + p.peakUserMemoryReservation, + "OperatorStats", + "DataSize", + "peakUserMemoryReservation"); + 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"); + "peakSystemMemoryReservation", + p.peakSystemMemoryReservation, + "OperatorStats", + "DataSize", + "peakSystemMemoryReservation"); from_json_key( j, - "assignments", - p.assignments, - "ProjectNode", - "Assignments", - "assignments"); + "peakTotalMemoryReservation", + p.peakTotalMemoryReservation, + "OperatorStats", + "DataSize", + "peakTotalMemoryReservation"); from_json_key( - j, "locality", p.locality, "ProjectNode", "Locality", "locality"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair CacheQuotaScope_enum_table[] = - { // NOLINT: cert-err58-cpp - {CacheQuotaScope::GLOBAL, "GLOBAL"}, - {CacheQuotaScope::SCHEMA, "SCHEMA"}, - {CacheQuotaScope::TABLE, "TABLE"}, - {CacheQuotaScope::PARTITION, "PARTITION"}}; -void to_json(json& j, const CacheQuotaScope& e) { - static_assert( - std::is_enum::value, "CacheQuotaScope must be an enum!"); - const auto* it = std::find_if( - std::begin(CacheQuotaScope_enum_table), - std::end(CacheQuotaScope_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(CacheQuotaScope_enum_table)) - ? it - : std::begin(CacheQuotaScope_enum_table)) - ->second; -} -void from_json(const json& j, CacheQuotaScope& e) { - static_assert( - std::is_enum::value, "CacheQuotaScope must be an enum!"); - const auto* it = std::find_if( - std::begin(CacheQuotaScope_enum_table), - std::end(CacheQuotaScope_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(CacheQuotaScope_enum_table)) - ? it - : std::begin(CacheQuotaScope_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -/* - * 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 { - -void to_json(json& j, const CacheQuotaRequirement& p) { - j = json::object(); - to_json_key( j, - "cacheQuotaScope", - p.cacheQuotaScope, - "CacheQuotaRequirement", - "CacheQuotaScope", - "cacheQuotaScope"); - to_json_key( - j, "quota", p.quota, "CacheQuotaRequirement", "DataSize", "quota"); -} - -void from_json(const json& j, CacheQuotaRequirement& p) { + "spilledDataSize", + p.spilledDataSize, + "OperatorStats", + "DataSize", + "spilledDataSize"); from_json_key( j, - "cacheQuotaScope", - p.cacheQuotaScope, - "CacheQuotaRequirement", - "CacheQuotaScope", - "cacheQuotaScope"); + "blockedReason", + p.blockedReason, + "OperatorStats", + "BlockedReason", + "blockedReason"); + from_json_key(j, "info", p.info, "OperatorStats", "OperatorInfo", "info"); from_json_key( - j, "quota", p.quota, "CacheQuotaRequirement", "DataSize", "quota"); + j, + "runtimeStats", + p.runtimeStats, + "OperatorStats", + "RuntimeStats", + "runtimeStats"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -SemiJoinNode::SemiJoinNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.SemiJoinNode"; -} -void to_json(json& j, const SemiJoinNode& p) { +void to_json(json& j, const DriverStats& 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, "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, - "filteringSource", - p.filteringSource, - "SemiJoinNode", - "PlanNode", - "filteringSource"); + "elapsedTime", + p.elapsedTime, + "DriverStats", + "Duration", + "elapsedTime"); to_json_key( j, - "sourceJoinVariable", - p.sourceJoinVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "sourceJoinVariable"); + "userMemoryReservation", + p.userMemoryReservation, + "DriverStats", + "DataSize", + "userMemoryReservation"); to_json_key( j, - "filteringSourceJoinVariable", - p.filteringSourceJoinVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "filteringSourceJoinVariable"); + "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, + "rawInputReadTime", + p.rawInputReadTime, + "DriverStats", + "Duration", + "rawInputReadTime"); + to_json_key( + j, + "processedInputDataSize", + p.processedInputDataSize, + "DriverStats", + "DataSize", + "processedInputDataSize"); to_json_key( j, - "semiJoinOutput", - p.semiJoinOutput, - "SemiJoinNode", - "VariableReferenceExpression", - "semiJoinOutput"); + "processedInputPositions", + p.processedInputPositions, + "DriverStats", + "int64_t", + "processedInputPositions"); to_json_key( j, - "sourceHashVariable", - p.sourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "sourceHashVariable"); + "outputDataSize", + p.outputDataSize, + "DriverStats", + "DataSize", + "outputDataSize"); to_json_key( j, - "filteringSourceHashVariable", - p.filteringSourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "filteringSourceHashVariable"); + "outputPositions", + p.outputPositions, + "DriverStats", + "int64_t", + "outputPositions"); to_json_key( j, - "distributionType", - p.distributionType, - "SemiJoinNode", - "DistributionType", - "distributionType"); + "physicalWrittenDataSize", + p.physicalWrittenDataSize, + "DriverStats", + "DataSize", + "physicalWrittenDataSize"); to_json_key( j, - "dynamicFilters", - p.dynamicFilters, - "SemiJoinNode", - "Map", - "dynamicFilters"); + "operatorStats", + p.operatorStats, + "DriverStats", + "List", + "operatorStats"); } -void from_json(const json& j, SemiJoinNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "SemiJoinNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "SemiJoinNode", "PlanNode", "source"); - from_json_key( - j, - "filteringSource", - p.filteringSource, - "SemiJoinNode", - "PlanNode", - "filteringSource"); - from_json_key( - j, - "sourceJoinVariable", - p.sourceJoinVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "sourceJoinVariable"); - from_json_key( - j, - "filteringSourceJoinVariable", - p.filteringSourceJoinVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "filteringSourceJoinVariable"); - from_json_key( - j, - "semiJoinOutput", - p.semiJoinOutput, - "SemiJoinNode", - "VariableReferenceExpression", - "semiJoinOutput"); - from_json_key( - j, - "sourceHashVariable", - p.sourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "sourceHashVariable"); +void from_json(const json& j, DriverStats& p) { from_json_key( - j, - "filteringSourceHashVariable", - p.filteringSourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "filteringSourceHashVariable"); + j, "lifespan", p.lifespan, "DriverStats", "Lifespan", "lifespan"); from_json_key( - j, - "distributionType", - p.distributionType, - "SemiJoinNode", - "DistributionType", - "distributionType"); + j, "createTime", p.createTime, "DriverStats", "DateTime", "createTime"); from_json_key( - j, - "dynamicFilters", - p.dynamicFilters, - "SemiJoinNode", - "Map", - "dynamicFilters"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (type == "equatable") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "sortable") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "allOrNone") { - j = *std::static_pointer_cast(p); - return; - } - - throw TypeError(type + " no abstract type ValueSet "); -} - -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " ValueSet ValueSet"); - } - - if (type == "equatable") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "sortable") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "allOrNone") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - - throw TypeError(type + " no abstract type ValueSet "); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Domain& p) { - j = json::object(); - to_json_key(j, "values", p.values, "Domain", "ValueSet", "values"); - to_json_key(j, "nullAllowed", p.nullAllowed, "Domain", "bool", "nullAllowed"); -} - -void from_json(const json& j, Domain& p) { - from_json_key(j, "values", p.values, "Domain", "ValueSet", "values"); + j, "startTime", p.startTime, "DriverStats", "DateTime", "startTime"); + from_json_key(j, "endTime", p.endTime, "DriverStats", "DateTime", "endTime"); from_json_key( - j, "nullAllowed", p.nullAllowed, "Domain", "bool", "nullAllowed"); -} -} // 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"]; + j, "queuedTime", p.queuedTime, "DriverStats", "Duration", "queuedTime"); from_json_key( j, - "connectorId", - p.connectorId, - "EmptySplit", - "ConnectorId", - "connectorId"); -} -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -namespace facebook::presto::protocol { - -void to_json(json& j, const Duration& p) { - j = p.toString(); -} - -void from_json(const json& j, Duration& p) { - p = Duration(std::string(j)); -} - -std::ostream& operator<<(std::ostream& os, const Duration& d) { - return os << d.toString(); -} - -} // namespace facebook::presto::protocol -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( + "elapsedTime", + p.elapsedTime, + "DriverStats", + "Duration", + "elapsedTime"); + from_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) { + "userMemoryReservation", + p.userMemoryReservation, + "DriverStats", + "DataSize", + "userMemoryReservation"); from_json_key( j, - "nodeVersion", - p.nodeVersion, - "ServerInfo", - "NodeVersion", - "nodeVersion"); + "revocableMemoryReservation", + p.revocableMemoryReservation, + "DriverStats", + "DataSize", + "revocableMemoryReservation"); from_json_key( - j, "environment", p.environment, "ServerInfo", "String", "environment"); + j, + "systemMemoryReservation", + p.systemMemoryReservation, + "DriverStats", + "DataSize", + "systemMemoryReservation"); from_json_key( - j, "coordinator", p.coordinator, "ServerInfo", "bool", "coordinator"); - from_json_key(j, "starting", p.starting, "ServerInfo", "bool", "starting"); - from_json_key(j, "uptime", p.uptime, "ServerInfo", "Duration", "uptime"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -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"); + j, + "totalScheduledTime", + p.totalScheduledTime, + "DriverStats", + "Duration", + "totalScheduledTime"); from_json_key( - j, "predicate", p.predicate, "FilterNode", "RowExpression", "predicate"); -} -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); - return; - } - - throw TypeError(type + " no abstract type ConnectorOutputTableHandle "); -} - -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError( - std::string(e.what()) + - " ConnectorOutputTableHandle ConnectorOutputTableHandle"); - } - - if (getConnectorKey(type) == "hive") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - - throw TypeError(type + " no abstract type ConnectorOutputTableHandle "); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const OutputTableHandle& p) { - j = json::object(); - to_json_key( j, - "connectorId", - p.connectorId, - "OutputTableHandle", - "ConnectorId", - "connectorId"); - to_json_key( + "totalCpuTime", + p.totalCpuTime, + "DriverStats", + "Duration", + "totalCpuTime"); + from_json_key( j, - "transactionHandle", - p.transactionHandle, - "OutputTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); - to_json_key( + "totalBlockedTime", + p.totalBlockedTime, + "DriverStats", + "Duration", + "totalBlockedTime"); + from_json_key( + j, "fullyBlocked", p.fullyBlocked, "DriverStats", "bool", "fullyBlocked"); + from_json_key( j, - "connectorHandle", - p.connectorHandle, - "OutputTableHandle", - "ConnectorOutputTableHandle", - "connectorHandle"); -} - -void from_json(const json& j, OutputTableHandle& p) { + "blockedReasons", + p.blockedReasons, + "DriverStats", + "List", + "blockedReasons"); from_json_key( j, - "connectorId", - p.connectorId, - "OutputTableHandle", - "ConnectorId", - "connectorId"); + "totalAllocation", + p.totalAllocation, + "DriverStats", + "DataSize", + "totalAllocation"); from_json_key( j, - "transactionHandle", - p.transactionHandle, - "OutputTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "rawInputDataSize", + p.rawInputDataSize, + "DriverStats", + "DataSize", + "rawInputDataSize"); from_json_key( j, - "connectorHandle", - p.connectorHandle, - "OutputTableHandle", - "ConnectorOutputTableHandle", - "connectorHandle"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair 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 ResourceEstimates& p) { - j = json::object(); - to_json_key( + "rawInputPositions", + p.rawInputPositions, + "DriverStats", + "int64_t", + "rawInputPositions"); + from_json_key( j, - "executionTime", - p.executionTime, - "ResourceEstimates", + "rawInputReadTime", + p.rawInputReadTime, + "DriverStats", "Duration", - "executionTime"); - to_json_key( - j, "cpuTime", p.cpuTime, "ResourceEstimates", "Duration", "cpuTime"); - to_json_key( + "rawInputReadTime"); + from_json_key( j, - "peakMemory", - p.peakMemory, - "ResourceEstimates", + "processedInputDataSize", + p.processedInputDataSize, + "DriverStats", "DataSize", - "peakMemory"); - to_json_key( + "processedInputDataSize"); + from_json_key( j, - "peakTaskMemory", - p.peakTaskMemory, - "ResourceEstimates", - "DataSize", - "peakTaskMemory"); -} - -void from_json(const json& j, ResourceEstimates& p) { + "processedInputPositions", + p.processedInputPositions, + "DriverStats", + "int64_t", + "processedInputPositions"); from_json_key( j, - "executionTime", - p.executionTime, - "ResourceEstimates", - "Duration", - "executionTime"); + "outputDataSize", + p.outputDataSize, + "DriverStats", + "DataSize", + "outputDataSize"); from_json_key( - j, "cpuTime", p.cpuTime, "ResourceEstimates", "Duration", "cpuTime"); + j, + "outputPositions", + p.outputPositions, + "DriverStats", + "int64_t", + "outputPositions"); from_json_key( j, - "peakMemory", - p.peakMemory, - "ResourceEstimates", + "physicalWrittenDataSize", + p.physicalWrittenDataSize, + "DriverStats", "DataSize", - "peakMemory"); + "physicalWrittenDataSize"); from_json_key( j, - "peakTaskMemory", - p.peakTaskMemory, - "ResourceEstimates", - "DataSize", - "peakTaskMemory"); + "operatorStats", + p.operatorStats, + "DriverStats", + "List", + "operatorStats"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const SessionRepresentation& p) { +void to_json(json& j, const PipelineStats& p) { j = json::object(); to_json_key( - j, "queryId", p.queryId, "SessionRepresentation", "String", "queryId"); + j, "pipelineId", p.pipelineId, "PipelineStats", "int", "pipelineId"); to_json_key( j, - "transactionId", - p.transactionId, - "SessionRepresentation", - "TransactionId", - "transactionId"); + "firstStartTime", + p.firstStartTime, + "PipelineStats", + "DateTime", + "firstStartTime"); to_json_key( j, - "clientTransactionSupport", - p.clientTransactionSupport, - "SessionRepresentation", + "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", - "clientTransactionSupport"); - to_json_key(j, "user", p.user, "SessionRepresentation", "String", "user"); + "inputPipeline"); to_json_key( j, - "principal", - p.principal, - "SessionRepresentation", - "String", - "principal"); + "outputPipeline", + p.outputPipeline, + "PipelineStats", + "bool", + "outputPipeline"); to_json_key( - j, "source", p.source, "SessionRepresentation", "String", "source"); + j, + "totalDrivers", + p.totalDrivers, + "PipelineStats", + "int", + "totalDrivers"); to_json_key( - j, "catalog", p.catalog, "SessionRepresentation", "String", "catalog"); + j, + "queuedDrivers", + p.queuedDrivers, + "PipelineStats", + "int", + "queuedDrivers"); to_json_key( - j, "schema", p.schema, "SessionRepresentation", "String", "schema"); + j, + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "PipelineStats", + "int", + "queuedPartitionedDrivers"); to_json_key( j, - "traceToken", - p.traceToken, - "SessionRepresentation", - "String", - "traceToken"); + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "PipelineStats", + "int64_t", + "queuedPartitionedSplitsWeight"); to_json_key( j, - "timeZoneKey", - p.timeZoneKey, - "SessionRepresentation", - "TimeZoneKey", - "timeZoneKey"); + "runningDrivers", + p.runningDrivers, + "PipelineStats", + "int", + "runningDrivers"); to_json_key( - j, "locale", p.locale, "SessionRepresentation", "Locale", "locale"); + j, + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "PipelineStats", + "int", + "runningPartitionedDrivers"); to_json_key( j, - "remoteUserAddress", - p.remoteUserAddress, - "SessionRepresentation", - "String", - "remoteUserAddress"); + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "PipelineStats", + "int64_t", + "runningPartitionedSplitsWeight"); to_json_key( j, - "userAgent", - p.userAgent, - "SessionRepresentation", - "String", - "userAgent"); + "blockedDrivers", + p.blockedDrivers, + "PipelineStats", + "int", + "blockedDrivers"); + to_json_key( + j, + "completedDrivers", + p.completedDrivers, + "PipelineStats", + "int", + "completedDrivers"); + to_json_key( + j, + "userMemoryReservationInBytes", + p.userMemoryReservationInBytes, + "PipelineStats", + "int64_t", + "userMemoryReservationInBytes"); + to_json_key( + j, + "revocableMemoryReservationInBytes", + p.revocableMemoryReservationInBytes, + "PipelineStats", + "int64_t", + "revocableMemoryReservationInBytes"); + to_json_key( + j, + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "PipelineStats", + "int64_t", + "systemMemoryReservationInBytes"); + to_json_key( + j, + "queuedTime", + p.queuedTime, + "PipelineStats", + "DistributionSnapshot", + "queuedTime"); + to_json_key( + j, + "elapsedTime", + p.elapsedTime, + "PipelineStats", + "DistributionSnapshot", + "elapsedTime"); + to_json_key( + j, + "totalScheduledTimeInNanos", + p.totalScheduledTimeInNanos, + "PipelineStats", + "int64_t", + "totalScheduledTimeInNanos"); + to_json_key( + j, + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "PipelineStats", + "int64_t", + "totalCpuTimeInNanos"); + to_json_key( + j, + "totalBlockedTimeInNanos", + p.totalBlockedTimeInNanos, + "PipelineStats", + "int64_t", + "totalBlockedTimeInNanos"); + to_json_key( + j, + "fullyBlocked", + p.fullyBlocked, + "PipelineStats", + "bool", + "fullyBlocked"); to_json_key( j, - "clientInfo", - p.clientInfo, - "SessionRepresentation", - "String", - "clientInfo"); + "blockedReasons", + p.blockedReasons, + "PipelineStats", + "List", + "blockedReasons"); to_json_key( j, - "clientTags", - p.clientTags, - "SessionRepresentation", - "List", - "clientTags"); + "totalAllocationInBytes", + p.totalAllocationInBytes, + "PipelineStats", + "int64_t", + "totalAllocationInBytes"); to_json_key( j, - "resourceEstimates", - p.resourceEstimates, - "SessionRepresentation", - "ResourceEstimates", - "resourceEstimates"); + "rawInputDataSizeInBytes", + p.rawInputDataSizeInBytes, + "PipelineStats", + "int64_t", + "rawInputDataSizeInBytes"); to_json_key( j, - "startTime", - p.startTime, - "SessionRepresentation", + "rawInputPositions", + p.rawInputPositions, + "PipelineStats", "int64_t", - "startTime"); + "rawInputPositions"); to_json_key( j, - "systemProperties", - p.systemProperties, - "SessionRepresentation", - "Map", - "systemProperties"); + "processedInputDataSizeInBytes", + p.processedInputDataSizeInBytes, + "PipelineStats", + "int64_t", + "processedInputDataSizeInBytes"); to_json_key( j, - "catalogProperties", - p.catalogProperties, - "SessionRepresentation", - "Map>", - "catalogProperties"); + "processedInputPositions", + p.processedInputPositions, + "PipelineStats", + "int64_t", + "processedInputPositions"); to_json_key( j, - "unprocessedCatalogProperties", - p.unprocessedCatalogProperties, - "SessionRepresentation", - "Map>", - "unprocessedCatalogProperties"); + "outputDataSizeInBytes", + p.outputDataSizeInBytes, + "PipelineStats", + "int64_t", + "outputDataSizeInBytes"); to_json_key( j, - "roles", - p.roles, - "SessionRepresentation", - "Map", - "roles"); + "outputPositions", + p.outputPositions, + "PipelineStats", + "int64_t", + "outputPositions"); to_json_key( j, - "preparedStatements", - p.preparedStatements, - "SessionRepresentation", - "Map", - "preparedStatements"); + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "PipelineStats", + "int64_t", + "physicalWrittenDataSizeInBytes"); to_json_key( j, - "sessionFunctions", - p.sessionFunctions, - "SessionRepresentation", - "Map", - "sessionFunctions"); + "operatorSummaries", + p.operatorSummaries, + "PipelineStats", + "List", + "operatorSummaries"); + to_json_key( + j, "drivers", p.drivers, "PipelineStats", "List", "drivers"); } -void from_json(const json& j, SessionRepresentation& p) { +void from_json(const json& j, PipelineStats& p) { from_json_key( - j, "queryId", p.queryId, "SessionRepresentation", "String", "queryId"); + j, "pipelineId", p.pipelineId, "PipelineStats", "int", "pipelineId"); from_json_key( j, - "transactionId", - p.transactionId, - "SessionRepresentation", - "TransactionId", - "transactionId"); + "firstStartTime", + p.firstStartTime, + "PipelineStats", + "DateTime", + "firstStartTime"); from_json_key( j, - "clientTransactionSupport", - p.clientTransactionSupport, - "SessionRepresentation", + "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", - "clientTransactionSupport"); - from_json_key(j, "user", p.user, "SessionRepresentation", "String", "user"); + "inputPipeline"); from_json_key( j, - "principal", - p.principal, - "SessionRepresentation", - "String", - "principal"); + "outputPipeline", + p.outputPipeline, + "PipelineStats", + "bool", + "outputPipeline"); from_json_key( - j, "source", p.source, "SessionRepresentation", "String", "source"); + j, + "totalDrivers", + p.totalDrivers, + "PipelineStats", + "int", + "totalDrivers"); from_json_key( - j, "catalog", p.catalog, "SessionRepresentation", "String", "catalog"); + j, + "queuedDrivers", + p.queuedDrivers, + "PipelineStats", + "int", + "queuedDrivers"); from_json_key( - j, "schema", p.schema, "SessionRepresentation", "String", "schema"); + j, + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "PipelineStats", + "int", + "queuedPartitionedDrivers"); from_json_key( j, - "traceToken", - p.traceToken, - "SessionRepresentation", - "String", - "traceToken"); + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "PipelineStats", + "int64_t", + "queuedPartitionedSplitsWeight"); from_json_key( j, - "timeZoneKey", - p.timeZoneKey, - "SessionRepresentation", - "TimeZoneKey", - "timeZoneKey"); + "runningDrivers", + p.runningDrivers, + "PipelineStats", + "int", + "runningDrivers"); + from_json_key( + j, + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "PipelineStats", + "int", + "runningPartitionedDrivers"); + from_json_key( + j, + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "PipelineStats", + "int64_t", + "runningPartitionedSplitsWeight"); + from_json_key( + j, + "blockedDrivers", + p.blockedDrivers, + "PipelineStats", + "int", + "blockedDrivers"); + from_json_key( + j, + "completedDrivers", + p.completedDrivers, + "PipelineStats", + "int", + "completedDrivers"); + from_json_key( + j, + "userMemoryReservationInBytes", + p.userMemoryReservationInBytes, + "PipelineStats", + "int64_t", + "userMemoryReservationInBytes"); + from_json_key( + j, + "revocableMemoryReservationInBytes", + p.revocableMemoryReservationInBytes, + "PipelineStats", + "int64_t", + "revocableMemoryReservationInBytes"); + from_json_key( + j, + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "PipelineStats", + "int64_t", + "systemMemoryReservationInBytes"); + from_json_key( + j, + "queuedTime", + p.queuedTime, + "PipelineStats", + "DistributionSnapshot", + "queuedTime"); + from_json_key( + j, + "elapsedTime", + p.elapsedTime, + "PipelineStats", + "DistributionSnapshot", + "elapsedTime"); + from_json_key( + j, + "totalScheduledTimeInNanos", + p.totalScheduledTimeInNanos, + "PipelineStats", + "int64_t", + "totalScheduledTimeInNanos"); from_json_key( - j, "locale", p.locale, "SessionRepresentation", "Locale", "locale"); + j, + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "PipelineStats", + "int64_t", + "totalCpuTimeInNanos"); from_json_key( j, - "remoteUserAddress", - p.remoteUserAddress, - "SessionRepresentation", - "String", - "remoteUserAddress"); + "totalBlockedTimeInNanos", + p.totalBlockedTimeInNanos, + "PipelineStats", + "int64_t", + "totalBlockedTimeInNanos"); from_json_key( j, - "userAgent", - p.userAgent, - "SessionRepresentation", - "String", - "userAgent"); + "fullyBlocked", + p.fullyBlocked, + "PipelineStats", + "bool", + "fullyBlocked"); from_json_key( j, - "clientInfo", - p.clientInfo, - "SessionRepresentation", - "String", - "clientInfo"); + "blockedReasons", + p.blockedReasons, + "PipelineStats", + "List", + "blockedReasons"); from_json_key( j, - "clientTags", - p.clientTags, - "SessionRepresentation", - "List", - "clientTags"); + "totalAllocationInBytes", + p.totalAllocationInBytes, + "PipelineStats", + "int64_t", + "totalAllocationInBytes"); from_json_key( j, - "resourceEstimates", - p.resourceEstimates, - "SessionRepresentation", - "ResourceEstimates", - "resourceEstimates"); + "rawInputDataSizeInBytes", + p.rawInputDataSizeInBytes, + "PipelineStats", + "int64_t", + "rawInputDataSizeInBytes"); from_json_key( j, - "startTime", - p.startTime, - "SessionRepresentation", + "rawInputPositions", + p.rawInputPositions, + "PipelineStats", "int64_t", - "startTime"); + "rawInputPositions"); from_json_key( j, - "systemProperties", - p.systemProperties, - "SessionRepresentation", - "Map", - "systemProperties"); + "processedInputDataSizeInBytes", + p.processedInputDataSizeInBytes, + "PipelineStats", + "int64_t", + "processedInputDataSizeInBytes"); from_json_key( j, - "catalogProperties", - p.catalogProperties, - "SessionRepresentation", - "Map>", - "catalogProperties"); + "processedInputPositions", + p.processedInputPositions, + "PipelineStats", + "int64_t", + "processedInputPositions"); from_json_key( j, - "unprocessedCatalogProperties", - p.unprocessedCatalogProperties, - "SessionRepresentation", - "Map>", - "unprocessedCatalogProperties"); + "outputDataSizeInBytes", + p.outputDataSizeInBytes, + "PipelineStats", + "int64_t", + "outputDataSizeInBytes"); from_json_key( j, - "roles", - p.roles, - "SessionRepresentation", - "Map", - "roles"); + "outputPositions", + p.outputPositions, + "PipelineStats", + "int64_t", + "outputPositions"); from_json_key( j, - "preparedStatements", - p.preparedStatements, - "SessionRepresentation", - "Map", - "preparedStatements"); + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "PipelineStats", + "int64_t", + "physicalWrittenDataSizeInBytes"); from_json_key( j, - "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 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"); + "operatorSummaries", + p.operatorSummaries, + "PipelineStats", + "List", + "operatorSummaries"); + from_json_key( + j, "drivers", p.drivers, "PipelineStats", "List", "drivers"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const ErrorLocation& p) { +void to_json(json& j, const TaskStats& p) { j = json::object(); to_json_key( - j, "lineNumber", p.lineNumber, "ErrorLocation", "int", "lineNumber"); + j, "createTime", p.createTime, "TaskStats", "DateTime", "createTime"); 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( + "firstStartTime", + p.firstStartTime, + "TaskStats", + "DateTime", + "firstStartTime"); + to_json_key( j, - "columnNumber", - p.columnNumber, - "ErrorLocation", + "lastStartTime", + p.lastStartTime, + "TaskStats", + "DateTime", + "lastStartTime"); + to_json_key( + j, "lastEndTime", p.lastEndTime, "TaskStats", "DateTime", "lastEndTime"); + to_json_key(j, "endTime", p.endTime, "TaskStats", "DateTime", "endTime"); + to_json_key( + j, + "elapsedTimeInNanos", + p.elapsedTimeInNanos, + "TaskStats", + "int64_t", + "elapsedTimeInNanos"); + to_json_key( + j, + "queuedTimeInNanos", + p.queuedTimeInNanos, + "TaskStats", + "int64_t", + "queuedTimeInNanos"); + to_json_key( + j, "totalDrivers", p.totalDrivers, "TaskStats", "int", "totalDrivers"); + to_json_key( + j, "queuedDrivers", p.queuedDrivers, "TaskStats", "int", "queuedDrivers"); + to_json_key( + j, + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "TaskStats", "int", - "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"); + "queuedPartitionedDrivers"); to_json_key( - j, "message", p.message, "ExecutionFailureInfo", "String", "message"); + j, + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "TaskStats", + "int64_t", + "queuedPartitionedSplitsWeight"); to_json_key( j, - "cause", - p.cause, - "ExecutionFailureInfo", - "ExecutionFailureInfo", - "cause"); + "runningDrivers", + p.runningDrivers, + "TaskStats", + "int", + "runningDrivers"); to_json_key( j, - "suppressed", - p.suppressed, - "ExecutionFailureInfo", - "List", - "suppressed"); + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "TaskStats", + "int", + "runningPartitionedDrivers"); to_json_key( - j, "stack", p.stack, "ExecutionFailureInfo", "List", "stack"); + j, + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "TaskStats", + "int64_t", + "runningPartitionedSplitsWeight"); to_json_key( j, - "errorLocation", - p.errorLocation, - "ExecutionFailureInfo", - "ErrorLocation", - "errorLocation"); + "blockedDrivers", + p.blockedDrivers, + "TaskStats", + "int", + "blockedDrivers"); to_json_key( j, - "errorCode", - p.errorCode, - "ExecutionFailureInfo", - "ErrorCode", - "errorCode"); + "completedDrivers", + p.completedDrivers, + "TaskStats", + "int", + "completedDrivers"); to_json_key( j, - "remoteHost", - p.remoteHost, - "ExecutionFailureInfo", - "HostAddress", - "remoteHost"); + "cumulativeUserMemory", + p.cumulativeUserMemory, + "TaskStats", + "double", + "cumulativeUserMemory"); to_json_key( j, - "errorCause", - p.errorCause, - "ExecutionFailureInfo", - "ErrorCause", - "errorCause"); -} - -void from_json(const json& j, ExecutionFailureInfo& p) { - from_json_key(j, "type", p.type, "ExecutionFailureInfo", "String", "type"); - from_json_key( - j, "message", p.message, "ExecutionFailureInfo", "String", "message"); - from_json_key( + "cumulativeTotalMemory", + p.cumulativeTotalMemory, + "TaskStats", + "double", + "cumulativeTotalMemory"); + to_json_key( j, - "cause", - p.cause, - "ExecutionFailureInfo", - "ExecutionFailureInfo", - "cause"); - from_json_key( + "userMemoryReservationInBytes", + p.userMemoryReservationInBytes, + "TaskStats", + "int64_t", + "userMemoryReservationInBytes"); + to_json_key( j, - "suppressed", - p.suppressed, - "ExecutionFailureInfo", - "List", - "suppressed"); - from_json_key( - j, "stack", p.stack, "ExecutionFailureInfo", "List", "stack"); - from_json_key( + "revocableMemoryReservationInBytes", + p.revocableMemoryReservationInBytes, + "TaskStats", + "int64_t", + "revocableMemoryReservationInBytes"); + to_json_key( j, - "errorLocation", - p.errorLocation, - "ExecutionFailureInfo", - "ErrorLocation", - "errorLocation"); - from_json_key( + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "TaskStats", + "int64_t", + "systemMemoryReservationInBytes"); + to_json_key( j, - "errorCode", - p.errorCode, - "ExecutionFailureInfo", - "ErrorCode", - "errorCode"); - from_json_key( + "peakTotalMemoryInBytes", + p.peakTotalMemoryInBytes, + "TaskStats", + "int64_t", + "peakTotalMemoryInBytes"); + to_json_key( j, - "remoteHost", - p.remoteHost, - "ExecutionFailureInfo", - "HostAddress", - "remoteHost"); - from_json_key( + "peakUserMemoryInBytes", + p.peakUserMemoryInBytes, + "TaskStats", + "int64_t", + "peakUserMemoryInBytes"); + to_json_key( j, - "errorCause", - p.errorCause, - "ExecutionFailureInfo", - "ErrorCause", - "errorCause"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const TaskStatus& p) { - j = json::object(); + "peakNodeTotalMemoryInBytes", + p.peakNodeTotalMemoryInBytes, + "TaskStats", + "int64_t", + "peakNodeTotalMemoryInBytes"); to_json_key( j, - "taskInstanceIdLeastSignificantBits", - p.taskInstanceIdLeastSignificantBits, - "TaskStatus", + "totalScheduledTimeInNanos", + p.totalScheduledTimeInNanos, + "TaskStats", "int64_t", - "taskInstanceIdLeastSignificantBits"); + "totalScheduledTimeInNanos"); to_json_key( j, - "taskInstanceIdMostSignificantBits", - p.taskInstanceIdMostSignificantBits, - "TaskStatus", + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "TaskStats", "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"); + "totalCpuTimeInNanos"); to_json_key( j, - "completedDriverGroups", - p.completedDriverGroups, - "TaskStatus", - "List", - "completedDriverGroups"); + "totalBlockedTimeInNanos", + p.totalBlockedTimeInNanos, + "TaskStats", + "int64_t", + "totalBlockedTimeInNanos"); + to_json_key( + j, "fullyBlocked", p.fullyBlocked, "TaskStats", "bool", "fullyBlocked"); to_json_key( j, - "failures", - p.failures, - "TaskStatus", - "List", - "failures"); + "blockedReasons", + p.blockedReasons, + "TaskStats", + "List", + "blockedReasons"); to_json_key( j, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "TaskStatus", - "int", - "queuedPartitionedDrivers"); + "totalAllocationInBytes", + p.totalAllocationInBytes, + "TaskStats", + "int64_t", + "totalAllocationInBytes"); to_json_key( j, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "TaskStatus", - "int", - "runningPartitionedDrivers"); + "rawInputDataSizeInBytes", + p.rawInputDataSizeInBytes, + "TaskStats", + "int64_t", + "rawInputDataSizeInBytes"); to_json_key( j, - "outputBufferUtilization", - p.outputBufferUtilization, - "TaskStatus", - "double", - "outputBufferUtilization"); + "rawInputPositions", + p.rawInputPositions, + "TaskStats", + "int64_t", + "rawInputPositions"); to_json_key( j, - "outputBufferOverutilized", - p.outputBufferOverutilized, - "TaskStatus", - "bool", - "outputBufferOverutilized"); + "processedInputDataSizeInBytes", + p.processedInputDataSizeInBytes, + "TaskStats", + "int64_t", + "processedInputDataSizeInBytes"); to_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "TaskStatus", + "processedInputPositions", + p.processedInputPositions, + "TaskStats", "int64_t", - "physicalWrittenDataSizeInBytes"); + "processedInputPositions"); to_json_key( j, - "memoryReservationInBytes", - p.memoryReservationInBytes, - "TaskStatus", + "outputDataSizeInBytes", + p.outputDataSizeInBytes, + "TaskStats", "int64_t", - "memoryReservationInBytes"); + "outputDataSizeInBytes"); to_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "TaskStatus", + "outputPositions", + p.outputPositions, + "TaskStats", "int64_t", - "systemMemoryReservationInBytes"); + "outputPositions"); to_json_key( j, - "peakNodeTotalMemoryReservationInBytes", - p.peakNodeTotalMemoryReservationInBytes, - "TaskStatus", + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "TaskStats", "int64_t", - "peakNodeTotalMemoryReservationInBytes"); + "physicalWrittenDataSizeInBytes"); to_json_key( - j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); + j, "fullGcCount", p.fullGcCount, "TaskStats", "int", "fullGcCount"); to_json_key( j, "fullGcTimeInMillis", p.fullGcTimeInMillis, - "TaskStatus", + "TaskStats", "int64_t", "fullGcTimeInMillis"); to_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "TaskStatus", - "int64_t", - "totalCpuTimeInNanos"); + "pipelines", + p.pipelines, + "TaskStats", + "List", + "pipelines"); to_json_key( j, - "taskAgeInMillis", - p.taskAgeInMillis, - "TaskStatus", + "runtimeStats", + p.runtimeStats, + "TaskStats", + "RuntimeStats", + "runtimeStats"); +} + +void from_json(const json& j, TaskStats& p) { + from_json_key( + j, "createTime", p.createTime, "TaskStats", "DateTime", "createTime"); + from_json_key( + j, + "firstStartTime", + p.firstStartTime, + "TaskStats", + "DateTime", + "firstStartTime"); + from_json_key( + j, + "lastStartTime", + p.lastStartTime, + "TaskStats", + "DateTime", + "lastStartTime"); + from_json_key( + j, "lastEndTime", p.lastEndTime, "TaskStats", "DateTime", "lastEndTime"); + from_json_key(j, "endTime", p.endTime, "TaskStats", "DateTime", "endTime"); + from_json_key( + j, + "elapsedTimeInNanos", + p.elapsedTimeInNanos, + "TaskStats", "int64_t", - "taskAgeInMillis"); - to_json_key( + "elapsedTimeInNanos"); + from_json_key( + j, + "queuedTimeInNanos", + p.queuedTimeInNanos, + "TaskStats", + "int64_t", + "queuedTimeInNanos"); + from_json_key( + j, "totalDrivers", p.totalDrivers, "TaskStats", "int", "totalDrivers"); + from_json_key( + j, "queuedDrivers", p.queuedDrivers, "TaskStats", "int", "queuedDrivers"); + from_json_key( + j, + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "TaskStats", + "int", + "queuedPartitionedDrivers"); + from_json_key( j, "queuedPartitionedSplitsWeight", p.queuedPartitionedSplitsWeight, - "TaskStatus", + "TaskStats", "int64_t", "queuedPartitionedSplitsWeight"); - to_json_key( + from_json_key( + j, + "runningDrivers", + p.runningDrivers, + "TaskStats", + "int", + "runningDrivers"); + from_json_key( + j, + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "TaskStats", + "int", + "runningPartitionedDrivers"); + from_json_key( j, "runningPartitionedSplitsWeight", p.runningPartitionedSplitsWeight, - "TaskStatus", + "TaskStats", "int64_t", "runningPartitionedSplitsWeight"); -} - -void from_json(const json& j, TaskStatus& p) { from_json_key( j, - "taskInstanceIdLeastSignificantBits", - p.taskInstanceIdLeastSignificantBits, - "TaskStatus", + "blockedDrivers", + p.blockedDrivers, + "TaskStats", + "int", + "blockedDrivers"); + from_json_key( + j, + "completedDrivers", + p.completedDrivers, + "TaskStats", + "int", + "completedDrivers"); + from_json_key( + j, + "cumulativeUserMemory", + p.cumulativeUserMemory, + "TaskStats", + "double", + "cumulativeUserMemory"); + from_json_key( + j, + "cumulativeTotalMemory", + p.cumulativeTotalMemory, + "TaskStats", + "double", + "cumulativeTotalMemory"); + from_json_key( + j, + "userMemoryReservationInBytes", + p.userMemoryReservationInBytes, + "TaskStats", "int64_t", - "taskInstanceIdLeastSignificantBits"); + "userMemoryReservationInBytes"); from_json_key( j, - "taskInstanceIdMostSignificantBits", - p.taskInstanceIdMostSignificantBits, - "TaskStatus", + "revocableMemoryReservationInBytes", + p.revocableMemoryReservationInBytes, + "TaskStats", "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"); + "revocableMemoryReservationInBytes"); from_json_key( j, - "completedDriverGroups", - p.completedDriverGroups, - "TaskStatus", - "List", - "completedDriverGroups"); + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "TaskStats", + "int64_t", + "systemMemoryReservationInBytes"); from_json_key( j, - "failures", - p.failures, - "TaskStatus", - "List", - "failures"); + "peakTotalMemoryInBytes", + p.peakTotalMemoryInBytes, + "TaskStats", + "int64_t", + "peakTotalMemoryInBytes"); + from_json_key( + j, + "peakUserMemoryInBytes", + p.peakUserMemoryInBytes, + "TaskStats", + "int64_t", + "peakUserMemoryInBytes"); + from_json_key( + j, + "peakNodeTotalMemoryInBytes", + p.peakNodeTotalMemoryInBytes, + "TaskStats", + "int64_t", + "peakNodeTotalMemoryInBytes"); from_json_key( j, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "TaskStatus", - "int", - "queuedPartitionedDrivers"); + "totalScheduledTimeInNanos", + p.totalScheduledTimeInNanos, + "TaskStats", + "int64_t", + "totalScheduledTimeInNanos"); from_json_key( j, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "TaskStatus", - "int", - "runningPartitionedDrivers"); + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "TaskStats", + "int64_t", + "totalCpuTimeInNanos"); from_json_key( j, - "outputBufferUtilization", - p.outputBufferUtilization, - "TaskStatus", - "double", - "outputBufferUtilization"); + "totalBlockedTimeInNanos", + p.totalBlockedTimeInNanos, + "TaskStats", + "int64_t", + "totalBlockedTimeInNanos"); + from_json_key( + j, "fullyBlocked", p.fullyBlocked, "TaskStats", "bool", "fullyBlocked"); from_json_key( j, - "outputBufferOverutilized", - p.outputBufferOverutilized, - "TaskStatus", - "bool", - "outputBufferOverutilized"); + "blockedReasons", + p.blockedReasons, + "TaskStats", + "List", + "blockedReasons"); from_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "TaskStatus", + "totalAllocationInBytes", + p.totalAllocationInBytes, + "TaskStats", "int64_t", - "physicalWrittenDataSizeInBytes"); + "totalAllocationInBytes"); from_json_key( j, - "memoryReservationInBytes", - p.memoryReservationInBytes, - "TaskStatus", + "rawInputDataSizeInBytes", + p.rawInputDataSizeInBytes, + "TaskStats", "int64_t", - "memoryReservationInBytes"); + "rawInputDataSizeInBytes"); from_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "TaskStatus", + "rawInputPositions", + p.rawInputPositions, + "TaskStats", "int64_t", - "systemMemoryReservationInBytes"); + "rawInputPositions"); from_json_key( j, - "peakNodeTotalMemoryReservationInBytes", - p.peakNodeTotalMemoryReservationInBytes, - "TaskStatus", + "processedInputDataSizeInBytes", + p.processedInputDataSizeInBytes, + "TaskStats", "int64_t", - "peakNodeTotalMemoryReservationInBytes"); - from_json_key( - j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); + "processedInputDataSizeInBytes"); from_json_key( j, - "fullGcTimeInMillis", - p.fullGcTimeInMillis, - "TaskStatus", + "processedInputPositions", + p.processedInputPositions, + "TaskStats", "int64_t", - "fullGcTimeInMillis"); + "processedInputPositions"); from_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "TaskStatus", + "outputDataSizeInBytes", + p.outputDataSizeInBytes, + "TaskStats", "int64_t", - "totalCpuTimeInNanos"); + "outputDataSizeInBytes"); from_json_key( j, - "taskAgeInMillis", - p.taskAgeInMillis, - "TaskStatus", + "outputPositions", + p.outputPositions, + "TaskStats", "int64_t", - "taskAgeInMillis"); + "outputPositions"); from_json_key( j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "TaskStatus", + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "TaskStats", "int64_t", - "queuedPartitionedSplitsWeight"); + "physicalWrittenDataSizeInBytes"); + from_json_key( + j, "fullGcCount", p.fullGcCount, "TaskStats", "int", "fullGcCount"); from_json_key( j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "TaskStatus", + "fullGcTimeInMillis", + p.fullGcTimeInMillis, + "TaskStats", "int64_t", - "runningPartitionedSplitsWeight"); + "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 /* @@ -6282,670 +5451,759 @@ std::string JsonEncodedSubclass::getSubclassKey(nlohmann::json j) { } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +HiveTransactionHandle::HiveTransactionHandle() noexcept { + _type = "hive"; +} -void to_json(json& j, const PageBufferInfo& p) { +void to_json(json& j, const HiveTransactionHandle& p) { j = json::object(); - to_json_key( - j, "partition", p.partition, "PageBufferInfo", "int", "partition"); - to_json_key( - j, - "bufferedPages", - p.bufferedPages, - "PageBufferInfo", - "int64_t", - "bufferedPages"); - to_json_key( - j, - "bufferedBytes", - p.bufferedBytes, - "PageBufferInfo", - "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"); + j["@type"] = "hive"; + to_json_key(j, "uuid", p.uuid, "HiveTransactionHandle", "UUID", "uuid"); } -void from_json(const json& j, PageBufferInfo& p) { - from_json_key( - j, "partition", p.partition, "PageBufferInfo", "int", "partition"); - from_json_key( - j, - "bufferedPages", - p.bufferedPages, - "PageBufferInfo", - "int64_t", - "bufferedPages"); - from_json_key( - j, - "bufferedBytes", - p.bufferedBytes, - "PageBufferInfo", - "int64_t", - "bufferedBytes"); - from_json_key( - j, "rowsAdded", p.rowsAdded, "PageBufferInfo", "int64_t", "rowsAdded"); - from_json_key( - j, "pagesAdded", p.pagesAdded, "PageBufferInfo", "int64_t", "pagesAdded"); +void from_json(const json& j, HiveTransactionHandle& p) { + p._type = j["@type"]; + from_json_key(j, "uuid", p.uuid, "HiveTransactionHandle", "UUID", "uuid"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +ValuesNode::ValuesNode() noexcept { + _type = ".ValuesNode"; +} -void to_json(json& j, const BufferInfo& p) { +void to_json(json& j, const ValuesNode& p) { j = json::object(); + j["@type"] = ".ValuesNode"; to_json_key( - j, "bufferId", p.bufferId, "BufferInfo", "OutputBufferId", "bufferId"); - to_json_key(j, "finished", p.finished, "BufferInfo", "bool", "finished"); + j, "location", p.location, "ValuesNode", "SourceLocation", "location"); + to_json_key(j, "id", p.id, "ValuesNode", "PlanNodeId", "id"); to_json_key( j, - "bufferedPages", - p.bufferedPages, - "BufferInfo", - "int", - "bufferedPages"); + "outputVariables", + p.outputVariables, + "ValuesNode", + "List", + "outputVariables"); to_json_key( - j, "pagesSent", p.pagesSent, "BufferInfo", "int64_t", "pagesSent"); + j, + "rows", + p.rows, + "ValuesNode", + "List>>", + "rows"); to_json_key( j, - "pageBufferInfo", - p.pageBufferInfo, - "BufferInfo", - "PageBufferInfo", - "pageBufferInfo"); + "valuesNodeLabel", + p.valuesNodeLabel, + "ValuesNode", + "String", + "valuesNodeLabel"); } -void from_json(const json& j, BufferInfo& p) { +void from_json(const json& j, ValuesNode& p) { + p._type = j["@type"]; from_json_key( - j, "bufferId", p.bufferId, "BufferInfo", "OutputBufferId", "bufferId"); - from_json_key(j, "finished", p.finished, "BufferInfo", "bool", "finished"); + j, "location", p.location, "ValuesNode", "SourceLocation", "location"); + from_json_key(j, "id", p.id, "ValuesNode", "PlanNodeId", "id"); from_json_key( j, - "bufferedPages", - p.bufferedPages, - "BufferInfo", - "int", - "bufferedPages"); + "outputVariables", + p.outputVariables, + "ValuesNode", + "List", + "outputVariables"); from_json_key( - j, "pagesSent", p.pagesSent, "BufferInfo", "int64_t", "pagesSent"); + j, + "rows", + p.rows, + "ValuesNode", + "List>>", + "rows"); from_json_key( j, - "pageBufferInfo", - p.pageBufferInfo, - "BufferInfo", - "PageBufferInfo", - "pageBufferInfo"); + "valuesNodeLabel", + p.valuesNodeLabel, + "ValuesNode", + "String", + "valuesNodeLabel"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair Step_enum_table[] = +static const std::pair PrestoTableType_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!"); + {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(Step_enum_table), - std::end(Step_enum_table), - [e](const std::pair& ej_pair) -> bool { + 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(Step_enum_table)) ? it : std::begin(Step_enum_table)) + j = ((it != std::end(PrestoTableType_enum_table)) + ? it + : std::begin(PrestoTableType_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, PrestoTableType& e) { + static_assert( + std::is_enum::value, "PrestoTableType 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(PrestoTableType_enum_table), + std::end(PrestoTableType_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(PrestoTableType_enum_table)) + ? it + : std::begin(PrestoTableType_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TopNNode::TopNNode() noexcept { - _type = ".TopNNode"; -} -void to_json(json& j, const TopNNode& p) { +void to_json(json& j, const Table& 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, "databaseName", p.databaseName, "Table", "String", "databaseName"); + to_json_key(j, "tableName", p.tableName, "Table", "String", "tableName"); + to_json_key(j, "owner", p.owner, "Table", "String", "owner"); + to_json_key( + j, "tableType", p.tableType, "Table", "PrestoTableType", "tableType"); + to_json_key(j, "storage", p.storage, "Table", "Storage", "storage"); + to_json_key( + j, "dataColumns", p.dataColumns, "Table", "List", "dataColumns"); to_json_key( j, - "orderingScheme", - p.orderingScheme, - "TopNNode", - "OrderingScheme", - "orderingScheme"); - to_json_key(j, "step", p.step, "TopNNode", "Step", "step"); + "partitionColumns", + p.partitionColumns, + "Table", + "List", + "partitionColumns"); + to_json_key( + j, + "parameters", + p.parameters, + "Table", + "Map", + "parameters"); + to_json_key( + j, + "viewOriginalText", + p.viewOriginalText, + "Table", + "String", + "viewOriginalText"); + to_json_key( + j, + "viewExpandedText", + p.viewExpandedText, + "Table", + "String", + "viewExpandedText"); } -void from_json(const json& j, 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"); +void from_json(const json& j, Table& p) { + from_json_key( + j, "databaseName", p.databaseName, "Table", "String", "databaseName"); + from_json_key(j, "tableName", p.tableName, "Table", "String", "tableName"); + from_json_key(j, "owner", p.owner, "Table", "String", "owner"); + from_json_key( + j, "tableType", p.tableType, "Table", "PrestoTableType", "tableType"); + from_json_key(j, "storage", p.storage, "Table", "Storage", "storage"); + from_json_key( + j, "dataColumns", p.dataColumns, "Table", "List", "dataColumns"); from_json_key( j, - "orderingScheme", - p.orderingScheme, - "TopNNode", - "OrderingScheme", - "orderingScheme"); - from_json_key(j, "step", p.step, "TopNNode", "Step", "step"); + "partitionColumns", + p.partitionColumns, + "Table", + "List", + "partitionColumns"); + from_json_key( + j, + "parameters", + p.parameters, + "Table", + "Map", + "parameters"); + from_json_key( + j, + "viewOriginalText", + p.viewOriginalText, + "Table", + "String", + "viewOriginalText"); + from_json_key( + j, + "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 std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); - return; - } - - if (getConnectorKey(type) == "tpch") { - j = *std::static_pointer_cast(p); - return; - } - throw TypeError(type + " no abstract type ColumnHandle "); +void 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, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " ColumnHandle ColumnHandle"); - } - - if (getConnectorKey(type) == "hive") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } +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 { +HiveMetadataUpdateHandle::HiveMetadataUpdateHandle() noexcept { + _type = "hive"; +} - if (getConnectorKey(type) == "tpch") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } +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"); +} - throw TypeError(type + " no abstract type ColumnHandle "); +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"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TableScanNode::TableScanNode() noexcept { - _type = ".TableScanNode"; +InsertHandle::InsertHandle() noexcept { + _type = "InsertHandle"; } -void to_json(json& j, const TableScanNode& p) { +void to_json(json& j, const InsertHandle& 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"] = "InsertHandle"; to_json_key( - j, - "outputVariables", - p.outputVariables, - "TableScanNode", - "List", - "outputVariables"); + j, "handle", p.handle, "InsertHandle", "InsertTableHandle", "handle"); to_json_key( j, - "assignments", - p.assignments, - "TableScanNode", - "Map>", - "assignments"); + "schemaTableName", + p.schemaTableName, + "InsertHandle", + "SchemaTableName", + "schemaTableName"); } -void from_json(const json& j, TableScanNode& p) { +void from_json(const json& j, InsertHandle& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "TableScanNode", "PlanNodeId", "id"); - from_json_key(j, "table", p.table, "TableScanNode", "TableHandle", "table"); from_json_key( - j, - "outputVariables", - p.outputVariables, - "TableScanNode", - "List", - "outputVariables"); + j, "handle", p.handle, "InsertHandle", "InsertTableHandle", "handle"); from_json_key( j, - "assignments", - p.assignments, - "TableScanNode", - "Map>", - "assignments"); + "schemaTableName", + p.schemaTableName, + "InsertHandle", + "SchemaTableName", + "schemaTableName"); +} +} // 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 { // 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 JoinNodeType_enum_table[] = + { // NOLINT: cert-err58-cpp + {JoinNodeType::INNER, "INNER"}, + {JoinNodeType::LEFT, "LEFT"}, + {JoinNodeType::RIGHT, "RIGHT"}, + {JoinNodeType::FULL, "FULL"}}; +void to_json(json& j, const JoinNodeType& e) { static_assert( - std::is_enum::value, - "NodeSelectionStrategy must be an enum!"); + std::is_enum::value, "JoinNodeType 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(JoinNodeType_enum_table), + std::end(JoinNodeType_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(JoinNodeType_enum_table)) ? it - : std::begin(NodeSelectionStrategy_enum_table)) + : std::begin(JoinNodeType_enum_table)) ->second; } -void from_json(const json& j, NodeSelectionStrategy& e) { +void from_json(const json& j, JoinNodeType& e) { static_assert( - std::is_enum::value, - "NodeSelectionStrategy must be an enum!"); + std::is_enum::value, "JoinNodeType 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(JoinNodeType_enum_table), + std::end(JoinNodeType_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(JoinNodeType_enum_table)) ? it - : std::begin(NodeSelectionStrategy_enum_table)) + : std::begin(JoinNodeType_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const HiveFileSplit& p) { +void to_json(json& j, const EquiJoinClause& p) { j = json::object(); - to_json_key(j, "path", p.path, "HiveFileSplit", "String", "path"); - to_json_key(j, "start", p.start, "HiveFileSplit", "int64_t", "start"); - to_json_key(j, "length", p.length, "HiveFileSplit", "int64_t", "length"); - to_json_key( - j, "fileSize", p.fileSize, "HiveFileSplit", "int64_t", "fileSize"); - to_json_key( - j, - "fileModifiedTime", - p.fileModifiedTime, - "HiveFileSplit", - "int64_t", - "fileModifiedTime"); to_json_key( j, - "extraFileInfo", - p.extraFileInfo, - "HiveFileSplit", - "String", - "extraFileInfo"); + "left", + p.left, + "EquiJoinClause", + "VariableReferenceExpression", + "left"); to_json_key( j, - "customSplitInfo", - p.customSplitInfo, - "HiveFileSplit", - "Map", - "customSplitInfo"); + "right", + p.right, + "EquiJoinClause", + "VariableReferenceExpression", + "right"); } -void from_json(const json& j, HiveFileSplit& p) { - from_json_key(j, "path", p.path, "HiveFileSplit", "String", "path"); - from_json_key(j, "start", p.start, "HiveFileSplit", "int64_t", "start"); - from_json_key(j, "length", p.length, "HiveFileSplit", "int64_t", "length"); - from_json_key( - j, "fileSize", p.fileSize, "HiveFileSplit", "int64_t", "fileSize"); - from_json_key( - j, - "fileModifiedTime", - p.fileModifiedTime, - "HiveFileSplit", - "int64_t", - "fileModifiedTime"); +void from_json(const json& j, EquiJoinClause& p) { from_json_key( j, - "extraFileInfo", - p.extraFileInfo, - "HiveFileSplit", - "String", - "extraFileInfo"); + "left", + p.left, + "EquiJoinClause", + "VariableReferenceExpression", + "left"); from_json_key( j, - "customSplitInfo", - p.customSplitInfo, - "HiveFileSplit", - "Map", - "customSplitInfo"); + "right", + p.right, + "EquiJoinClause", + "VariableReferenceExpression", + "right"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -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"); +// 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, TableToPartitionMapping& p) { - from_json_key( - j, - "tableToPartitionColumns", - p.tableToPartitionColumns, - "TableToPartitionMapping", - "Map", - "tableToPartitionColumns"); - from_json_key( - j, - "partitionSchemaDifference", - p.partitionSchemaDifference, - "TableToPartitionMapping", - "Map", - "partitionSchemaDifference"); +void from_json(const json& j, DistributionType& e) { + static_assert( + std::is_enum::value, + "DistributionType must be an enum!"); + const auto* it = std::find_if( + std::begin(DistributionType_enum_table), + std::end(DistributionType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(DistributionType_enum_table)) + ? it + : std::begin(DistributionType_enum_table)) + ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveSplit::HiveSplit() noexcept { - _type = "hive"; +JoinNode::JoinNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.JoinNode"; } -void to_json(json& j, const HiveSplit& p) { +void to_json(json& j, const JoinNode& p) { j = json::object(); - j["@type"] = "hive"; - to_json_key( - j, "fileSplit", p.fileSplit, "HiveSplit", "HiveFileSplit", "fileSplit"); - to_json_key(j, "database", p.database, "HiveSplit", "String", "database"); - to_json_key(j, "table", p.table, "HiveSplit", "String", "table"); - to_json_key( - j, - "partitionName", - p.partitionName, - "HiveSplit", - "String", - "partitionName"); - to_json_key(j, "storage", p.storage, "HiveSplit", "Storage", "storage"); - to_json_key( - j, - "partitionKeys", - p.partitionKeys, - "HiveSplit", - "List", - "partitionKeys"); - to_json_key( - j, - "addresses", - p.addresses, - "HiveSplit", - "List", - "addresses"); - to_json_key( - j, - "readBucketNumber", - p.readBucketNumber, - "HiveSplit", - "int", - "readBucketNumber"); - to_json_key( - j, - "tableBucketNumber", - p.tableBucketNumber, - "HiveSplit", - "int", - "tableBucketNumber"); - to_json_key( - j, - "nodeSelectionStrategy", - p.nodeSelectionStrategy, - "HiveSplit", - "NodeSelectionStrategy", - "nodeSelectionStrategy"); - to_json_key( - j, - "partitionDataColumnCount", - p.partitionDataColumnCount, - "HiveSplit", - "int", - "partitionDataColumnCount"); + j["@type"] = "com.facebook.presto.sql.planner.plan.JoinNode"; + to_json_key(j, "id", p.id, "JoinNode", "PlanNodeId", "id"); + to_json_key(j, "type", p.type, "JoinNode", "JoinNodeType", "type"); + to_json_key(j, "left", p.left, "JoinNode", "PlanNode", "left"); + to_json_key(j, "right", p.right, "JoinNode", "PlanNode", "right"); to_json_key( j, - "tableToPartitionMapping", - p.tableToPartitionMapping, - "HiveSplit", - "TableToPartitionMapping", - "tableToPartitionMapping"); + "criteria", + p.criteria, + "JoinNode", + "List", + "criteria"); to_json_key( j, - "bucketConversion", - p.bucketConversion, - "HiveSplit", - "BucketConversion", - "bucketConversion"); + "outputVariables", + p.outputVariables, + "JoinNode", + "List", + "outputVariables"); to_json_key( j, - "s3SelectPushdownEnabled", - p.s3SelectPushdownEnabled, - "HiveSplit", - "bool", - "s3SelectPushdownEnabled"); + "filter", + p.filter, + "JoinNode", + "std::shared_ptr", + "filter"); to_json_key( j, - "cacheQuota", - p.cacheQuota, - "HiveSplit", - "CacheQuotaRequirement", - "cacheQuota"); + "leftHashVariable", + p.leftHashVariable, + "JoinNode", + "VariableReferenceExpression", + "leftHashVariable"); to_json_key( j, - "encryptionMetadata", - p.encryptionMetadata, - "HiveSplit", - "EncryptionInformation", - "encryptionMetadata"); + "rightHashVariable", + p.rightHashVariable, + "JoinNode", + "VariableReferenceExpression", + "rightHashVariable"); to_json_key( j, - "redundantColumnDomains", - p.redundantColumnDomains, - "HiveSplit", - "List>", - "redundantColumnDomains"); + "distributionType", + p.distributionType, + "JoinNode", + "DistributionType", + "distributionType"); to_json_key( j, - "splitWeight", - p.splitWeight, - "HiveSplit", - "SplitWeight", - "splitWeight"); + "dynamicFilters", + p.dynamicFilters, + "JoinNode", + "Map", + "dynamicFilters"); } -void from_json(const json& j, HiveSplit& p) { +void from_json(const json& j, JoinNode& 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, "id", p.id, "JoinNode", "PlanNodeId", "id"); + from_json_key(j, "type", p.type, "JoinNode", "JoinNodeType", "type"); + from_json_key(j, "left", p.left, "JoinNode", "PlanNode", "left"); + from_json_key(j, "right", p.right, "JoinNode", "PlanNode", "right"); from_json_key( j, - "partitionName", - p.partitionName, - "HiveSplit", - "String", - "partitionName"); - from_json_key(j, "storage", p.storage, "HiveSplit", "Storage", "storage"); + "criteria", + p.criteria, + "JoinNode", + "List", + "criteria"); from_json_key( j, - "partitionKeys", - p.partitionKeys, - "HiveSplit", - "List", - "partitionKeys"); + "outputVariables", + p.outputVariables, + "JoinNode", + "List", + "outputVariables"); from_json_key( j, - "addresses", - p.addresses, - "HiveSplit", - "List", - "addresses"); + "filter", + p.filter, + "JoinNode", + "std::shared_ptr", + "filter"); from_json_key( j, - "readBucketNumber", - p.readBucketNumber, - "HiveSplit", - "int", - "readBucketNumber"); + "leftHashVariable", + p.leftHashVariable, + "JoinNode", + "VariableReferenceExpression", + "leftHashVariable"); from_json_key( j, - "tableBucketNumber", - p.tableBucketNumber, - "HiveSplit", - "int", - "tableBucketNumber"); + "rightHashVariable", + p.rightHashVariable, + "JoinNode", + "VariableReferenceExpression", + "rightHashVariable"); from_json_key( j, - "nodeSelectionStrategy", - p.nodeSelectionStrategy, - "HiveSplit", - "NodeSelectionStrategy", - "nodeSelectionStrategy"); + "distributionType", + p.distributionType, + "JoinNode", + "DistributionType", + "distributionType"); from_json_key( j, - "partitionDataColumnCount", - p.partitionDataColumnCount, - "HiveSplit", - "int", - "partitionDataColumnCount"); + "dynamicFilters", + p.dynamicFilters, + "JoinNode", + "Map", + "dynamicFilters"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +FilterNode::FilterNode() noexcept { + _type = ".FilterNode"; +} + +void to_json(json& j, const FilterNode& p) { + j = json::object(); + j["@type"] = ".FilterNode"; + to_json_key(j, "id", p.id, "FilterNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "FilterNode", "PlanNode", "source"); + to_json_key( + j, "predicate", p.predicate, "FilterNode", "RowExpression", "predicate"); +} + +void from_json(const json& j, FilterNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "FilterNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "FilterNode", "PlanNode", "source"); from_json_key( + j, "predicate", p.predicate, "FilterNode", "RowExpression", "predicate"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const ResourceEstimates& p) { + j = json::object(); + to_json_key( j, - "tableToPartitionMapping", - p.tableToPartitionMapping, - "HiveSplit", - "TableToPartitionMapping", - "tableToPartitionMapping"); - from_json_key( + "executionTime", + p.executionTime, + "ResourceEstimates", + "Duration", + "executionTime"); + to_json_key( + j, "cpuTime", p.cpuTime, "ResourceEstimates", "Duration", "cpuTime"); + to_json_key( + j, + "peakMemory", + p.peakMemory, + "ResourceEstimates", + "DataSize", + "peakMemory"); + to_json_key( j, - "bucketConversion", - p.bucketConversion, - "HiveSplit", - "BucketConversion", - "bucketConversion"); + "peakTaskMemory", + p.peakTaskMemory, + "ResourceEstimates", + "DataSize", + "peakTaskMemory"); +} + +void from_json(const json& j, ResourceEstimates& p) { from_json_key( j, - "s3SelectPushdownEnabled", - p.s3SelectPushdownEnabled, - "HiveSplit", - "bool", - "s3SelectPushdownEnabled"); + "executionTime", + p.executionTime, + "ResourceEstimates", + "Duration", + "executionTime"); from_json_key( - j, - "cacheQuota", - p.cacheQuota, - "HiveSplit", - "CacheQuotaRequirement", - "cacheQuota"); + j, "cpuTime", p.cpuTime, "ResourceEstimates", "Duration", "cpuTime"); from_json_key( j, - "encryptionMetadata", - p.encryptionMetadata, - "HiveSplit", - "EncryptionInformation", - "encryptionMetadata"); + "peakMemory", + p.peakMemory, + "ResourceEstimates", + "DataSize", + "peakMemory"); from_json_key( j, - "redundantColumnDomains", - p.redundantColumnDomains, - "HiveSplit", - "List>", - "redundantColumnDomains"); + "peakTaskMemory", + p.peakTaskMemory, + "ResourceEstimates", + "DataSize", + "peakTaskMemory"); +} +} // 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, - "splitWeight", - p.splitWeight, - "HiveSplit", - "SplitWeight", - "splitWeight"); + j, "nullAllowed", p.nullAllowed, "Domain", "bool", "nullAllowed"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair - 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!"); +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(SystemPartitionFunction_enum_table), - std::end(SystemPartitionFunction_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(BoundType_enum_table), + std::end(BoundType_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(SystemPartitionFunction_enum_table)) + j = ((it != std::end(BoundType_enum_table)) ? it - : std::begin(SystemPartitionFunction_enum_table)) + : std::begin(BoundType_enum_table)) ->second; } -void from_json(const json& j, SystemPartitionFunction& e) { - static_assert( - std::is_enum::value, - "SystemPartitionFunction must be an enum!"); +void from_json(const json& j, BoundType& e) { + static_assert(std::is_enum::value, "BoundType must be an enum!"); const auto* it = std::find_if( - std::begin(SystemPartitionFunction_enum_table), - std::end(SystemPartitionFunction_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(BoundType_enum_table), + std::end(BoundType_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(SystemPartitionFunction_enum_table)) + e = ((it != std::end(BoundType_enum_table)) ? it - : std::begin(SystemPartitionFunction_enum_table)) + : std::begin(BoundType_enum_table)) ->first; } } // namespace facebook::presto::protocol @@ -6953,5387 +6211,6143 @@ 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!"); +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(SystemPartitioning_enum_table), - std::end(SystemPartitioning_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(SystemPartitioning_enum_table)) + j = ((it != std::end(WindowType_enum_table)) ? it - : std::begin(SystemPartitioning_enum_table)) + : std::begin(WindowType_enum_table)) ->second; } -void from_json(const json& j, SystemPartitioning& e) { - static_assert( - std::is_enum::value, - "SystemPartitioning 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(SystemPartitioning_enum_table), - std::end(SystemPartitioning_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(SystemPartitioning_enum_table)) + e = ((it != std::end(WindowType_enum_table)) ? it - : std::begin(SystemPartitioning_enum_table)) + : std::begin(WindowType_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -SystemPartitioningHandle::SystemPartitioningHandle() noexcept { - _type = "$remote"; -} -void to_json(json& j, const SystemPartitioningHandle& p) { +void to_json(json& j, const Frame& p) { j = json::object(); - j["@type"] = "$remote"; + 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, - "partitioning", - p.partitioning, - "SystemPartitioningHandle", - "SystemPartitioning", - "partitioning"); + "originalStartValue", + p.originalStartValue, + "Frame", + "String", + "originalStartValue"); to_json_key( j, - "function", - p.function, - "SystemPartitioningHandle", - "SystemPartitionFunction", - "function"); + "originalEndValue", + p.originalEndValue, + "Frame", + "String", + "originalEndValue"); } -void from_json(const json& j, SystemPartitioningHandle& p) { - p._type = j["@type"]; +void from_json(const json& j, Frame& p) { + from_json_key(j, "type", p.type, "Frame", "WindowType", "type"); + from_json_key(j, "startType", p.startType, "Frame", "BoundType", "startType"); from_json_key( j, - "partitioning", - p.partitioning, - "SystemPartitioningHandle", - "SystemPartitioning", - "partitioning"); + "startValue", + p.startValue, + "Frame", + "VariableReferenceExpression", + "startValue"); from_json_key( j, - "function", - p.function, - "SystemPartitioningHandle", - "SystemPartitionFunction", - "function"); + "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"); + from_json_key( + j, + "originalEndValue", + p.originalEndValue, + "Frame", + "String", + "originalEndValue"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Range& p) { +void to_json(json& j, const Function& p) { j = json::object(); - to_json_key(j, "low", p.low, "Range", "Marker", "low"); - to_json_key(j, "high", p.high, "Range", "Marker", "high"); + to_json_key( + j, + "functionCall", + p.functionCall, + "Function", + "CallExpression", + "functionCall"); + to_json_key(j, "frame", p.frame, "Function", "Frame", "frame"); + to_json_key( + j, "ignoreNulls", p.ignoreNulls, "Function", "bool", "ignoreNulls"); } -void 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"); +void from_json(const json& j, Function& p) { + from_json_key( + j, + "functionCall", + p.functionCall, + "Function", + "CallExpression", + "functionCall"); + from_json_key(j, "frame", p.frame, "Function", "Frame", "frame"); + from_json_key( + j, "ignoreNulls", p.ignoreNulls, "Function", "bool", "ignoreNulls"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -SortedRangeSet::SortedRangeSet() noexcept { - _type = "sortable"; -} -void to_json(json& j, const SortedRangeSet& p) { +void to_json(json& j, const MemoryAllocation& 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"); + 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, SortedRangeSet& p) { - p._type = j["@type"]; - from_json_key(j, "type", p.type, "SortedRangeSet", "Type", "type"); +void from_json(const json& j, MemoryAllocation& p) { + from_json_key(j, "tag", p.tag, "MemoryAllocation", "String", "tag"); from_json_key( - j, "ranges", p.ranges, "SortedRangeSet", "List", "ranges"); + j, + "allocation", + p.allocation, + "MemoryAllocation", + "int64_t", + "allocation"); } } // 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 MemoryPoolInfo& p) { j = json::object(); - j["@type"] = "hive"; + to_json_key( + j, "maxBytes", p.maxBytes, "MemoryPoolInfo", "int64_t", "maxBytes"); to_json_key( j, - "schemaTableName", - p.schemaTableName, - "HiveTableLayoutHandle", - "SchemaTableName", - "schemaTableName"); + "reservedBytes", + p.reservedBytes, + "MemoryPoolInfo", + "int64_t", + "reservedBytes"); to_json_key( j, - "tablePath", - p.tablePath, - "HiveTableLayoutHandle", - "String", - "tablePath"); + "reservedRevocableBytes", + p.reservedRevocableBytes, + "MemoryPoolInfo", + "int64_t", + "reservedRevocableBytes"); to_json_key( j, - "partitionColumns", - p.partitionColumns, - "HiveTableLayoutHandle", - "List", - "partitionColumns"); + "queryMemoryReservations", + p.queryMemoryReservations, + "MemoryPoolInfo", + "Map", + "queryMemoryReservations"); to_json_key( j, - "dataColumns", - p.dataColumns, - "HiveTableLayoutHandle", - "List", - "dataColumns"); + "queryMemoryAllocations", + p.queryMemoryAllocations, + "MemoryPoolInfo", + "Map>", + "queryMemoryAllocations"); to_json_key( j, - "tableParameters", - p.tableParameters, - "HiveTableLayoutHandle", - "Map", - "tableParameters"); + "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 { + +void to_json(json& j, const MemoryInfo& p) { + j = json::object(); to_json_key( j, - "domainPredicate", - p.domainPredicate, - "HiveTableLayoutHandle", - "TupleDomain", - "domainPredicate"); + "totalNodeMemory", + p.totalNodeMemory, + "MemoryInfo", + "DataSize", + "totalNodeMemory"); to_json_key( j, - "remainingPredicate", - p.remainingPredicate, - "HiveTableLayoutHandle", - "RowExpression", - "remainingPredicate"); - to_json_key( + "pools", + p.pools, + "MemoryInfo", + "Map", + "pools"); +} + +void from_json(const json& j, MemoryInfo& p) { + from_json_key( + j, + "totalNodeMemory", + p.totalNodeMemory, + "MemoryInfo", + "DataSize", + "totalNodeMemory"); + from_json_key( j, - "predicateColumns", - p.predicateColumns, - "HiveTableLayoutHandle", - "Map", - "predicateColumns"); + "pools", + p.pools, + "MemoryInfo", + "Map", + "pools"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const NodeStatus& p) { + j = json::object(); + to_json_key(j, "nodeId", p.nodeId, "NodeStatus", "String", "nodeId"); to_json_key( j, - "partitionColumnPredicate", - p.partitionColumnPredicate, - "HiveTableLayoutHandle", - "TupleDomain>", - "partitionColumnPredicate"); + "nodeVersion", + p.nodeVersion, + "NodeStatus", + "NodeVersion", + "nodeVersion"); to_json_key( - j, - "bucketHandle", - p.bucketHandle, - "HiveTableLayoutHandle", - "HiveBucketHandle", - "bucketHandle"); + j, "environment", p.environment, "NodeStatus", "String", "environment"); to_json_key( - j, - "bucketFilter", - p.bucketFilter, - "HiveTableLayoutHandle", - "HiveBucketFilter", - "bucketFilter"); + j, "coordinator", p.coordinator, "NodeStatus", "bool", "coordinator"); + to_json_key(j, "uptime", p.uptime, "NodeStatus", "Duration", "uptime"); to_json_key( j, - "pushdownFilterEnabled", - p.pushdownFilterEnabled, - "HiveTableLayoutHandle", - "bool", - "pushdownFilterEnabled"); + "externalAddress", + p.externalAddress, + "NodeStatus", + "String", + "externalAddress"); to_json_key( j, - "layoutString", - p.layoutString, - "HiveTableLayoutHandle", + "internalAddress", + p.internalAddress, + "NodeStatus", "String", - "layoutString"); + "internalAddress"); to_json_key( - j, - "requestedColumns", - p.requestedColumns, - "HiveTableLayoutHandle", - "List", - "requestedColumns"); + j, "memoryInfo", p.memoryInfo, "NodeStatus", "MemoryInfo", "memoryInfo"); + to_json_key(j, "processors", p.processors, "NodeStatus", "int", "processors"); to_json_key( j, - "partialAggregationsPushedDown", - p.partialAggregationsPushedDown, - "HiveTableLayoutHandle", - "bool", - "partialAggregationsPushedDown"); + "processCpuLoad", + p.processCpuLoad, + "NodeStatus", + "double", + "processCpuLoad"); to_json_key( j, - "appendRowNumber", - p.appendRowNumber, - "HiveTableLayoutHandle", - "bool", - "appendRowNumber"); + "systemCpuLoad", + p.systemCpuLoad, + "NodeStatus", + "double", + "systemCpuLoad"); + to_json_key(j, "heapUsed", p.heapUsed, "NodeStatus", "int64_t", "heapUsed"); to_json_key( j, - "footerStatsUnreliable", - p.footerStatsUnreliable, - "HiveTableLayoutHandle", - "bool", - "footerStatsUnreliable"); + "heapAvailable", + p.heapAvailable, + "NodeStatus", + "int64_t", + "heapAvailable"); + to_json_key( + j, "nonHeapUsed", p.nonHeapUsed, "NodeStatus", "int64_t", "nonHeapUsed"); } -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"); +void from_json(const json& j, NodeStatus& p) { + from_json_key(j, "nodeId", p.nodeId, "NodeStatus", "String", "nodeId"); from_json_key( j, - "partitionColumnPredicate", - p.partitionColumnPredicate, - "HiveTableLayoutHandle", - "TupleDomain>", - "partitionColumnPredicate"); + "nodeVersion", + p.nodeVersion, + "NodeStatus", + "NodeVersion", + "nodeVersion"); from_json_key( - j, - "bucketHandle", - p.bucketHandle, - "HiveTableLayoutHandle", - "HiveBucketHandle", - "bucketHandle"); + j, "environment", p.environment, "NodeStatus", "String", "environment"); from_json_key( - j, - "bucketFilter", - p.bucketFilter, - "HiveTableLayoutHandle", - "HiveBucketFilter", - "bucketFilter"); + j, "coordinator", p.coordinator, "NodeStatus", "bool", "coordinator"); + from_json_key(j, "uptime", p.uptime, "NodeStatus", "Duration", "uptime"); from_json_key( j, - "pushdownFilterEnabled", - p.pushdownFilterEnabled, - "HiveTableLayoutHandle", - "bool", - "pushdownFilterEnabled"); + "externalAddress", + p.externalAddress, + "NodeStatus", + "String", + "externalAddress"); from_json_key( j, - "layoutString", - p.layoutString, - "HiveTableLayoutHandle", + "internalAddress", + p.internalAddress, + "NodeStatus", "String", - "layoutString"); + "internalAddress"); from_json_key( - j, - "requestedColumns", - p.requestedColumns, - "HiveTableLayoutHandle", - "List", - "requestedColumns"); + j, "memoryInfo", p.memoryInfo, "NodeStatus", "MemoryInfo", "memoryInfo"); + from_json_key( + j, "processors", p.processors, "NodeStatus", "int", "processors"); from_json_key( j, - "partialAggregationsPushedDown", - p.partialAggregationsPushedDown, - "HiveTableLayoutHandle", - "bool", - "partialAggregationsPushedDown"); + "processCpuLoad", + p.processCpuLoad, + "NodeStatus", + "double", + "processCpuLoad"); from_json_key( j, - "appendRowNumber", - p.appendRowNumber, - "HiveTableLayoutHandle", - "bool", - "appendRowNumber"); + "systemCpuLoad", + p.systemCpuLoad, + "NodeStatus", + "double", + "systemCpuLoad"); + from_json_key(j, "heapUsed", p.heapUsed, "NodeStatus", "int64_t", "heapUsed"); from_json_key( j, - "footerStatsUnreliable", - p.footerStatsUnreliable, - "HiveTableLayoutHandle", - "bool", - "footerStatsUnreliable"); + "heapAvailable", + p.heapAvailable, + "NodeStatus", + "int64_t", + "heapAvailable"); + from_json_key( + j, "nonHeapUsed", p.nonHeapUsed, "NodeStatus", "int64_t", "nonHeapUsed"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -SortNode::SortNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.SortNode"; -} -void to_json(json& j, const SortNode& p) { +void to_json(json& j, const ServerInfo& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.SortNode"; - to_json_key(j, "id", p.id, "SortNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "SortNode", "PlanNode", "source"); to_json_key( j, - "orderingScheme", - p.orderingScheme, - "SortNode", - "OrderingScheme", - "orderingScheme"); - to_json_key(j, "isPartial", p.isPartial, "SortNode", "bool", "isPartial"); + "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, 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"); +void from_json(const json& j, ServerInfo& p) { from_json_key( j, - "orderingScheme", - p.orderingScheme, - "SortNode", - "OrderingScheme", - "orderingScheme"); - from_json_key(j, "isPartial", p.isPartial, "SortNode", "bool", "isPartial"); + "nodeVersion", + p.nodeVersion, + "ServerInfo", + "NodeVersion", + "nodeVersion"); + from_json_key( + j, "environment", p.environment, "ServerInfo", "String", "environment"); + from_json_key( + j, "coordinator", p.coordinator, "ServerInfo", "bool", "coordinator"); + from_json_key(j, "starting", p.starting, "ServerInfo", "bool", "starting"); + from_json_key(j, "uptime", p.uptime, "ServerInfo", "Duration", "uptime"); } } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair 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; + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ConnectorOutputTableHandle "); } -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; + +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError( + std::string(e.what()) + + " ConnectorOutputTableHandle ConnectorOutputTableHandle"); + } + + if (getConnectorKey(type) == "hive") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + + throw TypeError(type + " no abstract type ConnectorOutputTableHandle "); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -SpecialFormExpression::SpecialFormExpression() noexcept { - _type = "special"; -} -void to_json(json& j, const SpecialFormExpression& p) { +void to_json(json& j, const OutputTableHandle& 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"); + "connectorId", + p.connectorId, + "OutputTableHandle", + "ConnectorId", + "connectorId"); to_json_key( j, - "returnType", - p.returnType, - "SpecialFormExpression", - "Type", - "returnType"); + "transactionHandle", + p.transactionHandle, + "OutputTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); to_json_key( j, - "arguments", - p.arguments, - "SpecialFormExpression", - "List>", - "arguments"); + "connectorHandle", + p.connectorHandle, + "OutputTableHandle", + "ConnectorOutputTableHandle", + "connectorHandle"); } -void from_json(const json& j, SpecialFormExpression& 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"); +void from_json(const json& j, OutputTableHandle& p) { from_json_key( j, - "returnType", - p.returnType, - "SpecialFormExpression", - "Type", - "returnType"); + "connectorId", + p.connectorId, + "OutputTableHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "arguments", - p.arguments, - "SpecialFormExpression", - "List>", - "arguments"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -SqlFunctionHandle::SqlFunctionHandle() noexcept { - _type = "json_file"; -} - -void to_json(json& j, const SqlFunctionHandle& p) { - j = json::object(); - j["@type"] = "json_file"; - to_json_key( - j, - "functionId", - p.functionId, - "SqlFunctionHandle", - "SqlFunctionId", - "functionId"); - to_json_key( - j, "version", p.version, "SqlFunctionHandle", "String", "version"); -} - -void from_json(const json& j, SqlFunctionHandle& p) { - p._type = j["@type"]; + "transactionHandle", + p.transactionHandle, + "OutputTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); from_json_key( j, - "functionId", - p.functionId, - "SqlFunctionHandle", - "SqlFunctionId", - "functionId"); - from_json_key( - j, "version", p.version, "SqlFunctionHandle", "String", "version"); + "connectorHandle", + p.connectorHandle, + "OutputTableHandle", + "ConnectorOutputTableHandle", + "connectorHandle"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -MarkDistinctNode::MarkDistinctNode() noexcept { - _type = ".MarkDistinctNode"; +CreateHandle::CreateHandle() noexcept { + _type = "CreateHandle"; } -void to_json(json& j, const MarkDistinctNode& p) { +void to_json(json& j, const CreateHandle& p) { j = json::object(); - j["@type"] = ".MarkDistinctNode"; - to_json_key(j, "id", p.id, "MarkDistinctNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "MarkDistinctNode", "PlanNode", "source"); - to_json_key( - j, - "markerVariable", - p.markerVariable, - "MarkDistinctNode", - "VariableReferenceExpression", - "markerVariable"); + j["@type"] = "CreateHandle"; to_json_key( - j, - "distinctVariables", - p.distinctVariables, - "MarkDistinctNode", - "List", - "distinctVariables"); + j, "handle", p.handle, "CreateHandle", "OutputTableHandle", "handle"); to_json_key( j, - "hashVariable", - p.hashVariable, - "MarkDistinctNode", - "VariableReferenceExpression", - "hashVariable"); + "schemaTableName", + p.schemaTableName, + "CreateHandle", + "SchemaTableName", + "schemaTableName"); } -void from_json(const json& j, MarkDistinctNode& p) { +void from_json(const json& j, CreateHandle& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "MarkDistinctNode", "PlanNodeId", "id"); - from_json_key( - j, "source", p.source, "MarkDistinctNode", "PlanNode", "source"); from_json_key( - j, - "markerVariable", - p.markerVariable, - "MarkDistinctNode", - "VariableReferenceExpression", - "markerVariable"); - from_json_key( - j, - "distinctVariables", - p.distinctVariables, - "MarkDistinctNode", - "List", - "distinctVariables"); + j, "handle", p.handle, "CreateHandle", "OutputTableHandle", "handle"); from_json_key( j, - "hashVariable", - p.hashVariable, - "MarkDistinctNode", - "VariableReferenceExpression", - "hashVariable"); + "schemaTableName", + p.schemaTableName, + "CreateHandle", + "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 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; +SemiJoinNode::SemiJoinNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.SemiJoinNode"; } -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -void to_json(json& j, const OutputBufferInfo& p) { +void to_json(json& j, const SemiJoinNode& p) { j = json::object(); - to_json_key(j, "type", p.type, "OutputBufferInfo", "String", "type"); - to_json_key(j, "state", p.state, "OutputBufferInfo", "BufferState", "state"); + 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, - "canAddBuffers", - p.canAddBuffers, - "OutputBufferInfo", - "bool", - "canAddBuffers"); + "filteringSource", + p.filteringSource, + "SemiJoinNode", + "PlanNode", + "filteringSource"); to_json_key( j, - "canAddPages", - p.canAddPages, - "OutputBufferInfo", - "bool", - "canAddPages"); + "sourceJoinVariable", + p.sourceJoinVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "sourceJoinVariable"); to_json_key( j, - "totalBufferedBytes", - p.totalBufferedBytes, - "OutputBufferInfo", - "int64_t", - "totalBufferedBytes"); + "filteringSourceJoinVariable", + p.filteringSourceJoinVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "filteringSourceJoinVariable"); to_json_key( j, - "totalBufferedPages", - p.totalBufferedPages, - "OutputBufferInfo", - "int64_t", - "totalBufferedPages"); + "semiJoinOutput", + p.semiJoinOutput, + "SemiJoinNode", + "VariableReferenceExpression", + "semiJoinOutput"); to_json_key( j, - "totalRowsSent", - p.totalRowsSent, - "OutputBufferInfo", - "int64_t", - "totalRowsSent"); + "sourceHashVariable", + p.sourceHashVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "sourceHashVariable"); to_json_key( j, - "totalPagesSent", - p.totalPagesSent, - "OutputBufferInfo", - "int64_t", - "totalPagesSent"); + "filteringSourceHashVariable", + p.filteringSourceHashVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "filteringSourceHashVariable"); to_json_key( j, - "buffers", - p.buffers, - "OutputBufferInfo", - "List", - "buffers"); + "distributionType", + p.distributionType, + "SemiJoinNode", + "DistributionType", + "distributionType"); + to_json_key( + j, + "dynamicFilters", + p.dynamicFilters, + "SemiJoinNode", + "Map", + "dynamicFilters"); } -void from_json(const json& j, OutputBufferInfo& p) { - from_json_key(j, "type", p.type, "OutputBufferInfo", "String", "type"); +void from_json(const json& j, SemiJoinNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "SemiJoinNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "SemiJoinNode", "PlanNode", "source"); from_json_key( - j, "state", p.state, "OutputBufferInfo", "BufferState", "state"); + j, + "filteringSource", + p.filteringSource, + "SemiJoinNode", + "PlanNode", + "filteringSource"); from_json_key( j, - "canAddBuffers", - p.canAddBuffers, - "OutputBufferInfo", - "bool", - "canAddBuffers"); + "sourceJoinVariable", + p.sourceJoinVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "sourceJoinVariable"); from_json_key( j, - "canAddPages", - p.canAddPages, - "OutputBufferInfo", - "bool", - "canAddPages"); + "filteringSourceJoinVariable", + p.filteringSourceJoinVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "filteringSourceJoinVariable"); from_json_key( j, - "totalBufferedBytes", - p.totalBufferedBytes, - "OutputBufferInfo", - "int64_t", - "totalBufferedBytes"); + "semiJoinOutput", + p.semiJoinOutput, + "SemiJoinNode", + "VariableReferenceExpression", + "semiJoinOutput"); from_json_key( j, - "totalBufferedPages", - p.totalBufferedPages, - "OutputBufferInfo", - "int64_t", - "totalBufferedPages"); + "sourceHashVariable", + p.sourceHashVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "sourceHashVariable"); from_json_key( j, - "totalRowsSent", - p.totalRowsSent, - "OutputBufferInfo", - "int64_t", - "totalRowsSent"); + "filteringSourceHashVariable", + p.filteringSourceHashVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "filteringSourceHashVariable"); from_json_key( j, - "totalPagesSent", - p.totalPagesSent, - "OutputBufferInfo", - "int64_t", - "totalPagesSent"); + "distributionType", + p.distributionType, + "SemiJoinNode", + "DistributionType", + "distributionType"); from_json_key( j, - "buffers", - p.buffers, - "OutputBufferInfo", - "List", - "buffers"); + "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 RuntimeUnit_enum_table[] = - { // NOLINT: cert-err58-cpp - {RuntimeUnit::NONE, "NONE"}, - {RuntimeUnit::NANO, "NANO"}, - {RuntimeUnit::BYTE, "BYTE"}}; -void to_json(json& j, const RuntimeUnit& e) { +// 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, "RuntimeUnit must be an enum!"); + std::is_enum::value, + "StageExecutionStrategy 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(StageExecutionStrategy_enum_table), + std::end(StageExecutionStrategy_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(StageExecutionStrategy_enum_table)) ? it - : std::begin(RuntimeUnit_enum_table)) + : std::begin(StageExecutionStrategy_enum_table)) ->second; } -void from_json(const json& j, RuntimeUnit& e) { +void from_json(const json& j, StageExecutionStrategy& e) { static_assert( - std::is_enum::value, "RuntimeUnit must be an enum!"); + std::is_enum::value, + "StageExecutionStrategy 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(StageExecutionStrategy_enum_table), + std::end(StageExecutionStrategy_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(StageExecutionStrategy_enum_table)) ? it - : std::begin(RuntimeUnit_enum_table)) + : std::begin(StageExecutionStrategy_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 StageExecutionDescriptor& 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, + "stageExecutionStrategy", + p.stageExecutionStrategy, + "StageExecutionDescriptor", + "StageExecutionStrategy", + "stageExecutionStrategy"); + to_json_key( + j, + "groupedExecutionScanNodes", + p.groupedExecutionScanNodes, + "StageExecutionDescriptor", + "List", + "groupedExecutionScanNodes"); + to_json_key( + j, + "totalLifespans", + p.totalLifespans, + "StageExecutionDescriptor", + "int", + "totalLifespans"); } -void from_json(const json& j, 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, StageExecutionDescriptor& p) { + from_json_key( + j, + "stageExecutionStrategy", + p.stageExecutionStrategy, + "StageExecutionDescriptor", + "StageExecutionStrategy", + "stageExecutionStrategy"); + from_json_key( + j, + "groupedExecutionScanNodes", + p.groupedExecutionScanNodes, + "StageExecutionDescriptor", + "List", + "groupedExecutionScanNodes"); + from_json_key( + j, + "totalLifespans", + p.totalLifespans, + "StageExecutionDescriptor", + "int", + "totalLifespans"); } } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); - return; - } - - throw TypeError(type + " no abstract type ConnectorMetadataUpdateHandle"); +RemoteTransactionHandle::RemoteTransactionHandle() noexcept { + _type = "$remote"; } -void from_json( - const json& j, - std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " ConnectorMetadataUpdateHandle"); - } - - if (getConnectorKey(type) == "hive") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } +void to_json(json& j, const RemoteTransactionHandle& p) { + j = json::object(); + j["@type"] = "$remote"; + to_json_key( + j, "dummy", p.dummy, "RemoteTransactionHandle", "String", "dummy"); +} - throw TypeError(type + " no abstract type ConnectorMetadataUpdateHandle"); +void from_json(const json& j, RemoteTransactionHandle& p) { + p._type = j["@type"]; + from_json_key( + j, "dummy", p.dummy, "RemoteTransactionHandle", "String", "dummy"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const MetadataUpdates& p) { +void to_json(json& j, const ErrorLocation& p) { j = json::object(); to_json_key( - j, - "connectorId", - p.connectorId, - "MetadataUpdates", - "ConnectorId", - "connectorId"); + j, "lineNumber", p.lineNumber, "ErrorLocation", "int", "lineNumber"); to_json_key( j, - "metadataUpdates", - p.metadataUpdates, - "MetadataUpdates", - "List>", - "metadataUpdates"); + "columnNumber", + p.columnNumber, + "ErrorLocation", + "int", + "columnNumber"); } -void from_json(const json& j, MetadataUpdates& p) { +void from_json(const json& j, ErrorLocation& p) { from_json_key( - j, - "connectorId", - p.connectorId, - "MetadataUpdates", - "ConnectorId", - "connectorId"); + j, "lineNumber", p.lineNumber, "ErrorLocation", "int", "lineNumber"); from_json_key( j, - "metadataUpdates", - p.metadataUpdates, - "MetadataUpdates", - "List>", - "metadataUpdates"); + "columnNumber", + p.columnNumber, + "ErrorLocation", + "int", + "columnNumber"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair ErrorCause_enum_table[] = + { // NOLINT: cert-err58-cpp + {ErrorCause::UNKNOWN, "UNKNOWN"}, + {ErrorCause::LOW_PARTITION_COUNT, "LOW_PARTITION_COUNT"}, + {ErrorCause::EXCEEDS_BROADCAST_MEMORY_LIMIT, + "EXCEEDS_BROADCAST_MEMORY_LIMIT"}}; +void to_json(json& j, const ErrorCause& e) { + static_assert(std::is_enum::value, "ErrorCause must be an enum!"); + const auto* it = std::find_if( + std::begin(ErrorCause_enum_table), + std::end(ErrorCause_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(ErrorCause_enum_table)) + ? it + : std::begin(ErrorCause_enum_table)) + ->second; +} +void from_json(const json& j, ErrorCause& e) { + static_assert(std::is_enum::value, "ErrorCause must be an enum!"); + const auto* it = std::find_if( + std::begin(ErrorCause_enum_table), + std::end(ErrorCause_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(ErrorCause_enum_table)) + ? it + : std::begin(ErrorCause_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair ErrorType_enum_table[] = + { // NOLINT: cert-err58-cpp + {ErrorType::USER_ERROR, "USER_ERROR"}, + {ErrorType::INTERNAL_ERROR, "INTERNAL_ERROR"}, + {ErrorType::INSUFFICIENT_RESOURCES, "INSUFFICIENT_RESOURCES"}, + {ErrorType::EXTERNAL, "EXTERNAL"}}; +void to_json(json& j, const ErrorType& e) { + static_assert(std::is_enum::value, "ErrorType must be an enum!"); + const auto* it = std::find_if( + std::begin(ErrorType_enum_table), + std::end(ErrorType_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(ErrorType_enum_table)) + ? it + : std::begin(ErrorType_enum_table)) + ->second; +} +void from_json(const json& j, ErrorType& e) { + static_assert(std::is_enum::value, "ErrorType must be an enum!"); + const auto* it = std::find_if( + std::begin(ErrorType_enum_table), + std::end(ErrorType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(ErrorType_enum_table)) + ? it + : std::begin(ErrorType_enum_table)) + ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TpchTableLayoutHandle::TpchTableLayoutHandle() noexcept { - _type = "tpch"; -} -void to_json(json& j, const TpchTableLayoutHandle& p) { +void to_json(json& j, const ErrorCode& 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"); + 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, 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"); +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 { -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 ExecutionFailureInfo& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.MergeJoinNode"; - to_json_key(j, "id", p.id, "MergeJoinNode", "PlanNodeId", "id"); - to_json_key(j, "type", p.type, "MergeJoinNode", "JoinNode.Type", "type"); - to_json_key(j, "left", p.left, "MergeJoinNode", "PlanNode", "left"); - to_json_key(j, "right", p.right, "MergeJoinNode", "PlanNode", "right"); + to_json_key(j, "type", p.type, "ExecutionFailureInfo", "String", "type"); + to_json_key( + j, "message", p.message, "ExecutionFailureInfo", "String", "message"); to_json_key( j, - "criteria", - p.criteria, - "MergeJoinNode", - "List", - "criteria"); + "cause", + p.cause, + "ExecutionFailureInfo", + "ExecutionFailureInfo", + "cause"); to_json_key( j, - "outputVariables", - p.outputVariables, - "MergeJoinNode", - "List", - "outputVariables"); + "suppressed", + p.suppressed, + "ExecutionFailureInfo", + "List", + "suppressed"); + to_json_key( + j, "stack", p.stack, "ExecutionFailureInfo", "List", "stack"); to_json_key( j, - "filter", - p.filter, - "MergeJoinNode", - "std::shared_ptr", - "filter"); + "errorLocation", + p.errorLocation, + "ExecutionFailureInfo", + "ErrorLocation", + "errorLocation"); to_json_key( j, - "leftHashVariable", - p.leftHashVariable, - "MergeJoinNode", - "VariableReferenceExpression", - "leftHashVariable"); + "errorCode", + p.errorCode, + "ExecutionFailureInfo", + "ErrorCode", + "errorCode"); to_json_key( j, - "rightHashVariable", - p.rightHashVariable, - "MergeJoinNode", - "VariableReferenceExpression", - "rightHashVariable"); + "remoteHost", + p.remoteHost, + "ExecutionFailureInfo", + "HostAddress", + "remoteHost"); + to_json_key( + j, + "errorCause", + p.errorCause, + "ExecutionFailureInfo", + "ErrorCause", + "errorCause"); } -void from_json(const json& j, MergeJoinNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "MergeJoinNode", "PlanNodeId", "id"); - from_json_key(j, "type", p.type, "MergeJoinNode", "JoinNode.Type", "type"); - from_json_key(j, "left", p.left, "MergeJoinNode", "PlanNode", "left"); - from_json_key(j, "right", p.right, "MergeJoinNode", "PlanNode", "right"); +void from_json(const json& j, 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, - "criteria", - p.criteria, - "MergeJoinNode", - "List", - "criteria"); + "cause", + p.cause, + "ExecutionFailureInfo", + "ExecutionFailureInfo", + "cause"); from_json_key( j, - "outputVariables", - p.outputVariables, - "MergeJoinNode", - "List", - "outputVariables"); + "suppressed", + p.suppressed, + "ExecutionFailureInfo", + "List", + "suppressed"); + from_json_key( + j, "stack", p.stack, "ExecutionFailureInfo", "List", "stack"); from_json_key( j, - "filter", - p.filter, - "MergeJoinNode", - "std::shared_ptr", - "filter"); + "errorLocation", + p.errorLocation, + "ExecutionFailureInfo", + "ErrorLocation", + "errorLocation"); from_json_key( j, - "leftHashVariable", - p.leftHashVariable, - "MergeJoinNode", - "VariableReferenceExpression", - "leftHashVariable"); + "errorCode", + p.errorCode, + "ExecutionFailureInfo", + "ErrorCode", + "errorCode"); + from_json_key( + j, + "remoteHost", + p.remoteHost, + "ExecutionFailureInfo", + "HostAddress", + "remoteHost"); from_json_key( j, - "rightHashVariable", - p.rightHashVariable, - "MergeJoinNode", - "VariableReferenceExpression", - "rightHashVariable"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -TpchSplit::TpchSplit() noexcept { - _type = "tpch"; -} - -void to_json(json& j, const TpchSplit& p) { - j = json::object(); - j["@type"] = "tpch"; + "errorCause", + p.errorCause, + "ExecutionFailureInfo", + "ErrorCause", + "errorCause"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair TaskState_enum_table[] = + { // NOLINT: cert-err58-cpp + {TaskState::PLANNED, "PLANNED"}, + {TaskState::RUNNING, "RUNNING"}, + {TaskState::FINISHED, "FINISHED"}, + {TaskState::CANCELED, "CANCELED"}, + {TaskState::ABORTED, "ABORTED"}, + {TaskState::FAILED, "FAILED"}}; +void to_json(json& j, const TaskState& e) { + static_assert(std::is_enum::value, "TaskState must be an enum!"); + const auto* it = std::find_if( + std::begin(TaskState_enum_table), + std::end(TaskState_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(TaskState_enum_table)) + ? it + : std::begin(TaskState_enum_table)) + ->second; +} +void from_json(const json& j, TaskState& e) { + static_assert(std::is_enum::value, "TaskState must be an enum!"); + const auto* it = std::find_if( + std::begin(TaskState_enum_table), + std::end(TaskState_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(TaskState_enum_table)) + ? it + : std::begin(TaskState_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const TaskStatus& p) { + j = json::object(); + to_json_key( + j, + "taskInstanceIdLeastSignificantBits", + p.taskInstanceIdLeastSignificantBits, + "TaskStatus", + "int64_t", + "taskInstanceIdLeastSignificantBits"); + to_json_key( + j, + "taskInstanceIdMostSignificantBits", + p.taskInstanceIdMostSignificantBits, + "TaskStatus", + "int64_t", + "taskInstanceIdMostSignificantBits"); + to_json_key(j, "version", p.version, "TaskStatus", "int64_t", "version"); + to_json_key(j, "state", p.state, "TaskStatus", "TaskState", "state"); + to_json_key(j, "self", p.self, "TaskStatus", "URI", "self"); + to_json_key( + j, + "completedDriverGroups", + p.completedDriverGroups, + "TaskStatus", + "List", + "completedDriverGroups"); + to_json_key( + j, + "failures", + p.failures, + "TaskStatus", + "List", + "failures"); + to_json_key( + j, + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "TaskStatus", + "int", + "queuedPartitionedDrivers"); 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"); + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "TaskStatus", + "int", + "runningPartitionedDrivers"); to_json_key( j, - "addresses", - p.addresses, - "TpchSplit", - "List", - "addresses"); + "outputBufferUtilization", + p.outputBufferUtilization, + "TaskStatus", + "double", + "outputBufferUtilization"); to_json_key( j, - "predicate", - p.predicate, - "TpchSplit", - "TupleDomain>", - "predicate"); -} - -void from_json(const json& j, TpchSplit& p) { - p._type = j["@type"]; - from_json_key( + "outputBufferOverutilized", + p.outputBufferOverutilized, + "TaskStatus", + "bool", + "outputBufferOverutilized"); + to_json_key( j, - "tableHandle", - p.tableHandle, - "TpchSplit", - "TpchTableHandle", - "tableHandle"); - from_json_key( - j, "partNumber", p.partNumber, "TpchSplit", "int", "partNumber"); - from_json_key( - j, "totalParts", p.totalParts, "TpchSplit", "int", "totalParts"); - from_json_key( + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "TaskStatus", + "int64_t", + "physicalWrittenDataSizeInBytes"); + to_json_key( j, - "addresses", - p.addresses, - "TpchSplit", - "List", - "addresses"); - from_json_key( + "memoryReservationInBytes", + p.memoryReservationInBytes, + "TaskStatus", + "int64_t", + "memoryReservationInBytes"); + to_json_key( j, - "predicate", - p.predicate, - "TpchSplit", - "TupleDomain>", - "predicate"); -} -} // 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"; + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "TaskStatus", + "int64_t", + "systemMemoryReservationInBytes"); 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( + "peakNodeTotalMemoryReservationInBytes", + p.peakNodeTotalMemoryReservationInBytes, + "TaskStatus", + "int64_t", + "peakNodeTotalMemoryReservationInBytes"); + to_json_key( + j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); + to_json_key( j, - "columnName", - p.columnName, - "TpchColumnHandle", - "String", - "columnName"); - from_json_key(j, "type", p.type, "TpchColumnHandle", "Type", "type"); -} -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { -TableWriterNode::TableWriterNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.TableWriterNode"; -} - -void to_json(json& j, const StatisticAggregations& p) { - j = json::object(); + "fullGcTimeInMillis", + p.fullGcTimeInMillis, + "TaskStatus", + "int64_t", + "fullGcTimeInMillis"); to_json_key( j, - "outputVariables", - p.outputVariables, - "StatisticAggregations", - "std::vector", - "outputVariables"); + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "TaskStatus", + "int64_t", + "totalCpuTimeInNanos"); to_json_key( j, - "aggregations", - p.aggregations, - "StatisticAggregations", - "Map", - "aggregations"); + "taskAgeInMillis", + p.taskAgeInMillis, + "TaskStatus", + "int64_t", + "taskAgeInMillis"); to_json_key( j, - "groupingVariables", - p.groupingVariables, - "StatisticAggregations", - "std::vector", - "groupingVariables"); + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "TaskStatus", + "int64_t", + "queuedPartitionedSplitsWeight"); + to_json_key( + j, + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "TaskStatus", + "int64_t", + "runningPartitionedSplitsWeight"); } -void from_json(const json& j, StatisticAggregations& p) { +void from_json(const json& j, TaskStatus& p) { from_json_key( j, - "outputVariables", - p.outputVariables, - "StatisticAggregations", - "std::vector", - "outputVariables"); + "taskInstanceIdLeastSignificantBits", + p.taskInstanceIdLeastSignificantBits, + "TaskStatus", + "int64_t", + "taskInstanceIdLeastSignificantBits"); from_json_key( j, - "aggregations", - p.aggregations, - "StatisticAggregations", - "Map", - "aggregations"); + "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, - "groupingVariables", - p.groupingVariables, - "StatisticAggregations", - "std::vector", - "groupingVariables"); -} - -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( + "completedDriverGroups", + p.completedDriverGroups, + "TaskStatus", + "List", + "completedDriverGroups"); + from_json_key( j, - "columns", - p.columns, - "TableWriterNode", - "List", - "columns"); - to_json_key( + "failures", + p.failures, + "TaskStatus", + "List", + "failures"); + from_json_key( j, - "columnNames", - p.columnNames, - "TableWriterNode", - "List", - "columnNames"); - to_json_key( + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "TaskStatus", + "int", + "queuedPartitionedDrivers"); + from_json_key( j, - "notNullColumnVariables", - p.notNullColumnVariables, - "TableWriterNode", - "List", - "notNullColumnVariables"); - to_json_key( + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "TaskStatus", + "int", + "runningPartitionedDrivers"); + from_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "partitioningScheme"); - to_json_key( + "outputBufferUtilization", + p.outputBufferUtilization, + "TaskStatus", + "double", + "outputBufferUtilization"); + from_json_key( j, - "preferredShufflePartitioningScheme", - p.preferredShufflePartitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "preferredShufflePartitioningScheme"); - to_json_key( + "outputBufferOverutilized", + p.outputBufferOverutilized, + "TaskStatus", + "bool", + "outputBufferOverutilized"); + 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"); + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "TaskStatus", + "int64_t", + "physicalWrittenDataSizeInBytes"); from_json_key( j, - "rowCountVariable", - p.rowCountVariable, - "TableWriterNode", - "VariableReferenceExpression", - "rowCountVariable"); + "memoryReservationInBytes", + p.memoryReservationInBytes, + "TaskStatus", + "int64_t", + "memoryReservationInBytes"); from_json_key( j, - "fragmentVariable", - p.fragmentVariable, - "TableWriterNode", - "VariableReferenceExpression", - "fragmentVariable"); + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "TaskStatus", + "int64_t", + "systemMemoryReservationInBytes"); from_json_key( j, - "tableCommitContextVariable", - p.tableCommitContextVariable, - "TableWriterNode", - "VariableReferenceExpression", - "tableCommitContextVariable"); + "peakNodeTotalMemoryReservationInBytes", + p.peakNodeTotalMemoryReservationInBytes, + "TaskStatus", + "int64_t", + "peakNodeTotalMemoryReservationInBytes"); + from_json_key( + j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); from_json_key( j, - "columns", - p.columns, - "TableWriterNode", - "List", - "columns"); + "fullGcTimeInMillis", + p.fullGcTimeInMillis, + "TaskStatus", + "int64_t", + "fullGcTimeInMillis"); from_json_key( j, - "columnNames", - p.columnNames, - "TableWriterNode", - "List", - "columnNames"); + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "TaskStatus", + "int64_t", + "totalCpuTimeInNanos"); from_json_key( j, - "notNullColumnVariables", - p.notNullColumnVariables, - "TableWriterNode", - "List", - "notNullColumnVariables"); + "taskAgeInMillis", + p.taskAgeInMillis, + "TaskStatus", + "int64_t", + "taskAgeInMillis"); from_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "partitioningScheme"); + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "TaskStatus", + "int64_t", + "queuedPartitionedSplitsWeight"); from_json_key( j, - "preferredShufflePartitioningScheme", - p.preferredShufflePartitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "preferredShufflePartitioningScheme"); + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "TaskStatus", + "int64_t", + "runningPartitionedSplitsWeight"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +TpchColumnHandle::TpchColumnHandle() noexcept { + _type = "tpch"; +} + +void to_json(json& j, const TpchColumnHandle& p) { + j = json::object(); + j["@type"] = "tpch"; + to_json_key( + j, + "columnName", + p.columnName, + "TpchColumnHandle", + "String", + "columnName"); + to_json_key(j, "type", p.type, "TpchColumnHandle", "Type", "type"); +} + +void from_json(const json& j, TpchColumnHandle& p) { + p._type = j["@type"]; from_json_key( j, - "statisticsAggregation", - p.statisticsAggregation, - "TableWriterNode", - "StatisticAggregations", - "statisticsAggregation"); + "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() // 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 + 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(WindowType_enum_table), - std::end(WindowType_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(WindowType_enum_table)) + j = ((it != std::end(SystemPartitioning_enum_table)) ? it - : std::begin(WindowType_enum_table)) + : std::begin(SystemPartitioning_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, SystemPartitioning& e) { + static_assert( + std::is_enum::value, + "SystemPartitioning 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(SystemPartitioning_enum_table), + std::end(SystemPartitioning_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(WindowType_enum_table)) + e = ((it != std::end(SystemPartitioning_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() - -// 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!"); + : std::begin(SystemPartitioning_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair + SystemPartitionFunction_enum_table[] = { // NOLINT: cert-err58-cpp + {SystemPartitionFunction::SINGLE, "SINGLE"}, + {SystemPartitionFunction::HASH, "HASH"}, + {SystemPartitionFunction::ROUND_ROBIN, "ROUND_ROBIN"}, + {SystemPartitionFunction::BROADCAST, "BROADCAST"}, + {SystemPartitionFunction::UNKNOWN, "UNKNOWN"}}; +void to_json(json& j, const SystemPartitionFunction& e) { + static_assert( + std::is_enum::value, + "SystemPartitionFunction must be an enum!"); const auto* it = std::find_if( - std::begin(BoundType_enum_table), - std::end(BoundType_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(BoundType_enum_table)) + j = ((it != std::end(SystemPartitionFunction_enum_table)) ? it - : std::begin(BoundType_enum_table)) + : std::begin(SystemPartitionFunction_enum_table)) ->second; } -void from_json(const json& j, BoundType& e) { - static_assert(std::is_enum::value, "BoundType 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(BoundType_enum_table), - std::end(BoundType_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(BoundType_enum_table)) + e = ((it != std::end(SystemPartitionFunction_enum_table)) ? it - : std::begin(BoundType_enum_table)) + : 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 Frame& p) { +void to_json(json& j, const SystemPartitioningHandle& 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"); + j["@type"] = "$remote"; to_json_key( j, - "originalStartValue", - p.originalStartValue, - "Frame", - "String", - "originalStartValue"); + "partitioning", + p.partitioning, + "SystemPartitioningHandle", + "SystemPartitioning", + "partitioning"); to_json_key( j, - "originalEndValue", - p.originalEndValue, - "Frame", - "String", - "originalEndValue"); + "function", + p.function, + "SystemPartitioningHandle", + "SystemPartitionFunction", + "function"); } -void from_json(const json& j, Frame& p) { - from_json_key(j, "type", p.type, "Frame", "WindowType", "type"); - from_json_key(j, "startType", p.startType, "Frame", "BoundType", "startType"); +void from_json(const json& j, SystemPartitioningHandle& p) { + p._type = j["@type"]; from_json_key( j, - "startValue", - p.startValue, - "Frame", - "VariableReferenceExpression", - "startValue"); + "partitioning", + p.partitioning, + "SystemPartitioningHandle", + "SystemPartitioning", + "partitioning"); from_json_key( j, - "sortKeyCoercedForFrameStartComparison", - p.sortKeyCoercedForFrameStartComparison, - "Frame", - "VariableReferenceExpression", - "sortKeyCoercedForFrameStartComparison"); - from_json_key(j, "endType", p.endType, "Frame", "BoundType", "endType"); - from_json_key( + "function", + p.function, + "SystemPartitioningHandle", + "SystemPartitionFunction", + "function"); +} +} // 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, - "endValue", - p.endValue, - "Frame", - "VariableReferenceExpression", - "endValue"); + "analyzePartitionValues", + p.analyzePartitionValues, + "HiveTableHandle", + "List>", + "analyzePartitionValues"); +} + +void from_json(const json& j, HiveTableHandle& p) { + p._type = j["@type"]; from_json_key( - j, - "sortKeyCoercedForFrameEndComparison", - p.sortKeyCoercedForFrameEndComparison, - "Frame", - "VariableReferenceExpression", - "sortKeyCoercedForFrameEndComparison"); + j, "schemaName", p.schemaName, "HiveTableHandle", "String", "schemaName"); from_json_key( - j, - "originalStartValue", - p.originalStartValue, - "Frame", - "String", - "originalStartValue"); + j, "tableName", p.tableName, "HiveTableHandle", "String", "tableName"); from_json_key( j, - "originalEndValue", - p.originalEndValue, - "Frame", - "String", - "originalEndValue"); + "analyzePartitionValues", + p.analyzePartitionValues, + "HiveTableHandle", + "List>", + "analyzePartitionValues"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Function& p) { +void to_json(json& j, const Language& p) { j = json::object(); - to_json_key( - j, - "functionCall", - p.functionCall, - "Function", - "CallExpression", - "functionCall"); - to_json_key(j, "frame", p.frame, "Function", "Frame", "frame"); - to_json_key( - j, "ignoreNulls", p.ignoreNulls, "Function", "bool", "ignoreNulls"); + to_json_key(j, "language", p.language, "Language", "String", "language"); } -void from_json(const json& j, Function& p) { - from_json_key( - j, - "functionCall", - p.functionCall, - "Function", - "CallExpression", - "functionCall"); - from_json_key(j, "frame", p.frame, "Function", "Frame", "frame"); - from_json_key( - j, "ignoreNulls", p.ignoreNulls, "Function", "bool", "ignoreNulls"); +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 { -WindowNode::WindowNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.WindowNode"; +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair 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 WindowNode& p) { +void to_json(json& j, const SpecialFormExpression& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.WindowNode"; + j["@type"] = "special"; to_json_key( j, "sourceLocation", p.sourceLocation, - "WindowNode", + "SpecialFormExpression", "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"); + to_json_key(j, "form", p.form, "SpecialFormExpression", "Form", "form"); to_json_key( j, - "prePartitionedInputs", - p.prePartitionedInputs, - "WindowNode", - "List", - "prePartitionedInputs"); + "returnType", + p.returnType, + "SpecialFormExpression", + "Type", + "returnType"); to_json_key( j, - "preSortedOrderPrefix", - p.preSortedOrderPrefix, - "WindowNode", - "int", - "preSortedOrderPrefix"); + "arguments", + p.arguments, + "SpecialFormExpression", + "List>", + "arguments"); } -void from_json(const json& j, WindowNode& p) { +void from_json(const json& j, SpecialFormExpression& p) { p._type = j["@type"]; from_json_key( j, "sourceLocation", p.sourceLocation, - "WindowNode", + "SpecialFormExpression", "SourceLocation", "sourceLocation"); - from_json_key(j, "id", p.id, "WindowNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "WindowNode", "PlanNode", "source"); - from_json_key( - j, - "specification", - p.specification, - "WindowNode", - "Specification", - "specification"); - from_json_key( - j, - "windowFunctions", - p.windowFunctions, - "WindowNode", - "Map", - "windowFunctions"); - from_json_key( - j, - "hashVariable", - p.hashVariable, - "WindowNode", - "VariableReferenceExpression", - "hashVariable"); + from_json_key(j, "form", p.form, "SpecialFormExpression", "Form", "form"); from_json_key( j, - "prePartitionedInputs", - p.prePartitionedInputs, - "WindowNode", - "List", - "prePartitionedInputs"); + "returnType", + p.returnType, + "SpecialFormExpression", + "Type", + "returnType"); from_json_key( j, - "preSortedOrderPrefix", - p.preSortedOrderPrefix, - "WindowNode", - "int", - "preSortedOrderPrefix"); + "arguments", + p.arguments, + "SpecialFormExpression", + "List>", + "arguments"); } } // 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 { -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 to_json(json& j, const Block& p) { + j = p.data; } -void from_json(const json& j, HiveTransactionHandle& p) { - p._type = j["@type"]; - from_json_key(j, "uuid", p.uuid, "HiveTransactionHandle", "UUID", "uuid"); +void from_json(const json& j, Block& p) { + p.data = std::string(j); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const MemoryInfo& p) { +void to_json(json& j, const ValueEntry& p) { j = json::object(); - to_json_key( - j, - "totalNodeMemory", - p.totalNodeMemory, - "MemoryInfo", - "DataSize", - "totalNodeMemory"); - to_json_key( - j, - "pools", - p.pools, - "MemoryInfo", - "Map", - "pools"); + 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, MemoryInfo& p) { - from_json_key( - j, - "totalNodeMemory", - p.totalNodeMemory, - "MemoryInfo", - "DataSize", - "totalNodeMemory"); - from_json_key( - j, - "pools", - p.pools, - "MemoryInfo", - "Map", - "pools"); +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 DeleteScanInfo& p) { +void to_json(json& j, const EquatableValueSet& p) { j = json::object(); - to_json_key(j, "id", p.id, "DeleteScanInfo", "PlanNodeId", "id"); + j["@type"] = "equatable"; + to_json_key(j, "type", p.type, "EquatableValueSet", "Type", "type"); + to_json_key( + j, "whiteList", p.whiteList, "EquatableValueSet", "bool", "whiteList"); to_json_key( j, - "tableHandle", - p.tableHandle, - "DeleteScanInfo", - "TableHandle", - "tableHandle"); + "entries", + p.entries, + "EquatableValueSet", + "List", + "entries"); } -void from_json(const json& j, DeleteScanInfo& p) { - from_json_key(j, "id", p.id, "DeleteScanInfo", "PlanNodeId", "id"); +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, - "tableHandle", - p.tableHandle, - "DeleteScanInfo", - "TableHandle", - "tableHandle"); + "entries", + p.entries, + "EquatableValueSet", + "List", + "entries"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (type == "CreateHandle") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "InsertHandle") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "DeleteHandle") { - j = *std::static_pointer_cast(p); - return; - } +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - throw TypeError(type + " no abstract type ExecutionWriterTarget "); +// 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 from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError( - std::string(e.what()) + - " ExecutionWriterTarget ExecutionWriterTarget"); - } - - if (type == "CreateHandle") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "InsertHandle") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "DeleteHandle") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } +void to_json(json& j, const 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"); +} - throw TypeError(type + " no abstract type ExecutionWriterTarget "); +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 TableWriteInfo& p) { +void to_json(json& j, const Parameter& p) { j = json::object(); - to_json_key( - j, - "writerTarget", - p.writerTarget, - "TableWriteInfo", - "ExecutionWriterTarget", - "writerTarget"); - to_json_key( - j, - "analyzeTableHandle", - p.analyzeTableHandle, - "TableWriteInfo", - "AnalyzeTableHandle", - "analyzeTableHandle"); - to_json_key( - j, - "deleteScanInfo", - p.deleteScanInfo, - "TableWriteInfo", - "DeleteScanInfo", - "deleteScanInfo"); + 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, TableWriteInfo& p) { - from_json_key( - j, - "writerTarget", - p.writerTarget, - "TableWriteInfo", - "ExecutionWriterTarget", - "writerTarget"); - from_json_key( - j, - "analyzeTableHandle", - p.analyzeTableHandle, - "TableWriteInfo", - "AnalyzeTableHandle", - "analyzeTableHandle"); - from_json_key( - j, - "deleteScanInfo", - p.deleteScanInfo, - "TableWriteInfo", - "DeleteScanInfo", - "deleteScanInfo"); +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 BufferType_enum_table[] = +static const std::pair NullCallClause_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!"); + {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(BufferType_enum_table), - std::end(BufferType_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(NullCallClause_enum_table), + std::end(NullCallClause_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(BufferType_enum_table)) + j = ((it != std::end(NullCallClause_enum_table)) ? it - : std::begin(BufferType_enum_table)) + : std::begin(NullCallClause_enum_table)) ->second; } -void from_json(const json& j, BufferType& e) { - static_assert(std::is_enum::value, "BufferType must be an enum!"); +void from_json(const json& j, NullCallClause& e) { + static_assert( + std::is_enum::value, "NullCallClause must be an enum!"); const auto* it = std::find_if( - std::begin(BufferType_enum_table), - std::end(BufferType_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(NullCallClause_enum_table), + std::end(NullCallClause_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(BufferType_enum_table)) + e = ((it != std::end(NullCallClause_enum_table)) ? it - : std::begin(BufferType_enum_table)) + : std::begin(NullCallClause_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 OutputBuffers& p) { +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair Determinism_enum_table[] = { + // NOLINT: cert-err58-cpp + {Determinism::DETERMINISTIC, "DETERMINISTIC"}, + {Determinism::NOT_DETERMINISTIC, "NOT_DETERMINISTIC"}, +}; +void to_json(json& j, const Determinism& e) { + static_assert( + std::is_enum::value, "Determinism must be an enum!"); + const auto* it = std::find_if( + std::begin(Determinism_enum_table), + std::end(Determinism_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(Determinism_enum_table)) + ? it + : std::begin(Determinism_enum_table)) + ->second; +} +void from_json(const json& j, Determinism& e) { + static_assert( + std::is_enum::value, "Determinism must be an enum!"); + const auto* it = std::find_if( + std::begin(Determinism_enum_table), + std::end(Determinism_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(Determinism_enum_table)) + ? it + : std::begin(Determinism_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const RoutineCharacteristics& p) { j = json::object(); - to_json_key(j, "type", p.type, "OutputBuffers", "BufferType", "type"); - to_json_key(j, "version", p.version, "OutputBuffers", "int64_t", "version"); to_json_key( j, - "noMoreBufferIds", - p.noMoreBufferIds, - "OutputBuffers", - "bool", - "noMoreBufferIds"); + "language", + p.language, + "RoutineCharacteristics", + "Language", + "language"); to_json_key( j, - "buffers", - p.buffers, - "OutputBuffers", - "Map", - "buffers"); + "determinism", + p.determinism, + "RoutineCharacteristics", + "Determinism", + "determinism"); + to_json_key( + j, + "nullCallClause", + p.nullCallClause, + "RoutineCharacteristics", + "NullCallClause", + "nullCallClause"); } -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, RoutineCharacteristics& p) { from_json_key( j, - "noMoreBufferIds", - p.noMoreBufferIds, - "OutputBuffers", - "bool", - "noMoreBufferIds"); + "language", + p.language, + "RoutineCharacteristics", + "Language", + "language"); from_json_key( j, - "buffers", - p.buffers, - "OutputBuffers", - "Map", - "buffers"); + "determinism", + p.determinism, + "RoutineCharacteristics", + "Determinism", + "determinism"); + from_json_key( + j, + "nullCallClause", + p.nullCallClause, + "RoutineCharacteristics", + "NullCallClause", + "nullCallClause"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair FunctionKind_enum_table[] = + { // NOLINT: cert-err58-cpp + {FunctionKind::SCALAR, "SCALAR"}, + {FunctionKind::AGGREGATE, "AGGREGATE"}, + {FunctionKind::WINDOW, "WINDOW"}}; +void to_json(json& j, const FunctionKind& e) { + static_assert( + std::is_enum::value, "FunctionKind must be an enum!"); + const auto* it = std::find_if( + std::begin(FunctionKind_enum_table), + std::end(FunctionKind_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(FunctionKind_enum_table)) + ? it + : std::begin(FunctionKind_enum_table)) + ->second; +} +void from_json(const json& j, FunctionKind& e) { + static_assert( + std::is_enum::value, "FunctionKind must be an enum!"); + const auto* it = std::find_if( + std::begin(FunctionKind_enum_table), + std::end(FunctionKind_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(FunctionKind_enum_table)) + ? it + : std::begin(FunctionKind_enum_table)) + ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const TaskUpdateRequest& p) { +void to_json(json& j, const TypeVariableConstraint& p) { j = json::object(); + to_json_key(j, "name", p.name, "TypeVariableConstraint", "String", "name"); to_json_key( j, - "session", - p.session, - "TaskUpdateRequest", - "SessionRepresentation", - "session"); - to_json_key( - j, - "extraCredentials", - p.extraCredentials, - "TaskUpdateRequest", - "Map", - "extraCredentials"); - to_json_key( - j, "fragment", p.fragment, "TaskUpdateRequest", "String", "fragment"); + "comparableRequired", + p.comparableRequired, + "TypeVariableConstraint", + "bool", + "comparableRequired"); to_json_key( j, - "sources", - p.sources, - "TaskUpdateRequest", - "List", - "sources"); + "orderableRequired", + p.orderableRequired, + "TypeVariableConstraint", + "bool", + "orderableRequired"); to_json_key( j, - "outputIds", - p.outputIds, - "TaskUpdateRequest", - "OutputBuffers", - "outputIds"); + "variadicBound", + p.variadicBound, + "TypeVariableConstraint", + "String", + "variadicBound"); to_json_key( j, - "tableWriteInfo", - p.tableWriteInfo, - "TaskUpdateRequest", - "TableWriteInfo", - "tableWriteInfo"); + "nonDecimalNumericRequired", + p.nonDecimalNumericRequired, + "TypeVariableConstraint", + "bool", + "nonDecimalNumericRequired"); } -void from_json(const json& j, TaskUpdateRequest& p) { - from_json_key( - j, - "session", - p.session, - "TaskUpdateRequest", - "SessionRepresentation", - "session"); +void from_json(const json& j, TypeVariableConstraint& p) { + from_json_key(j, "name", p.name, "TypeVariableConstraint", "String", "name"); from_json_key( j, - "extraCredentials", - p.extraCredentials, - "TaskUpdateRequest", - "Map", - "extraCredentials"); - from_json_key( - j, "fragment", p.fragment, "TaskUpdateRequest", "String", "fragment"); + "comparableRequired", + p.comparableRequired, + "TypeVariableConstraint", + "bool", + "comparableRequired"); from_json_key( j, - "sources", - p.sources, - "TaskUpdateRequest", - "List", - "sources"); + "orderableRequired", + p.orderableRequired, + "TypeVariableConstraint", + "bool", + "orderableRequired"); from_json_key( j, - "outputIds", - p.outputIds, - "TaskUpdateRequest", - "OutputBuffers", - "outputIds"); + "variadicBound", + p.variadicBound, + "TypeVariableConstraint", + "String", + "variadicBound"); from_json_key( j, - "tableWriteInfo", - p.tableWriteInfo, - "TaskUpdateRequest", - "TableWriteInfo", - "tableWriteInfo"); + "nonDecimalNumericRequired", + p.nonDecimalNumericRequired, + "TypeVariableConstraint", + "bool", + "nonDecimalNumericRequired"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const BatchTaskUpdateRequest& p) { +void to_json(json& j, const LongVariableConstraint& p) { j = json::object(); + to_json_key(j, "name", p.name, "LongVariableConstraint", "String", "name"); to_json_key( j, - "taskUpdateRequest", - p.taskUpdateRequest, - "BatchTaskUpdateRequest", - "TaskUpdateRequest", - "taskUpdateRequest"); - to_json_key( - j, - "shuffleWriteInfo", - p.shuffleWriteInfo, - "BatchTaskUpdateRequest", + "expression", + p.expression, + "LongVariableConstraint", "String", - "shuffleWriteInfo"); + "expression"); } -void from_json(const json& j, BatchTaskUpdateRequest& p) { - from_json_key( - j, - "taskUpdateRequest", - p.taskUpdateRequest, - "BatchTaskUpdateRequest", - "TaskUpdateRequest", - "taskUpdateRequest"); +void from_json(const json& j, LongVariableConstraint& p) { + from_json_key(j, "name", p.name, "LongVariableConstraint", "String", "name"); from_json_key( j, - "shuffleWriteInfo", - p.shuffleWriteInfo, - "BatchTaskUpdateRequest", + "expression", + p.expression, + "LongVariableConstraint", "String", - "shuffleWriteInfo"); + "expression"); } } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); - return; - } - throw TypeError(type + " no abstract type ConnectorInsertTableHandle "); +void to_json(json& j, const Signature& p) { + j = json::object(); + to_json_key(j, "name", p.name, "Signature", "QualifiedObjectName", "name"); + to_json_key(j, "kind", p.kind, "Signature", "FunctionKind", "kind"); + to_json_key( + 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"); + to_json_key( + j, + "variableArity", + p.variableArity, + "Signature", + "bool", + "variableArity"); } -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError( - std::string(e.what()) + - " ConnectorInsertTableHandle ConnectorInsertTableHandle"); - } - - if (getConnectorKey(type) == "hive") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - - throw TypeError(type + " no abstract type ConnectorInsertTableHandle "); +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"); + from_json_key( + j, + "argumentTypes", + p.argumentTypes, + "Signature", + "List", + "argumentTypes"); + from_json_key( + j, + "variableArity", + p.variableArity, + "Signature", + "bool", + "variableArity"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const InsertTableHandle& p) { +void to_json(json& j, const SqlInvokedFunction& p) { j = json::object(); to_json_key( j, - "connectorId", - p.connectorId, - "InsertTableHandle", - "ConnectorId", - "connectorId"); + "parameters", + p.parameters, + "SqlInvokedFunction", + "List", + "parameters"); to_json_key( j, - "transactionHandle", - p.transactionHandle, - "InsertTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "description", + p.description, + "SqlInvokedFunction", + "String", + "description"); to_json_key( j, - "connectorHandle", - p.connectorHandle, - "InsertTableHandle", - "ConnectorInsertTableHandle", - "connectorHandle"); + "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, InsertTableHandle& p) { +void from_json(const json& j, SqlInvokedFunction& p) { from_json_key( j, - "connectorId", - p.connectorId, - "InsertTableHandle", - "ConnectorId", - "connectorId"); + "parameters", + p.parameters, + "SqlInvokedFunction", + "List", + "parameters"); from_json_key( j, - "transactionHandle", - p.transactionHandle, - "InsertTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "description", + p.description, + "SqlInvokedFunction", + "String", + "description"); from_json_key( j, - "connectorHandle", - p.connectorHandle, - "InsertTableHandle", - "ConnectorInsertTableHandle", - "connectorHandle"); + "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"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -OutputNode::OutputNode() noexcept { - _type = ".OutputNode"; -} -void to_json(json& j, const OutputNode& p) { +void to_json(json& j, const SessionRepresentation& 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, "queryId", p.queryId, "SessionRepresentation", "String", "queryId"); to_json_key( j, - "columnNames", - p.columnNames, - "OutputNode", - "List", - "columnNames"); + "transactionId", + p.transactionId, + "SessionRepresentation", + "TransactionId", + "transactionId"); to_json_key( j, - "outputVariables", - p.outputVariables, - "OutputNode", - "List", - "outputVariables"); -} - -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( + "clientTransactionSupport", + p.clientTransactionSupport, + "SessionRepresentation", + "bool", + "clientTransactionSupport"); + to_json_key(j, "user", p.user, "SessionRepresentation", "String", "user"); + to_json_key( j, - "columnNames", - p.columnNames, - "OutputNode", - "List", - "columnNames"); - 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, - "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 - 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(); + "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"); to_json_key( j, - "stageExecutionStrategy", - p.stageExecutionStrategy, - "StageExecutionDescriptor", - "StageExecutionStrategy", - "stageExecutionStrategy"); + "resourceEstimates", + p.resourceEstimates, + "SessionRepresentation", + "ResourceEstimates", + "resourceEstimates"); to_json_key( j, - "groupedExecutionScanNodes", - p.groupedExecutionScanNodes, - "StageExecutionDescriptor", - "List", - "groupedExecutionScanNodes"); + "startTime", + p.startTime, + "SessionRepresentation", + "int64_t", + "startTime"); to_json_key( j, - "totalLifespans", - p.totalLifespans, - "StageExecutionDescriptor", - "int", - "totalLifespans"); -} - -void from_json(const json& j, StageExecutionDescriptor& p) { - from_json_key( + "systemProperties", + p.systemProperties, + "SessionRepresentation", + "Map", + "systemProperties"); + to_json_key( j, - "stageExecutionStrategy", - p.stageExecutionStrategy, - "StageExecutionDescriptor", - "StageExecutionStrategy", - "stageExecutionStrategy"); - from_json_key( + "catalogProperties", + p.catalogProperties, + "SessionRepresentation", + "Map>", + "catalogProperties"); + to_json_key( j, - "groupedExecutionScanNodes", - p.groupedExecutionScanNodes, - "StageExecutionDescriptor", - "List", - "groupedExecutionScanNodes"); - from_json_key( + "unprocessedCatalogProperties", + p.unprocessedCatalogProperties, + "SessionRepresentation", + "Map>", + "unprocessedCatalogProperties"); + to_json_key( j, - "totalLifespans", - p.totalLifespans, - "StageExecutionDescriptor", - "int", - "totalLifespans"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const StatsAndCosts& p) { - j = json::object(); + "roles", + p.roles, + "SessionRepresentation", + "Map", + "roles"); to_json_key( j, - "stats", - p.stats, - "StatsAndCosts", - "Map", - "stats"); + "preparedStatements", + p.preparedStatements, + "SessionRepresentation", + "Map", + "preparedStatements"); to_json_key( j, - "costs", - p.costs, - "StatsAndCosts", - "Map", - "costs"); + "sessionFunctions", + p.sessionFunctions, + "SessionRepresentation", + "Map", + "sessionFunctions"); } -void from_json(const json& j, StatsAndCosts& p) { +void from_json(const json& j, SessionRepresentation& p) { from_json_key( - j, - "stats", - p.stats, - "StatsAndCosts", - "Map", - "stats"); + j, "queryId", p.queryId, "SessionRepresentation", "String", "queryId"); from_json_key( j, - "costs", - p.costs, - "StatsAndCosts", - "Map", - "costs"); -} -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -namespace facebook::presto::protocol { - -void to_json(json& j, const PlanFragment& p) { - j = json::object(); - to_json_key(j, "id", p.id, "PlanFragment", "PlanFragmentId", "id"); - to_json_key(j, "root", p.root, "PlanFragment", "PlanNode", "root"); - to_json_key( - j, - "variables", - p.variables, - "PlanFragment", - "List", - "variables"); - to_json_key( + "transactionId", + p.transactionId, + "SessionRepresentation", + "TransactionId", + "transactionId"); + from_json_key( j, - "partitioning", - p.partitioning, - "PlanFragment", - "PartitioningHandle", - "partitioning"); - to_json_key( + "clientTransactionSupport", + p.clientTransactionSupport, + "SessionRepresentation", + "bool", + "clientTransactionSupport"); + from_json_key(j, "user", p.user, "SessionRepresentation", "String", "user"); + from_json_key( j, - "tableScanSchedulingOrder", - p.tableScanSchedulingOrder, - "PlanFragment", - "List", - "tableScanSchedulingOrder"); - 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, - "partitioningScheme", - p.partitioningScheme, - "PlanFragment", - "PartitioningScheme", - "partitioningScheme"); - to_json_key( + "traceToken", + p.traceToken, + "SessionRepresentation", + "String", + "traceToken"); + from_json_key( j, - "stageExecutionDescriptor", - p.stageExecutionDescriptor, - "PlanFragment", - "StageExecutionDescriptor", - "stageExecutionDescriptor"); - to_json_key( + "timeZoneKey", + p.timeZoneKey, + "SessionRepresentation", + "TimeZoneKey", + "timeZoneKey"); + from_json_key( + j, "locale", p.locale, "SessionRepresentation", "Locale", "locale"); + from_json_key( j, - "outputTableWriterFragment", - p.outputTableWriterFragment, - "PlanFragment", - "bool", - "outputTableWriterFragment"); - to_json_key( + "remoteUserAddress", + p.remoteUserAddress, + "SessionRepresentation", + "String", + "remoteUserAddress"); + from_json_key( j, - "jsonRepresentation", - p.jsonRepresentation, - "PlanFragment", + "userAgent", + p.userAgent, + "SessionRepresentation", "String", - "jsonRepresentation"); -} - -void from_json(const json& j, PlanFragment& p) { - from_json_key(j, "id", p.id, "PlanFragment", "PlanFragmentId", "id"); - from_json_key(j, "root", p.root, "PlanFragment", "PlanNode", "root"); + "userAgent"); from_json_key( j, - "variables", - p.variables, - "PlanFragment", - "List", - "variables"); + "clientInfo", + p.clientInfo, + "SessionRepresentation", + "String", + "clientInfo"); from_json_key( j, - "partitioning", - p.partitioning, - "PlanFragment", - "PartitioningHandle", - "partitioning"); + "clientTags", + p.clientTags, + "SessionRepresentation", + "List", + "clientTags"); from_json_key( j, - "tableScanSchedulingOrder", - p.tableScanSchedulingOrder, - "PlanFragment", - "List", - "tableScanSchedulingOrder"); + "resourceEstimates", + p.resourceEstimates, + "SessionRepresentation", + "ResourceEstimates", + "resourceEstimates"); from_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "PlanFragment", - "PartitioningScheme", - "partitioningScheme"); + "startTime", + p.startTime, + "SessionRepresentation", + "int64_t", + "startTime"); from_json_key( j, - "stageExecutionDescriptor", - p.stageExecutionDescriptor, - "PlanFragment", - "StageExecutionDescriptor", - "stageExecutionDescriptor"); + "systemProperties", + p.systemProperties, + "SessionRepresentation", + "Map", + "systemProperties"); from_json_key( j, - "outputTableWriterFragment", - p.outputTableWriterFragment, - "PlanFragment", - "bool", - "outputTableWriterFragment"); + "catalogProperties", + p.catalogProperties, + "SessionRepresentation", + "Map>", + "catalogProperties"); from_json_key( j, - "jsonRepresentation", - p.jsonRepresentation, - "PlanFragment", - "String", - "jsonRepresentation"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const RefreshMaterializedViewHandle& p) { - j = json::object(); - to_json_key( - j, - "handle", - p.handle, - "RefreshMaterializedViewHandle", - "InsertTableHandle", - "handle"); - to_json_key( + "unprocessedCatalogProperties", + p.unprocessedCatalogProperties, + "SessionRepresentation", + "Map>", + "unprocessedCatalogProperties"); + from_json_key( j, - "schemaTableName", - p.schemaTableName, - "RefreshMaterializedViewHandle", - "SchemaTableName", - "schemaTableName"); -} - -void from_json(const json& j, RefreshMaterializedViewHandle& p) { + "roles", + p.roles, + "SessionRepresentation", + "Map", + "roles"); from_json_key( j, - "handle", - p.handle, - "RefreshMaterializedViewHandle", - "InsertTableHandle", - "handle"); + "preparedStatements", + p.preparedStatements, + "SessionRepresentation", + "Map", + "preparedStatements"); from_json_key( j, - "schemaTableName", - p.schemaTableName, - "RefreshMaterializedViewHandle", - "SchemaTableName", - "schemaTableName"); + "sessionFunctions", + p.sessionFunctions, + "SessionRepresentation", + "Map", + "sessionFunctions"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveInsertTableHandle::HiveInsertTableHandle() noexcept { - _type = "hive"; -} -void to_json(json& j, const HiveInsertTableHandle& p) { +void to_json(json& j, const TaskSource& p) { j = json::object(); - j["@type"] = "hive"; - to_json_key( - j, - "schemaName", - p.schemaName, - "HiveInsertTableHandle", - "String", - "schemaName"); - to_json_key( - j, - "tableName", - p.tableName, - "HiveInsertTableHandle", - "String", - "tableName"); - to_json_key( - j, - "inputColumns", - p.inputColumns, - "HiveInsertTableHandle", - "List", - "inputColumns"); - to_json_key( - j, - "pageSinkMetadata", - p.pageSinkMetadata, - "HiveInsertTableHandle", - "HivePageSinkMetadata", - "pageSinkMetadata"); - to_json_key( - j, - "locationHandle", - p.locationHandle, - "HiveInsertTableHandle", - "LocationHandle", - "locationHandle"); - to_json_key( - j, - "bucketProperty", - p.bucketProperty, - "HiveInsertTableHandle", - "HiveBucketProperty", - "bucketProperty"); - to_json_key( - j, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveInsertTableHandle", - "List", - "preferredOrderingColumns"); - to_json_key( - j, - "tableStorageFormat", - p.tableStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "tableStorageFormat"); to_json_key( - j, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); + j, "planNodeId", p.planNodeId, "TaskSource", "PlanNodeId", "planNodeId"); to_json_key( - j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); + j, "splits", p.splits, "TaskSource", "List", "splits"); to_json_key( j, - "compressionCodec", - p.compressionCodec, - "HiveInsertTableHandle", - "HiveCompressionCodec", - "compressionCodec"); + "noMoreSplitsForLifespan", + p.noMoreSplitsForLifespan, + "TaskSource", + "List", + "noMoreSplitsForLifespan"); to_json_key( - j, - "encryptionInformation", - p.encryptionInformation, - "HiveInsertTableHandle", - "EncryptionInformation", - "encryptionInformation"); + j, "noMoreSplits", p.noMoreSplits, "TaskSource", "bool", "noMoreSplits"); } -void from_json(const json& j, HiveInsertTableHandle& p) { - p._type = j["@type"]; - from_json_key( - j, - "schemaName", - p.schemaName, - "HiveInsertTableHandle", - "String", - "schemaName"); - from_json_key( - j, - "tableName", - p.tableName, - "HiveInsertTableHandle", - "String", - "tableName"); - from_json_key( - j, - "inputColumns", - p.inputColumns, - "HiveInsertTableHandle", - "List", - "inputColumns"); +void from_json(const json& j, TaskSource& p) { from_json_key( - j, - "pageSinkMetadata", - p.pageSinkMetadata, - "HiveInsertTableHandle", - "HivePageSinkMetadata", - "pageSinkMetadata"); + j, "planNodeId", p.planNodeId, "TaskSource", "PlanNodeId", "planNodeId"); from_json_key( - j, - "locationHandle", - p.locationHandle, - "HiveInsertTableHandle", - "LocationHandle", - "locationHandle"); + j, "splits", p.splits, "TaskSource", "List", "splits"); from_json_key( j, - "bucketProperty", - p.bucketProperty, - "HiveInsertTableHandle", - "HiveBucketProperty", - "bucketProperty"); + "noMoreSplitsForLifespan", + p.noMoreSplitsForLifespan, + "TaskSource", + "List", + "noMoreSplitsForLifespan"); from_json_key( + j, "noMoreSplits", p.noMoreSplits, "TaskSource", "bool", "noMoreSplits"); +} +} // 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; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const OutputBuffers& p) { + j = json::object(); + to_json_key(j, "type", p.type, "OutputBuffers", "BufferType", "type"); + to_json_key(j, "version", p.version, "OutputBuffers", "int64_t", "version"); + to_json_key( j, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveInsertTableHandle", - "List", - "preferredOrderingColumns"); - from_json_key( + "noMoreBufferIds", + p.noMoreBufferIds, + "OutputBuffers", + "bool", + "noMoreBufferIds"); + to_json_key( j, - "tableStorageFormat", - p.tableStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "tableStorageFormat"); + "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, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); + "noMoreBufferIds", + p.noMoreBufferIds, + "OutputBuffers", + "bool", + "noMoreBufferIds"); from_json_key( j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); - from_json_key( + "buffers", + p.buffers, + "OutputBuffers", + "Map", + "buffers"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const DeleteScanInfo& p) { + j = json::object(); + to_json_key(j, "id", p.id, "DeleteScanInfo", "PlanNodeId", "id"); + to_json_key( j, - "compressionCodec", - p.compressionCodec, - "HiveInsertTableHandle", - "HiveCompressionCodec", - "compressionCodec"); + "tableHandle", + p.tableHandle, + "DeleteScanInfo", + "TableHandle", + "tableHandle"); +} + +void from_json(const json& j, DeleteScanInfo& p) { + from_json_key(j, "id", p.id, "DeleteScanInfo", "PlanNodeId", "id"); from_json_key( j, - "encryptionInformation", - p.encryptionInformation, - "HiveInsertTableHandle", - "EncryptionInformation", - "encryptionInformation"); + "tableHandle", + p.tableHandle, + "DeleteScanInfo", + "TableHandle", + "tableHandle"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TableWriterMergeNode::TableWriterMergeNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.TableWriterMergeNode"; -} -void to_json(json& j, const TableWriterMergeNode& p) { +void to_json(json& j, const AnalyzeTableHandle& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.TableWriterMergeNode"; - to_json_key(j, "id", p.id, "TableWriterMergeNode", "PlanNodeId", "id"); - to_json_key( - j, "source", p.source, "TableWriterMergeNode", "PlanNode", "source"); to_json_key( j, - "rowCountVariable", - p.rowCountVariable, - "TableWriterMergeNode", - "VariableReferenceExpression", - "rowCountVariable"); - to_json_key( - j, - "fragmentVariable", - p.fragmentVariable, - "TableWriterMergeNode", - "VariableReferenceExpression", - "fragmentVariable"); + "connectorId", + p.connectorId, + "AnalyzeTableHandle", + "ConnectorId", + "connectorId"); to_json_key( j, - "tableCommitContextVariable", - p.tableCommitContextVariable, - "TableWriterMergeNode", - "VariableReferenceExpression", - "tableCommitContextVariable"); + "transactionHandle", + p.transactionHandle, + "AnalyzeTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); to_json_key( j, - "statisticsAggregation", - p.statisticsAggregation, - "TableWriterMergeNode", - "StatisticAggregations", - "statisticsAggregation"); + "connectorHandle", + p.connectorHandle, + "AnalyzeTableHandle", + "ConnectorTableHandle", + "connectorHandle"); } -void from_json(const json& j, TableWriterMergeNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "TableWriterMergeNode", "PlanNodeId", "id"); - from_json_key( - j, "source", p.source, "TableWriterMergeNode", "PlanNode", "source"); - from_json_key( - j, - "rowCountVariable", - p.rowCountVariable, - "TableWriterMergeNode", - "VariableReferenceExpression", - "rowCountVariable"); +void from_json(const json& j, AnalyzeTableHandle& p) { from_json_key( j, - "fragmentVariable", - p.fragmentVariable, - "TableWriterMergeNode", - "VariableReferenceExpression", - "fragmentVariable"); + "connectorId", + p.connectorId, + "AnalyzeTableHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "tableCommitContextVariable", - p.tableCommitContextVariable, - "TableWriterMergeNode", - "VariableReferenceExpression", - "tableCommitContextVariable"); + "transactionHandle", + p.transactionHandle, + "AnalyzeTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); from_json_key( j, - "statisticsAggregation", - p.statisticsAggregation, - "TableWriterMergeNode", - "StatisticAggregations", - "statisticsAggregation"); + "connectorHandle", + p.connectorHandle, + "AnalyzeTableHandle", + "ConnectorTableHandle", + "connectorHandle"); } } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { -void to_json(json& j, const OperatorInfo& p) {} -void from_json(const json& j, OperatorInfo& p) {} -} // namespace facebook::presto::protocol namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + + if (type == "CreateHandle") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "InsertHandle") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "DeleteHandle") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ExecutionWriterTarget "); +} + +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError( + std::string(e.what()) + + " ExecutionWriterTarget ExecutionWriterTarget"); + } + + if (type == "CreateHandle") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "InsertHandle") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "DeleteHandle") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } -// 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; + throw TypeError(type + " no abstract type ExecutionWriterTarget "); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const OperatorStats& p) { +void to_json(json& j, const TableWriteInfo& p) { j = json::object(); - to_json_key(j, "stageId", p.stageId, "OperatorStats", "int", "stageId"); to_json_key( j, - "stageExecutionId", - p.stageExecutionId, - "OperatorStats", - "int", - "stageExecutionId"); - to_json_key( - j, "pipelineId", p.pipelineId, "OperatorStats", "int", "pipelineId"); - to_json_key( - j, "operatorId", p.operatorId, "OperatorStats", "int", "operatorId"); + "writerTarget", + p.writerTarget, + "TableWriteInfo", + "ExecutionWriterTarget", + "writerTarget"); to_json_key( j, - "planNodeId", - p.planNodeId, - "OperatorStats", - "PlanNodeId", - "planNodeId"); + "analyzeTableHandle", + p.analyzeTableHandle, + "TableWriteInfo", + "AnalyzeTableHandle", + "analyzeTableHandle"); to_json_key( j, - "operatorType", - p.operatorType, - "OperatorStats", - "String", - "operatorType"); - to_json_key( + "deleteScanInfo", + p.deleteScanInfo, + "TableWriteInfo", + "DeleteScanInfo", + "deleteScanInfo"); +} + +void from_json(const json& j, TableWriteInfo& p) { + from_json_key( j, - "totalDrivers", - p.totalDrivers, - "OperatorStats", - "int64_t", - "totalDrivers"); - to_json_key( + "writerTarget", + p.writerTarget, + "TableWriteInfo", + "ExecutionWriterTarget", + "writerTarget"); + from_json_key( j, - "addInputCalls", - p.addInputCalls, - "OperatorStats", - "int64_t", - "addInputCalls"); - to_json_key( + "analyzeTableHandle", + p.analyzeTableHandle, + "TableWriteInfo", + "AnalyzeTableHandle", + "analyzeTableHandle"); + from_json_key( j, - "addInputWall", - p.addInputWall, - "OperatorStats", - "Duration", - "addInputWall"); + "deleteScanInfo", + p.deleteScanInfo, + "TableWriteInfo", + "DeleteScanInfo", + "deleteScanInfo"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const TaskUpdateRequest& p) { + j = json::object(); to_json_key( j, - "addInputCpu", - p.addInputCpu, - "OperatorStats", - "Duration", - "addInputCpu"); + "session", + p.session, + "TaskUpdateRequest", + "SessionRepresentation", + "session"); to_json_key( j, - "addInputAllocation", - p.addInputAllocation, - "OperatorStats", - "DataSize", - "addInputAllocation"); + "extraCredentials", + p.extraCredentials, + "TaskUpdateRequest", + "Map", + "extraCredentials"); to_json_key( - j, - "rawInputDataSize", - p.rawInputDataSize, - "OperatorStats", - "DataSize", - "rawInputDataSize"); + j, "fragment", p.fragment, "TaskUpdateRequest", "String", "fragment"); to_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "OperatorStats", - "int64_t", - "rawInputPositions"); + "sources", + p.sources, + "TaskUpdateRequest", + "List", + "sources"); to_json_key( j, - "inputDataSize", - p.inputDataSize, - "OperatorStats", - "DataSize", - "inputDataSize"); + "outputIds", + p.outputIds, + "TaskUpdateRequest", + "OutputBuffers", + "outputIds"); to_json_key( j, - "inputPositions", - p.inputPositions, - "OperatorStats", - "int64_t", - "inputPositions"); - to_json_key( + "tableWriteInfo", + p.tableWriteInfo, + "TaskUpdateRequest", + "TableWriteInfo", + "tableWriteInfo"); +} + +void from_json(const json& j, TaskUpdateRequest& p) { + from_json_key( j, - "sumSquaredInputPositions", - p.sumSquaredInputPositions, - "OperatorStats", - "double", - "sumSquaredInputPositions"); + "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 { + +void to_json(json& j, const BatchTaskUpdateRequest& p) { + j = json::object(); to_json_key( j, - "getOutputCalls", - p.getOutputCalls, - "OperatorStats", - "int64_t", - "getOutputCalls"); + "taskUpdateRequest", + p.taskUpdateRequest, + "BatchTaskUpdateRequest", + "TaskUpdateRequest", + "taskUpdateRequest"); to_json_key( j, - "getOutputWall", - p.getOutputWall, - "OperatorStats", - "Duration", - "getOutputWall"); + "shuffleWriteInfo", + p.shuffleWriteInfo, + "BatchTaskUpdateRequest", + "String", + "shuffleWriteInfo"); to_json_key( j, - "getOutputCpu", - p.getOutputCpu, - "OperatorStats", - "Duration", - "getOutputCpu"); - 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, + "shuffleWriteInfo", + p.shuffleWriteInfo, + "BatchTaskUpdateRequest", + "String", + "shuffleWriteInfo"); + from_json_key( j, - "getOutputAllocation", - p.getOutputAllocation, - "OperatorStats", - "DataSize", - "getOutputAllocation"); + "broadcastBasePath", + p.broadcastBasePath, + "BatchTaskUpdateRequest", + "String", + "broadcastBasePath"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const PageBufferInfo& p) { + j = json::object(); to_json_key( - j, - "outputDataSize", - p.outputDataSize, - "OperatorStats", - "DataSize", - "outputDataSize"); + j, "partition", p.partition, "PageBufferInfo", "int", "partition"); to_json_key( j, - "outputPositions", - p.outputPositions, - "OperatorStats", + "bufferedPages", + p.bufferedPages, + "PageBufferInfo", "int64_t", - "outputPositions"); + "bufferedPages"); to_json_key( j, - "physicalWrittenDataSize", - p.physicalWrittenDataSize, - "OperatorStats", - "DataSize", - "physicalWrittenDataSize"); + "bufferedBytes", + p.bufferedBytes, + "PageBufferInfo", + "int64_t", + "bufferedBytes"); to_json_key( - j, - "additionalCpu", - p.additionalCpu, - "OperatorStats", - "Duration", - "additionalCpu"); + j, "rowsAdded", p.rowsAdded, "PageBufferInfo", "int64_t", "rowsAdded"); to_json_key( + j, "pagesAdded", p.pagesAdded, "PageBufferInfo", "int64_t", "pagesAdded"); +} + +void from_json(const json& j, PageBufferInfo& p) { + from_json_key( + j, "partition", p.partition, "PageBufferInfo", "int", "partition"); + from_json_key( j, - "blockedWall", - p.blockedWall, - "OperatorStats", - "Duration", - "blockedWall"); - to_json_key( + "bufferedPages", + p.bufferedPages, + "PageBufferInfo", + "int64_t", + "bufferedPages"); + from_json_key( j, - "finishCalls", - p.finishCalls, - "OperatorStats", + "bufferedBytes", + p.bufferedBytes, + "PageBufferInfo", "int64_t", - "finishCalls"); - to_json_key( - j, "finishWall", p.finishWall, "OperatorStats", "Duration", "finishWall"); + "bufferedBytes"); + from_json_key( + j, "rowsAdded", p.rowsAdded, "PageBufferInfo", "int64_t", "rowsAdded"); + from_json_key( + j, "pagesAdded", p.pagesAdded, "PageBufferInfo", "int64_t", "pagesAdded"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const BufferInfo& p) { + j = json::object(); to_json_key( - j, "finishCpu", p.finishCpu, "OperatorStats", "Duration", "finishCpu"); + j, "bufferId", p.bufferId, "BufferInfo", "OutputBufferId", "bufferId"); + to_json_key(j, "finished", p.finished, "BufferInfo", "bool", "finished"); to_json_key( j, - "finishAllocation", - p.finishAllocation, - "OperatorStats", - "DataSize", - "finishAllocation"); + "bufferedPages", + p.bufferedPages, + "BufferInfo", + "int", + "bufferedPages"); to_json_key( - j, - "userMemoryReservation", - p.userMemoryReservation, - "OperatorStats", - "DataSize", - "userMemoryReservation"); + j, "pagesSent", p.pagesSent, "BufferInfo", "int64_t", "pagesSent"); to_json_key( j, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "OperatorStats", - "DataSize", - "revocableMemoryReservation"); - to_json_key( + "pageBufferInfo", + p.pageBufferInfo, + "BufferInfo", + "PageBufferInfo", + "pageBufferInfo"); +} + +void from_json(const json& j, BufferInfo& p) { + from_json_key( + j, "bufferId", p.bufferId, "BufferInfo", "OutputBufferId", "bufferId"); + from_json_key(j, "finished", p.finished, "BufferInfo", "bool", "finished"); + from_json_key( j, - "systemMemoryReservation", - p.systemMemoryReservation, - "OperatorStats", - "DataSize", - "systemMemoryReservation"); - to_json_key( + "bufferedPages", + p.bufferedPages, + "BufferInfo", + "int", + "bufferedPages"); + from_json_key( + j, "pagesSent", p.pagesSent, "BufferInfo", "int64_t", "pagesSent"); + from_json_key( j, - "peakUserMemoryReservation", - p.peakUserMemoryReservation, - "OperatorStats", - "DataSize", - "peakUserMemoryReservation"); + "pageBufferInfo", + p.pageBufferInfo, + "BufferInfo", + "PageBufferInfo", + "pageBufferInfo"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +TpchSplit::TpchSplit() noexcept { + _type = "tpch"; +} + +void to_json(json& j, const TpchSplit& p) { + j = json::object(); + j["@type"] = "tpch"; to_json_key( j, - "peakSystemMemoryReservation", - p.peakSystemMemoryReservation, - "OperatorStats", - "DataSize", - "peakSystemMemoryReservation"); + "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, - "peakTotalMemoryReservation", - p.peakTotalMemoryReservation, - "OperatorStats", - "DataSize", - "peakTotalMemoryReservation"); + "addresses", + p.addresses, + "TpchSplit", + "List", + "addresses"); to_json_key( j, - "spilledDataSize", - p.spilledDataSize, - "OperatorStats", - "DataSize", - "spilledDataSize"); - to_json_key( + "predicate", + p.predicate, + "TpchSplit", + "TupleDomain>", + "predicate"); +} + +void from_json(const json& j, TpchSplit& p) { + p._type = j["@type"]; + from_json_key( j, - "blockedReason", - p.blockedReason, - "OperatorStats", - "BlockedReason", - "blockedReason"); - to_json_key(j, "info", p.info, "OperatorStats", "OperatorInfo", "info"); + "tableHandle", + p.tableHandle, + "TpchSplit", + "TpchTableHandle", + "tableHandle"); + from_json_key( + j, "partNumber", p.partNumber, "TpchSplit", "int", "partNumber"); + from_json_key( + j, "totalParts", p.totalParts, "TpchSplit", "int", "totalParts"); + from_json_key( + j, + "addresses", + p.addresses, + "TpchSplit", + "List", + "addresses"); + from_json_key( + j, + "predicate", + p.predicate, + "TpchSplit", + "TupleDomain>", + "predicate"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +SqlFunctionHandle::SqlFunctionHandle() noexcept { + _type = "json_file"; +} + +void to_json(json& j, const SqlFunctionHandle& p) { + j = json::object(); + j["@type"] = "json_file"; to_json_key( j, - "runtimeStats", - p.runtimeStats, - "OperatorStats", - "RuntimeStats", - "runtimeStats"); + "functionId", + p.functionId, + "SqlFunctionHandle", + "SqlFunctionId", + "functionId"); + to_json_key( + j, "version", p.version, "SqlFunctionHandle", "String", "version"); } -void from_json(const json& j, OperatorStats& p) { - from_json_key(j, "stageId", p.stageId, "OperatorStats", "int", "stageId"); +void from_json(const json& j, SqlFunctionHandle& p) { + p._type = j["@type"]; from_json_key( j, - "stageExecutionId", - p.stageExecutionId, - "OperatorStats", - "int", - "stageExecutionId"); - from_json_key( - j, "pipelineId", p.pipelineId, "OperatorStats", "int", "pipelineId"); - from_json_key( - j, "operatorId", p.operatorId, "OperatorStats", "int", "operatorId"); + "functionId", + p.functionId, + "SqlFunctionHandle", + "SqlFunctionId", + "functionId"); from_json_key( + j, "version", p.version, "SqlFunctionHandle", "String", "version"); +} +} // 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, - "planNodeId", - p.planNodeId, - "OperatorStats", - "PlanNodeId", - "planNodeId"); - from_json_key( + "replicateVariables", + p.replicateVariables, + "UnnestNode", + "List", + "replicateVariables"); + to_json_key( j, - "operatorType", - p.operatorType, - "OperatorStats", - "String", - "operatorType"); - from_json_key( + "unnestVariables", + p.unnestVariables, + "UnnestNode", + "Map>", + "unnestVariables"); + to_json_key( j, - "totalDrivers", - p.totalDrivers, - "OperatorStats", - "int64_t", - "totalDrivers"); + "ordinalityVariable", + p.ordinalityVariable, + "UnnestNode", + "VariableReferenceExpression", + "ordinalityVariable"); +} + +void from_json(const json& j, UnnestNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "UnnestNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "UnnestNode", "PlanNode", "source"); from_json_key( j, - "addInputCalls", - p.addInputCalls, - "OperatorStats", - "int64_t", - "addInputCalls"); + "replicateVariables", + p.replicateVariables, + "UnnestNode", + "List", + "replicateVariables"); from_json_key( j, - "addInputWall", - p.addInputWall, - "OperatorStats", - "Duration", - "addInputWall"); + "unnestVariables", + p.unnestVariables, + "UnnestNode", + "Map>", + "unnestVariables"); from_json_key( j, - "addInputCpu", - p.addInputCpu, - "OperatorStats", - "Duration", - "addInputCpu"); - from_json_key( + "ordinalityVariable", + p.ordinalityVariable, + "UnnestNode", + "VariableReferenceExpression", + "ordinalityVariable"); +} +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +namespace facebook::presto::protocol { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ConnectorMetadataUpdateHandle"); +} + +void from_json( + const json& j, + std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError(std::string(e.what()) + " ConnectorMetadataUpdateHandle"); + } + + if (getConnectorKey(type) == "hive") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + + throw TypeError(type + " no abstract type ConnectorMetadataUpdateHandle"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const MetadataUpdates& p) { + j = json::object(); + to_json_key( j, - "addInputAllocation", - p.addInputAllocation, - "OperatorStats", - "DataSize", - "addInputAllocation"); - from_json_key( + "connectorId", + p.connectorId, + "MetadataUpdates", + "ConnectorId", + "connectorId"); + to_json_key( j, - "rawInputDataSize", - p.rawInputDataSize, - "OperatorStats", - "DataSize", - "rawInputDataSize"); + "metadataUpdates", + p.metadataUpdates, + "MetadataUpdates", + "List>", + "metadataUpdates"); +} + +void from_json(const json& j, MetadataUpdates& p) { from_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "OperatorStats", - "int64_t", - "rawInputPositions"); + "connectorId", + p.connectorId, + "MetadataUpdates", + "ConnectorId", + "connectorId"); from_json_key( j, - "inputDataSize", - p.inputDataSize, - "OperatorStats", - "DataSize", - "inputDataSize"); - from_json_key( + "metadataUpdates", + p.metadataUpdates, + "MetadataUpdates", + "List>", + "metadataUpdates"); +} +} // 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, - "inputPositions", - p.inputPositions, - "OperatorStats", - "int64_t", - "inputPositions"); - from_json_key( + "canAddBuffers", + p.canAddBuffers, + "OutputBufferInfo", + "bool", + "canAddBuffers"); + to_json_key( j, - "sumSquaredInputPositions", - p.sumSquaredInputPositions, - "OperatorStats", - "double", - "sumSquaredInputPositions"); - from_json_key( + "canAddPages", + p.canAddPages, + "OutputBufferInfo", + "bool", + "canAddPages"); + to_json_key( j, - "getOutputCalls", - p.getOutputCalls, - "OperatorStats", + "totalBufferedBytes", + p.totalBufferedBytes, + "OutputBufferInfo", "int64_t", - "getOutputCalls"); - from_json_key( - j, - "getOutputWall", - p.getOutputWall, - "OperatorStats", - "Duration", - "getOutputWall"); - from_json_key( - j, - "getOutputCpu", - p.getOutputCpu, - "OperatorStats", - "Duration", - "getOutputCpu"); - from_json_key( - j, - "getOutputAllocation", - p.getOutputAllocation, - "OperatorStats", - "DataSize", - "getOutputAllocation"); - from_json_key( - j, - "outputDataSize", - p.outputDataSize, - "OperatorStats", - "DataSize", - "outputDataSize"); - from_json_key( + "totalBufferedBytes"); + to_json_key( j, - "outputPositions", - p.outputPositions, - "OperatorStats", + "totalBufferedPages", + p.totalBufferedPages, + "OutputBufferInfo", "int64_t", - "outputPositions"); - from_json_key( - j, - "physicalWrittenDataSize", - p.physicalWrittenDataSize, - "OperatorStats", - "DataSize", - "physicalWrittenDataSize"); - from_json_key( - j, - "additionalCpu", - p.additionalCpu, - "OperatorStats", - "Duration", - "additionalCpu"); - from_json_key( - j, - "blockedWall", - p.blockedWall, - "OperatorStats", - "Duration", - "blockedWall"); - from_json_key( + "totalBufferedPages"); + to_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"); - from_json_key( + "totalRowsSent"); + to_json_key( j, - "finishAllocation", - p.finishAllocation, - "OperatorStats", - "DataSize", - "finishAllocation"); - from_json_key( + "totalPagesSent", + p.totalPagesSent, + "OutputBufferInfo", + "int64_t", + "totalPagesSent"); + to_json_key( j, - "userMemoryReservation", - p.userMemoryReservation, - "OperatorStats", - "DataSize", - "userMemoryReservation"); + "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, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "OperatorStats", - "DataSize", - "revocableMemoryReservation"); + j, "state", p.state, "OutputBufferInfo", "BufferState", "state"); from_json_key( j, - "systemMemoryReservation", - p.systemMemoryReservation, - "OperatorStats", - "DataSize", - "systemMemoryReservation"); + "canAddBuffers", + p.canAddBuffers, + "OutputBufferInfo", + "bool", + "canAddBuffers"); from_json_key( j, - "peakUserMemoryReservation", - p.peakUserMemoryReservation, - "OperatorStats", - "DataSize", - "peakUserMemoryReservation"); + "canAddPages", + p.canAddPages, + "OutputBufferInfo", + "bool", + "canAddPages"); from_json_key( j, - "peakSystemMemoryReservation", - p.peakSystemMemoryReservation, - "OperatorStats", - "DataSize", - "peakSystemMemoryReservation"); + "totalBufferedBytes", + p.totalBufferedBytes, + "OutputBufferInfo", + "int64_t", + "totalBufferedBytes"); from_json_key( j, - "peakTotalMemoryReservation", - p.peakTotalMemoryReservation, - "OperatorStats", - "DataSize", - "peakTotalMemoryReservation"); + "totalBufferedPages", + p.totalBufferedPages, + "OutputBufferInfo", + "int64_t", + "totalBufferedPages"); from_json_key( j, - "spilledDataSize", - p.spilledDataSize, - "OperatorStats", - "DataSize", - "spilledDataSize"); + "totalRowsSent", + p.totalRowsSent, + "OutputBufferInfo", + "int64_t", + "totalRowsSent"); from_json_key( j, - "blockedReason", - p.blockedReason, - "OperatorStats", - "BlockedReason", - "blockedReason"); - from_json_key(j, "info", p.info, "OperatorStats", "OperatorInfo", "info"); + "totalPagesSent", + p.totalPagesSent, + "OutputBufferInfo", + "int64_t", + "totalPagesSent"); from_json_key( j, - "runtimeStats", - p.runtimeStats, - "OperatorStats", - "RuntimeStats", - "runtimeStats"); + "buffers", + p.buffers, + "OutputBufferInfo", + "List", + "buffers"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const NodeStatus& p) { +void to_json(json& j, const TaskInfo& p) { j = json::object(); - to_json_key(j, "nodeId", p.nodeId, "NodeStatus", "String", "nodeId"); - to_json_key( - j, - "nodeVersion", - p.nodeVersion, - "NodeStatus", - "NodeVersion", - "nodeVersion"); - to_json_key( - j, "environment", p.environment, "NodeStatus", "String", "environment"); - to_json_key( - j, "coordinator", p.coordinator, "NodeStatus", "bool", "coordinator"); - to_json_key(j, "uptime", p.uptime, "NodeStatus", "Duration", "uptime"); + to_json_key(j, "taskId", p.taskId, "TaskInfo", "TaskId", "taskId"); to_json_key( - j, - "externalAddress", - p.externalAddress, - "NodeStatus", - "String", - "externalAddress"); + j, "taskStatus", p.taskStatus, "TaskInfo", "TaskStatus", "taskStatus"); to_json_key( j, - "internalAddress", - p.internalAddress, - "NodeStatus", - "String", - "internalAddress"); - to_json_key( - j, "memoryInfo", p.memoryInfo, "NodeStatus", "MemoryInfo", "memoryInfo"); - to_json_key(j, "processors", p.processors, "NodeStatus", "int", "processors"); + "lastHeartbeat", + p.lastHeartbeat, + "TaskInfo", + "DateTime", + "lastHeartbeat"); to_json_key( j, - "processCpuLoad", - p.processCpuLoad, - "NodeStatus", - "double", - "processCpuLoad"); + "outputBuffers", + p.outputBuffers, + "TaskInfo", + "OutputBufferInfo", + "outputBuffers"); to_json_key( j, - "systemCpuLoad", - p.systemCpuLoad, - "NodeStatus", - "double", - "systemCpuLoad"); - to_json_key(j, "heapUsed", p.heapUsed, "NodeStatus", "int64_t", "heapUsed"); + "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, - "heapAvailable", - p.heapAvailable, - "NodeStatus", - "int64_t", - "heapAvailable"); - to_json_key( - j, "nonHeapUsed", p.nonHeapUsed, "NodeStatus", "int64_t", "nonHeapUsed"); + "metadataUpdates", + p.metadataUpdates, + "TaskInfo", + "MetadataUpdates", + "metadataUpdates"); + to_json_key(j, "nodeId", p.nodeId, "TaskInfo", "String", "nodeId"); } -void from_json(const json& j, NodeStatus& p) { - from_json_key(j, "nodeId", p.nodeId, "NodeStatus", "String", "nodeId"); - from_json_key( - j, - "nodeVersion", - p.nodeVersion, - "NodeStatus", - "NodeVersion", - "nodeVersion"); - from_json_key( - j, "environment", p.environment, "NodeStatus", "String", "environment"); - from_json_key( - j, "coordinator", p.coordinator, "NodeStatus", "bool", "coordinator"); - from_json_key(j, "uptime", p.uptime, "NodeStatus", "Duration", "uptime"); +void from_json(const json& j, TaskInfo& p) { + from_json_key(j, "taskId", p.taskId, "TaskInfo", "TaskId", "taskId"); from_json_key( - j, - "externalAddress", - p.externalAddress, - "NodeStatus", - "String", - "externalAddress"); + j, "taskStatus", p.taskStatus, "TaskInfo", "TaskStatus", "taskStatus"); from_json_key( j, - "internalAddress", - p.internalAddress, - "NodeStatus", - "String", - "internalAddress"); - from_json_key( - j, "memoryInfo", p.memoryInfo, "NodeStatus", "MemoryInfo", "memoryInfo"); - from_json_key( - j, "processors", p.processors, "NodeStatus", "int", "processors"); + "lastHeartbeat", + p.lastHeartbeat, + "TaskInfo", + "DateTime", + "lastHeartbeat"); from_json_key( j, - "processCpuLoad", - p.processCpuLoad, - "NodeStatus", - "double", - "processCpuLoad"); + "outputBuffers", + p.outputBuffers, + "TaskInfo", + "OutputBufferInfo", + "outputBuffers"); from_json_key( j, - "systemCpuLoad", - p.systemCpuLoad, - "NodeStatus", - "double", - "systemCpuLoad"); - from_json_key(j, "heapUsed", p.heapUsed, "NodeStatus", "int64_t", "heapUsed"); + "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, - "heapAvailable", - p.heapAvailable, - "NodeStatus", - "int64_t", - "heapAvailable"); - from_json_key( - j, "nonHeapUsed", p.nonHeapUsed, "NodeStatus", "int64_t", "nonHeapUsed"); + "metadataUpdates", + p.metadataUpdates, + "TaskInfo", + "MetadataUpdates", + "metadataUpdates"); + from_json_key(j, "nodeId", p.nodeId, "TaskInfo", "String", "nodeId"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -RemoteTransactionHandle::RemoteTransactionHandle() noexcept { - _type = "$remote"; -} +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const RemoteTransactionHandle& p) { - j = json::object(); - j["@type"] = "$remote"; - to_json_key( - j, "dummy", p.dummy, "RemoteTransactionHandle", "String", "dummy"); +// 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, RemoteTransactionHandle& p) { - p._type = j["@type"]; - from_json_key( - j, "dummy", p.dummy, "RemoteTransactionHandle", "String", "dummy"); +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 namespace facebook::presto::protocol { -EnforceSingleRowNode::EnforceSingleRowNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode"; -} -void to_json(json& j, const EnforceSingleRowNode& p) { +void to_json(json& j, const Marker& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode"; - to_json_key(j, "id", p.id, "EnforceSingleRowNode", "PlanNodeId", "id"); - to_json_key( - j, "source", p.source, "EnforceSingleRowNode", "PlanNode", "source"); + 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, EnforceSingleRowNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "EnforceSingleRowNode", "PlanNodeId", "id"); - from_json_key( - j, "source", p.source, "EnforceSingleRowNode", "PlanNode", "source"); +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 Location& p) { +void to_json(json& j, const Range& p) { j = json::object(); - to_json_key(j, "location", p.location, "Location", "String", "location"); + 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, Location& p) { - from_json_key(j, "location", p.location, "Location", "String", "location"); +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 { -RemoteSplit::RemoteSplit() noexcept { - _type = "$remote"; -} +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const RemoteSplit& p) { - j = json::object(); - j["@type"] = "$remote"; - to_json_key(j, "location", p.location, "RemoteSplit", "Location", "location"); - to_json_key( - j, - "remoteSourceTaskId", - p.remoteSourceTaskId, - "RemoteSplit", - "TaskId", - "remoteSourceTaskId"); +// 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, 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"); +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 { -HiveMetadataUpdateHandle::HiveMetadataUpdateHandle() noexcept { - _type = "hive"; -} - -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"); +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, - "fileName", - p.fileName, - "HiveMetadataUpdateHandle", - "String", - "fileName"); + "orderingScheme", + p.orderingScheme, + "TopNNode", + "OrderingScheme", + "orderingScheme"); + to_json_key(j, "step", p.step, "TopNNode", "Step", "step"); } -void from_json(const json& j, HiveMetadataUpdateHandle& p) { +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, - "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"); + "orderingScheme", + p.orderingScheme, + "TopNNode", + "OrderingScheme", + "orderingScheme"); + from_json_key(j, "step", p.step, "TopNNode", "Step", "step"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -UnnestNode::UnnestNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.UnnestNode"; +TableWriterMergeNode::TableWriterMergeNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.TableWriterMergeNode"; } -void to_json(json& j, const UnnestNode& p) { +void to_json(json& j, const TableWriterMergeNode& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.UnnestNode"; - to_json_key(j, "id", p.id, "UnnestNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "UnnestNode", "PlanNode", "source"); + j["@type"] = "com.facebook.presto.sql.planner.plan.TableWriterMergeNode"; + to_json_key(j, "id", p.id, "TableWriterMergeNode", "PlanNodeId", "id"); + to_json_key( + j, "source", p.source, "TableWriterMergeNode", "PlanNode", "source"); to_json_key( j, - "replicateVariables", - p.replicateVariables, - "UnnestNode", - "List", - "replicateVariables"); + "rowCountVariable", + p.rowCountVariable, + "TableWriterMergeNode", + "VariableReferenceExpression", + "rowCountVariable"); to_json_key( j, - "unnestVariables", - p.unnestVariables, - "UnnestNode", - "Map>", - "unnestVariables"); + "fragmentVariable", + p.fragmentVariable, + "TableWriterMergeNode", + "VariableReferenceExpression", + "fragmentVariable"); to_json_key( j, - "ordinalityVariable", - p.ordinalityVariable, - "UnnestNode", + "tableCommitContextVariable", + p.tableCommitContextVariable, + "TableWriterMergeNode", "VariableReferenceExpression", - "ordinalityVariable"); + "tableCommitContextVariable"); + to_json_key( + j, + "statisticsAggregation", + p.statisticsAggregation, + "TableWriterMergeNode", + "StatisticAggregations", + "statisticsAggregation"); } -void from_json(const json& j, UnnestNode& p) { +void from_json(const json& j, TableWriterMergeNode& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "UnnestNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "UnnestNode", "PlanNode", "source"); + from_json_key(j, "id", p.id, "TableWriterMergeNode", "PlanNodeId", "id"); from_json_key( - j, - "replicateVariables", - p.replicateVariables, - "UnnestNode", - "List", - "replicateVariables"); + j, "source", p.source, "TableWriterMergeNode", "PlanNode", "source"); from_json_key( j, - "unnestVariables", - p.unnestVariables, - "UnnestNode", - "Map>", - "unnestVariables"); + "rowCountVariable", + p.rowCountVariable, + "TableWriterMergeNode", + "VariableReferenceExpression", + "rowCountVariable"); from_json_key( j, - "ordinalityVariable", - p.ordinalityVariable, - "UnnestNode", + "fragmentVariable", + p.fragmentVariable, + "TableWriterMergeNode", "VariableReferenceExpression", - "ordinalityVariable"); -} -} // 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( + "fragmentVariable"); + from_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"); + "tableCommitContextVariable", + p.tableCommitContextVariable, + "TableWriterMergeNode", + "VariableReferenceExpression", + "tableCommitContextVariable"); from_json_key( j, - "schemaTableName", - p.schemaTableName, - "DeleteHandle", - "SchemaTableName", - "schemaTableName"); + "statisticsAggregation", + p.statisticsAggregation, + "TableWriterMergeNode", + "StatisticAggregations", + "statisticsAggregation"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const DriverStats& p) { - j = json::object(); - to_json_key(j, "lifespan", p.lifespan, "DriverStats", "Lifespan", "lifespan"); - to_json_key( - j, "createTime", p.createTime, "DriverStats", "DateTime", "createTime"); - to_json_key( - j, "startTime", p.startTime, "DriverStats", "DateTime", "startTime"); - to_json_key(j, "endTime", p.endTime, "DriverStats", "DateTime", "endTime"); - to_json_key( - j, "queuedTime", p.queuedTime, "DriverStats", "Duration", "queuedTime"); - to_json_key( - j, - "elapsedTime", - p.elapsedTime, - "DriverStats", - "Duration", - "elapsedTime"); - to_json_key( - j, - "userMemoryReservation", - p.userMemoryReservation, - "DriverStats", - "DataSize", - "userMemoryReservation"); - to_json_key( - j, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "DriverStats", - "DataSize", - "revocableMemoryReservation"); - to_json_key( - j, - "systemMemoryReservation", - p.systemMemoryReservation, - "DriverStats", - "DataSize", - "systemMemoryReservation"); - to_json_key( - j, - "totalScheduledTime", - p.totalScheduledTime, - "DriverStats", - "Duration", - "totalScheduledTime"); - to_json_key( - j, - "totalCpuTime", - p.totalCpuTime, - "DriverStats", - "Duration", - "totalCpuTime"); +// 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, - "totalBlockedTime", - p.totalBlockedTime, - "DriverStats", - "Duration", - "totalBlockedTime"); - to_json_key( - j, "fullyBlocked", p.fullyBlocked, "DriverStats", "bool", "fullyBlocked"); - to_json_key( + "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, - "blockedReasons", - p.blockedReasons, - "DriverStats", - "List", - "blockedReasons"); + "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 Specification& p) { + j = json::object(); to_json_key( j, - "totalAllocation", - p.totalAllocation, - "DriverStats", - "DataSize", - "totalAllocation"); + "partitionBy", + p.partitionBy, + "Specification", + "List", + "partitionBy"); to_json_key( j, - "rawInputDataSize", - p.rawInputDataSize, - "DriverStats", - "DataSize", - "rawInputDataSize"); - to_json_key( + "orderingScheme", + p.orderingScheme, + "Specification", + "OrderingScheme", + "orderingScheme"); +} + +void from_json(const json& j, Specification& p) { + from_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "DriverStats", - "int64_t", - "rawInputPositions"); - to_json_key( + "partitionBy", + p.partitionBy, + "Specification", + "List", + "partitionBy"); + from_json_key( j, - "rawInputReadTime", - p.rawInputReadTime, - "DriverStats", - "Duration", - "rawInputReadTime"); + "orderingScheme", + p.orderingScheme, + "Specification", + "OrderingScheme", + "orderingScheme"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +WindowNode::WindowNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.WindowNode"; +} + +void to_json(json& j, const WindowNode& p) { + j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.WindowNode"; to_json_key( j, - "processedInputDataSize", - p.processedInputDataSize, - "DriverStats", - "DataSize", - "processedInputDataSize"); + "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, - "processedInputPositions", - p.processedInputPositions, - "DriverStats", - "int64_t", - "processedInputPositions"); + "specification", + p.specification, + "WindowNode", + "Specification", + "specification"); to_json_key( j, - "outputDataSize", - p.outputDataSize, - "DriverStats", - "DataSize", - "outputDataSize"); + "windowFunctions", + p.windowFunctions, + "WindowNode", + "Map", + "windowFunctions"); to_json_key( j, - "outputPositions", - p.outputPositions, - "DriverStats", - "int64_t", - "outputPositions"); + "hashVariable", + p.hashVariable, + "WindowNode", + "VariableReferenceExpression", + "hashVariable"); to_json_key( j, - "physicalWrittenDataSize", - p.physicalWrittenDataSize, - "DriverStats", - "DataSize", - "physicalWrittenDataSize"); + "prePartitionedInputs", + p.prePartitionedInputs, + "WindowNode", + "List", + "prePartitionedInputs"); to_json_key( j, - "operatorStats", - p.operatorStats, - "DriverStats", - "List", - "operatorStats"); + "preSortedOrderPrefix", + p.preSortedOrderPrefix, + "WindowNode", + "int", + "preSortedOrderPrefix"); } -void from_json(const json& j, DriverStats& p) { - from_json_key( - j, "lifespan", p.lifespan, "DriverStats", "Lifespan", "lifespan"); - from_json_key( - j, "createTime", p.createTime, "DriverStats", "DateTime", "createTime"); - from_json_key( - j, "startTime", p.startTime, "DriverStats", "DateTime", "startTime"); - from_json_key(j, "endTime", p.endTime, "DriverStats", "DateTime", "endTime"); - from_json_key( - j, "queuedTime", p.queuedTime, "DriverStats", "Duration", "queuedTime"); - from_json_key( - j, - "elapsedTime", - p.elapsedTime, - "DriverStats", - "Duration", - "elapsedTime"); - from_json_key( - j, - "userMemoryReservation", - p.userMemoryReservation, - "DriverStats", - "DataSize", - "userMemoryReservation"); - from_json_key( - j, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "DriverStats", - "DataSize", - "revocableMemoryReservation"); - from_json_key( - j, - "systemMemoryReservation", - p.systemMemoryReservation, - "DriverStats", - "DataSize", - "systemMemoryReservation"); - from_json_key( - j, - "totalScheduledTime", - p.totalScheduledTime, - "DriverStats", - "Duration", - "totalScheduledTime"); - from_json_key( - j, - "totalCpuTime", - p.totalCpuTime, - "DriverStats", - "Duration", - "totalCpuTime"); +void from_json(const json& j, WindowNode& p) { + p._type = j["@type"]; from_json_key( j, - "totalBlockedTime", - p.totalBlockedTime, - "DriverStats", - "Duration", - "totalBlockedTime"); - from_json_key( - j, "fullyBlocked", p.fullyBlocked, "DriverStats", "bool", "fullyBlocked"); + "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, - "blockedReasons", - p.blockedReasons, - "DriverStats", - "List", - "blockedReasons"); + "specification", + p.specification, + "WindowNode", + "Specification", + "specification"); from_json_key( j, - "totalAllocation", - p.totalAllocation, - "DriverStats", - "DataSize", - "totalAllocation"); + "windowFunctions", + p.windowFunctions, + "WindowNode", + "Map", + "windowFunctions"); from_json_key( j, - "rawInputDataSize", - p.rawInputDataSize, - "DriverStats", - "DataSize", - "rawInputDataSize"); + "hashVariable", + p.hashVariable, + "WindowNode", + "VariableReferenceExpression", + "hashVariable"); from_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "DriverStats", - "int64_t", - "rawInputPositions"); + "prePartitionedInputs", + p.prePartitionedInputs, + "WindowNode", + "List", + "prePartitionedInputs"); from_json_key( j, - "rawInputReadTime", - p.rawInputReadTime, - "DriverStats", - "Duration", - "rawInputReadTime"); - from_json_key( + "preSortedOrderPrefix", + p.preSortedOrderPrefix, + "WindowNode", + "int", + "preSortedOrderPrefix"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +OutputNode::OutputNode() noexcept { + _type = ".OutputNode"; +} + +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, - "processedInputDataSize", - p.processedInputDataSize, - "DriverStats", - "DataSize", - "processedInputDataSize"); + "columnNames", + p.columnNames, + "OutputNode", + "List", + "columnNames"); + to_json_key( + j, + "outputVariables", + p.outputVariables, + "OutputNode", + "List", + "outputVariables"); +} + +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, - "processedInputPositions", - p.processedInputPositions, - "DriverStats", - "int64_t", - "processedInputPositions"); + "columnNames", + p.columnNames, + "OutputNode", + "List", + "columnNames"); from_json_key( j, - "outputDataSize", - p.outputDataSize, - "DriverStats", - "DataSize", - "outputDataSize"); + "outputVariables", + p.outputVariables, + "OutputNode", + "List", + "outputVariables"); +} +} // 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, - "outputPositions", - p.outputPositions, - "DriverStats", - "int64_t", - "outputPositions"); + "columns", + p.columns, + "HiveBucketHandle", + "List", + "columns"); from_json_key( j, - "physicalWrittenDataSize", - p.physicalWrittenDataSize, - "DriverStats", - "DataSize", - "physicalWrittenDataSize"); + "tableBucketCount", + p.tableBucketCount, + "HiveBucketHandle", + "int", + "tableBucketCount"); from_json_key( j, - "operatorStats", - p.operatorStats, - "DriverStats", - "List", - "operatorStats"); + "readBucketCount", + p.readBucketCount, + "HiveBucketHandle", + "int", + "readBucketCount"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const DistributionSnapshot& p) { +void to_json(json& j, const HiveBucketFilter& 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"); + j, + "bucketsToKeep", + p.bucketsToKeep, + "HiveBucketFilter", + "List", + "bucketsToKeep"); } -void from_json(const json& j, DistributionSnapshot& p) { +void from_json(const json& j, HiveBucketFilter& 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"); + 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 PipelineStats& p) { +void to_json(json& j, const HiveTableLayoutHandle& p) { j = json::object(); + j["@type"] = "hive"; to_json_key( - j, "pipelineId", p.pipelineId, "PipelineStats", "int", "pipelineId"); + 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, - "firstStartTime", - p.firstStartTime, - "PipelineStats", - "DateTime", - "firstStartTime"); + "partitionColumnPredicate", + p.partitionColumnPredicate, + "HiveTableLayoutHandle", + "TupleDomain>", + "partitionColumnPredicate"); to_json_key( j, - "lastStartTime", - p.lastStartTime, - "PipelineStats", - "DateTime", - "lastStartTime"); + "bucketHandle", + p.bucketHandle, + "HiveTableLayoutHandle", + "HiveBucketHandle", + "bucketHandle"); to_json_key( j, - "lastEndTime", - p.lastEndTime, - "PipelineStats", - "DateTime", - "lastEndTime"); + "bucketFilter", + p.bucketFilter, + "HiveTableLayoutHandle", + "HiveBucketFilter", + "bucketFilter"); to_json_key( j, - "inputPipeline", - p.inputPipeline, - "PipelineStats", + "pushdownFilterEnabled", + p.pushdownFilterEnabled, + "HiveTableLayoutHandle", "bool", - "inputPipeline"); + "pushdownFilterEnabled"); to_json_key( j, - "outputPipeline", - p.outputPipeline, - "PipelineStats", - "bool", - "outputPipeline"); + "layoutString", + p.layoutString, + "HiveTableLayoutHandle", + "String", + "layoutString"); to_json_key( j, - "totalDrivers", - p.totalDrivers, - "PipelineStats", - "int", - "totalDrivers"); + "requestedColumns", + p.requestedColumns, + "HiveTableLayoutHandle", + "List", + "requestedColumns"); to_json_key( j, - "queuedDrivers", - p.queuedDrivers, - "PipelineStats", - "int", - "queuedDrivers"); + "partialAggregationsPushedDown", + p.partialAggregationsPushedDown, + "HiveTableLayoutHandle", + "bool", + "partialAggregationsPushedDown"); to_json_key( j, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "PipelineStats", - "int", - "queuedPartitionedDrivers"); + "appendRowNumber", + p.appendRowNumber, + "HiveTableLayoutHandle", + "bool", + "appendRowNumber"); to_json_key( j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "PipelineStats", - "int64_t", - "queuedPartitionedSplitsWeight"); - to_json_key( + "footerStatsUnreliable", + p.footerStatsUnreliable, + "HiveTableLayoutHandle", + "bool", + "footerStatsUnreliable"); +} + +void from_json(const json& j, HiveTableLayoutHandle& p) { + p._type = j["@type"]; + from_json_key( j, - "runningDrivers", - p.runningDrivers, - "PipelineStats", - "int", - "runningDrivers"); - to_json_key( + "schemaTableName", + p.schemaTableName, + "HiveTableLayoutHandle", + "SchemaTableName", + "schemaTableName"); + from_json_key( j, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "PipelineStats", - "int", - "runningPartitionedDrivers"); - to_json_key( + "tablePath", + p.tablePath, + "HiveTableLayoutHandle", + "String", + "tablePath"); + from_json_key( j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "PipelineStats", - "int64_t", - "runningPartitionedSplitsWeight"); - to_json_key( + "partitionColumns", + p.partitionColumns, + "HiveTableLayoutHandle", + "List", + "partitionColumns"); + from_json_key( j, - "blockedDrivers", - p.blockedDrivers, - "PipelineStats", - "int", - "blockedDrivers"); - to_json_key( + "dataColumns", + p.dataColumns, + "HiveTableLayoutHandle", + "List", + "dataColumns"); + from_json_key( j, - "completedDrivers", - p.completedDrivers, - "PipelineStats", - "int", - "completedDrivers"); - to_json_key( + "tableParameters", + p.tableParameters, + "HiveTableLayoutHandle", + "Map", + "tableParameters"); + from_json_key( j, - "userMemoryReservationInBytes", - p.userMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "userMemoryReservationInBytes"); - to_json_key( + "domainPredicate", + p.domainPredicate, + "HiveTableLayoutHandle", + "TupleDomain", + "domainPredicate"); + from_json_key( j, - "revocableMemoryReservationInBytes", - p.revocableMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "revocableMemoryReservationInBytes"); - to_json_key( + "remainingPredicate", + p.remainingPredicate, + "HiveTableLayoutHandle", + "RowExpression", + "remainingPredicate"); + from_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "systemMemoryReservationInBytes"); - to_json_key( + "predicateColumns", + p.predicateColumns, + "HiveTableLayoutHandle", + "Map", + "predicateColumns"); + from_json_key( j, - "queuedTime", - p.queuedTime, - "PipelineStats", - "DistributionSnapshot", - "queuedTime"); - to_json_key( + "partitionColumnPredicate", + p.partitionColumnPredicate, + "HiveTableLayoutHandle", + "TupleDomain>", + "partitionColumnPredicate"); + from_json_key( j, - "elapsedTime", - p.elapsedTime, - "PipelineStats", - "DistributionSnapshot", - "elapsedTime"); - to_json_key( + "bucketHandle", + p.bucketHandle, + "HiveTableLayoutHandle", + "HiveBucketHandle", + "bucketHandle"); + from_json_key( j, - "totalScheduledTimeInNanos", - p.totalScheduledTimeInNanos, - "PipelineStats", - "int64_t", - "totalScheduledTimeInNanos"); - to_json_key( + "bucketFilter", + p.bucketFilter, + "HiveTableLayoutHandle", + "HiveBucketFilter", + "bucketFilter"); + from_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "PipelineStats", - "int64_t", - "totalCpuTimeInNanos"); - to_json_key( + "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, - "totalBlockedTimeInNanos", - p.totalBlockedTimeInNanos, - "PipelineStats", - "int64_t", - "totalBlockedTimeInNanos"); - to_json_key( + "appendRowNumber", + p.appendRowNumber, + "HiveTableLayoutHandle", + "bool", + "appendRowNumber"); + from_json_key( j, - "fullyBlocked", - p.fullyBlocked, - "PipelineStats", + "footerStatsUnreliable", + p.footerStatsUnreliable, + "HiveTableLayoutHandle", "bool", - "fullyBlocked"); + "footerStatsUnreliable"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +TableScanNode::TableScanNode() noexcept { + _type = ".TableScanNode"; +} + +void to_json(json& j, const TableScanNode& p) { + j = json::object(); + j["@type"] = ".TableScanNode"; + to_json_key(j, "id", p.id, "TableScanNode", "PlanNodeId", "id"); + to_json_key(j, "table", p.table, "TableScanNode", "TableHandle", "table"); to_json_key( j, - "blockedReasons", - p.blockedReasons, - "PipelineStats", - "List", - "blockedReasons"); + "outputVariables", + p.outputVariables, + "TableScanNode", + "List", + "outputVariables"); to_json_key( j, - "totalAllocationInBytes", - p.totalAllocationInBytes, - "PipelineStats", - "int64_t", - "totalAllocationInBytes"); - to_json_key( + "assignments", + p.assignments, + "TableScanNode", + "Map>", + "assignments"); +} + +void from_json(const json& j, TableScanNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "TableScanNode", "PlanNodeId", "id"); + from_json_key(j, "table", p.table, "TableScanNode", "TableHandle", "table"); + from_json_key( j, - "rawInputDataSizeInBytes", - p.rawInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "rawInputDataSizeInBytes"); - to_json_key( + "outputVariables", + p.outputVariables, + "TableScanNode", + "List", + "outputVariables"); + from_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "PipelineStats", - "int64_t", - "rawInputPositions"); + "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 { + +static const std::pair HiveStorageFormat_enum_table[] = + { // NOLINT: cert-err58-cpp + {HiveStorageFormat::ORC, "ORC"}, + {HiveStorageFormat::DWRF, "DWRF"}, + {HiveStorageFormat::PARQUET, "PARQUET"}, + {HiveStorageFormat::AVRO, "AVRO"}, + {HiveStorageFormat::RCBINARY, "RCBINARY"}, + {HiveStorageFormat::RCTEXT, "RCTEXT"}, + {HiveStorageFormat::SEQUENCEFILE, "SEQUENCEFILE"}, + {HiveStorageFormat::JSON, "JSON"}, + {HiveStorageFormat::TEXTFILE, "TEXTFILE"}, + {HiveStorageFormat::CSV, "CSV"}, + {HiveStorageFormat::PAGEFILE, "PAGEFILE"}}; + +void 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 +/* + * 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, - "processedInputDataSizeInBytes", - p.processedInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "processedInputDataSizeInBytes"); - to_json_key( + "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, - "processedInputPositions", - p.processedInputPositions, - "PipelineStats", - "int64_t", - "processedInputPositions"); + "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() + +// 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, - "outputDataSizeInBytes", - p.outputDataSizeInBytes, - "PipelineStats", - "int64_t", - "outputDataSizeInBytes"); + j, "targetPath", p.targetPath, "LocationHandle", "String", "targetPath"); to_json_key( - j, - "outputPositions", - p.outputPositions, - "PipelineStats", - "int64_t", - "outputPositions"); + j, "writePath", p.writePath, "LocationHandle", "String", "writePath"); to_json_key( - j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "PipelineStats", - "int64_t", - "physicalWrittenDataSizeInBytes"); + j, "tempPath", p.tempPath, "LocationHandle", "String", "tempPath"); to_json_key( - j, - "operatorSummaries", - p.operatorSummaries, - "PipelineStats", - "List", - "operatorSummaries"); + j, "tableType", p.tableType, "LocationHandle", "TableType", "tableType"); to_json_key( - j, "drivers", p.drivers, "PipelineStats", "List", "drivers"); + j, "writeMode", p.writeMode, "LocationHandle", "WriteMode", "writeMode"); } -void from_json(const json& j, PipelineStats& p) { - from_json_key( - j, "pipelineId", p.pipelineId, "PipelineStats", "int", "pipelineId"); - from_json_key( - j, - "firstStartTime", - p.firstStartTime, - "PipelineStats", - "DateTime", - "firstStartTime"); - from_json_key( - j, - "lastStartTime", - p.lastStartTime, - "PipelineStats", - "DateTime", - "lastStartTime"); +void from_json(const json& j, LocationHandle& p) { from_json_key( - j, - "lastEndTime", - p.lastEndTime, - "PipelineStats", - "DateTime", - "lastEndTime"); + j, "targetPath", p.targetPath, "LocationHandle", "String", "targetPath"); from_json_key( - j, - "inputPipeline", - p.inputPipeline, - "PipelineStats", - "bool", - "inputPipeline"); + j, "writePath", p.writePath, "LocationHandle", "String", "writePath"); from_json_key( - j, - "outputPipeline", - p.outputPipeline, - "PipelineStats", - "bool", - "outputPipeline"); + j, "tempPath", p.tempPath, "LocationHandle", "String", "tempPath"); from_json_key( - j, - "totalDrivers", - p.totalDrivers, - "PipelineStats", - "int", - "totalDrivers"); + 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 { +// 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, - "queuedDrivers", - p.queuedDrivers, - "PipelineStats", - "int", - "queuedDrivers"); - from_json_key( + "schemaName", + p.schemaName, + "HiveOutputTableHandle", + "String", + "schemaName"); + to_json_key( j, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "PipelineStats", - "int", - "queuedPartitionedDrivers"); - from_json_key( + "tableName", + p.tableName, + "HiveOutputTableHandle", + "String", + "tableName"); + to_json_key( j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "PipelineStats", - "int64_t", - "queuedPartitionedSplitsWeight"); - from_json_key( + "inputColumns", + p.inputColumns, + "HiveOutputTableHandle", + "List", + "inputColumns"); + to_json_key( j, - "runningDrivers", - p.runningDrivers, - "PipelineStats", - "int", - "runningDrivers"); - from_json_key( + "pageSinkMetadata", + p.pageSinkMetadata, + "HiveOutputTableHandle", + "HivePageSinkMetadata", + "pageSinkMetadata"); + to_json_key( j, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "PipelineStats", - "int", - "runningPartitionedDrivers"); - from_json_key( + "locationHandle", + p.locationHandle, + "HiveOutputTableHandle", + "LocationHandle", + "locationHandle"); + to_json_key( j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "PipelineStats", - "int64_t", - "runningPartitionedSplitsWeight"); - from_json_key( + "tableStorageFormat", + p.tableStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "tableStorageFormat"); + to_json_key( j, - "blockedDrivers", - p.blockedDrivers, - "PipelineStats", - "int", - "blockedDrivers"); - from_json_key( + "partitionStorageFormat", + p.partitionStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "partitionStorageFormat"); + to_json_key( j, - "completedDrivers", - p.completedDrivers, - "PipelineStats", - "int", - "completedDrivers"); - from_json_key( + "actualStorageFormat", + p.actualStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "actualStorageFormat"); + to_json_key( j, - "userMemoryReservationInBytes", - p.userMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "userMemoryReservationInBytes"); - from_json_key( + "compressionCodec", + p.compressionCodec, + "HiveOutputTableHandle", + "HiveCompressionCodec", + "compressionCodec"); + to_json_key( j, - "revocableMemoryReservationInBytes", - p.revocableMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "revocableMemoryReservationInBytes"); - from_json_key( + "partitionedBy", + p.partitionedBy, + "HiveOutputTableHandle", + "List", + "partitionedBy"); + to_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "systemMemoryReservationInBytes"); - from_json_key( + "bucketProperty", + p.bucketProperty, + "HiveOutputTableHandle", + "HiveBucketProperty", + "bucketProperty"); + to_json_key( j, - "queuedTime", - p.queuedTime, - "PipelineStats", - "DistributionSnapshot", - "queuedTime"); - from_json_key( + "preferredOrderingColumns", + p.preferredOrderingColumns, + "HiveOutputTableHandle", + "List", + "preferredOrderingColumns"); + to_json_key( j, - "elapsedTime", - p.elapsedTime, - "PipelineStats", - "DistributionSnapshot", - "elapsedTime"); - from_json_key( + "tableOwner", + p.tableOwner, + "HiveOutputTableHandle", + "String", + "tableOwner"); + to_json_key( j, - "totalScheduledTimeInNanos", - p.totalScheduledTimeInNanos, - "PipelineStats", - "int64_t", - "totalScheduledTimeInNanos"); - from_json_key( + "additionalTableParameters", + p.additionalTableParameters, + "HiveOutputTableHandle", + "Map", + "additionalTableParameters"); + to_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "PipelineStats", - "int64_t", - "totalCpuTimeInNanos"); + "encryptionInformation", + p.encryptionInformation, + "HiveOutputTableHandle", + "EncryptionInformation", + "encryptionInformation"); +} + +void from_json(const json& j, HiveOutputTableHandle& p) { + p._type = j["@type"]; from_json_key( j, - "totalBlockedTimeInNanos", - p.totalBlockedTimeInNanos, - "PipelineStats", - "int64_t", - "totalBlockedTimeInNanos"); + "schemaName", + p.schemaName, + "HiveOutputTableHandle", + "String", + "schemaName"); from_json_key( j, - "fullyBlocked", - p.fullyBlocked, - "PipelineStats", - "bool", - "fullyBlocked"); + "tableName", + p.tableName, + "HiveOutputTableHandle", + "String", + "tableName"); from_json_key( j, - "blockedReasons", - p.blockedReasons, - "PipelineStats", - "List", - "blockedReasons"); + "inputColumns", + p.inputColumns, + "HiveOutputTableHandle", + "List", + "inputColumns"); from_json_key( j, - "totalAllocationInBytes", - p.totalAllocationInBytes, - "PipelineStats", - "int64_t", - "totalAllocationInBytes"); + "pageSinkMetadata", + p.pageSinkMetadata, + "HiveOutputTableHandle", + "HivePageSinkMetadata", + "pageSinkMetadata"); from_json_key( j, - "rawInputDataSizeInBytes", - p.rawInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "rawInputDataSizeInBytes"); + "locationHandle", + p.locationHandle, + "HiveOutputTableHandle", + "LocationHandle", + "locationHandle"); from_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "PipelineStats", - "int64_t", - "rawInputPositions"); + "tableStorageFormat", + p.tableStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "tableStorageFormat"); from_json_key( j, - "processedInputDataSizeInBytes", - p.processedInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "processedInputDataSizeInBytes"); + "partitionStorageFormat", + p.partitionStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "partitionStorageFormat"); from_json_key( j, - "processedInputPositions", - p.processedInputPositions, - "PipelineStats", - "int64_t", - "processedInputPositions"); + "actualStorageFormat", + p.actualStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "actualStorageFormat"); from_json_key( j, - "outputDataSizeInBytes", - p.outputDataSizeInBytes, - "PipelineStats", - "int64_t", - "outputDataSizeInBytes"); + "compressionCodec", + p.compressionCodec, + "HiveOutputTableHandle", + "HiveCompressionCodec", + "compressionCodec"); from_json_key( j, - "outputPositions", - p.outputPositions, - "PipelineStats", - "int64_t", - "outputPositions"); + "partitionedBy", + p.partitionedBy, + "HiveOutputTableHandle", + "List", + "partitionedBy"); from_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "PipelineStats", - "int64_t", - "physicalWrittenDataSizeInBytes"); + "bucketProperty", + p.bucketProperty, + "HiveOutputTableHandle", + "HiveBucketProperty", + "bucketProperty"); from_json_key( j, - "operatorSummaries", - p.operatorSummaries, - "PipelineStats", - "List", - "operatorSummaries"); + "preferredOrderingColumns", + p.preferredOrderingColumns, + "HiveOutputTableHandle", + "List", + "preferredOrderingColumns"); from_json_key( - j, "drivers", p.drivers, "PipelineStats", "List", "drivers"); -} -} // 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"; - to_json_key( - j, "handle", p.handle, "CreateHandle", "OutputTableHandle", "handle"); - to_json_key( j, - "schemaTableName", - p.schemaTableName, - "CreateHandle", - "SchemaTableName", - "schemaTableName"); -} - -void from_json(const json& j, CreateHandle& p) { - p._type = j["@type"]; - from_json_key( - j, "handle", p.handle, "CreateHandle", "OutputTableHandle", "handle"); + "tableOwner", + p.tableOwner, + "HiveOutputTableHandle", + "String", + "tableOwner"); from_json_key( j, - "schemaTableName", - p.schemaTableName, - "CreateHandle", - "SchemaTableName", - "schemaTableName"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -InsertHandle::InsertHandle() noexcept { - _type = "InsertHandle"; -} - -void to_json(json& j, const InsertHandle& p) { - j = json::object(); - j["@type"] = "InsertHandle"; - to_json_key( - j, "handle", p.handle, "InsertHandle", "InsertTableHandle", "handle"); - to_json_key( - j, - "schemaTableName", - p.schemaTableName, - "InsertHandle", - "SchemaTableName", - "schemaTableName"); -} - -void from_json(const json& j, InsertHandle& p) { - p._type = j["@type"]; - from_json_key( - j, "handle", p.handle, "InsertHandle", "InsertTableHandle", "handle"); + "additionalTableParameters", + p.additionalTableParameters, + "HiveOutputTableHandle", + "Map", + "additionalTableParameters"); from_json_key( j, - "schemaTableName", - p.schemaTableName, - "InsertHandle", - "SchemaTableName", - "schemaTableName"); + "encryptionInformation", + p.encryptionInformation, + "HiveOutputTableHandle", + "EncryptionInformation", + "encryptionInformation"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -ValuesNode::ValuesNode() noexcept { - _type = ".ValuesNode"; +MergeJoinNode::MergeJoinNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.MergeJoinNode"; } -void to_json(json& j, const ValuesNode& p) { +void to_json(json& j, const MergeJoinNode& p) { j = json::object(); - j["@type"] = ".ValuesNode"; + j["@type"] = "com.facebook.presto.sql.planner.plan.MergeJoinNode"; + to_json_key(j, "id", p.id, "MergeJoinNode", "PlanNodeId", "id"); + to_json_key(j, "type", p.type, "MergeJoinNode", "JoinNode.Type", "type"); + to_json_key(j, "left", p.left, "MergeJoinNode", "PlanNode", "left"); + to_json_key(j, "right", p.right, "MergeJoinNode", "PlanNode", "right"); to_json_key( - j, "location", p.location, "ValuesNode", "SourceLocation", "location"); - to_json_key(j, "id", p.id, "ValuesNode", "PlanNodeId", "id"); + j, + "criteria", + p.criteria, + "MergeJoinNode", + "List", + "criteria"); to_json_key( j, "outputVariables", p.outputVariables, - "ValuesNode", + "MergeJoinNode", "List", "outputVariables"); to_json_key( j, - "rows", - p.rows, - "ValuesNode", - "List>>", - "rows"); + "filter", + p.filter, + "MergeJoinNode", + "std::shared_ptr", + "filter"); to_json_key( j, - "valuesNodeLabel", - p.valuesNodeLabel, - "ValuesNode", - "String", - "valuesNodeLabel"); + "leftHashVariable", + p.leftHashVariable, + "MergeJoinNode", + "VariableReferenceExpression", + "leftHashVariable"); + to_json_key( + j, + "rightHashVariable", + p.rightHashVariable, + "MergeJoinNode", + "VariableReferenceExpression", + "rightHashVariable"); } -void from_json(const json& j, ValuesNode& 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", "JoinNode.Type", "type"); + from_json_key(j, "left", p.left, "MergeJoinNode", "PlanNode", "left"); + from_json_key(j, "right", p.right, "MergeJoinNode", "PlanNode", "right"); from_json_key( - j, "location", p.location, "ValuesNode", "SourceLocation", "location"); - from_json_key(j, "id", p.id, "ValuesNode", "PlanNodeId", "id"); + j, + "criteria", + p.criteria, + "MergeJoinNode", + "List", + "criteria"); from_json_key( j, "outputVariables", p.outputVariables, - "ValuesNode", + "MergeJoinNode", "List", "outputVariables"); from_json_key( j, - "rows", - p.rows, - "ValuesNode", - "List>>", - "rows"); + "filter", + p.filter, + "MergeJoinNode", + "std::shared_ptr", + "filter"); from_json_key( j, - "valuesNodeLabel", - p.valuesNodeLabel, - "ValuesNode", - "String", - "valuesNodeLabel"); + "leftHashVariable", + p.leftHashVariable, + "MergeJoinNode", + "VariableReferenceExpression", + "leftHashVariable"); + from_json_key( + j, + "rightHashVariable", + p.rightHashVariable, + "MergeJoinNode", + "VariableReferenceExpression", + "rightHashVariable"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -AllOrNoneValueSet::AllOrNoneValueSet() noexcept { - _type = "allOrNone"; +SortNode::SortNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.SortNode"; } -void to_json(json& j, const AllOrNoneValueSet& p) { +void to_json(json& j, const SortNode& p) { j = json::object(); - j["@type"] = "allOrNone"; - to_json_key(j, "type", p.type, "AllOrNoneValueSet", "Type", "type"); - to_json_key(j, "all", p.all, "AllOrNoneValueSet", "bool", "all"); + j["@type"] = "com.facebook.presto.sql.planner.plan.SortNode"; + to_json_key(j, "id", p.id, "SortNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "SortNode", "PlanNode", "source"); + to_json_key( + j, + "orderingScheme", + p.orderingScheme, + "SortNode", + "OrderingScheme", + "orderingScheme"); + to_json_key(j, "isPartial", p.isPartial, "SortNode", "bool", "isPartial"); } -void from_json(const json& j, AllOrNoneValueSet& p) { +void from_json(const json& j, SortNode& p) { p._type = j["@type"]; - from_json_key(j, "type", p.type, "AllOrNoneValueSet", "Type", "type"); - from_json_key(j, "all", p.all, "AllOrNoneValueSet", "bool", "all"); + from_json_key(j, "id", p.id, "SortNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "SortNode", "PlanNode", "source"); + from_json_key( + j, + "orderingScheme", + p.orderingScheme, + "SortNode", + "OrderingScheme", + "orderingScheme"); + from_json_key(j, "isPartial", p.isPartial, "SortNode", "bool", "isPartial"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +ConstantExpression::ConstantExpression() noexcept { + _type = "constant"; +} -void to_json(json& j, const TaskStats& p) { +void to_json(json& j, const ConstantExpression& p) { j = json::object(); - to_json_key( - j, "createTime", p.createTime, "TaskStats", "DateTime", "createTime"); - to_json_key( - j, - "firstStartTime", - p.firstStartTime, - "TaskStats", - "DateTime", - "firstStartTime"); - to_json_key( - j, - "lastStartTime", - p.lastStartTime, - "TaskStats", - "DateTime", - "lastStartTime"); - to_json_key( - j, "lastEndTime", p.lastEndTime, "TaskStats", "DateTime", "lastEndTime"); - to_json_key(j, "endTime", p.endTime, "TaskStats", "DateTime", "endTime"); + j["@type"] = "constant"; to_json_key( j, - "elapsedTimeInNanos", - p.elapsedTimeInNanos, - "TaskStats", - "int64_t", - "elapsedTimeInNanos"); - to_json_key( + "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, - "queuedTimeInNanos", - p.queuedTimeInNanos, - "TaskStats", - "int64_t", - "queuedTimeInNanos"); - to_json_key( - j, "totalDrivers", p.totalDrivers, "TaskStats", "int", "totalDrivers"); - to_json_key( - j, "queuedDrivers", p.queuedDrivers, "TaskStats", "int", "queuedDrivers"); + "valueBlock", + p.valueBlock, + "ConstantExpression", + "Block", + "valueBlock"); + from_json_key(j, "type", p.type, "ConstantExpression", "Type", "type"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair 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 PlanCostEstimate& p) { + j = json::object(); + to_json_key(j, "cpuCost", p.cpuCost, "PlanCostEstimate", "double", "cpuCost"); to_json_key( - j, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "TaskStats", - "int", - "queuedPartitionedDrivers"); + j, "maxMemory", p.maxMemory, "PlanCostEstimate", "double", "maxMemory"); to_json_key( j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "TaskStats", - "int64_t", - "queuedPartitionedSplitsWeight"); + "maxMemoryWhenOutputting", + p.maxMemoryWhenOutputting, + "PlanCostEstimate", + "double", + "maxMemoryWhenOutputting"); to_json_key( j, - "runningDrivers", - p.runningDrivers, - "TaskStats", - "int", - "runningDrivers"); - to_json_key( + "networkCost", + p.networkCost, + "PlanCostEstimate", + "double", + "networkCost"); +} + +void from_json(const json& j, PlanCostEstimate& p) { + from_json_key( + j, "cpuCost", p.cpuCost, "PlanCostEstimate", "double", "cpuCost"); + from_json_key( + j, "maxMemory", p.maxMemory, "PlanCostEstimate", "double", "maxMemory"); + from_json_key( j, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "TaskStats", - "int", - "runningPartitionedDrivers"); - to_json_key( + "maxMemoryWhenOutputting", + p.maxMemoryWhenOutputting, + "PlanCostEstimate", + "double", + "maxMemoryWhenOutputting"); + from_json_key( j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "TaskStats", - "int64_t", - "runningPartitionedSplitsWeight"); + "networkCost", + p.networkCost, + "PlanCostEstimate", + "double", + "networkCost"); +} +} // 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, - "blockedDrivers", - p.blockedDrivers, - "TaskStats", - "int", - "blockedDrivers"); + j, "table", p.table, "TpchTableLayoutHandle", "TpchTableHandle", "table"); to_json_key( j, - "completedDrivers", - p.completedDrivers, - "TaskStats", - "int", - "completedDrivers"); - to_json_key( + "predicate", + p.predicate, + "TpchTableLayoutHandle", + "TupleDomain>", + "predicate"); +} + +void from_json(const json& j, TpchTableLayoutHandle& p) { + p._type = j["@type"]; + from_json_key( + j, "table", p.table, "TpchTableLayoutHandle", "TpchTableHandle", "table"); + from_json_key( j, - "cumulativeUserMemory", - p.cumulativeUserMemory, - "TaskStats", - "double", - "cumulativeUserMemory"); + "predicate", + p.predicate, + "TpchTableLayoutHandle", + "TupleDomain>", + "predicate"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const RefreshMaterializedViewHandle& p) { + j = json::object(); to_json_key( j, - "cumulativeTotalMemory", - p.cumulativeTotalMemory, - "TaskStats", - "double", - "cumulativeTotalMemory"); + "handle", + p.handle, + "RefreshMaterializedViewHandle", + "InsertTableHandle", + "handle"); to_json_key( j, - "userMemoryReservationInBytes", - p.userMemoryReservationInBytes, - "TaskStats", - "int64_t", - "userMemoryReservationInBytes"); - to_json_key( + "schemaTableName", + p.schemaTableName, + "RefreshMaterializedViewHandle", + "SchemaTableName", + "schemaTableName"); +} + +void from_json(const json& j, RefreshMaterializedViewHandle& p) { + from_json_key( j, - "revocableMemoryReservationInBytes", - p.revocableMemoryReservationInBytes, - "TaskStats", - "int64_t", - "revocableMemoryReservationInBytes"); - to_json_key( + "handle", + p.handle, + "RefreshMaterializedViewHandle", + "InsertTableHandle", + "handle"); + from_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "TaskStats", - "int64_t", - "systemMemoryReservationInBytes"); + "schemaTableName", + p.schemaTableName, + "RefreshMaterializedViewHandle", + "SchemaTableName", + "schemaTableName"); +} +} // 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, - "peakTotalMemoryInBytes", - p.peakTotalMemoryInBytes, - "TaskStats", - "int64_t", - "peakTotalMemoryInBytes"); - to_json_key( + "connectorId", + p.connectorId, + "EmptySplit", + "ConnectorId", + "connectorId"); +} + +void from_json(const json& j, EmptySplit& p) { + p._type = j["@type"]; + from_json_key( j, - "peakUserMemoryInBytes", - p.peakUserMemoryInBytes, - "TaskStats", - "int64_t", - "peakUserMemoryInBytes"); + "connectorId", + p.connectorId, + "EmptySplit", + "ConnectorId", + "connectorId"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const StatsAndCosts& p) { + j = json::object(); to_json_key( j, - "peakNodeTotalMemoryInBytes", - p.peakNodeTotalMemoryInBytes, - "TaskStats", - "int64_t", - "peakNodeTotalMemoryInBytes"); + "stats", + p.stats, + "StatsAndCosts", + "Map", + "stats"); to_json_key( j, - "totalScheduledTimeInNanos", - p.totalScheduledTimeInNanos, - "TaskStats", - "int64_t", - "totalScheduledTimeInNanos"); - to_json_key( + "costs", + p.costs, + "StatsAndCosts", + "Map", + "costs"); +} + +void from_json(const json& j, StatsAndCosts& p) { + from_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "TaskStats", - "int64_t", - "totalCpuTimeInNanos"); - to_json_key( + "stats", + p.stats, + "StatsAndCosts", + "Map", + "stats"); + from_json_key( j, - "totalBlockedTimeInNanos", - p.totalBlockedTimeInNanos, - "TaskStats", - "int64_t", - "totalBlockedTimeInNanos"); - to_json_key( - j, "fullyBlocked", p.fullyBlocked, "TaskStats", "bool", "fullyBlocked"); + "costs", + p.costs, + "StatsAndCosts", + "Map", + "costs"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +HiveInsertTableHandle::HiveInsertTableHandle() noexcept { + _type = "hive"; +} + +void to_json(json& j, const HiveInsertTableHandle& p) { + j = json::object(); + j["@type"] = "hive"; to_json_key( j, - "blockedReasons", - p.blockedReasons, - "TaskStats", - "List", - "blockedReasons"); + "schemaName", + p.schemaName, + "HiveInsertTableHandle", + "String", + "schemaName"); to_json_key( j, - "totalAllocationInBytes", - p.totalAllocationInBytes, - "TaskStats", - "int64_t", - "totalAllocationInBytes"); + "tableName", + p.tableName, + "HiveInsertTableHandle", + "String", + "tableName"); to_json_key( j, - "rawInputDataSizeInBytes", - p.rawInputDataSizeInBytes, - "TaskStats", - "int64_t", - "rawInputDataSizeInBytes"); + "inputColumns", + p.inputColumns, + "HiveInsertTableHandle", + "List", + "inputColumns"); to_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "TaskStats", - "int64_t", - "rawInputPositions"); + "pageSinkMetadata", + p.pageSinkMetadata, + "HiveInsertTableHandle", + "HivePageSinkMetadata", + "pageSinkMetadata"); to_json_key( j, - "processedInputDataSizeInBytes", - p.processedInputDataSizeInBytes, - "TaskStats", - "int64_t", - "processedInputDataSizeInBytes"); + "locationHandle", + p.locationHandle, + "HiveInsertTableHandle", + "LocationHandle", + "locationHandle"); to_json_key( j, - "processedInputPositions", - p.processedInputPositions, - "TaskStats", - "int64_t", - "processedInputPositions"); + "bucketProperty", + p.bucketProperty, + "HiveInsertTableHandle", + "HiveBucketProperty", + "bucketProperty"); to_json_key( j, - "outputDataSizeInBytes", - p.outputDataSizeInBytes, - "TaskStats", - "int64_t", - "outputDataSizeInBytes"); + "preferredOrderingColumns", + p.preferredOrderingColumns, + "HiveInsertTableHandle", + "List", + "preferredOrderingColumns"); to_json_key( j, - "outputPositions", - p.outputPositions, - "TaskStats", - "int64_t", - "outputPositions"); + "tableStorageFormat", + p.tableStorageFormat, + "HiveInsertTableHandle", + "HiveStorageFormat", + "tableStorageFormat"); to_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "TaskStats", - "int64_t", - "physicalWrittenDataSizeInBytes"); - to_json_key( - j, "fullGcCount", p.fullGcCount, "TaskStats", "int", "fullGcCount"); + "partitionStorageFormat", + p.partitionStorageFormat, + "HiveInsertTableHandle", + "HiveStorageFormat", + "partitionStorageFormat"); to_json_key( j, - "fullGcTimeInMillis", - p.fullGcTimeInMillis, - "TaskStats", - "int64_t", - "fullGcTimeInMillis"); + "actualStorageFormat", + p.actualStorageFormat, + "HiveInsertTableHandle", + "HiveStorageFormat", + "actualStorageFormat"); to_json_key( j, - "pipelines", - p.pipelines, - "TaskStats", - "List", - "pipelines"); + "compressionCodec", + p.compressionCodec, + "HiveInsertTableHandle", + "HiveCompressionCodec", + "compressionCodec"); to_json_key( j, - "runtimeStats", - p.runtimeStats, - "TaskStats", - "RuntimeStats", - "runtimeStats"); + "encryptionInformation", + p.encryptionInformation, + "HiveInsertTableHandle", + "EncryptionInformation", + "encryptionInformation"); } -void from_json(const json& j, TaskStats& p) { +void from_json(const json& j, HiveInsertTableHandle& p) { + p._type = j["@type"]; from_json_key( - j, "createTime", p.createTime, "TaskStats", "DateTime", "createTime"); + j, + "schemaName", + p.schemaName, + "HiveInsertTableHandle", + "String", + "schemaName"); from_json_key( j, - "firstStartTime", - p.firstStartTime, - "TaskStats", - "DateTime", - "firstStartTime"); + "tableName", + p.tableName, + "HiveInsertTableHandle", + "String", + "tableName"); from_json_key( j, - "lastStartTime", - p.lastStartTime, - "TaskStats", - "DateTime", - "lastStartTime"); + "inputColumns", + p.inputColumns, + "HiveInsertTableHandle", + "List", + "inputColumns"); from_json_key( - j, "lastEndTime", p.lastEndTime, "TaskStats", "DateTime", "lastEndTime"); - from_json_key(j, "endTime", p.endTime, "TaskStats", "DateTime", "endTime"); + j, + "pageSinkMetadata", + p.pageSinkMetadata, + "HiveInsertTableHandle", + "HivePageSinkMetadata", + "pageSinkMetadata"); from_json_key( j, - "elapsedTimeInNanos", - p.elapsedTimeInNanos, - "TaskStats", - "int64_t", - "elapsedTimeInNanos"); + "locationHandle", + p.locationHandle, + "HiveInsertTableHandle", + "LocationHandle", + "locationHandle"); + from_json_key( + j, + "bucketProperty", + p.bucketProperty, + "HiveInsertTableHandle", + "HiveBucketProperty", + "bucketProperty"); + from_json_key( + j, + "preferredOrderingColumns", + p.preferredOrderingColumns, + "HiveInsertTableHandle", + "List", + "preferredOrderingColumns"); from_json_key( j, - "queuedTimeInNanos", - p.queuedTimeInNanos, - "TaskStats", - "int64_t", - "queuedTimeInNanos"); - from_json_key( - j, "totalDrivers", p.totalDrivers, "TaskStats", "int", "totalDrivers"); - from_json_key( - j, "queuedDrivers", p.queuedDrivers, "TaskStats", "int", "queuedDrivers"); - from_json_key( + "tableStorageFormat", + p.tableStorageFormat, + "HiveInsertTableHandle", + "HiveStorageFormat", + "tableStorageFormat"); + from_json_key( + j, + "partitionStorageFormat", + p.partitionStorageFormat, + "HiveInsertTableHandle", + "HiveStorageFormat", + "partitionStorageFormat"); + from_json_key( + j, + "actualStorageFormat", + p.actualStorageFormat, + "HiveInsertTableHandle", + "HiveStorageFormat", + "actualStorageFormat"); + from_json_key( + j, + "compressionCodec", + p.compressionCodec, + "HiveInsertTableHandle", + "HiveCompressionCodec", + "compressionCodec"); + from_json_key( + j, + "encryptionInformation", + p.encryptionInformation, + "HiveInsertTableHandle", + "EncryptionInformation", + "encryptionInformation"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +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 { +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, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "TaskStats", - "int", - "queuedPartitionedDrivers"); - from_json_key( + "rowNumberVariable", + p.rowNumberVariable, + "RowNumberNode", + "VariableReferenceExpression", + "rowNumberVariable"); + to_json_key( j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "TaskStats", - "int64_t", - "queuedPartitionedSplitsWeight"); - from_json_key( + "maxRowCountPerPartition", + p.maxRowCountPerPartition, + "RowNumberNode", + "Integer", + "maxRowCountPerPartition"); + to_json_key(j, "partial", p.partial, "RowNumberNode", "bool", "partial"); + to_json_key( j, - "runningDrivers", - p.runningDrivers, - "TaskStats", - "int", - "runningDrivers"); + "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, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "TaskStats", - "int", - "runningPartitionedDrivers"); + "partitionBy", + p.partitionBy, + "RowNumberNode", + "List", + "partitionBy"); from_json_key( j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "TaskStats", - "int64_t", - "runningPartitionedSplitsWeight"); + "rowNumberVariable", + p.rowNumberVariable, + "RowNumberNode", + "VariableReferenceExpression", + "rowNumberVariable"); from_json_key( j, - "blockedDrivers", - p.blockedDrivers, - "TaskStats", - "int", - "blockedDrivers"); + "maxRowCountPerPartition", + p.maxRowCountPerPartition, + "RowNumberNode", + "Integer", + "maxRowCountPerPartition"); + from_json_key(j, "partial", p.partial, "RowNumberNode", "bool", "partial"); from_json_key( j, - "completedDrivers", - p.completedDrivers, - "TaskStats", - "int", - "completedDrivers"); - from_json_key( + "hashVariable", + p.hashVariable, + "RowNumberNode", + "VariableReferenceExpression", + "hashVariable"); +} +} // 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, - "cumulativeUserMemory", - p.cumulativeUserMemory, - "TaskStats", - "double", - "cumulativeUserMemory"); + "signature", + p.signature, + "BuiltInFunctionHandle", + "Signature", + "signature"); +} + +void from_json(const json& j, BuiltInFunctionHandle& p) { + p._type = j["@type"]; from_json_key( j, - "cumulativeTotalMemory", - p.cumulativeTotalMemory, - "TaskStats", - "double", - "cumulativeTotalMemory"); - from_json_key( + "signature", + p.signature, + "BuiltInFunctionHandle", + "Signature", + "signature"); +} +} // 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, - "userMemoryReservationInBytes", - p.userMemoryReservationInBytes, - "TaskStats", - "int64_t", - "userMemoryReservationInBytes"); - from_json_key( + "groupingSets", + p.groupingSets, + "GroupIdNode", + "List>", + "groupingSets"); + to_json_key( j, - "revocableMemoryReservationInBytes", - p.revocableMemoryReservationInBytes, - "TaskStats", - "int64_t", - "revocableMemoryReservationInBytes"); - from_json_key( + "groupingColumns", + p.groupingColumns, + "GroupIdNode", + "Map", + "groupingColumns"); + to_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "TaskStats", - "int64_t", - "systemMemoryReservationInBytes"); - from_json_key( + "aggregationArguments", + p.aggregationArguments, + "GroupIdNode", + "List", + "aggregationArguments"); + to_json_key( j, - "peakTotalMemoryInBytes", - p.peakTotalMemoryInBytes, - "TaskStats", - "int64_t", - "peakTotalMemoryInBytes"); + "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, - "peakUserMemoryInBytes", - p.peakUserMemoryInBytes, - "TaskStats", - "int64_t", - "peakUserMemoryInBytes"); + "groupingSets", + p.groupingSets, + "GroupIdNode", + "List>", + "groupingSets"); from_json_key( j, - "peakNodeTotalMemoryInBytes", - p.peakNodeTotalMemoryInBytes, - "TaskStats", - "int64_t", - "peakNodeTotalMemoryInBytes"); + "groupingColumns", + p.groupingColumns, + "GroupIdNode", + "Map", + "groupingColumns"); from_json_key( j, - "totalScheduledTimeInNanos", - p.totalScheduledTimeInNanos, - "TaskStats", - "int64_t", - "totalScheduledTimeInNanos"); + "aggregationArguments", + p.aggregationArguments, + "GroupIdNode", + "List", + "aggregationArguments"); from_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "TaskStats", - "int64_t", - "totalCpuTimeInNanos"); - from_json_key( + "groupIdVariable", + p.groupIdVariable, + "GroupIdNode", + "VariableReferenceExpression", + "groupIdVariable"); +} +} // 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, - "totalBlockedTimeInNanos", - p.totalBlockedTimeInNanos, - "TaskStats", - "int64_t", - "totalBlockedTimeInNanos"); - from_json_key( - j, "fullyBlocked", p.fullyBlocked, "TaskStats", "bool", "fullyBlocked"); - from_json_key( + "distinctVariables", + p.distinctVariables, + "DistinctLimitNode", + "List", + "distinctVariables"); + to_json_key( j, - "blockedReasons", - p.blockedReasons, - "TaskStats", - "List", - "blockedReasons"); - from_json_key( + "hashVariable", + p.hashVariable, + "DistinctLimitNode", + "VariableReferenceExpression", + "hashVariable"); + to_json_key( j, - "totalAllocationInBytes", - p.totalAllocationInBytes, - "TaskStats", - "int64_t", - "totalAllocationInBytes"); + "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, - "rawInputDataSizeInBytes", - p.rawInputDataSizeInBytes, - "TaskStats", - "int64_t", - "rawInputDataSizeInBytes"); + j, "source", p.source, "DistinctLimitNode", "PlanNode", "source"); + from_json_key(j, "limit", p.limit, "DistinctLimitNode", "int64_t", "limit"); from_json_key( - j, - "rawInputPositions", - p.rawInputPositions, - "TaskStats", - "int64_t", - "rawInputPositions"); + j, "partial", p.partial, "DistinctLimitNode", "bool", "partial"); from_json_key( j, - "processedInputDataSizeInBytes", - p.processedInputDataSizeInBytes, - "TaskStats", - "int64_t", - "processedInputDataSizeInBytes"); + "distinctVariables", + p.distinctVariables, + "DistinctLimitNode", + "List", + "distinctVariables"); from_json_key( j, - "processedInputPositions", - p.processedInputPositions, - "TaskStats", - "int64_t", - "processedInputPositions"); + "hashVariable", + p.hashVariable, + "DistinctLimitNode", + "VariableReferenceExpression", + "hashVariable"); from_json_key( j, - "outputDataSizeInBytes", - p.outputDataSizeInBytes, - "TaskStats", - "int64_t", - "outputDataSizeInBytes"); - from_json_key( + "timeoutMillis", + p.timeoutMillis, + "DistinctLimitNode", + "int", + "timeoutMillis"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +MarkDistinctNode::MarkDistinctNode() noexcept { + _type = ".MarkDistinctNode"; +} + +void to_json(json& j, const MarkDistinctNode& p) { + j = json::object(); + j["@type"] = ".MarkDistinctNode"; + to_json_key(j, "id", p.id, "MarkDistinctNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "MarkDistinctNode", "PlanNode", "source"); + to_json_key( j, - "outputPositions", - p.outputPositions, - "TaskStats", - "int64_t", - "outputPositions"); - from_json_key( + "markerVariable", + p.markerVariable, + "MarkDistinctNode", + "VariableReferenceExpression", + "markerVariable"); + to_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "TaskStats", - "int64_t", - "physicalWrittenDataSizeInBytes"); + "distinctVariables", + p.distinctVariables, + "MarkDistinctNode", + "List", + "distinctVariables"); + to_json_key( + j, + "hashVariable", + p.hashVariable, + "MarkDistinctNode", + "VariableReferenceExpression", + "hashVariable"); +} + +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, "fullGcCount", p.fullGcCount, "TaskStats", "int", "fullGcCount"); + j, "source", p.source, "MarkDistinctNode", "PlanNode", "source"); from_json_key( j, - "fullGcTimeInMillis", - p.fullGcTimeInMillis, - "TaskStats", - "int64_t", - "fullGcTimeInMillis"); + "markerVariable", + p.markerVariable, + "MarkDistinctNode", + "VariableReferenceExpression", + "markerVariable"); from_json_key( j, - "pipelines", - p.pipelines, - "TaskStats", - "List", - "pipelines"); + "distinctVariables", + p.distinctVariables, + "MarkDistinctNode", + "List", + "distinctVariables"); from_json_key( j, - "runtimeStats", - p.runtimeStats, - "TaskStats", - "RuntimeStats", - "runtimeStats"); + "hashVariable", + p.hashVariable, + "MarkDistinctNode", + "VariableReferenceExpression", + "hashVariable"); } } // 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"); + j, "source", p.source, "EnforceSingleRowNode", "PlanNode", "source"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair RuntimeUnit_enum_table[] = + { // NOLINT: cert-err58-cpp + {RuntimeUnit::NONE, "NONE"}, + {RuntimeUnit::NANO, "NANO"}, + {RuntimeUnit::BYTE, "BYTE"}}; +void to_json(json& j, const RuntimeUnit& e) { + static_assert( + std::is_enum::value, "RuntimeUnit must be an enum!"); + const auto* it = std::find_if( + std::begin(RuntimeUnit_enum_table), + std::end(RuntimeUnit_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(RuntimeUnit_enum_table)) + ? it + : std::begin(RuntimeUnit_enum_table)) + ->second; +} +void from_json(const json& j, RuntimeUnit& e) { + static_assert( + std::is_enum::value, "RuntimeUnit must be an enum!"); + const auto* it = std::find_if( + std::begin(RuntimeUnit_enum_table), + std::end(RuntimeUnit_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(RuntimeUnit_enum_table)) + ? it + : std::begin(RuntimeUnit_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const RuntimeMetric& p) { + j = json::object(); + to_json_key(j, "name", p.name, "RuntimeMetric", "String", "name"); + to_json_key(j, "unit", p.unit, "RuntimeMetric", "RuntimeUnit", "unit"); + to_json_key(j, "sum", p.sum, "RuntimeMetric", "int64_t", "sum"); + to_json_key(j, "count", p.count, "RuntimeMetric", "int64_t", "count"); + to_json_key(j, "max", p.max, "RuntimeMetric", "int64_t", "max"); + to_json_key(j, "min", p.min, "RuntimeMetric", "int64_t", "min"); +} + +void from_json(const json& j, RuntimeMetric& p) { + from_json_key(j, "name", p.name, "RuntimeMetric", "String", "name"); + from_json_key(j, "unit", p.unit, "RuntimeMetric", "RuntimeUnit", "unit"); + from_json_key(j, "sum", p.sum, "RuntimeMetric", "int64_t", "sum"); + from_json_key(j, "count", p.count, "RuntimeMetric", "int64_t", "count"); + from_json_key(j, "max", p.max, "RuntimeMetric", "int64_t", "max"); + from_json_key(j, "min", p.min, "RuntimeMetric", "int64_t", "min"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -EquatableValueSet::EquatableValueSet() noexcept { - _type = "equatable"; +TopNRowNumberNode::TopNRowNumberNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.TopNRowNumberNode"; } -void to_json(json& j, const EquatableValueSet& p) { +void to_json(json& j, const TopNRowNumberNode& p) { j = json::object(); - j["@type"] = "equatable"; - to_json_key(j, "type", p.type, "EquatableValueSet", "Type", "type"); + 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, "whiteList", p.whiteList, "EquatableValueSet", "bool", "whiteList"); + j, + "specification", + p.specification, + "TopNRowNumberNode", + "Specification", + "specification"); to_json_key( j, - "entries", - p.entries, - "EquatableValueSet", - "List", - "entries"); + "rowNumberVariable", + p.rowNumberVariable, + "TopNRowNumberNode", + "VariableReferenceExpression", + "rowNumberVariable"); + to_json_key( + j, + "maxRowCountPerPartition", + p.maxRowCountPerPartition, + "TopNRowNumberNode", + "int", + "maxRowCountPerPartition"); + to_json_key(j, "partial", p.partial, "TopNRowNumberNode", "bool", "partial"); + to_json_key( + j, + "hashVariable", + p.hashVariable, + "TopNRowNumberNode", + "VariableReferenceExpression", + "hashVariable"); } -void from_json(const json& j, EquatableValueSet& p) { +void from_json(const json& j, TopNRowNumberNode& p) { p._type = j["@type"]; - from_json_key(j, "type", p.type, "EquatableValueSet", "Type", "type"); + from_json_key(j, "id", p.id, "TopNRowNumberNode", "PlanNodeId", "id"); from_json_key( - j, "whiteList", p.whiteList, "EquatableValueSet", "bool", "whiteList"); + j, "source", p.source, "TopNRowNumberNode", "PlanNode", "source"); from_json_key( j, - "entries", - p.entries, - "EquatableValueSet", - "List", - "entries"); + "specification", + p.specification, + "TopNRowNumberNode", + "Specification", + "specification"); + from_json_key( + j, + "rowNumberVariable", + p.rowNumberVariable, + "TopNRowNumberNode", + "VariableReferenceExpression", + "rowNumberVariable"); + from_json_key( + j, + "maxRowCountPerPartition", + p.maxRowCountPerPartition, + "TopNRowNumberNode", + "int", + "maxRowCountPerPartition"); + from_json_key( + j, "partial", p.partial, "TopNRowNumberNode", "bool", "partial"); + from_json_key( + j, + "hashVariable", + p.hashVariable, + "TopNRowNumberNode", + "VariableReferenceExpression", + "hashVariable"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -BuiltInFunctionHandle::BuiltInFunctionHandle() noexcept { - _type = "$static"; +LambdaDefinitionExpression::LambdaDefinitionExpression() noexcept { + _type = "lambda"; } -void to_json(json& j, const BuiltInFunctionHandle& p) { +void to_json(json& j, const LambdaDefinitionExpression& p) { j = json::object(); - j["@type"] = "$static"; + j["@type"] = "lambda"; to_json_key( j, - "signature", - p.signature, - "BuiltInFunctionHandle", - "Signature", - "signature"); + "sourceLocation", + p.sourceLocation, + "LambdaDefinitionExpression", + "SourceLocation", + "sourceLocation"); + to_json_key( + j, + "argumentTypes", + p.argumentTypes, + "LambdaDefinitionExpression", + "List", + "argumentTypes"); + to_json_key( + j, + "arguments", + p.arguments, + "LambdaDefinitionExpression", + "List", + "arguments"); + to_json_key( + j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); } -void from_json(const json& j, BuiltInFunctionHandle& p) { +void from_json(const json& j, LambdaDefinitionExpression& p) { p._type = j["@type"]; from_json_key( j, - "signature", - p.signature, - "BuiltInFunctionHandle", - "Signature", - "signature"); + "sourceLocation", + p.sourceLocation, + "LambdaDefinitionExpression", + "SourceLocation", + "sourceLocation"); + from_json_key( + j, + "argumentTypes", + p.argumentTypes, + "LambdaDefinitionExpression", + "List", + "argumentTypes"); + from_json_key( + j, + "arguments", + p.arguments, + "LambdaDefinitionExpression", + "List", + "arguments"); + from_json_key( + j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); } } // 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 { -DistinctLimitNode::DistinctLimitNode() noexcept { - _type = ".DistinctLimitNode"; -} -void to_json(json& j, const DistinctLimitNode& p) { +void to_json(json& j, const PlanFragment& 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, "id", p.id, "PlanFragment", "PlanFragmentId", "id"); + to_json_key(j, "root", p.root, "PlanFragment", "PlanNode", "root"); to_json_key( j, - "distinctVariables", - p.distinctVariables, - "DistinctLimitNode", + "variables", + p.variables, + "PlanFragment", "List", - "distinctVariables"); + "variables"); to_json_key( j, - "hashVariable", - p.hashVariable, - "DistinctLimitNode", - "VariableReferenceExpression", - "hashVariable"); + "partitioning", + p.partitioning, + "PlanFragment", + "PartitioningHandle", + "partitioning"); to_json_key( j, - "timeoutMillis", - p.timeoutMillis, - "DistinctLimitNode", - "int", - "timeoutMillis"); + "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, DistinctLimitNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "DistinctLimitNode", "PlanNodeId", "id"); +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, "source", p.source, "DistinctLimitNode", "PlanNode", "source"); - from_json_key(j, "limit", p.limit, "DistinctLimitNode", "int64_t", "limit"); + j, + "variables", + p.variables, + "PlanFragment", + "List", + "variables"); from_json_key( - j, "partial", p.partial, "DistinctLimitNode", "bool", "partial"); + j, + "partitioning", + p.partitioning, + "PlanFragment", + "PartitioningHandle", + "partitioning"); + from_json_key( + j, + "tableScanSchedulingOrder", + p.tableScanSchedulingOrder, + "PlanFragment", + "List", + "tableScanSchedulingOrder"); + from_json_key( + j, + "partitioningScheme", + p.partitioningScheme, + "PlanFragment", + "PartitioningScheme", + "partitioningScheme"); from_json_key( j, - "distinctVariables", - p.distinctVariables, - "DistinctLimitNode", - "List", - "distinctVariables"); + "stageExecutionDescriptor", + p.stageExecutionDescriptor, + "PlanFragment", + "StageExecutionDescriptor", + "stageExecutionDescriptor"); from_json_key( j, - "hashVariable", - p.hashVariable, - "DistinctLimitNode", - "VariableReferenceExpression", - "hashVariable"); + "outputTableWriterFragment", + p.outputTableWriterFragment, + "PlanFragment", + "bool", + "outputTableWriterFragment"); from_json_key( j, - "timeoutMillis", - p.timeoutMillis, - "DistinctLimitNode", - "int", - "timeoutMillis"); + "jsonRepresentation", + p.jsonRepresentation, + "PlanFragment", + "String", + "jsonRepresentation"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TpchPartitioningHandle::TpchPartitioningHandle() noexcept { - _type = "tpch"; +HivePartitioningHandle::HivePartitioningHandle() noexcept { + _type = "hive"; } -void to_json(json& j, const TpchPartitioningHandle& p) { +void to_json(json& j, const HivePartitioningHandle& p) { j = json::object(); - j["@type"] = "tpch"; - to_json_key(j, "table", p.table, "TpchPartitioningHandle", "String", "table"); + j["@type"] = "hive"; to_json_key( j, - "totalRows", - p.totalRows, - "TpchPartitioningHandle", - "int64_t", - "totalRows"); -} - -void from_json(const json& j, TpchPartitioningHandle& p) { - p._type = j["@type"]; - from_json_key( - j, "table", p.table, "TpchPartitioningHandle", "String", "table"); - from_json_key( - j, - "totalRows", - p.totalRows, - "TpchPartitioningHandle", - "int64_t", - "totalRows"); -} -} // 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"; + "bucketCount", + p.bucketCount, + "HivePartitioningHandle", + "int", + "bucketCount"); to_json_key( j, - "sourceLocation", - p.sourceLocation, - "LambdaDefinitionExpression", - "SourceLocation", - "sourceLocation"); + "maxCompatibleBucketCount", + p.maxCompatibleBucketCount, + "HivePartitioningHandle", + "int", + "maxCompatibleBucketCount"); to_json_key( j, - "argumentTypes", - p.argumentTypes, - "LambdaDefinitionExpression", - "List", - "argumentTypes"); + "bucketFunctionType", + p.bucketFunctionType, + "HivePartitioningHandle", + "BucketFunctionType", + "bucketFunctionType"); to_json_key( j, - "arguments", - p.arguments, - "LambdaDefinitionExpression", - "List", - "arguments"); + "hiveTypes", + p.hiveTypes, + "HivePartitioningHandle", + "List", + "hiveTypes"); to_json_key( - j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); + j, "types", p.types, "HivePartitioningHandle", "List", "types"); } -void from_json(const json& j, LambdaDefinitionExpression& p) { +void from_json(const json& j, HivePartitioningHandle& p) { p._type = j["@type"]; from_json_key( j, - "sourceLocation", - p.sourceLocation, - "LambdaDefinitionExpression", - "SourceLocation", - "sourceLocation"); + "bucketCount", + p.bucketCount, + "HivePartitioningHandle", + "int", + "bucketCount"); from_json_key( j, - "argumentTypes", - p.argumentTypes, - "LambdaDefinitionExpression", - "List", - "argumentTypes"); + "maxCompatibleBucketCount", + p.maxCompatibleBucketCount, + "HivePartitioningHandle", + "int", + "maxCompatibleBucketCount"); from_json_key( j, - "arguments", - p.arguments, - "LambdaDefinitionExpression", - "List", - "arguments"); + "bucketFunctionType", + p.bucketFunctionType, + "HivePartitioningHandle", + "BucketFunctionType", + "bucketFunctionType"); from_json_key( - j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); + j, + "hiveTypes", + p.hiveTypes, + "HivePartitioningHandle", + "List", + "hiveTypes"); + from_json_key( + j, "types", p.types, "HivePartitioningHandle", "List", "types"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +SortedRangeSet::SortedRangeSet() noexcept { + _type = "sortable"; +} -void to_json(json& j, const TaskInfo& p) { +void to_json(json& j, const SortedRangeSet& 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"); + 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, 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"); +void from_json(const json& j, SortedRangeSet& p) { + p._type = j["@type"]; + from_json_key(j, "type", p.type, "SortedRangeSet", "Type", "type"); from_json_key( + j, "ranges", p.ranges, "SortedRangeSet", "List", "ranges"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +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, - "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"); + "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, - "metadataUpdates", - p.metadataUpdates, - "TaskInfo", - "MetadataUpdates", - "metadataUpdates"); - from_json_key(j, "nodeId", p.nodeId, "TaskInfo", "String", "nodeId"); + "idVariable", + p.idVariable, + "AssignUniqueId", + "VariableReferenceExpression", + "idVariable"); } } // 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 82f9d8c427b91..fd94ee4d2e84b 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.h +++ b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.h @@ -252,11 +252,19 @@ 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); +struct ConnectorTransactionHandle : 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 RowExpression : public JsonEncodedSubclass { @@ -266,24 +274,16 @@ void to_json(json& j, const std::shared_ptr& p); void from_json(const json& j, std::shared_ptr& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ConnectorSplit : public JsonEncodedSubclass {}; -void to_json(json& j, const std::shared_ptr& p); -void from_json(const json& j, std::shared_ptr& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct ConnectorTransactionHandle : 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 FunctionHandle : public JsonEncodedSubclass {}; void to_json(json& j, const std::shared_ptr& p); void from_json(const json& j, std::shared_ptr& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ConnectorTableHandle : public JsonEncodedSubclass {}; -void to_json(json& j, const std::shared_ptr& p); -void from_json(const json& j, std::shared_ptr& p); +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 ConnectorPartitioningHandle : public JsonEncodedSubclass {}; @@ -291,9 +291,23 @@ 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); +struct ConnectorSplit : public JsonEncodedSubclass {}; +void to_json(json& j, const std::shared_ptr& p); +void from_json(const json& j, std::shared_ptr& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct ConnectorInsertTableHandle : public JsonEncodedSubclass {}; +void to_json(json& j, const std::shared_ptr& p); +void from_json(const json& j, std::shared_ptr& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct ColumnHandle : public JsonEncodedSubclass { + 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 ValueSet : public JsonEncodedSubclass {}; @@ -306,13 +320,9 @@ void to_json(json& j, const std::shared_ptr& p); void from_json(const json& j, std::shared_ptr& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct 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 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 {}; @@ -321,17 +331,35 @@ 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 TpchPartitioningHandle : public ConnectorPartitioningHandle { + String table = {}; + int64_t totalRows = {}; + + TpchPartitioningHandle() noexcept; +}; +void to_json(json& j, const TpchPartitioningHandle& p); +void from_json(const json& j, TpchPartitioningHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ConnectorInsertTableHandle : public JsonEncodedSubclass {}; -void to_json(json& j, const std::shared_ptr& p); -void from_json(const json& j, std::shared_ptr& p); +struct Location { + String location = {}; +}; +void to_json(json& j, const Location& p); +void from_json(const json& j, Location& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct VariableStatsEstimate { + double lowValue = {}; + double highValue = {}; + double nullsFraction = {}; + double averageRowSize = {}; + double distinctValuesCount = {}; +}; +void to_json(json& j, const VariableStatsEstimate& p); +void from_json(const json& j, VariableStatsEstimate& p); } // namespace facebook::presto::protocol - namespace facebook::presto::protocol { struct SourceLocation { int line = {}; @@ -387,138 +415,46 @@ std::string json_map_key( } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct RowNumberNode : public PlanNode { - std::shared_ptr source = {}; - List partitionBy = {}; - VariableReferenceExpression rowNumberVariable = {}; - std::shared_ptr maxRowCountPerPartition = {}; - bool partial = {}; - std::shared_ptr hashVariable = {}; - - RowNumberNode() noexcept; -}; -void to_json(json& j, const RowNumberNode& p); -void from_json(const json& j, RowNumberNode& p); -} // namespace facebook::presto::protocol -/* - * 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. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { - -class ValueEntry { - public: - Type type; - std::shared_ptr block; -}; - -void to_json(json& j, const ValueEntry& p); -void from_json(const json& j, ValueEntry& p); - -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct EquiJoinClause { - VariableReferenceExpression left = {}; - VariableReferenceExpression right = {}; -}; -void to_json(json& j, const EquiJoinClause& p); -void from_json(const json& j, EquiJoinClause& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class JoinNodeType { INNER, LEFT, RIGHT, FULL }; -extern void to_json(json& j, const JoinNodeType& e); -extern void from_json(const json& j, JoinNodeType& e); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class DistributionType { PARTITIONED, REPLICATED }; -extern void to_json(json& j, const DistributionType& e); -extern void from_json(const json& j, DistributionType& e); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct JoinNode : public PlanNode { - JoinNodeType type = {}; - std::shared_ptr left = {}; - std::shared_ptr right = {}; - List criteria = {}; - List outputVariables = {}; - std::shared_ptr> filter = {}; - std::shared_ptr leftHashVariable = {}; - std::shared_ptr rightHashVariable = {}; - std::shared_ptr distributionType = {}; - Map dynamicFilters = {}; - - JoinNode() noexcept; +struct PlanNodeStatsEstimate { + double outputRowCount = {}; + double totalSize = {}; + bool confident = {}; + Map variableStatistics = + {}; }; -void to_json(json& j, const JoinNode& p); -void from_json(const json& j, JoinNode& p); +void to_json(json& j, const PlanNodeStatsEstimate& p); +void from_json(const json& j, PlanNodeStatsEstimate& 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 TpchTableHandle : public ConnectorTableHandle { + String tableName = {}; + double scaleFactor = {}; -struct Lifespan { - bool isgroup = false; - long groupid = 0; - - bool operator<(const Lifespan& o) const { - return groupid < o.groupid; - } + TpchTableHandle() noexcept; }; - -void to_json(json& j, const Lifespan& p); -void from_json(const json& j, Lifespan& 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 SplitContext { - bool cacheable = {}; +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 SplitContext& p); -void from_json(const json& j, SplitContext& p); +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"); @@ -549,35 +485,47 @@ void from_json(const json& j, TpchTransactionHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Split { +struct TableHandle { ConnectorId connectorId = {}; - std::shared_ptr transactionHandle = {}; - std::shared_ptr connectorSplit = {}; - Lifespan lifespan = {}; - SplitContext splitContext = {}; + std::shared_ptr connectorHandle = {}; + std::shared_ptr transaction = {}; + std::shared_ptr connectorTableLayout = {}; }; -void to_json(json& j, const Split& p); -void from_json(const json& j, Split& p); +void to_json(json& j, const TableHandle& p); +void from_json(const json& j, TableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct MemoryAllocation { - String tag = {}; - int64_t allocation = {}; +struct SchemaTableName { + String schema = {}; + String table = {}; }; -void to_json(json& j, const MemoryAllocation& p); -void from_json(const json& j, MemoryAllocation& p); +void to_json(json& j, const SchemaTableName& p); +void from_json(const json& j, SchemaTableName& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct MemoryPoolInfo { - int64_t maxBytes = {}; - int64_t reservedBytes = {}; - int64_t reservedRevocableBytes = {}; - Map queryMemoryReservations = {}; - Map> queryMemoryAllocations = {}; - Map queryMemoryRevocableReservations = {}; +struct DeleteHandle : public ExecutionWriterTarget { + TableHandle handle = {}; + SchemaTableName schemaTableName = {}; + + DeleteHandle() noexcept; }; -void to_json(json& j, const MemoryPoolInfo& p); -void from_json(const json& j, MemoryPoolInfo& 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 RemoteSplit : public ConnectorSplit { + Location location = {}; + TaskId remoteSourceTaskId = {}; + + RemoteSplit() noexcept; +}; +void to_json(json& j, const RemoteSplit& p); +void from_json(const json& j, RemoteSplit& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +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 CallExpression : public RowExpression { @@ -630,9 +578,105 @@ void to_json(json& j, const Aggregation& p); void from_json(const json& j, Aggregation& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class ColumnType { PARTITION_KEY, REGULAR, SYNTHESIZED, AGGREGATED }; -extern void to_json(json& j, const ColumnType& e); -extern void from_json(const json& j, ColumnType& e); +struct 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 +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 = {}; + + 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 PartitioningHandle { + std::shared_ptr connectorId = {}; + std::shared_ptr transactionHandle = {}; + std::shared_ptr connectorHandle = {}; +}; +void to_json(json& j, const PartitioningHandle& p); +void from_json(const json& j, PartitioningHandle& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct Partitioning { + PartitioningHandle handle = {}; + List> arguments = {}; +}; +void to_json(json& j, const Partitioning& p); +void from_json(const json& j, Partitioning& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct PartitioningScheme { + Partitioning partitioning = {}; + List outputLayout = {}; + std::shared_ptr hashColumn = {}; + bool replicateNullsAndAny = {}; + std::shared_ptr> bucketToPartition = {}; +}; +void to_json(json& j, const PartitioningScheme& p); +void from_json(const json& j, PartitioningScheme& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +enum class ExchangeNodeScope { LOCAL, REMOTE_STREAMING, REMOTE_MATERIALIZED }; +extern void to_json(json& j, const ExchangeNodeScope& e); +extern void from_json(const json& j, ExchangeNodeScope& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +enum class 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 ExchangeNode : public PlanNode { + ExchangeNodeType type = {}; + ExchangeNodeScope scope = {}; + PartitioningScheme partitioningScheme = {}; + List> sources = {}; + List> inputs = {}; + bool ensureSourceOrdering = {}; + std::shared_ptr orderingScheme = {}; + + ExchangeNode() noexcept; +}; +void to_json(json& j, const ExchangeNode& p); +void from_json(const json& j, ExchangeNode& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct RemoteSourceNode : public PlanNode { + List sourceFragmentIds = {}; + List outputVariables = {}; + bool ensureSourceOrdering = {}; + std::shared_ptr orderingScheme = {}; + ExchangeNodeType exchangeType = {}; + + RemoteSourceNode() noexcept; +}; +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 SplitContext { + bool cacheable = {}; +}; +void to_json(json& j, const SplitContext& p); +void from_json(const json& j, SplitContext& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -647,48 +691,31 @@ extern void from_json(const json& j, ColumnType& e); * See the License for the specific language governing permissions and * limitations under the License. */ - 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; +struct Lifespan { + bool isgroup = false; + long groupid = 0; - bool operator<(const ColumnHandle& o) const override { - return name < dynamic_cast(o).name; + bool operator<(const Lifespan& o) const { + return groupid < o.groupid; } }; -void to_json(json& j, const HiveColumnHandle& p); -void from_json(const json& j, HiveColumnHandle& p); +void to_json(json& j, const Lifespan& p); +void from_json(const json& j, Lifespan& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct BucketConversion { - int tableBucketCount = {}; - int partitionBucketCount = {}; - List bucketColumnHandles = {}; -}; -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 TpchTableHandle : public ConnectorTableHandle { - String tableName = {}; - double scaleFactor = {}; - - TpchTableHandle() noexcept; +struct Split { + ConnectorId connectorId = {}; + std::shared_ptr transactionHandle = {}; + std::shared_ptr connectorSplit = {}; + Lifespan lifespan = {}; + SplitContext splitContext = {}; }; -void to_json(json& j, const TpchTableHandle& p); -void from_json(const json& j, TpchTableHandle& p); +void to_json(json& j, const Split& p); +void from_json(const json& j, Split& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -719,34 +746,57 @@ void to_json(json& j, const ScheduledSplit& p); void from_json(const json& j, ScheduledSplit& 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 TaskSource { - PlanNodeId planNodeId = {}; - List splits = {}; - List noMoreSplitsForLifespan = {}; - bool noMoreSplits = {}; +struct StatisticAggregations { + std::vector outputVariables = {}; + Map aggregations = {}; + std::vector groupingVariables = {}; }; -void to_json(json& j, const TaskSource& p); -void from_json(const json& j, TaskSource& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class BucketFunctionType { HIVE_COMPATIBLE, PRESTO_NATIVE }; -extern void to_json(json& j, const BucketFunctionType& e); -extern void from_json(const json& j, BucketFunctionType& e); +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 HivePartitioningHandle : public ConnectorPartitioningHandle { - int bucketCount = {}; - std::shared_ptr maxCompatibleBucketCount = {}; - BucketFunctionType bucketFunctionType = {}; - std::shared_ptr> hiveTypes = {}; - std::shared_ptr> types = {}; +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 = {}; - HivePartitioningHandle() noexcept; + TableWriterNode() noexcept; }; -void to_json(json& j, const HivePartitioningHandle& p); -void from_json(const json& j, HivePartitioningHandle& p); -} // namespace facebook::presto::protocol +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 InsertTableHandle { + ConnectorId connectorId = {}; + std::shared_ptr transactionHandle = {}; + std::shared_ptr connectorHandle = {}; +}; +void to_json(json& j, const InsertTableHandle& p); +void from_json(const json& j, InsertTableHandle& p); +} // namespace facebook::presto::protocol namespace facebook::presto::protocol { enum class Order { ASCENDING, DESCENDING }; extern void to_json(json& j, const Order& e); @@ -761,29 +811,83 @@ void to_json(json& j, const SortingColumn& p); void from_json(const json& j, SortingColumn& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class TableType { NEW, EXISTING, TEMPORARY }; -extern void to_json(json& j, const TableType& e); -extern void from_json(const json& j, TableType& e); +enum class BucketFunctionType { HIVE_COMPATIBLE, PRESTO_NATIVE }; +extern void to_json(json& j, const BucketFunctionType& e); +extern void from_json(const json& j, BucketFunctionType& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class WriteMode { - STAGE_AND_MOVE_TO_TARGET_DIRECTORY, - DIRECT_TO_TARGET_NEW_DIRECTORY, - DIRECT_TO_TARGET_EXISTING_DIRECTORY +struct HiveBucketProperty { + List bucketedBy = {}; + int bucketCount = {}; + List sortedBy = {}; + BucketFunctionType bucketFunctionType = {}; + std::shared_ptr> types = {}; }; -extern void to_json(json& j, const WriteMode& e); -extern void from_json(const json& j, WriteMode& e); +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 LocationHandle { - String targetPath = {}; - String writePath = {}; - std::shared_ptr tempPath = {}; - TableType tableType = {}; - WriteMode writeMode = {}; +struct StorageFormat { + String serDe = {}; + String inputFormat = {}; + String outputFormat = {}; }; -void to_json(json& j, const LocationHandle& p); -void from_json(const json& j, LocationHandle& 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 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 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 +/* + * 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 { + +std::ostream& operator<<(std::ostream& os, const DataSize& d); + +void to_json(nlohmann::json& j, const DataSize& p); +void from_json(const nlohmann::json& j, DataSize& p); + +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct CacheQuotaRequirement { + CacheQuotaScope cacheQuotaScope = {}; + std::shared_ptr quota = {}; +}; +void to_json(json& j, const CacheQuotaRequirement& p); +void from_json(const json& j, CacheQuotaRequirement& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct HivePartitionKey { + String name = {}; + std::shared_ptr value = {}; +}; +void to_json(json& j, const HivePartitionKey& p); +void from_json(const json& j, HivePartitionKey& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct DwrfEncryptionMetadata { @@ -803,12 +907,9 @@ 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 SchemaTableName { - String schema = {}; - String table = {}; -}; -void to_json(json& j, const SchemaTableName& p); -void from_json(const json& j, SchemaTableName& 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 /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -826,79 +927,35 @@ void from_json(const json& j, SchemaTableName& p); namespace facebook::presto::protocol { -struct Column { - String name; - String type; +struct HiveColumnHandle : public ColumnHandle { + String name = {}; + HiveType hiveType = {}; + TypeSignature typeSignature = {}; + int hiveColumnIndex = {}; + ColumnType columnType = {}; + std::shared_ptr comment = {}; + List requiredSubfields = {}; + std::shared_ptr partialAggregation = {}; - Column() = default; - explicit Column(const String& str) { - name = str; + HiveColumnHandle() noexcept; + + bool operator<(const ColumnHandle& o) const override { + return name < dynamic_cast(o).name; } }; -void to_json(json& j, const Column& p); -void from_json(const json& j, Column& p); +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 HiveBucketProperty { - List bucketedBy = {}; - int bucketCount = {}; - List sortedBy = {}; - BucketFunctionType bucketFunctionType = {}; - std::shared_ptr> types = {}; -}; -void to_json(json& j, const HiveBucketProperty& p); -void from_json(const json& j, HiveBucketProperty& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct StorageFormat { - String serDe = {}; - String inputFormat = {}; - String outputFormat = {}; -}; -void to_json(json& j, const StorageFormat& p); -void from_json(const json& j, StorageFormat& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct Storage { - StorageFormat storageFormat = {}; - String location = {}; - std::shared_ptr bucketProperty = {}; - bool skewed = {}; - Map serdeParameters = {}; - Map parameters = {}; -}; -void to_json(json& j, const Storage& p); -void from_json(const json& j, Storage& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class PrestoTableType { - MANAGED_TABLE, - EXTERNAL_TABLE, - VIRTUAL_VIEW, - MATERIALIZED_VIEW, - TEMPORARY_TABLE, - OTHER -}; -extern void to_json(json& j, const PrestoTableType& e); -extern void from_json(const json& j, PrestoTableType& e); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct Table { - String databaseName = {}; - String tableName = {}; - String owner = {}; - PrestoTableType tableType = {}; - Storage storage = {}; - List dataColumns = {}; - List partitionColumns = {}; - Map parameters = {}; - std::shared_ptr viewOriginalText = {}; - std::shared_ptr viewExpandedText = {}; +struct BucketConversion { + int tableBucketCount = {}; + int partitionBucketCount = {}; + List bucketColumnHandles = {}; }; -void to_json(json& j, const Table& p); -void from_json(const json& j, Table& p); +void to_json(json& j, const BucketConversion& p); +void from_json(const json& j, BucketConversion& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -914,19 +971,29 @@ void from_json(const json& j, Table& p); * limitations under the License. */ -// dependency Table -// dependency SchemaTableName - namespace facebook::presto::protocol { -struct HivePageSinkMetadata { - SchemaTableName schemaTableName = {}; - std::shared_ptr table = {}; - // TODO Add modifiedPartitions +struct Column { + String name; + String type; + + Column() = default; + explicit Column(const String& str) { + name = str; + } }; -void to_json(json& j, const HivePageSinkMetadata& p); -void from_json(const json& j, HivePageSinkMetadata& 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 TableToPartitionMapping { + std::shared_ptr> tableToPartitionColumns = {}; + Map partitionSchemaDifference = {}; +}; +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"); @@ -944,153 +1011,79 @@ void from_json(const json& j, HivePageSinkMetadata& p); namespace facebook::presto::protocol { -enum class HiveStorageFormat { - ORC, - DWRF, - PARQUET, - AVRO, - RCBINARY, - RCTEXT, - SEQUENCEFILE, - JSON, - TEXTFILE, - CSV, - PAGEFILE -}; - -void to_json(json& j, const HiveStorageFormat& p); -void from_json(const json& j, HiveStorageFormat& p); +using HostAddress = std::string; } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class HiveCompressionCodec { NONE, SNAPPY, GZIP, LZ4, ZSTD }; -extern void to_json(json& j, const HiveCompressionCodec& e); -extern void from_json(const json& j, HiveCompressionCodec& e); +enum class 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 namespace facebook::presto::protocol { -struct HiveOutputTableHandle : public ConnectorOutputTableHandle { - String schemaName = {}; - String tableName = {}; - List inputColumns = {}; - HivePageSinkMetadata pageSinkMetadata = {}; - LocationHandle locationHandle = {}; - HiveStorageFormat tableStorageFormat = {}; - HiveStorageFormat partitionStorageFormat = {}; - HiveStorageFormat actualStorageFormat = {}; - HiveCompressionCodec compressionCodec = {}; - List partitionedBy = {}; - std::shared_ptr bucketProperty = {}; - List preferredOrderingColumns = {}; - String tableOwner = {}; - Map additionalTableParameters = {}; - std::shared_ptr encryptionInformation = {}; - - HiveOutputTableHandle() noexcept; -}; -void to_json(json& j, const HiveOutputTableHandle& p); -void from_json(const json& j, HiveOutputTableHandle& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct HivePartitionKey { - String name = {}; - std::shared_ptr value = {}; -}; -void to_json(json& j, const HivePartitionKey& p); -void from_json(const json& j, HivePartitionKey& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct PlanCostEstimate { - double cpuCost = {}; - double maxMemory = {}; - double maxMemoryWhenOutputting = {}; - double networkCost = {}; -}; -void to_json(json& j, const PlanCostEstimate& p); -void from_json(const json& j, PlanCostEstimate& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct Assignments { - Map> assignments = - {}; -}; -void to_json(json& j, const Assignments& p); -void from_json(const json& j, Assignments& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct 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 GroupIdNode : public PlanNode { - std::shared_ptr source = {}; - List> groupingSets = {}; - Map - groupingColumns = {}; - List aggregationArguments = {}; - VariableReferenceExpression groupIdVariable = {}; - - GroupIdNode() noexcept; +struct HiveFileSplit { + String path = {}; + int64_t start = {}; + int64_t length = {}; + int64_t fileSize = {}; + int64_t fileModifiedTime = {}; + std::shared_ptr extraFileInfo = {}; + Map customSplitInfo = {}; }; -void to_json(json& j, const GroupIdNode& p); -void from_json(const json& j, GroupIdNode& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class LimitNodeStep { PARTIAL, FINAL }; -extern void to_json(json& j, const LimitNodeStep& e); -extern void from_json(const json& j, LimitNodeStep& e); +void to_json(json& j, const HiveFileSplit& p); +void from_json(const json& j, HiveFileSplit& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct LimitNode : public PlanNode { - std::shared_ptr source = {}; - int64_t count = {}; - LimitNodeStep step = {}; +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 = {}; - LimitNode() noexcept; -}; -void to_json(json& j, const LimitNode& p); -void from_json(const json& j, LimitNode& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct 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, + HiveSplit() noexcept; }; -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); +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 Language { - String language = {}; -}; -void to_json(json& j, const Language& p); -void from_json(const json& j, Language& p); +enum class BlockedReason { WAITING_FOR_MEMORY }; +extern void to_json(json& j, const BlockedReason& e); +extern void from_json(const json& j, BlockedReason& e); } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ namespace facebook::presto::protocol { -struct RoutineCharacteristics { - std::shared_ptr language = {}; - std::shared_ptr determinism = {}; - std::shared_ptr nullCallClause = {}; -}; -void to_json(json& j, const RoutineCharacteristics& p); -void from_json(const json& j, RoutineCharacteristics& p); +struct OperatorInfo {}; +void to_json(json& j, const OperatorInfo& p); +void from_json(const json& j, OperatorInfo& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -1105,516 +1098,257 @@ void from_json(const json& j, RoutineCharacteristics& p); * See the License for the specific language governing permissions and * limitations under the License. */ + namespace facebook::presto::protocol { -struct TypeVariableConstraint { - String name = {}; - bool comparableRequired = {}; - bool orderableRequired = {}; - String variadicBound = {}; - bool nonDecimalNumericRequired = {}; - String boundedBy = {}; -}; -void to_json(json& j, const TypeVariableConstraint& p); -void from_json(const json& j, TypeVariableConstraint& p); +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 LongVariableConstraint { - String name = {}; - String expression = {}; +struct OperatorStats { + int stageId = {}; + int stageExecutionId = {}; + int pipelineId = {}; + int operatorId = {}; + PlanNodeId planNodeId = {}; + String operatorType = {}; + int64_t totalDrivers = {}; + int64_t addInputCalls = {}; + Duration addInputWall = {}; + Duration addInputCpu = {}; + DataSize addInputAllocation = {}; + DataSize rawInputDataSize = {}; + int64_t rawInputPositions = {}; + DataSize inputDataSize = {}; + int64_t inputPositions = {}; + double sumSquaredInputPositions = {}; + int64_t getOutputCalls = {}; + Duration getOutputWall = {}; + Duration getOutputCpu = {}; + DataSize getOutputAllocation = {}; + DataSize outputDataSize = {}; + int64_t outputPositions = {}; + DataSize physicalWrittenDataSize = {}; + Duration additionalCpu = {}; + Duration blockedWall = {}; + int64_t finishCalls = {}; + Duration finishWall = {}; + Duration finishCpu = {}; + DataSize finishAllocation = {}; + DataSize userMemoryReservation = {}; + DataSize revocableMemoryReservation = {}; + DataSize systemMemoryReservation = {}; + DataSize peakUserMemoryReservation = {}; + DataSize peakSystemMemoryReservation = {}; + DataSize peakTotalMemoryReservation = {}; + DataSize spilledDataSize = {}; + std::shared_ptr blockedReason = {}; + OperatorInfo info = {}; + RuntimeStats runtimeStats = {}; }; -void to_json(json& j, const LongVariableConstraint& p); -void from_json(const json& j, LongVariableConstraint& 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 { -enum class FunctionKind { SCALAR, AGGREGATE, WINDOW }; -extern void to_json(json& j, const FunctionKind& e); -extern void from_json(const json& j, FunctionKind& e); +struct 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 DriverStats& p); +void from_json(const json& j, DriverStats& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Signature { - QualifiedObjectName name = {}; - FunctionKind kind = {}; - List typeVariableConstraints = {}; - List longVariableConstraints = {}; - TypeSignature returnType = {}; - List argumentTypes = {}; - bool variableArity = {}; -}; -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 SqlInvokedFunction { - List parameters = {}; - String description = {}; - RoutineCharacteristics routineCharacteristics = {}; - String body = {}; - Signature signature = {}; - SqlFunctionId functionId = {}; -}; -void to_json(json& j, const SqlInvokedFunction& p); -void from_json(const json& j, SqlInvokedFunction& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct HiveBucketHandle { - List columns = {}; - int tableBucketCount = {}; - int readBucketCount = {}; -}; -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 VariableStatsEstimate { - double lowValue = {}; - double highValue = {}; - double nullsFraction = {}; - double averageRowSize = {}; - double distinctValuesCount = {}; -}; -void to_json(json& j, const VariableStatsEstimate& p); -void from_json(const json& j, VariableStatsEstimate& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct PlanNodeStatsEstimate { - double outputRowCount = {}; - double totalSize = {}; - bool confident = {}; - Map variableStatistics = - {}; -}; -void to_json(json& j, const PlanNodeStatsEstimate& p); -void from_json(const json& j, PlanNodeStatsEstimate& 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 { -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 PartitioningHandle { - std::shared_ptr connectorId = {}; - std::shared_ptr transactionHandle = {}; - std::shared_ptr connectorHandle = {}; -}; -void to_json(json& j, const PartitioningHandle& p); -void from_json(const json& j, PartitioningHandle& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct Partitioning { - PartitioningHandle handle = {}; - List> arguments = {}; +struct PipelineStats { + int pipelineId = {}; + DateTime firstStartTime = {}; + DateTime lastStartTime = {}; + DateTime lastEndTime = {}; + bool inputPipeline = {}; + bool outputPipeline = {}; + int totalDrivers = {}; + int queuedDrivers = {}; + int queuedPartitionedDrivers = {}; + int64_t queuedPartitionedSplitsWeight = {}; + int runningDrivers = {}; + int runningPartitionedDrivers = {}; + int64_t runningPartitionedSplitsWeight = {}; + int blockedDrivers = {}; + int completedDrivers = {}; + int64_t userMemoryReservationInBytes = {}; + int64_t revocableMemoryReservationInBytes = {}; + int64_t systemMemoryReservationInBytes = {}; + DistributionSnapshot queuedTime = {}; + DistributionSnapshot elapsedTime = {}; + int64_t totalScheduledTimeInNanos = {}; + int64_t totalCpuTimeInNanos = {}; + int64_t totalBlockedTimeInNanos = {}; + bool fullyBlocked = {}; + List blockedReasons = {}; + int64_t totalAllocationInBytes = {}; + int64_t rawInputDataSizeInBytes = {}; + int64_t rawInputPositions = {}; + int64_t processedInputDataSizeInBytes = {}; + int64_t processedInputPositions = {}; + int64_t outputDataSizeInBytes = {}; + int64_t outputPositions = {}; + int64_t physicalWrittenDataSizeInBytes = {}; + List operatorSummaries = {}; + List drivers = {}; }; -void to_json(json& j, const Partitioning& p); -void from_json(const json& j, Partitioning& p); +void to_json(json& j, const PipelineStats& p); +void from_json(const json& j, PipelineStats& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct PartitioningScheme { - Partitioning partitioning = {}; - List outputLayout = {}; - std::shared_ptr hashColumn = {}; - bool replicateNullsAndAny = {}; - std::shared_ptr> bucketToPartition = {}; +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 PartitioningScheme& p); -void from_json(const json& j, PartitioningScheme& 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 ExchangeNode : public PlanNode { - ExchangeNodeType type = {}; - ExchangeNodeScope scope = {}; - PartitioningScheme partitioningScheme = {}; - List> sources = {}; - List> inputs = {}; - bool ensureSourceOrdering = {}; - std::shared_ptr orderingScheme = {}; +struct HiveTransactionHandle : public ConnectorTransactionHandle { + UUID uuid = {}; - ExchangeNode() noexcept; + HiveTransactionHandle() noexcept; }; -void to_json(json& j, const ExchangeNode& p); -void from_json(const json& j, ExchangeNode& p); +void to_json(json& j, const HiveTransactionHandle& p); +void from_json(const json& j, HiveTransactionHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct RemoteSourceNode : public PlanNode { - List sourceFragmentIds = {}; +struct ValuesNode : public PlanNode { + std::shared_ptr location = {}; + 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); -} // 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); +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 GroupingSetDescriptor { - List groupingKeys = {}; - int groupingSetCount = {}; - List globalGroupingSets = {}; +enum class PrestoTableType { + MANAGED_TABLE, + EXTERNAL_TABLE, + VIRTUAL_VIEW, + MATERIALIZED_VIEW, + TEMPORARY_TABLE, + OTHER }; -void to_json(json& j, const GroupingSetDescriptor& p); -void from_json(const json& j, GroupingSetDescriptor& p); +extern void to_json(json& j, const PrestoTableType& e); +extern void from_json(const json& j, PrestoTableType& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct AggregationNode : public PlanNode { - std::shared_ptr source = {}; - Map aggregations = {}; - GroupingSetDescriptor groupingSets = {}; - List preGroupedVariables = {}; - AggregationNodeStep step = {}; - std::shared_ptr hashVariable = {}; - std::shared_ptr groupIdVariable = {}; - - AggregationNode() noexcept; +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 AggregationNode& p); -void from_json(const json& j, AggregationNode& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class Bound { BELOW, EXACTLY, ABOVE }; -extern void to_json(json& j, const Bound& e); -extern void from_json(const json& j, Bound& e); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct Marker { - Type type = {}; - std::shared_ptr valueBlock = {}; - Bound bound = {}; -}; -void to_json(json& j, const Marker& p); -void from_json(const json& j, Marker& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct HiveBucketFilter { - List bucketsToKeep = {}; -}; -void to_json(json& j, const HiveBucketFilter& p); -void from_json(const json& j, HiveBucketFilter& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct 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 HiveTableHandle : public ConnectorTableHandle { - String schemaName = {}; - String tableName = {}; - std::shared_ptr>> analyzePartitionValues = {}; - - HiveTableHandle() noexcept; -}; -void to_json(json& j, const HiveTableHandle& p); -void from_json(const json& j, HiveTableHandle& p); -} // namespace facebook::presto::protocol -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 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 -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); -} // 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); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct ProjectNode : public PlanNode { - std::shared_ptr source = {}; - Assignments assignments = {}; - Locality locality = {}; - - ProjectNode() noexcept; -}; -void to_json(json& j, const ProjectNode& p); -void from_json(const json& j, ProjectNode& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -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 -/* - * 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 { - -std::ostream& operator<<(std::ostream& os, const DataSize& d); - -void to_json(nlohmann::json& j, const DataSize& p); -void from_json(const nlohmann::json& j, DataSize& p); - -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct CacheQuotaRequirement { - CacheQuotaScope cacheQuotaScope = {}; - std::shared_ptr quota = {}; -}; -void to_json(json& j, const CacheQuotaRequirement& p); -void from_json(const json& j, CacheQuotaRequirement& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct SemiJoinNode : public PlanNode { - std::shared_ptr source = {}; - std::shared_ptr filteringSource = {}; - VariableReferenceExpression sourceJoinVariable = {}; - VariableReferenceExpression filteringSourceJoinVariable = {}; - VariableReferenceExpression semiJoinOutput = {}; - std::shared_ptr sourceHashVariable = {}; - std::shared_ptr filteringSourceHashVariable = {}; - std::shared_ptr distributionType = {}; - Map dynamicFilters = {}; - - SemiJoinNode() noexcept; -}; -void to_json(json& j, const SemiJoinNode& p); -void from_json(const json& j, SemiJoinNode& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct Domain { - std::shared_ptr values = {}; - bool nullAllowed = {}; -}; -void to_json(json& j, const Domain& p); -void from_json(const json& j, Domain& p); -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { - -template -struct pointerDerefCompare { - bool operator()(const std::shared_ptr& a, const std::shared_ptr& b) - const { - return *a < *b; - } -}; - -template -struct TupleDomain { - std::shared_ptr> domains; -}; - -template -struct TupleDomain> { - std::shared_ptr, Domain, pointerDerefCompare>> - domains; -}; - -template -struct ColumnDomain { - T column; - Domain domain; // dependency -}; - -} // namespace facebook::presto::protocol - -namespace nlohmann { - -template -struct adl_serializer> { - static void to_json( - json& j, - const facebook::presto::protocol::ColumnDomain& p) { - facebook::presto::protocol::to_json_key( - j, "column", p.column, "ColumnDomain", "T", "column"); - facebook::presto::protocol::to_json_key( - j, "domain", p.domain, "ColumnDomain", "Domain", "domain"); - } - - static void from_json( - const json& j, - facebook::presto::protocol::ColumnDomain& p) { - facebook::presto::protocol::from_json_key( - j, "column", p.column, "ColumnDomain", "T", "column"); - facebook::presto::protocol::from_json_key( - j, "domain", p.domain, "ColumnDomain", "Domain", "domain"); - } -}; - -template -struct adl_serializer> { - static void to_json( - json& j, - const facebook::presto::protocol::TupleDomain& tup) { - facebook::presto::protocol::List< - facebook::presto::protocol::ColumnDomain> - list; - if (tup.domains != nullptr) { - for (auto& el : *tup.domains) { - facebook::presto::protocol::ColumnDomain domain; - domain.column = el.first; - domain.domain = el.second; - list.push_back(domain); - } - } - - j["columnDomains"] = list; - } - - static void from_json( - const json& j, - facebook::presto::protocol::TupleDomain& tup) { - if (j.count("columnDomains") != 0U) { - std::shared_ptr> - map = std::make_shared< - std::map>(); - - facebook::presto::protocol::List< - facebook::presto::protocol::ColumnDomain> - list = j.at("columnDomains"); - for (const facebook::presto::protocol::ColumnDomain& value : list) { - map->insert(std::make_pair(T(value.column), value.domain)); - } - tup.domains = map; - } - } -}; - -template -struct adl_serializer< - facebook::presto::protocol::TupleDomain>> { - static void to_json( - json& j, - const facebook::presto::protocol::TupleDomain>& tup) { - facebook::presto::protocol::List< - facebook::presto::protocol::ColumnDomain>> - list; - if (tup.domains != nullptr) { - for (auto& el : *tup.domains) { - facebook::presto::protocol::ColumnDomain> domain; - domain.column = el.first; - domain.domain = el.second; - list.push_back(domain); - } - } - - j["columnDomains"] = list; - } - - static void from_json( - const json& j, - facebook::presto::protocol::TupleDomain>& tup) { - if (j.count("columnDomains") != 0U) { - auto map = std::make_shared, - facebook::presto::protocol::Domain, - facebook::presto::protocol::pointerDerefCompare>>(); - - facebook::presto::protocol::List< - facebook::presto::protocol::ColumnDomain>> - list = j.at("columnDomains"); - for (const facebook::presto::protocol::ColumnDomain>& - value : list) { - map->insert( - std::make_pair(std::shared_ptr(value.column), value.domain)); - } - tup.domains = map; - } - } +void to_json(json& j, const Table& p); +void from_json(const json& j, Table& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct Assignments { + Map> assignments = + {}; }; - -} // namespace nlohmann +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 EmptySplit : public ConnectorSplit { - ConnectorId connectorId = {}; +struct HiveMetadataUpdateHandle : public ConnectorMetadataUpdateHandle { + UUID requestId = {}; + SchemaTableName schemaTableName = {}; + std::shared_ptr partitionName = {}; + std::shared_ptr fileName = {}; - EmptySplit() noexcept; + HiveMetadataUpdateHandle() noexcept; }; -void to_json(json& j, const EmptySplit& p); -void from_json(const json& j, EmptySplit& p); +void to_json(json& j, const HiveMetadataUpdateHandle& p); +void from_json(const json& j, HiveMetadataUpdateHandle& p); } // namespace facebook::presto::protocol -/* - * 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 InsertHandle : public ExecutionWriterTarget { + InsertTableHandle handle = {}; + SchemaTableName schemaTableName = {}; -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); - + InsertHandle() noexcept; +}; +void to_json(json& j, const InsertHandle& p); +void from_json(const json& j, InsertHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct NodeVersion { @@ -1624,15 +1358,40 @@ void to_json(json& j, const NodeVersion& p); void from_json(const json& j, NodeVersion& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ServerInfo { - NodeVersion nodeVersion = {}; - String environment = {}; - bool coordinator = {}; - bool starting = {}; - std::shared_ptr uptime = {}; +enum class JoinNodeType { INNER, LEFT, RIGHT, FULL }; +extern void to_json(json& j, const JoinNodeType& e); +extern void from_json(const json& j, JoinNodeType& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct EquiJoinClause { + VariableReferenceExpression left = {}; + VariableReferenceExpression right = {}; }; -void to_json(json& j, const ServerInfo& p); -void from_json(const json& j, ServerInfo& 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 { +enum class DistributionType { PARTITIONED, REPLICATED }; +extern void to_json(json& j, const DistributionType& e); +extern void from_json(const json& j, DistributionType& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct JoinNode : public PlanNode { + JoinNodeType type = {}; + std::shared_ptr left = {}; + std::shared_ptr right = {}; + List criteria = {}; + List outputVariables = {}; + std::shared_ptr> filter = {}; + std::shared_ptr leftHashVariable = {}; + std::shared_ptr rightHashVariable = {}; + std::shared_ptr distributionType = {}; + Map dynamicFilters = {}; + + JoinNode() noexcept; +}; +void to_json(json& j, const JoinNode& p); +void from_json(const json& j, JoinNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct FilterNode : public PlanNode { @@ -1645,6 +1404,129 @@ void to_json(json& j, const FilterNode& p); void from_json(const json& j, FilterNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +struct 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 Domain { + std::shared_ptr values = {}; + bool nullAllowed = {}; +}; +void to_json(json& j, const Domain& p); +void from_json(const json& j, Domain& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +enum class 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 { +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 { +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 MemoryAllocation { + String tag = {}; + int64_t allocation = {}; +}; +void to_json(json& j, const MemoryAllocation& p); +void from_json(const json& j, MemoryAllocation& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct MemoryPoolInfo { + int64_t maxBytes = {}; + int64_t reservedBytes = {}; + int64_t reservedRevocableBytes = {}; + Map queryMemoryReservations = {}; + Map> queryMemoryAllocations = {}; + Map queryMemoryRevocableReservations = {}; +}; +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 MemoryInfo { + DataSize totalNodeMemory = {}; + Map pools = {}; +}; +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 NodeStatus { + String nodeId = {}; + NodeVersion nodeVersion = {}; + String environment = {}; + bool coordinator = {}; + Duration uptime = {}; + String externalAddress = {}; + String internalAddress = {}; + MemoryInfo memoryInfo = {}; + int processors = {}; + double processCpuLoad = {}; + double systemCpuLoad = {}; + int64_t heapUsed = {}; + int64_t heapAvailable = {}; + int64_t nonHeapUsed = {}; +}; +void to_json(json& j, const NodeStatus& p); +void from_json(const json& j, NodeStatus& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct ServerInfo { + NodeVersion nodeVersion = {}; + String environment = {}; + bool coordinator = {}; + bool starting = {}; + std::shared_ptr uptime = {}; +}; +void to_json(json& j, const ServerInfo& p); +void from_json(const json& j, ServerInfo& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { struct OutputTableHandle { ConnectorId connectorId = {}; std::shared_ptr transactionHandle = {}; @@ -1654,61 +1536,67 @@ 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 SelectedRoleType { ROLE, ALL, NONE }; -extern void to_json(json& j, const SelectedRoleType& e); -extern void from_json(const json& j, SelectedRoleType& e); +struct CreateHandle : public ExecutionWriterTarget { + OutputTableHandle handle = {}; + SchemaTableName schemaTableName = {}; + + CreateHandle() noexcept; +}; +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 SelectedRole { - SelectedRoleType type = {}; - std::shared_ptr role = {}; +struct SemiJoinNode : public PlanNode { + std::shared_ptr source = {}; + std::shared_ptr filteringSource = {}; + VariableReferenceExpression sourceJoinVariable = {}; + VariableReferenceExpression filteringSourceJoinVariable = {}; + VariableReferenceExpression semiJoinOutput = {}; + std::shared_ptr sourceHashVariable = {}; + std::shared_ptr filteringSourceHashVariable = {}; + std::shared_ptr distributionType = {}; + Map dynamicFilters = {}; + + SemiJoinNode() noexcept; }; -void to_json(json& j, const SelectedRole& p); -void from_json(const json& j, SelectedRole& p); +void to_json(json& j, const SemiJoinNode& p); +void from_json(const json& j, SemiJoinNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ResourceEstimates { - std::shared_ptr executionTime = {}; - std::shared_ptr cpuTime = {}; - std::shared_ptr peakMemory = {}; - std::shared_ptr peakTaskMemory = {}; +enum class StageExecutionStrategy { + UNGROUPED_EXECUTION, + FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, + DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, + RECOVERABLE_GROUPED_EXECUTION +}; +extern void to_json(json& j, const StageExecutionStrategy& e); +extern void from_json(const json& j, StageExecutionStrategy& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct StageExecutionDescriptor { + StageExecutionStrategy stageExecutionStrategy = {}; + List groupedExecutionScanNodes = {}; + int totalLifespans = {}; }; -void to_json(json& j, const ResourceEstimates& p); -void from_json(const json& j, ResourceEstimates& p); +void to_json(json& j, const StageExecutionDescriptor& p); +void from_json(const json& j, StageExecutionDescriptor& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct SessionRepresentation { - String queryId = {}; - std::shared_ptr transactionId = {}; - bool clientTransactionSupport = {}; - String user = {}; - std::shared_ptr principal = {}; - std::shared_ptr source = {}; - std::shared_ptr catalog = {}; - std::shared_ptr schema = {}; - std::shared_ptr traceToken = {}; - TimeZoneKey timeZoneKey = {}; - Locale locale = {}; - std::shared_ptr remoteUserAddress = {}; - std::shared_ptr userAgent = {}; - std::shared_ptr clientInfo = {}; - List clientTags = {}; - ResourceEstimates resourceEstimates = {}; - int64_t startTime = {}; - Map systemProperties = {}; - Map> catalogProperties = {}; - Map> unprocessedCatalogProperties = {}; - Map roles = {}; - Map preparedStatements = {}; - Map sessionFunctions = {}; +struct RemoteTransactionHandle : public ConnectorTransactionHandle { + std::shared_ptr dummy = {}; + + RemoteTransactionHandle() noexcept; }; -void to_json(json& j, const SessionRepresentation& p); -void from_json(const json& j, SessionRepresentation& p); +void to_json(json& j, const RemoteTransactionHandle& p); +void from_json(const json& j, RemoteTransactionHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class TaskState { PLANNED, RUNNING, FINISHED, CANCELED, ABORTED, FAILED }; -extern void to_json(json& j, const TaskState& e); -extern void from_json(const json& j, TaskState& e); +struct ErrorLocation { + int lineNumber = {}; + int columnNumber = {}; +}; +void to_json(json& j, const ErrorLocation& p); +void from_json(const json& j, ErrorLocation& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { enum class ErrorCause { @@ -1739,33 +1627,6 @@ struct ErrorCode { void to_json(json& j, const ErrorCode& p); void from_json(const json& j, ErrorCode& p); } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * 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 { - -using HostAddress = std::string; - -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct ErrorLocation { - int lineNumber = {}; - int columnNumber = {}; -}; -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 ExecutionFailureInfo { String type = {}; @@ -1782,6 +1643,11 @@ 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 = {}; @@ -1808,312 +1674,413 @@ struct TaskStatus { void to_json(json& j, const TaskStatus& p); void from_json(const json& j, TaskStatus& p); } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// TpchColumnHandle is special since we require an implementation of +// operator<(). + namespace facebook::presto::protocol { -struct PageBufferInfo { - int partition = {}; - int64_t bufferedPages = {}; - int64_t bufferedBytes = {}; - int64_t rowsAdded = {}; - int64_t pagesAdded = {}; +struct TpchColumnHandle : public ColumnHandle { + String columnName = {}; + Type type = {}; + + TpchColumnHandle() noexcept; + + bool operator<(const ColumnHandle& o) const override { + return columnName < dynamic_cast(o).columnName; + } }; -void to_json(json& j, const PageBufferInfo& p); -void from_json(const json& j, PageBufferInfo& p); +void to_json(json& j, const TpchColumnHandle& p); +void from_json(const json& j, TpchColumnHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct BufferInfo { - OutputBufferId bufferId = {}; - bool finished = {}; - int bufferedPages = {}; - int64_t pagesSent = {}; - PageBufferInfo pageBufferInfo = {}; +enum class SystemPartitioning { + SINGLE, + FIXED, + SOURCE, + SCALED, + COORDINATOR_ONLY, + ARBITRARY }; -void to_json(json& j, const BufferInfo& p); -void from_json(const json& j, BufferInfo& p); +extern void to_json(json& j, const SystemPartitioning& e); +extern void from_json(const json& j, SystemPartitioning& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -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 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 TopNNode : public PlanNode { - std::shared_ptr source = {}; - int64_t count = {}; - OrderingScheme orderingScheme = {}; - Step step = {}; +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 { +struct HiveTableHandle : public ConnectorTableHandle { + String schemaName = {}; + String tableName = {}; + std::shared_ptr>> analyzePartitionValues = {}; + + HiveTableHandle() noexcept; +}; +void to_json(json& j, const HiveTableHandle& p); +void from_json(const json& j, HiveTableHandle& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct Language { + String language = {}; +}; +void to_json(json& j, const Language& p); +void from_json(const json& j, Language& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +enum class Form { + IF, + NULL_IF, + SWITCH, + WHEN, + IS_NULL, + COALESCE, + IN, + AND, + OR, + DEREFERENCE, + ROW_CONSTRUCTOR, + BIND +}; +extern void to_json(json& j, const Form& e); +extern void from_json(const json& j, Form& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct SpecialFormExpression : public RowExpression { + Form form = {}; + Type returnType = {}; + List> arguments = {}; + + SpecialFormExpression() noexcept; +}; +void to_json(json& j, const SpecialFormExpression& p); +void from_json(const json& j, SpecialFormExpression& p); +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * 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. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +namespace facebook::presto::protocol { + +class ValueEntry { + public: + Type type; + std::shared_ptr block; +}; + +void to_json(json& j, const ValueEntry& p); +void from_json(const json& j, ValueEntry& p); - TopNNode() noexcept; -}; -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 TableScanNode : public PlanNode { - TableHandle table = {}; - List outputVariables = {}; - Map> assignments = - {}; +struct EquatableValueSet : public ValueSet { + Type type = {}; + bool whiteList = {}; + List entries = {}; - TableScanNode() noexcept; + EquatableValueSet() noexcept; }; -void to_json(json& j, const TableScanNode& p); -void from_json(const json& j, TableScanNode& p); +void to_json(json& j, const EquatableValueSet& p); +void from_json(const json& j, EquatableValueSet& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class 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); +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 HiveFileSplit { - String path = {}; - int64_t start = {}; - int64_t length = {}; - int64_t fileSize = {}; - int64_t fileModifiedTime = {}; - std::shared_ptr extraFileInfo = {}; - Map customSplitInfo = {}; +struct SelectedRole { + SelectedRoleType type = {}; + std::shared_ptr role = {}; }; -void to_json(json& j, const HiveFileSplit& p); -void from_json(const json& j, HiveFileSplit& 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 TableToPartitionMapping { - std::shared_ptr> tableToPartitionColumns = {}; - Map partitionSchemaDifference = {}; +struct Parameter { + String name = {}; + TypeSignature type = {}; }; -void to_json(json& j, const TableToPartitionMapping& p); -void from_json(const json& j, TableToPartitionMapping& 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 HiveSplit : public ConnectorSplit { - HiveFileSplit fileSplit = {}; - String database = {}; - String table = {}; - String partitionName = {}; - Storage storage = {}; - List partitionKeys = {}; - List addresses = {}; - std::shared_ptr readBucketNumber = {}; - std::shared_ptr tableBucketNumber = {}; - NodeSelectionStrategy nodeSelectionStrategy = {}; - int partitionDataColumnCount = {}; - TableToPartitionMapping tableToPartitionMapping = {}; - std::shared_ptr bucketConversion = {}; - bool s3SelectPushdownEnabled = {}; - CacheQuotaRequirement cacheQuota = {}; - std::shared_ptr encryptionMetadata = {}; - List> redundantColumnDomains = {}; - SplitWeight splitWeight = {}; - - HiveSplit() noexcept; -}; -void to_json(json& j, const HiveSplit& p); -void from_json(const json& j, HiveSplit& 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 { -enum class SystemPartitionFunction { - SINGLE, - HASH, - ROUND_ROBIN, - BROADCAST, - UNKNOWN +enum class Determinism { + DETERMINISTIC, + NOT_DETERMINISTIC, }; -extern void to_json(json& j, const SystemPartitionFunction& e); -extern void from_json(const json& j, SystemPartitionFunction& e); +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 SystemPartitioning { - SINGLE, - FIXED, - SOURCE, - SCALED, - COORDINATOR_ONLY, - ARBITRARY +struct RoutineCharacteristics { + std::shared_ptr language = {}; + std::shared_ptr determinism = {}; + std::shared_ptr nullCallClause = {}; }; -extern void to_json(json& j, const SystemPartitioning& e); -extern void from_json(const json& j, SystemPartitioning& e); +void to_json(json& j, const RoutineCharacteristics& p); +void from_json(const json& j, RoutineCharacteristics& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct SystemPartitioningHandle : public ConnectorPartitioningHandle { - SystemPartitioning partitioning = {}; - SystemPartitionFunction function = {}; +enum class FunctionKind { SCALAR, AGGREGATE, WINDOW }; +extern void to_json(json& j, const FunctionKind& e); +extern void from_json(const json& j, FunctionKind& e); +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * 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 { - SystemPartitioningHandle() noexcept; +struct TypeVariableConstraint { + String name = {}; + bool comparableRequired = {}; + bool orderableRequired = {}; + String variadicBound = {}; + bool nonDecimalNumericRequired = {}; + String boundedBy = {}; }; -void to_json(json& j, const SystemPartitioningHandle& p); -void from_json(const json& j, SystemPartitioningHandle& 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 Range { - Marker low = {}; - Marker high = {}; +struct LongVariableConstraint { + String name = {}; + String expression = {}; }; -void to_json(json& j, const Range& p); -void from_json(const json& j, Range& 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 SortedRangeSet : public ValueSet { - Type type = {}; - List ranges = {}; - - SortedRangeSet() noexcept; +struct Signature { + QualifiedObjectName name = {}; + FunctionKind kind = {}; + List typeVariableConstraints = {}; + List longVariableConstraints = {}; + TypeSignature returnType = {}; + List argumentTypes = {}; + bool variableArity = {}; }; -void to_json(json& j, const SortedRangeSet& p); -void from_json(const json& j, SortedRangeSet& 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 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 SqlInvokedFunction { + List parameters = {}; + String description = {}; + RoutineCharacteristics routineCharacteristics = {}; + String body = {}; + Signature signature = {}; + SqlFunctionId functionId = {}; }; -void to_json(json& j, const HiveTableLayoutHandle& p); -void from_json(const json& j, HiveTableLayoutHandle& 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 SortNode : public PlanNode { - std::shared_ptr source = {}; - OrderingScheme orderingScheme = {}; - bool isPartial = {}; - - SortNode() noexcept; +struct SessionRepresentation { + String queryId = {}; + std::shared_ptr transactionId = {}; + bool clientTransactionSupport = {}; + String user = {}; + std::shared_ptr principal = {}; + std::shared_ptr source = {}; + std::shared_ptr catalog = {}; + std::shared_ptr schema = {}; + std::shared_ptr traceToken = {}; + TimeZoneKey timeZoneKey = {}; + Locale locale = {}; + std::shared_ptr remoteUserAddress = {}; + std::shared_ptr userAgent = {}; + std::shared_ptr clientInfo = {}; + List clientTags = {}; + ResourceEstimates resourceEstimates = {}; + int64_t startTime = {}; + Map systemProperties = {}; + Map> catalogProperties = {}; + Map> unprocessedCatalogProperties = {}; + Map roles = {}; + Map preparedStatements = {}; + Map sessionFunctions = {}; }; -void to_json(json& j, const SortNode& p); -void from_json(const json& j, SortNode& 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 { -enum class Form { - IF, - NULL_IF, - SWITCH, - WHEN, - IS_NULL, - COALESCE, - IN, - AND, - OR, - DEREFERENCE, - ROW_CONSTRUCTOR, - BIND +struct TaskSource { + PlanNodeId planNodeId = {}; + List splits = {}; + List noMoreSplitsForLifespan = {}; + bool noMoreSplits = {}; }; -extern void to_json(json& j, const Form& e); -extern void from_json(const json& j, Form& e); +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 SpecialFormExpression : public RowExpression { - Form form = {}; - Type returnType = {}; - List> arguments = {}; - - SpecialFormExpression() noexcept; +enum class BufferType { + PARTITIONED, + BROADCAST, + ARBITRARY, + DISCARDING, + SPOOLING }; -void to_json(json& j, const SpecialFormExpression& p); -void from_json(const json& j, SpecialFormExpression& 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 SqlFunctionHandle : public FunctionHandle { - SqlFunctionId functionId = {}; - String version = {}; - - SqlFunctionHandle() noexcept; +struct OutputBuffers { + BufferType type = {}; + int64_t version = {}; + bool noMoreBufferIds = {}; + Map buffers = {}; }; -void to_json(json& j, const SqlFunctionHandle& p); -void from_json(const json& j, SqlFunctionHandle& 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 MarkDistinctNode : public PlanNode { - std::shared_ptr source = {}; - VariableReferenceExpression markerVariable = {}; - List distinctVariables = {}; - std::shared_ptr hashVariable = {}; - - MarkDistinctNode() noexcept; +struct DeleteScanInfo { + PlanNodeId id = {}; + TableHandle tableHandle = {}; }; -void to_json(json& j, const MarkDistinctNode& p); -void from_json(const json& j, MarkDistinctNode& 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 { -enum class BufferState { - OPEN, - NO_MORE_BUFFERS, - NO_MORE_PAGES, - FLUSHING, - FINISHED, - FAILED +struct AnalyzeTableHandle { + ConnectorId connectorId = {}; + std::shared_ptr transactionHandle = {}; + std::shared_ptr connectorHandle = {}; }; -extern void to_json(json& j, const BufferState& e); -extern void from_json(const json& j, BufferState& e); +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 OutputBufferInfo { - String type = {}; - BufferState state = {}; - bool canAddBuffers = {}; - bool canAddPages = {}; - int64_t totalBufferedBytes = {}; - int64_t totalBufferedPages = {}; - int64_t totalRowsSent = {}; - int64_t totalPagesSent = {}; - List buffers = {}; +struct TableWriteInfo { + std::shared_ptr writerTarget = {}; + std::shared_ptr analyzeTableHandle = {}; + std::shared_ptr deleteScanInfo = {}; }; -void to_json(json& j, const OutputBufferInfo& p); -void from_json(const json& j, OutputBufferInfo& p); +void to_json(json& j, const TableWriteInfo& p); +void from_json(const json& j, TableWriteInfo& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class RuntimeUnit { NONE, NANO, BYTE }; -extern void to_json(json& j, const RuntimeUnit& e); -extern void from_json(const json& j, RuntimeUnit& e); +struct 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 { -struct RuntimeMetric { - String name = {}; - RuntimeUnit unit = {}; - int64_t sum = {}; - int64_t count = {}; - int64_t max = {}; - int64_t min = {}; +struct BatchTaskUpdateRequest { + TaskUpdateRequest taskUpdateRequest = {}; + std::shared_ptr shuffleWriteInfo = {}; + std::shared_ptr broadcastBasePath = {}; }; -void to_json(json& j, const RuntimeMetric& p); -void from_json(const json& j, RuntimeMetric& 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 { -struct MetadataUpdates { - ConnectorId connectorId = {}; - List> metadataUpdates = {}; +struct PageBufferInfo { + int partition = {}; + int64_t bufferedPages = {}; + int64_t bufferedBytes = {}; + int64_t rowsAdded = {}; + int64_t pagesAdded = {}; }; -void to_json(json& j, const MetadataUpdates& p); -void from_json(const json& j, MetadataUpdates& 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 TpchTableLayoutHandle : public ConnectorTableLayoutHandle { - TpchTableHandle table = {}; - TupleDomain> predicate = {}; - - TpchTableLayoutHandle() noexcept; +struct BufferInfo { + OutputBufferId bufferId = {}; + bool finished = {}; + int bufferedPages = {}; + int64_t pagesSent = {}; + PageBufferInfo pageBufferInfo = {}; }; -void to_json(json& j, const TpchTableLayoutHandle& p); -void from_json(const json& j, TpchTableLayoutHandle& p); +void to_json(json& j, const BufferInfo& p); +void from_json(const json& j, BufferInfo& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -2129,25 +2096,141 @@ void from_json(const json& j, TpchTableLayoutHandle& p); * limitations under the License. */ namespace facebook::presto::protocol { -struct MergeJoinNode : public PlanNode { - MergeJoinNode() noexcept; - PlanNodeId id = {}; - // JoinNodeType is referenced as JoinNode.Type in Presto - // Since presto_cpp codegen can't nicely handle inner class references - // So a special hard-coded template is required here - JoinNodeType type = {}; - std::shared_ptr left = {}; - std::shared_ptr right = {}; - // EquiJoinClause is referenced as JoinNode.EquiJoinClause in Presto - List criteria = {}; - List outputVariables = {}; - std::shared_ptr> filter = {}; - std::shared_ptr leftHashVariable = {}; - std::shared_ptr rightHashVariable = {}; + +template +struct pointerDerefCompare { + bool operator()(const std::shared_ptr& a, const std::shared_ptr& b) + const { + return *a < *b; + } +}; + +template +struct TupleDomain { + std::shared_ptr> domains; +}; + +template +struct TupleDomain> { + std::shared_ptr, Domain, pointerDerefCompare>> + domains; +}; + +template +struct ColumnDomain { + T column; + Domain domain; // dependency +}; + +} // namespace facebook::presto::protocol + +namespace nlohmann { + +template +struct adl_serializer> { + static void to_json( + json& j, + const facebook::presto::protocol::ColumnDomain& p) { + facebook::presto::protocol::to_json_key( + j, "column", p.column, "ColumnDomain", "T", "column"); + facebook::presto::protocol::to_json_key( + j, "domain", p.domain, "ColumnDomain", "Domain", "domain"); + } + + static void from_json( + const json& j, + facebook::presto::protocol::ColumnDomain& p) { + facebook::presto::protocol::from_json_key( + j, "column", p.column, "ColumnDomain", "T", "column"); + facebook::presto::protocol::from_json_key( + j, "domain", p.domain, "ColumnDomain", "Domain", "domain"); + } +}; + +template +struct adl_serializer> { + static void to_json( + json& j, + const facebook::presto::protocol::TupleDomain& tup) { + facebook::presto::protocol::List< + facebook::presto::protocol::ColumnDomain> + list; + if (tup.domains != nullptr) { + for (auto& el : *tup.domains) { + facebook::presto::protocol::ColumnDomain domain; + domain.column = el.first; + domain.domain = el.second; + list.push_back(domain); + } + } + + j["columnDomains"] = list; + } + + static void from_json( + const json& j, + facebook::presto::protocol::TupleDomain& tup) { + if (j.count("columnDomains") != 0U) { + std::shared_ptr> + map = std::make_shared< + std::map>(); + + facebook::presto::protocol::List< + facebook::presto::protocol::ColumnDomain> + list = j.at("columnDomains"); + for (const facebook::presto::protocol::ColumnDomain& value : list) { + map->insert(std::make_pair(T(value.column), value.domain)); + } + tup.domains = map; + } + } +}; + +template +struct adl_serializer< + facebook::presto::protocol::TupleDomain>> { + static void to_json( + json& j, + const facebook::presto::protocol::TupleDomain>& tup) { + facebook::presto::protocol::List< + facebook::presto::protocol::ColumnDomain>> + list; + if (tup.domains != nullptr) { + for (auto& el : *tup.domains) { + facebook::presto::protocol::ColumnDomain> domain; + domain.column = el.first; + domain.domain = el.second; + list.push_back(domain); + } + } + + j["columnDomains"] = list; + } + + static void from_json( + const json& j, + facebook::presto::protocol::TupleDomain>& tup) { + if (j.count("columnDomains") != 0U) { + auto map = std::make_shared, + facebook::presto::protocol::Domain, + facebook::presto::protocol::pointerDerefCompare>>(); + + facebook::presto::protocol::List< + facebook::presto::protocol::ColumnDomain>> + list = j.at("columnDomains"); + for (const facebook::presto::protocol::ColumnDomain>& + value : list) { + map->insert( + std::make_pair(std::shared_ptr(value.column), value.domain)); + } + tup.domains = map; + } + } }; -void to_json(json& j, const MergeJoinNode& p); -void from_json(const json& j, MergeJoinNode& p); -} // namespace facebook::presto::protocol + +} // namespace nlohmann namespace facebook::presto::protocol { struct TpchSplit : public ConnectorSplit { TpchTableHandle tableHandle = {}; @@ -2161,224 +2244,170 @@ struct TpchSplit : public ConnectorSplit { void to_json(json& j, const TpchSplit& p); void from_json(const json& j, TpchSplit& p); } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -// TpchColumnHandle is special since we require an implementation of -// operator<(). - namespace facebook::presto::protocol { -struct TpchColumnHandle : public ColumnHandle { - String columnName = {}; - Type type = {}; - - TpchColumnHandle() noexcept; +struct SqlFunctionHandle : public FunctionHandle { + SqlFunctionId functionId = {}; + String version = {}; - bool operator<(const ColumnHandle& o) const override { - return columnName < dynamic_cast(o).columnName; - } -}; -void to_json(json& j, const TpchColumnHandle& p); -void from_json(const json& j, TpchColumnHandle& p); -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { -struct StatisticAggregations { - std::vector outputVariables = {}; - Map aggregations = {}; - std::vector groupingVariables = {}; + SqlFunctionHandle() noexcept; }; -void to_json(json& j, const StatisticAggregations& p); -void from_json(const json& j, StatisticAggregations& p); +void to_json(json& j, const SqlFunctionHandle& p); +void from_json(const json& j, SqlFunctionHandle& p); } // namespace facebook::presto::protocol - namespace facebook::presto::protocol { -struct TableWriterNode : public PlanNode { +struct UnnestNode : 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 = {}; + List replicateVariables = {}; + Map> + unnestVariables = {}; + std::shared_ptr ordinalityVariable = {}; - TableWriterNode() noexcept; + UnnestNode() noexcept; }; -void to_json(json& j, const TableWriterNode& p); -void from_json(const json& j, TableWriterNode& p); +void to_json(json& j, const UnnestNode& p); +void from_json(const json& j, UnnestNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class WindowType { - RANGE, - ROWS, - GROUPS, +struct MetadataUpdates { + ConnectorId connectorId = {}; + List> metadataUpdates = {}; }; -extern void to_json(json& j, const WindowType& e); -extern void from_json(const json& j, WindowType& e); +void to_json(json& j, const MetadataUpdates& p); +void from_json(const json& j, MetadataUpdates& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class BoundType { - UNBOUNDED_PRECEDING, - PRECEDING, - CURRENT_ROW, - FOLLOWING, - UNBOUNDED_FOLLOWING +enum class BufferState { + OPEN, + NO_MORE_BUFFERS, + NO_MORE_PAGES, + FLUSHING, + FINISHED, + FAILED }; -extern void to_json(json& j, const BoundType& e); -extern void from_json(const json& j, BoundType& e); +extern void to_json(json& j, const BufferState& e); +extern void from_json(const json& j, BufferState& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct 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 = {}; +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 Frame& p); -void from_json(const json& j, Frame& 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 Function { - CallExpression functionCall = {}; - Frame frame = {}; - bool ignoreNulls = {}; +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 Function& p); -void from_json(const json& j, Function& 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 WindowNode : public PlanNode { - std::shared_ptr sourceLocation = {}; - - std::shared_ptr source = {}; - Specification specification = {}; - Map windowFunctions = {}; - std::shared_ptr hashVariable = {}; - List prePartitionedInputs = {}; - int preSortedOrderPrefix = {}; - - WindowNode() noexcept; -}; -void to_json(json& j, const WindowNode& p); -void from_json(const json& j, WindowNode& p); +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 HiveTransactionHandle : public ConnectorTransactionHandle { - UUID uuid = {}; - - HiveTransactionHandle() noexcept; +struct Marker { + Type type = {}; + std::shared_ptr valueBlock = {}; + Bound bound = {}; }; -void to_json(json& j, const HiveTransactionHandle& p); -void from_json(const json& j, HiveTransactionHandle& 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 MemoryInfo { - DataSize totalNodeMemory = {}; - Map pools = {}; +struct Range { + Marker low = {}; + Marker high = {}; }; -void to_json(json& j, const MemoryInfo& p); -void from_json(const json& j, MemoryInfo& p); +void to_json(json& j, const Range& p); +void from_json(const json& j, Range& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct DeleteScanInfo { - PlanNodeId id = {}; - TableHandle tableHandle = {}; -}; -void to_json(json& j, const DeleteScanInfo& p); -void from_json(const json& j, DeleteScanInfo& p); +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 TableWriteInfo { - std::shared_ptr writerTarget = {}; - std::shared_ptr analyzeTableHandle = {}; - std::shared_ptr deleteScanInfo = {}; +struct TopNNode : public PlanNode { + std::shared_ptr source = {}; + int64_t count = {}; + OrderingScheme orderingScheme = {}; + Step step = {}; + + TopNNode() noexcept; }; -void to_json(json& j, const TableWriteInfo& p); -void from_json(const json& j, TableWriteInfo& 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 { -enum class BufferType { - PARTITIONED, - BROADCAST, - ARBITRARY, - DISCARDING, - SPOOLING +struct TableWriterMergeNode : public PlanNode { + std::shared_ptr source = {}; + VariableReferenceExpression rowCountVariable = {}; + VariableReferenceExpression fragmentVariable = {}; + VariableReferenceExpression tableCommitContextVariable = {}; + std::shared_ptr statisticsAggregation = {}; + + TableWriterMergeNode() noexcept; }; -extern void to_json(json& j, const BufferType& e); -extern void from_json(const json& j, BufferType& e); +void to_json(json& j, const TableWriterMergeNode& p); +void from_json(const json& j, TableWriterMergeNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct 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); +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 TaskUpdateRequest { - SessionRepresentation session = {}; - Map extraCredentials = {}; - std::shared_ptr fragment = {}; - List sources = {}; - OutputBuffers outputIds = {}; - std::shared_ptr tableWriteInfo = {}; +struct ProjectNode : public PlanNode { + std::shared_ptr source = {}; + Assignments assignments = {}; + Locality locality = {}; + + ProjectNode() noexcept; }; -void to_json(json& j, const TaskUpdateRequest& p); -void from_json(const json& j, TaskUpdateRequest& 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 BatchTaskUpdateRequest { - TaskUpdateRequest taskUpdateRequest = {}; - std::shared_ptr shuffleWriteInfo = {}; +struct Specification { + List partitionBy = {}; + std::shared_ptr orderingScheme = {}; }; -void to_json(json& j, const BatchTaskUpdateRequest& p); -void from_json(const json& j, BatchTaskUpdateRequest& p); +void to_json(json& j, const Specification& p); +void from_json(const json& j, Specification& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct InsertTableHandle { - ConnectorId connectorId = {}; - std::shared_ptr transactionHandle = {}; - std::shared_ptr connectorHandle = {}; +struct WindowNode : public PlanNode { + std::shared_ptr sourceLocation = {}; + + std::shared_ptr source = {}; + Specification specification = {}; + Map windowFunctions = {}; + std::shared_ptr hashVariable = {}; + List prePartitionedInputs = {}; + int preSortedOrderPrefix = {}; + + WindowNode() noexcept; }; -void to_json(json& j, const InsertTableHandle& p); -void from_json(const json& j, InsertTableHandle& p); +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 OutputNode : public PlanNode { @@ -2392,31 +2421,57 @@ void to_json(json& j, const OutputNode& p); void from_json(const json& j, OutputNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class StageExecutionStrategy { - UNGROUPED_EXECUTION, - FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, - DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, - RECOVERABLE_GROUPED_EXECUTION +struct HiveBucketHandle { + List columns = {}; + int tableBucketCount = {}; + int readBucketCount = {}; }; -extern void to_json(json& j, const StageExecutionStrategy& e); -extern void from_json(const json& j, StageExecutionStrategy& e); +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 StageExecutionDescriptor { - StageExecutionStrategy stageExecutionStrategy = {}; - List groupedExecutionScanNodes = {}; - int totalLifespans = {}; +struct HiveBucketFilter { + List bucketsToKeep = {}; }; -void to_json(json& j, const StageExecutionDescriptor& p); -void from_json(const json& j, StageExecutionDescriptor& 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 StatsAndCosts { - Map stats = {}; - Map costs = {}; +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; }; -void to_json(json& j, const StatsAndCosts& p); -void from_json(const json& j, StatsAndCosts& 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 TableScanNode : public PlanNode { + TableHandle table = {}; + List outputVariables = {}; + Map> assignments = + {}; + + TableScanNode() noexcept; +}; +void to_json(json& j, const TableScanNode& p); +void from_json(const json& j, TableScanNode& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -2433,60 +2488,24 @@ void from_json(const json& j, StatsAndCosts& p); */ namespace facebook::presto::protocol { -struct PlanFragment { - PlanFragmentId id = {}; - std::shared_ptr root = {}; - List variables = {}; - PartitioningHandle partitioning = {}; - List tableScanSchedulingOrder = {}; - PartitioningScheme partitioningScheme = {}; - StageExecutionDescriptor stageExecutionDescriptor = {}; - bool outputTableWriterFragment = {}; - std::shared_ptr jsonRepresentation = {}; -}; -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 RefreshMaterializedViewHandle { - InsertTableHandle handle = {}; - SchemaTableName schemaTableName = {}; -}; -void to_json(json& j, const RefreshMaterializedViewHandle& p); -void from_json(const json& j, RefreshMaterializedViewHandle& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct HiveInsertTableHandle : public ConnectorInsertTableHandle { - String schemaName = {}; - String tableName = {}; - List inputColumns = {}; - HivePageSinkMetadata pageSinkMetadata = {}; - LocationHandle locationHandle = {}; - std::shared_ptr bucketProperty = {}; - List preferredOrderingColumns = {}; - HiveStorageFormat tableStorageFormat = {}; - HiveStorageFormat partitionStorageFormat = {}; - HiveStorageFormat actualStorageFormat = {}; - HiveCompressionCodec compressionCodec = {}; - std::shared_ptr encryptionInformation = {}; - HiveInsertTableHandle() noexcept; +enum class HiveStorageFormat { + ORC, + DWRF, + PARQUET, + AVRO, + RCBINARY, + RCTEXT, + SEQUENCEFILE, + JSON, + TEXTFILE, + CSV, + PAGEFILE }; -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 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); +void to_json(json& j, const HiveStorageFormat& p); +void from_json(const json& j, HiveStorageFormat& p); + } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -2501,275 +2520,208 @@ void from_json(const json& j, TableWriterMergeNode& p); * See the License for the specific language governing permissions and * limitations under the License. */ + +// dependency Table +// dependency SchemaTableName + namespace facebook::presto::protocol { -struct OperatorInfo {}; -void to_json(json& j, const OperatorInfo& p); -void from_json(const json& j, OperatorInfo& p); + +struct HivePageSinkMetadata { + SchemaTableName schemaTableName = {}; + std::shared_ptr
table = {}; + // TODO Add modifiedPartitions +}; +void to_json(json& j, const HivePageSinkMetadata& p); +void from_json(const json& j, HivePageSinkMetadata& p); + } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class BlockedReason { WAITING_FOR_MEMORY }; -extern void to_json(json& j, const BlockedReason& e); -extern void from_json(const json& j, BlockedReason& e); +enum class TableType { NEW, EXISTING, TEMPORARY }; +extern void to_json(json& j, const TableType& e); +extern void from_json(const json& j, TableType& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct 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 = {}; +enum class WriteMode { + STAGE_AND_MOVE_TO_TARGET_DIRECTORY, + DIRECT_TO_TARGET_NEW_DIRECTORY, + DIRECT_TO_TARGET_EXISTING_DIRECTORY }; -void to_json(json& j, const OperatorStats& p); -void from_json(const json& j, OperatorStats& p); +extern void to_json(json& j, const WriteMode& e); +extern void from_json(const json& j, WriteMode& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct 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 LocationHandle { + String targetPath = {}; + String writePath = {}; + std::shared_ptr tempPath = {}; + TableType tableType = {}; + WriteMode writeMode = {}; }; -void to_json(json& j, const NodeStatus& p); -void from_json(const json& j, NodeStatus& p); +void to_json(json& j, const LocationHandle& p); +void from_json(const json& j, LocationHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct 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); +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 EnforceSingleRowNode : public PlanNode { - std::shared_ptr source = {}; +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 = {}; - EnforceSingleRowNode() noexcept; + HiveOutputTableHandle() noexcept; }; -void to_json(json& j, const EnforceSingleRowNode& p); -void from_json(const json& j, EnforceSingleRowNode& p); +void to_json(json& j, const HiveOutputTableHandle& p); +void from_json(const json& j, HiveOutputTableHandle& 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 Location { - String location = {}; +struct MergeJoinNode : public PlanNode { + MergeJoinNode() noexcept; + PlanNodeId id = {}; + // JoinNodeType is referenced as JoinNode.Type in Presto + // Since presto_cpp codegen can't nicely handle inner class references + // So a special hard-coded template is required here + JoinNodeType type = {}; + std::shared_ptr left = {}; + std::shared_ptr right = {}; + // EquiJoinClause is referenced as JoinNode.EquiJoinClause in Presto + List criteria = {}; + List outputVariables = {}; + std::shared_ptr> filter = {}; + std::shared_ptr leftHashVariable = {}; + std::shared_ptr rightHashVariable = {}; }; -void to_json(json& j, const Location& p); -void from_json(const json& j, Location& p); +void to_json(json& j, const MergeJoinNode& p); +void from_json(const json& j, MergeJoinNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct RemoteSplit : public ConnectorSplit { - Location location = {}; - TaskId remoteSourceTaskId = {}; +struct SortNode : public PlanNode { + std::shared_ptr source = {}; + OrderingScheme orderingScheme = {}; + bool isPartial = {}; - RemoteSplit() noexcept; + SortNode() noexcept; }; -void to_json(json& j, const RemoteSplit& p); -void from_json(const json& j, RemoteSplit& 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 HiveMetadataUpdateHandle : public ConnectorMetadataUpdateHandle { - UUID requestId = {}; - SchemaTableName schemaTableName = {}; - std::shared_ptr partitionName = {}; - std::shared_ptr fileName = {}; +struct ConstantExpression : public RowExpression { + Block valueBlock = {}; + Type type = {}; - HiveMetadataUpdateHandle() noexcept; + ConstantExpression() noexcept; }; -void to_json(json& j, const HiveMetadataUpdateHandle& p); -void from_json(const json& j, HiveMetadataUpdateHandle& p); +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 UnnestNode : public PlanNode { - std::shared_ptr source = {}; - List replicateVariables = {}; - Map> - unnestVariables = {}; - std::shared_ptr ordinalityVariable = {}; - - UnnestNode() noexcept; -}; -void to_json(json& j, const UnnestNode& p); -void from_json(const json& j, UnnestNode& 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 DeleteHandle : public ExecutionWriterTarget { - TableHandle handle = {}; - SchemaTableName schemaTableName = {}; +struct LimitNode : public PlanNode { + std::shared_ptr source = {}; + int64_t count = {}; + LimitNodeStep step = {}; - DeleteHandle() noexcept; + LimitNode() noexcept; }; -void to_json(json& j, const DeleteHandle& p); -void from_json(const json& j, DeleteHandle& 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 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 PlanCostEstimate { + double cpuCost = {}; + double maxMemory = {}; + double maxMemoryWhenOutputting = {}; + double networkCost = {}; }; -void to_json(json& j, const DriverStats& p); -void from_json(const json& j, DriverStats& p); +void to_json(json& j, const PlanCostEstimate& p); +void from_json(const json& j, PlanCostEstimate& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct 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 TpchTableLayoutHandle : public ConnectorTableLayoutHandle { + TpchTableHandle table = {}; + TupleDomain> predicate = {}; + + TpchTableLayoutHandle() noexcept; }; -void to_json(json& j, const DistributionSnapshot& p); -void from_json(const json& j, DistributionSnapshot& p); +void to_json(json& j, const TpchTableLayoutHandle& p); +void from_json(const json& j, TpchTableLayoutHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -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 RefreshMaterializedViewHandle { + InsertTableHandle handle = {}; + SchemaTableName schemaTableName = {}; }; -void to_json(json& j, const PipelineStats& p); -void from_json(const json& j, PipelineStats& p); +void to_json(json& j, const RefreshMaterializedViewHandle& p); +void from_json(const json& j, RefreshMaterializedViewHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct CreateHandle : public ExecutionWriterTarget { - OutputTableHandle handle = {}; - SchemaTableName schemaTableName = {}; +struct EmptySplit : public ConnectorSplit { + ConnectorId connectorId = {}; - CreateHandle() noexcept; + EmptySplit() noexcept; }; -void to_json(json& j, const CreateHandle& p); -void from_json(const json& j, CreateHandle& p); +void to_json(json& j, const EmptySplit& p); +void from_json(const json& j, EmptySplit& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct InsertHandle : public ExecutionWriterTarget { - InsertTableHandle handle = {}; - SchemaTableName schemaTableName = {}; - - InsertHandle() noexcept; +struct StatsAndCosts { + Map stats = {}; + Map costs = {}; }; -void to_json(json& j, const InsertHandle& p); -void from_json(const json& j, InsertHandle& p); +void to_json(json& j, const StatsAndCosts& p); +void from_json(const json& j, StatsAndCosts& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ValuesNode : public PlanNode { - std::shared_ptr location = {}; - - List outputVariables = {}; - List>> rows = {}; - std::shared_ptr valuesNodeLabel = {}; +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 = {}; - ValuesNode() noexcept; + HiveInsertTableHandle() noexcept; }; -void to_json(json& j, const ValuesNode& p); -void from_json(const json& j, ValuesNode& 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 AllOrNoneValueSet : public ValueSet { @@ -2782,72 +2734,18 @@ void to_json(json& j, const AllOrNoneValueSet& p); void from_json(const json& j, AllOrNoneValueSet& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct 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 TaskStats& p); -void from_json(const json& j, TaskStats& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct AssignUniqueId : public PlanNode { +struct RowNumberNode : 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 -namespace facebook::presto::protocol { -struct EquatableValueSet : public ValueSet { - Type type = {}; - bool whiteList = {}; - List entries = {}; + List partitionBy = {}; + VariableReferenceExpression rowNumberVariable = {}; + std::shared_ptr maxRowCountPerPartition = {}; + bool partial = {}; + std::shared_ptr hashVariable = {}; - EquatableValueSet() noexcept; + RowNumberNode() noexcept; }; -void to_json(json& j, const EquatableValueSet& p); -void from_json(const json& j, EquatableValueSet& 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 { struct BuiltInFunctionHandle : public FunctionHandle { @@ -2859,6 +2757,20 @@ 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 GroupIdNode : public PlanNode { + std::shared_ptr source = {}; + List> groupingSets = {}; + Map + groupingColumns = {}; + List aggregationArguments = {}; + VariableReferenceExpression groupIdVariable = {}; + + GroupIdNode() noexcept; +}; +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 DistinctLimitNode : public PlanNode { std::shared_ptr source = {}; int64_t limit = {}; @@ -2873,14 +2785,56 @@ void to_json(json& j, const DistinctLimitNode& p); void from_json(const json& j, DistinctLimitNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TpchPartitioningHandle : public ConnectorPartitioningHandle { - String table = {}; - int64_t totalRows = {}; +struct MarkDistinctNode : public PlanNode { + std::shared_ptr source = {}; + VariableReferenceExpression markerVariable = {}; + List distinctVariables = {}; + std::shared_ptr hashVariable = {}; - TpchPartitioningHandle() noexcept; + MarkDistinctNode() noexcept; }; -void to_json(json& j, const TpchPartitioningHandle& p); -void from_json(const json& j, TpchPartitioningHandle& 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 { +struct EnforceSingleRowNode : public PlanNode { + std::shared_ptr source = {}; + + EnforceSingleRowNode() noexcept; +}; +void to_json(json& j, const EnforceSingleRowNode& p); +void from_json(const json& j, EnforceSingleRowNode& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +enum class RuntimeUnit { NONE, NANO, BYTE }; +extern void to_json(json& j, const RuntimeUnit& e); +extern void from_json(const json& j, RuntimeUnit& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct RuntimeMetric { + String name = {}; + RuntimeUnit unit = {}; + int64_t sum = {}; + int64_t count = {}; + int64_t max = {}; + int64_t min = {}; +}; +void to_json(json& j, const RuntimeMetric& p); +void from_json(const json& j, RuntimeMetric& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct 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); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct LambdaDefinitionExpression : public RowExpression { @@ -2893,20 +2847,67 @@ struct LambdaDefinitionExpression : public RowExpression { void to_json(json& j, const LambdaDefinitionExpression& p); void from_json(const json& j, LambdaDefinitionExpression& p); } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +namespace facebook::presto::protocol { +struct PlanFragment { + PlanFragmentId id = {}; + std::shared_ptr root = {}; + List variables = {}; + PartitioningHandle partitioning = {}; + List tableScanSchedulingOrder = {}; + PartitioningScheme partitioningScheme = {}; + StageExecutionDescriptor stageExecutionDescriptor = {}; + bool outputTableWriterFragment = {}; + std::shared_ptr jsonRepresentation = {}; +}; +void to_json(json& j, const PlanFragment& p); +void from_json(const json& j, PlanFragment& p); +} // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TaskInfo { - TaskId taskId = {}; - TaskStatus taskStatus = {}; - DateTime lastHeartbeat = {}; - OutputBufferInfo outputBuffers = {}; - List noMoreSplits = {}; - TaskStats stats = {}; - bool needsPlan = {}; - MetadataUpdates metadataUpdates = {}; - String nodeId = {}; +struct HivePartitioningHandle : public ConnectorPartitioningHandle { + int bucketCount = {}; + std::shared_ptr maxCompatibleBucketCount = {}; + BucketFunctionType bucketFunctionType = {}; + std::shared_ptr> hiveTypes = {}; + std::shared_ptr> types = {}; + + HivePartitioningHandle() noexcept; }; -void to_json(json& j, const TaskInfo& p); -void from_json(const json& j, TaskInfo& 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 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 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 namespace facebook::presto::protocol { enum class NodeState { ACTIVE, INACTIVE, SHUTTING_DOWN }; diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeJoinQueries.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeJoinQueries.java index 284ea72075a3d..84bc301686afd 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeJoinQueries.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeJoinQueries.java @@ -105,6 +105,7 @@ public void testCrossJoin() assertQuery("SELECT * FROM nation n, region r WHERE n.regionkey < r.regionkey"); assertQueryReturnsEmptyResult("SELECT l.linenumber FROM lineitem l, orders o WHERE l.orderkey = o.orderkey AND o.orderkey = 12345 AND o.totalprice > 0"); + assertQuery("SELECT l.linenumber FROM lineitem l, orders o WHERE l.orderkey = o.orderkey AND o.orderkey = 14209 AND o.totalprice > 0"); assertQuery("SELECT * FROM nation_partitioned a, nation_partitioned b"); diff --git a/presto-native-execution/src/test/java/com/facebook/presto/spark/PrestoSparkNativeQueryRunnerUtils.java b/presto-native-execution/src/test/java/com/facebook/presto/spark/PrestoSparkNativeQueryRunnerUtils.java index 29775dfc458d5..72470d20ed351 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/spark/PrestoSparkNativeQueryRunnerUtils.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/spark/PrestoSparkNativeQueryRunnerUtils.java @@ -33,6 +33,8 @@ import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle; import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.util.Map; @@ -101,6 +103,14 @@ public static Map getNativeExecutionSessionConfigs() builder.put("native-execution-executable-path", path); } + try { + builder.put("native-execution-broadcast-base-path", + Files.createTempDirectory("native_broadcast").toAbsolutePath().toString()); + } + catch (IOException e) { + throw new UncheckedIOException("Error creating temporary directory for broadcast", e); + } + return builder.build(); } diff --git a/presto-native-execution/src/test/java/com/facebook/presto/spark/TestPrestoSparkNativeJoinQueries.java b/presto-native-execution/src/test/java/com/facebook/presto/spark/TestPrestoSparkNativeJoinQueries.java index 7a1cf97342c13..021158b39427e 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/spark/TestPrestoSparkNativeJoinQueries.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/spark/TestPrestoSparkNativeJoinQueries.java @@ -16,8 +16,6 @@ import com.facebook.presto.nativeworker.AbstractTestNativeJoinQueries; import com.facebook.presto.testing.ExpectedQueryRunner; import com.facebook.presto.testing.QueryRunner; -import org.testng.annotations.Ignore; -import org.testng.annotations.Test; public class TestPrestoSparkNativeJoinQueries extends AbstractTestNativeJoinQueries @@ -47,17 +45,4 @@ public Object[][] joinTypeProviderImpl() { return new Object[][] {{partitionedJoin()}}; } - - @Test - public void testBroadcastJoin() - { - assertQueryFails(broadcastJoin(), "SELECT * FROM orders o, lineitem l WHERE o.orderkey = l.orderkey", - ".*Broadcast shuffle is not supported"); - } - - // TODO: Enable following Ignored tests after fixing (Tests can be enabled by removing the method) - // Cross join requires broadcast join - @Override - @Ignore - public void testCrossJoin() {} } diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkConfig.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkConfig.java index 2f747182cf55b..b2adc17fb83c6 100644 --- a/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkConfig.java +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkConfig.java @@ -66,6 +66,7 @@ public class PrestoSparkConfig private boolean hashPartitionCountAllocationStrategyEnabled; private boolean adaptiveQueryExecutionEnabled; private boolean adaptiveJoinSideSwitchingEnabled; + private String nativeExecutionBroadcastBasePath; public boolean isSparkPartitionCountAutoTuneEnabled() { @@ -477,4 +478,17 @@ public PrestoSparkConfig setAdaptiveJoinSideSwitchingEnabled(boolean adaptiveJoi this.adaptiveJoinSideSwitchingEnabled = adaptiveJoinSideSwitchingEnabled; return this; } + + public String getNativeExecutionBroadcastBasePath() + { + return nativeExecutionBroadcastBasePath; + } + + @Config("native-execution-broadcast-base-path") + @ConfigDescription("Base path for temporary broadcast files for native execution") + public PrestoSparkConfig setNativeExecutionBroadcastBasePath(String nativeExecutionBroadcastBasePath) + { + this.nativeExecutionBroadcastBasePath = nativeExecutionBroadcastBasePath; + return this; + } } diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkModule.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkModule.java index 6139a867b0d29..9339824c8b9fb 100644 --- a/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkModule.java +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkModule.java @@ -108,6 +108,7 @@ import com.facebook.presto.spark.accesscontrol.PrestoSparkAuthenticatorProvider; import com.facebook.presto.spark.accesscontrol.PrestoSparkCredentialsProvider; import com.facebook.presto.spark.classloader_interface.SparkProcessType; +import com.facebook.presto.spark.execution.BroadcastFileInfo; import com.facebook.presto.spark.execution.PrestoSparkBroadcastTableCacheManager; import com.facebook.presto.spark.execution.PrestoSparkExecutionExceptionFactory; import com.facebook.presto.spark.execution.http.BatchTaskUpdateRequest; @@ -285,6 +286,7 @@ protected void setup(Binder binder) jsonCodecBinder(binder).bindJsonCodec(PrestoSparkLocalShuffleReadInfo.class); jsonCodecBinder(binder).bindJsonCodec(PrestoSparkLocalShuffleWriteInfo.class); jsonCodecBinder(binder).bindJsonCodec(BatchTaskUpdateRequest.class); + jsonCodecBinder(binder).bindJsonCodec(BroadcastFileInfo.class); // smile codecs smileCodecBinder(binder).bindSmileCodec(TaskSource.class); diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkSessionProperties.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkSessionProperties.java index 4f756305bd115..b0fbfdaa9a030 100644 --- a/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkSessionProperties.java +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkSessionProperties.java @@ -29,6 +29,7 @@ import static com.facebook.presto.spi.session.PropertyMetadata.dataSizeProperty; import static com.facebook.presto.spi.session.PropertyMetadata.doubleProperty; import static com.facebook.presto.spi.session.PropertyMetadata.integerProperty; +import static com.facebook.presto.spi.session.PropertyMetadata.stringProperty; import static com.google.common.base.Strings.nullToEmpty; public class PrestoSparkSessionProperties @@ -64,6 +65,7 @@ public class PrestoSparkSessionProperties public static final String SPARK_HASH_PARTITION_COUNT_SCALING_FACTOR_ON_OUT_OF_MEMORY = "spark_hash_partition_count_scaling_factor_on_out_of_memory"; public static final String SPARK_ADAPTIVE_QUERY_EXECUTION_ENABLED = "spark_adaptive_query_execution_enabled"; public static final String ADAPTIVE_JOIN_SIDE_SWITCHING_ENABLED = "adaptive_join_side_switching_enabled"; + public static final String NATIVE_EXECUTION_BROADCAST_BASE_PATH = "native_execution_broadcast_base_path"; private final List> sessionProperties; @@ -228,6 +230,11 @@ public PrestoSparkSessionProperties(PrestoSparkConfig prestoSparkConfig) ADAPTIVE_JOIN_SIDE_SWITCHING_ENABLED, "Enables the adaptive optimizer to switch the build and probe sides of a join", prestoSparkConfig.isAdaptiveJoinSideSwitchingEnabled(), + false), + stringProperty( + NATIVE_EXECUTION_BROADCAST_BASE_PATH, + "Base path for temporary storage of broadcast data", + prestoSparkConfig.getNativeExecutionBroadcastBasePath(), false)); } @@ -380,4 +387,9 @@ public static boolean isAdaptiveJoinSideSwitchingEnabled(Session session) { return session.getSystemProperty(ADAPTIVE_JOIN_SIDE_SWITCHING_ENABLED, Boolean.class); } + + public static String getNativeExecutionBroadcastBasePath(Session session) + { + return session.getSystemProperty(NATIVE_EXECUTION_BROADCAST_BASE_PATH, String.class); + } } diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/BroadcastFileInfo.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/BroadcastFileInfo.java new file mode 100644 index 0000000000000..a497611b00d8b --- /dev/null +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/BroadcastFileInfo.java @@ -0,0 +1,50 @@ +/* + * 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. + */ +package com.facebook.presto.spark.execution; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import static com.google.common.base.MoreObjects.toStringHelper; + +/** + * This class is a 1:1 strict API mapping to BroadcastFileInfo in + * presto-native-execution/presto_cpp/main/operators/BroadcastFactory.h. + * Please refrain changes to this API class. If any changes have to be made to + * this class, one should make sure to make corresponding changes in the above + * C++ struct and its corresponding serde functionalities. + */ +public class BroadcastFileInfo +{ + private final String filePath; + // TODO: Add additional stats including checksum, num rows, size. + + @JsonCreator + public BroadcastFileInfo(@JsonProperty("filePath") String filePath) + { + this.filePath = filePath; + } + + @JsonProperty("filePath") + public String getFilePath() + { + return filePath; + } + + @Override + public String toString() + { + return toStringHelper(this).toString(); + } +} diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/http/BatchTaskUpdateRequest.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/http/BatchTaskUpdateRequest.java index bb836c3eb1155..65550b557f3ff 100644 --- a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/http/BatchTaskUpdateRequest.java +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/http/BatchTaskUpdateRequest.java @@ -16,24 +16,34 @@ import com.facebook.presto.server.TaskUpdateRequest; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.MoreObjects.ToStringHelper; import java.util.Optional; import static com.google.common.base.MoreObjects.toStringHelper; import static java.util.Objects.requireNonNull; -public class BatchTaskUpdateRequest +// Marked as final to prevent finalizer attack. See Guideline 7-3 in java secure coding guidelines +public final class BatchTaskUpdateRequest { private final TaskUpdateRequest taskUpdateRequest; private final Optional shuffleWriteInfo; + private final Optional broadcastBasePath; @JsonCreator public BatchTaskUpdateRequest( @JsonProperty("taskUpdateRequest") TaskUpdateRequest taskUpdateRequest, - @JsonProperty("shuffleWriteInfo") Optional shuffleWriteInfo) + @JsonProperty("shuffleWriteInfo") Optional shuffleWriteInfo, + @JsonProperty("broadcastBasePath") Optional broadcastBasePath) { this.taskUpdateRequest = requireNonNull(taskUpdateRequest, "taskUpdateRequest is null"); this.shuffleWriteInfo = requireNonNull(shuffleWriteInfo, "shuffleWriteInfo is null"); + this.broadcastBasePath = requireNonNull(broadcastBasePath, "broadcastBasePath is null"); + + // shuffleWriteInfo and broadcastBasePath, both can't have value at the same time. + if (this.shuffleWriteInfo.isPresent() && this.broadcastBasePath.isPresent()) { + throw new IllegalArgumentException("shuffleWriteInfo and broadcastBasePath can not be specified in same request"); + } } @JsonProperty @@ -48,12 +58,19 @@ public Optional getShuffleWriteInfo() return shuffleWriteInfo; } + @JsonProperty + public Optional getBroadcastBasePath() + { + return broadcastBasePath; + } + @Override public String toString() { - return toStringHelper(this) - .add("taskUpdateRequest", taskUpdateRequest) - .add("shuffleWriteInfo", shuffleWriteInfo) - .toString(); + ToStringHelper stringHelper = toStringHelper(this) + .add("taskUpdateRequest", taskUpdateRequest); + shuffleWriteInfo.ifPresent(shuffleInfo -> stringHelper.add("shuffleWriteInfo", shuffleInfo)); + broadcastBasePath.ifPresent(basePath -> stringHelper.add("broadcastBasePath", basePath)); + return stringHelper.toString(); } } diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/http/PrestoSparkHttpTaskClient.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/http/PrestoSparkHttpTaskClient.java index b8023be16b66d..3c23af22558f1 100644 --- a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/http/PrestoSparkHttpTaskClient.java +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/http/PrestoSparkHttpTaskClient.java @@ -195,6 +195,7 @@ public ListenableFuture> updateTask( PlanFragment planFragment, TableWriteInfo tableWriteInfo, Optional shuffleWriteInfo, + Optional broadcastBasePath, Session session, OutputBuffers outputBuffers) { @@ -207,7 +208,7 @@ public ListenableFuture> updateTask( sources, outputBuffers, writeInfo); - BatchTaskUpdateRequest batchTaskUpdateRequest = new BatchTaskUpdateRequest(updateRequest, shuffleWriteInfo); + BatchTaskUpdateRequest batchTaskUpdateRequest = new BatchTaskUpdateRequest(updateRequest, shuffleWriteInfo, broadcastBasePath); URI batchTaskUri = uriBuilderFrom(taskUri) .appendPath("batch") diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/NativeExecutionTask.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/NativeExecutionTask.java index ac70aeabf2ff4..4ffff7bfe202b 100644 --- a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/NativeExecutionTask.java +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/NativeExecutionTask.java @@ -41,7 +41,6 @@ import static com.facebook.presto.execution.TaskState.ABORTED; import static com.facebook.presto.execution.TaskState.CANCELED; import static com.facebook.presto.execution.TaskState.FAILED; -import static com.facebook.presto.execution.buffer.OutputBuffers.BufferType.PARTITIONED; import static com.facebook.presto.execution.buffer.OutputBuffers.createInitialEmptyOutputBuffers; import static java.util.Objects.requireNonNull; @@ -67,6 +66,7 @@ public class NativeExecutionTask private final PrestoSparkHttpTaskClient workerClient; private final TableWriteInfo tableWriteInfo; private final Optional shuffleWriteInfo; + private final Optional broadcastBasePath; private final List sources; private final Executor executor; private final HttpNativeExecutionTaskInfoFetcher taskInfoFetcher; @@ -82,6 +82,7 @@ public NativeExecutionTask( List sources, TableWriteInfo tableWriteInfo, Optional shuffleWriteInfo, + Optional broadcastBasePath, Executor executor, ScheduledExecutorService updateScheduledExecutor, ScheduledExecutorService errorRetryScheduledExecutor, @@ -92,10 +93,11 @@ public NativeExecutionTask( this.planFragment = requireNonNull(planFragment, "planFragment is null"); this.tableWriteInfo = requireNonNull(tableWriteInfo, "tableWriteInfo is null"); this.shuffleWriteInfo = requireNonNull(shuffleWriteInfo, "shuffleWriteInfo is null"); + this.broadcastBasePath = requireNonNull(broadcastBasePath, "broadcastBasePath is null"); this.sources = requireNonNull(sources, "sources is null"); this.executor = requireNonNull(executor, "executor is null"); this.workerClient = requireNonNull(workerClient, "workerClient is null"); - this.outputBuffers = createInitialEmptyOutputBuffers(PARTITIONED); + this.outputBuffers = createInitialEmptyOutputBuffers(planFragment.getPartitioningScheme().getPartitioning().getHandle()).withNoMoreBufferIds(); requireNonNull(taskManagerConfig, "taskManagerConfig is null"); requireNonNull(updateScheduledExecutor, "updateScheduledExecutor is null"); requireNonNull(errorRetryScheduledExecutor, "errorRetryScheduledExecutor is null"); @@ -197,6 +199,7 @@ private TaskInfo sendUpdateRequest() planFragment, tableWriteInfo, shuffleWriteInfo, + broadcastBasePath, session, outputBuffers); BaseResponse response = future.get(); diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/NativeExecutionTaskFactory.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/NativeExecutionTaskFactory.java index d84960c88430b..ad030d421e75b 100644 --- a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/NativeExecutionTaskFactory.java +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/NativeExecutionTaskFactory.java @@ -86,7 +86,8 @@ public NativeExecutionTask createNativeExecutionTask( PlanFragment fragment, List sources, TableWriteInfo tableWriteInfo, - Optional shuffleWriteInfo) + Optional shuffleWriteInfo, + Optional broadcastBasePath) { PrestoSparkHttpTaskClient workerClient = new PrestoSparkHttpTaskClient( httpClient, @@ -104,6 +105,7 @@ public NativeExecutionTask createNativeExecutionTask( sources, tableWriteInfo, shuffleWriteInfo, + broadcastBasePath, executor, updateScheduledExecutor, errorRetryScheduledExecutor, diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/PrestoSparkNativeTaskExecutorFactory.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/PrestoSparkNativeTaskExecutorFactory.java index 63a0792716d3f..854d78d4f24e0 100644 --- a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/PrestoSparkNativeTaskExecutorFactory.java +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/PrestoSparkNativeTaskExecutorFactory.java @@ -17,6 +17,8 @@ import com.facebook.airlift.json.JsonCodec; import com.facebook.airlift.log.Logger; import com.facebook.presto.Session; +import com.facebook.presto.common.block.BlockEncodingManager; +import com.facebook.presto.common.type.VarcharType; import com.facebook.presto.execution.ExecutionFailureInfo; import com.facebook.presto.execution.Lifespan; import com.facebook.presto.execution.Location; @@ -44,13 +46,16 @@ import com.facebook.presto.spark.classloader_interface.SerializedPrestoSparkTaskDescriptor; import com.facebook.presto.spark.classloader_interface.SerializedPrestoSparkTaskSource; import com.facebook.presto.spark.classloader_interface.SerializedTaskInfo; +import com.facebook.presto.spark.execution.BroadcastFileInfo; import com.facebook.presto.spark.execution.PrestoSparkBroadcastTableCacheManager; import com.facebook.presto.spark.execution.PrestoSparkExecutionExceptionFactory; import com.facebook.presto.spark.execution.nativeprocess.NativeExecutionProcess; import com.facebook.presto.spark.execution.nativeprocess.NativeExecutionProcessFactory; import com.facebook.presto.spark.execution.shuffle.PrestoSparkShuffleInfoTranslator; import com.facebook.presto.spark.execution.shuffle.PrestoSparkShuffleWriteInfo; +import com.facebook.presto.spark.util.PrestoSparkUtils; import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.page.PagesSerde; import com.facebook.presto.spi.page.SerializedPage; import com.facebook.presto.spi.plan.OutputNode; import com.facebook.presto.spi.plan.PlanNode; @@ -65,6 +70,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; +import org.apache.spark.broadcast.Broadcast; import org.apache.spark.util.CollectionAccumulator; import scala.Tuple2; import scala.collection.AbstractIterator; @@ -80,16 +86,20 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import java.util.stream.IntStream; import static com.facebook.presto.operator.ExchangeOperator.REMOTE_CONNECTOR_ID; +import static com.facebook.presto.spark.PrestoSparkSessionProperties.getNativeExecutionBroadcastBasePath; import static com.facebook.presto.spark.execution.nativeprocess.NativeExecutionProcessFactory.DEFAULT_URI; import static com.facebook.presto.spark.util.PrestoSparkUtils.deserializeZstdCompressed; import static com.facebook.presto.spark.util.PrestoSparkUtils.serializeZstdCompressed; import static com.facebook.presto.spark.util.PrestoSparkUtils.toPrestoSparkSerializedPage; import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static com.facebook.presto.sql.planner.SchedulingOrderVisitor.scheduleOrder; +import static com.facebook.presto.sql.planner.SystemPartitioningHandle.FIXED_BROADCAST_DISTRIBUTION; import static com.facebook.presto.sql.planner.optimizations.PlanNodeSearcher.searchFrom; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.airlift.units.DataSize.succinctBytes; import static java.lang.String.format; import static java.util.Objects.requireNonNull; @@ -126,7 +136,9 @@ public class PrestoSparkNativeTaskExecutorFactory private static final TaskId DUMMY_TASK_ID = TaskId.valueOf("remotesourcetaskid.0.0.0.0"); private final SessionPropertyManager sessionPropertyManager; + private final BlockEncodingManager blockEncodingManager; private final JsonCodec taskDescriptorJsonCodec; + private final JsonCodec broadcastFileInfoJsonCodec; private final Codec taskSourceCodec; private final Codec taskInfoCodec; private final PrestoSparkExecutionExceptionFactory executionExceptionFactory; @@ -134,12 +146,15 @@ public class PrestoSparkNativeTaskExecutorFactory private final NativeExecutionProcessFactory nativeExecutionProcessFactory; private final NativeExecutionTaskFactory nativeExecutionTaskFactory; private final PrestoSparkShuffleInfoTranslator shuffleInfoTranslator; + private final PagesSerde pagesSerde; private NativeExecutionProcess nativeExecutionProcess; @Inject public PrestoSparkNativeTaskExecutorFactory( SessionPropertyManager sessionPropertyManager, + BlockEncodingManager blockEncodingManager, JsonCodec taskDescriptorJsonCodec, + JsonCodec broadcastFileInfoJsonCodec, Codec taskSourceCodec, Codec taskInfoCodec, PrestoSparkExecutionExceptionFactory executionExceptionFactory, @@ -150,14 +165,17 @@ public PrestoSparkNativeTaskExecutorFactory( PrestoSparkShuffleInfoTranslator shuffleInfoTranslator) { this.sessionPropertyManager = requireNonNull(sessionPropertyManager, "sessionPropertyManager is null"); + this.blockEncodingManager = requireNonNull(blockEncodingManager, "blockEncodingManager is null"); this.taskDescriptorJsonCodec = requireNonNull(taskDescriptorJsonCodec, "sparkTaskDescriptorJsonCodec is null"); this.taskSourceCodec = requireNonNull(taskSourceCodec, "taskSourceCodec is null"); this.taskInfoCodec = requireNonNull(taskInfoCodec, "taskInfoCodec is null"); + this.broadcastFileInfoJsonCodec = requireNonNull(broadcastFileInfoJsonCodec, "broadcastFileInfoJsonCodec is null"); this.executionExceptionFactory = requireNonNull(executionExceptionFactory, "executionExceptionFactory is null"); this.authenticatorProviders = ImmutableSet.copyOf(requireNonNull(authenticatorProviders, "authenticatorProviders is null")); this.nativeExecutionProcessFactory = requireNonNull(nativeExecutionProcessFactory, "processFactory is null"); this.nativeExecutionTaskFactory = requireNonNull(nativeExecutionTaskFactory, "taskFactory is null"); this.shuffleInfoTranslator = requireNonNull(shuffleInfoTranslator, "shuffleInfoFactory is null"); + this.pagesSerde = PrestoSparkUtils.createPagesSerde(blockEncodingManager); } @Override @@ -233,13 +251,19 @@ public IPrestoSparkTaskExecutor doCreate( // 2.a Populate Read info List taskSources = getTaskSources(serializedTaskSources, fragment, session, nativeInputs); - // 2.b Populate Write info + boolean isFixedBroadcastDistribution = fragment.getPartitioningScheme().getPartitioning().getHandle().equals(FIXED_BROADCAST_DISTRIBUTION); + // 2.b Populate Shuffle Write info Optional shuffleWriteInfo = nativeInputs.getShuffleWriteDescriptor().isPresent() && !findTableWriteNode(fragment.getRoot()).isPresent() - && !(fragment.getRoot() instanceof OutputNode) ? + && !(fragment.getRoot() instanceof OutputNode) + && !isFixedBroadcastDistribution ? Optional.of(shuffleInfoTranslator.createShuffleWriteInfo(session, nativeInputs.getShuffleWriteDescriptor().get())) : Optional.empty(); Optional serializedShuffleWriteInfo = shuffleWriteInfo.map(shuffleInfoTranslator::createSerializedWriteInfo); + // 2.c populate broadcast path + Optional broadcastDirectory = + isFixedBroadcastDistribution ? Optional.of(getBroadcastDirectoryPath(session)) : Optional.empty(); + // 3. Submit the task to cpp process for execution log.info("Submitting native execution task "); NativeExecutionTask task = nativeExecutionTaskFactory.createNativeExecutionTask( @@ -249,7 +273,8 @@ public IPrestoSparkTaskExecutor doCreate( fragment, ImmutableList.copyOf(taskSources), taskDescriptor.getTableWriteInfo(), - serializedShuffleWriteInfo); + serializedShuffleWriteInfo, + broadcastDirectory); log.info("Creating task and will wait for remote task completion"); TaskInfo taskInfo = task.start(); @@ -260,6 +285,11 @@ public IPrestoSparkTaskExecutor doCreate( return new PrestoSparkNativeTaskOutputIterator<>(partitionId, task, outputType, taskInfoCollector, taskInfoCodec, executionExceptionFactory); } + private String getBroadcastDirectoryPath(Session session) + { + return format("%s/%s", getNativeExecutionBroadcastBasePath(session), session.getQueryId().getId()); + } + @Override public void close() { @@ -346,8 +376,9 @@ private List getTaskSources( log.info("Total serialized size of all table scan task sources: %s", succinctBytes(totalSerializedSizeInBytes)); - // Populate ShuffleRead sources + // Populate remote sources - ShuffleRead & Broadcast. ImmutableList.Builder shuffleTaskSources = ImmutableList.builder(); + ImmutableList.Builder broadcastTaskSources = ImmutableList.builder(); AtomicLong nextSplitId = new AtomicLong(); taskSources.stream() .flatMap(source -> source.getSplits().stream()) @@ -368,11 +399,39 @@ private List getTaskSources( TaskSource source = new TaskSource(remoteSource.getId(), ImmutableSet.of(split), ImmutableSet.of(Lifespan.taskWide()), true); shuffleTaskSources.add(source); } + + Broadcast broadcast = nativeTaskInputs.getBroadcastInputs().get(sourceFragmentId.toString()); + if (broadcast != null) { + Set splits = + ((List) broadcast.value()).stream() + .map(PrestoSparkSerializedPage.class::cast) + .map(prestoSparkSerializedPage -> PrestoSparkUtils.toSerializedPage(prestoSparkSerializedPage)) + .map(serializedPage -> pagesSerde.deserialize(serializedPage)) + // Extract filePath. + .flatMap(page -> IntStream.range(0, page.getPositionCount()) + .mapToObj(position -> VarcharType.VARCHAR.getObjectValue(null, page.getBlock(0), position))) + .map(String.class::cast) + .map(filePath -> new BroadcastFileInfo(filePath)) + .map(broadcastFileInfo -> new ScheduledSplit( + nextSplitId.getAndIncrement(), + remoteSource.getId(), + new Split( + REMOTE_CONNECTOR_ID, + new RemoteTransactionHandle(), + new RemoteSplit( + new Location( + format("batch://%s?broadcastInfo=%s", DUMMY_TASK_ID, broadcastFileInfoJsonCodec.toJson(broadcastFileInfo))), + DUMMY_TASK_ID)))) + .collect(toImmutableSet()); + + TaskSource source = new TaskSource(remoteSource.getId(), splits, ImmutableSet.of(Lifespan.taskWide()), true); + broadcastTaskSources.add(source); + } } } taskSources.addAll(shuffleTaskSources.build()); - + taskSources.addAll(broadcastTaskSources.build()); return taskSources; } diff --git a/presto-spark-base/src/test/java/com/facebook/presto/spark/TestPrestoSparkConfig.java b/presto-spark-base/src/test/java/com/facebook/presto/spark/TestPrestoSparkConfig.java index 4ff096237aece..da78197ef9c85 100644 --- a/presto-spark-base/src/test/java/com/facebook/presto/spark/TestPrestoSparkConfig.java +++ b/presto-spark-base/src/test/java/com/facebook/presto/spark/TestPrestoSparkConfig.java @@ -62,7 +62,8 @@ public void testDefaults() .setAdaptiveQueryExecutionEnabled(false) .setAdaptiveJoinSideSwitchingEnabled(false) .setExecutorAllocationStrategyEnabled(false) - .setHashPartitionCountAllocationStrategyEnabled(false)); + .setHashPartitionCountAllocationStrategyEnabled(false) + .setNativeExecutionBroadcastBasePath(null)); } @Test @@ -100,6 +101,7 @@ public void testExplicitPropertyMappings() .put("optimizer.adaptive-join-side-switching-enabled", "true") .put("spark.executor-allocation-strategy-enabled", "true") .put("spark.hash-partition-count-allocation-strategy-enabled", "true") + .put("native-execution-broadcast-base-path", "/tmp/broadcast_path") .build(); PrestoSparkConfig expected = new PrestoSparkConfig() .setSparkPartitionCountAutoTuneEnabled(false) @@ -132,7 +134,8 @@ public void testExplicitPropertyMappings() .setAdaptiveQueryExecutionEnabled(true) .setAdaptiveJoinSideSwitchingEnabled(true) .setHashPartitionCountAllocationStrategyEnabled(true) - .setExecutorAllocationStrategyEnabled(true); + .setExecutorAllocationStrategyEnabled(true) + .setNativeExecutionBroadcastBasePath("/tmp/broadcast_path"); assertFullMapping(properties, expected); } } diff --git a/presto-spark-base/src/test/java/com/facebook/presto/spark/execution/TestBatchTaskUpdateRequest.java b/presto-spark-base/src/test/java/com/facebook/presto/spark/execution/TestBatchTaskUpdateRequest.java index 6db74e0292f34..94e0bf79ed67b 100644 --- a/presto-spark-base/src/test/java/com/facebook/presto/spark/execution/TestBatchTaskUpdateRequest.java +++ b/presto-spark-base/src/test/java/com/facebook/presto/spark/execution/TestBatchTaskUpdateRequest.java @@ -101,7 +101,7 @@ public void testJsonConversion() createInitialEmptyOutputBuffers(PARTITIONED), Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty()))); String shuffleWriteInfo = "dummy-shuffle-write-info"; - BatchTaskUpdateRequest batchUpdateRequest = new BatchTaskUpdateRequest(updateRequest, Optional.of(shuffleWriteInfo)); + BatchTaskUpdateRequest batchUpdateRequest = new BatchTaskUpdateRequest(updateRequest, Optional.of(shuffleWriteInfo), Optional.empty()); JsonCodec batchTaskUpdateRequestJsonCodec = getJsonCodec(); byte[] batchUpdateRequestJson = batchTaskUpdateRequestJsonCodec.toBytes(batchUpdateRequest); BatchTaskUpdateRequest recoveredBatchUpdateRequest = batchTaskUpdateRequestJsonCodec.fromBytes(batchUpdateRequestJson); diff --git a/presto-spark-base/src/test/java/com/facebook/presto/spark/execution/http/TestPrestoSparkHttpClient.java b/presto-spark-base/src/test/java/com/facebook/presto/spark/execution/http/TestPrestoSparkHttpClient.java index 3444a7512e80f..22e3fe044ac62 100644 --- a/presto-spark-base/src/test/java/com/facebook/presto/spark/execution/http/TestPrestoSparkHttpClient.java +++ b/presto-spark-base/src/test/java/com/facebook/presto/spark/execution/http/TestPrestoSparkHttpClient.java @@ -254,6 +254,7 @@ public void testUpdateTask() createPlanFragment(), new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty()), Optional.empty(), + Optional.empty(), TestingSession.testSessionBuilder().build(), createInitialEmptyOutputBuffers(PARTITIONED)); @@ -814,6 +815,7 @@ public void testNativeExecutionTask() createPlanFragment(), sources, new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty()), + Optional.empty(), Optional.empty()); assertNotNull(task); assertFalse(task.getTaskInfo().isPresent()); diff --git a/presto-spark-classloader-interface/src/main/java/com/facebook/presto/spark/classloader_interface/PrestoSparkNativeTaskInputs.java b/presto-spark-classloader-interface/src/main/java/com/facebook/presto/spark/classloader_interface/PrestoSparkNativeTaskInputs.java index fe836920310a1..383a88787c94a 100644 --- a/presto-spark-classloader-interface/src/main/java/com/facebook/presto/spark/classloader_interface/PrestoSparkNativeTaskInputs.java +++ b/presto-spark-classloader-interface/src/main/java/com/facebook/presto/spark/classloader_interface/PrestoSparkNativeTaskInputs.java @@ -14,6 +14,7 @@ package com.facebook.presto.spark.classloader_interface; import com.google.common.collect.ImmutableMap; +import org.apache.spark.broadcast.Broadcast; import java.util.Map; import java.util.Optional; @@ -27,12 +28,16 @@ public class PrestoSparkNativeTaskInputs private final Map shuffleReadDescriptors; private final Optional shuffleWriteDescriptor; + private final Map> broadcastInputs; + public PrestoSparkNativeTaskInputs( Map shuffleReadDescriptors, - Optional shuffleWriteDescriptor) + Optional shuffleWriteDescriptor, + Map> broadcastInputs) { this.shuffleReadDescriptors = ImmutableMap.copyOf(requireNonNull(shuffleReadDescriptors, "shuffleReadDescriptors is null")); this.shuffleWriteDescriptor = requireNonNull(shuffleWriteDescriptor, "shuffleWriteDescriptor is null"); + this.broadcastInputs = ImmutableMap.copyOf(requireNonNull(broadcastInputs, "broadcastInputs is null")); } public Map getShuffleReadDescriptors() @@ -44,4 +49,9 @@ public Optional getShuffleWriteDescriptor() { return shuffleWriteDescriptor; } + + public Map> getBroadcastInputs() + { + return broadcastInputs; + } } diff --git a/presto-spark-classloader-interface/src/main/java/com/facebook/presto/spark/classloader_interface/PrestoSparkTaskProcessor.java b/presto-spark-classloader-interface/src/main/java/com/facebook/presto/spark/classloader_interface/PrestoSparkTaskProcessor.java index 288f9e291a373..44db430b7a502 100644 --- a/presto-spark-classloader-interface/src/main/java/com/facebook/presto/spark/classloader_interface/PrestoSparkTaskProcessor.java +++ b/presto-spark-classloader-interface/src/main/java/com/facebook/presto/spark/classloader_interface/PrestoSparkTaskProcessor.java @@ -84,7 +84,7 @@ public Iterator> process( TaskContext.get().attemptNumber(), serializedTaskDescriptor, serializedTaskSources, - new PrestoSparkNativeTaskInputs(shuffleReadDescriptors, shuffleWriteDescriptor), + new PrestoSparkNativeTaskInputs(shuffleReadDescriptors, shuffleWriteDescriptor, broadcastInputs), taskInfoCollector, shuffleStatsCollector, outputType);