diff --git a/presto-native-execution/presto_cpp/main/TaskResource.cpp b/presto-native-execution/presto_cpp/main/TaskResource.cpp index ee3e7ff6ab5b3..afee3d1fbba36 100644 --- a/presto-native-execution/presto_cpp/main/TaskResource.cpp +++ b/presto-native-execution/presto_cpp/main/TaskResource.cpp @@ -208,27 +208,37 @@ proxygen::RequestHandler* TaskResource::createOrUpdateTaskImpl( const std::vector& pathMatch, const std::function( const protocol::TaskId& taskId, - const std::string& updateJson, + const std::string& requestBody, const bool summarize, - long startProcessCpuTime)>& createOrUpdateFunc) { + long startProcessCpuTime, + bool receiveThrift)>& createOrUpdateFunc) { protocol::TaskId taskId = pathMatch[1]; bool summarize = message->hasQueryParam("summarize"); + + auto& headers = message->getHeaders(); + const auto& acceptHeader = headers.getSingleOrEmpty(proxygen::HTTP_HEADER_ACCEPT); + const auto sendThrift = + acceptHeader.find(http::kMimeTypeApplicationThrift) != std::string::npos; + const auto& contentHeader = headers.getSingleOrEmpty(proxygen::HTTP_HEADER_CONTENT_TYPE); + const auto receiveThrift = + contentHeader.find(http::kMimeTypeApplicationThrift) != std::string::npos; + return new http::CallbackRequestHandler( - [this, taskId, summarize, createOrUpdateFunc]( + [this, taskId, summarize, createOrUpdateFunc, sendThrift, receiveThrift]( proxygen::HTTPMessage* /*message*/, const std::vector>& body, proxygen::ResponseHandler* downstream, std::shared_ptr handlerState) { folly::via( httpSrvCpuExecutor_, - [this, &body, taskId, summarize, createOrUpdateFunc]() { + [this, &body, taskId, summarize, createOrUpdateFunc, receiveThrift]() { const auto startProcessCpuTimeNs = util::getProcessCpuTimeNs(); - std::string updateJson = util::extractMessageBody(body); + std::string requestBody = util::extractMessageBody(body); std::unique_ptr taskInfo; try { taskInfo = createOrUpdateFunc( - taskId, updateJson, summarize, startProcessCpuTimeNs); + taskId, requestBody, summarize, startProcessCpuTimeNs, receiveThrift); } catch (const velox::VeloxException& e) { // Creating an empty task, putting errors inside so that next // status fetch from coordinator will catch the error and well @@ -243,12 +253,19 @@ proxygen::RequestHandler* TaskResource::createOrUpdateTaskImpl( throw; } } - return json(*taskInfo); + return taskInfo; }) .via(folly::EventBaseManager::get()->getEventBase()) - .thenValue([downstream, handlerState](auto&& taskInfoJson) { + .thenValue([downstream, handlerState, sendThrift](auto taskInfo) { if (!handlerState->requestExpired()) { - http::sendOkResponse(downstream, taskInfoJson); + if (sendThrift) { + thrift::TaskInfo thriftTaskInfo; + toThrift(*taskInfo, thriftTaskInfo); + http::sendOkThriftResponse( + downstream, thriftWrite(thriftTaskInfo)); + } else { + http::sendOkResponse(downstream, json(*taskInfo)); + } } }) .thenError( @@ -275,11 +292,12 @@ proxygen::RequestHandler* TaskResource::createOrUpdateBatchTask( message, pathMatch, [&](const protocol::TaskId& taskId, - const std::string& updateJson, + const std::string& requestBody, const bool summarize, - long startProcessCpuTime) { + long startProcessCpuTime, + bool /*receiveThrift*/) { protocol::BatchTaskUpdateRequest batchUpdateRequest = - json::parse(updateJson); + json::parse(requestBody); auto updateRequest = batchUpdateRequest.taskUpdateRequest; VELOX_USER_CHECK_NOT_NULL(updateRequest.fragment); @@ -327,16 +345,22 @@ proxygen::RequestHandler* TaskResource::createOrUpdateTask( message, pathMatch, [&](const protocol::TaskId& taskId, - const std::string& updateJson, + const std::string& requestBody, const bool summarize, - long startProcessCpuTime) { - protocol::TaskUpdateRequest updateRequest = json::parse(updateJson); + long startProcessCpuTime, + bool receiveThrift) { + protocol::TaskUpdateRequest updateRequest; + if (receiveThrift) { + auto thriftTaskUpdateRequest = std::make_shared(); + thriftRead(requestBody, thriftTaskUpdateRequest); + fromThrift(*thriftTaskUpdateRequest, updateRequest); + } else { + updateRequest = json::parse(requestBody); + } velox::core::PlanFragment planFragment; std::shared_ptr queryCtx; if (updateRequest.fragment) { - auto fragment = - velox::encoding::Base64::decode(*updateRequest.fragment); - protocol::PlanFragment prestoPlan = json::parse(fragment); + protocol::PlanFragment prestoPlan = json::parse(receiveThrift ? *updateRequest.fragment : velox::encoding::Base64::decode(*updateRequest.fragment)); queryCtx = taskManager_.getQueryContextManager()->findOrCreateQueryCtx( @@ -510,12 +534,12 @@ proxygen::RequestHandler* TaskResource::getTaskStatus( auto maxWait = getMaxWait(message); auto& headers = message->getHeaders(); - auto acceptHeader = headers.getSingleOrEmpty(proxygen::HTTP_HEADER_ACCEPT); - auto useThrift = + const auto& acceptHeader = headers.getSingleOrEmpty(proxygen::HTTP_HEADER_ACCEPT); + const auto sendThrift = acceptHeader.find(http::kMimeTypeApplicationThrift) != std::string::npos; return new http::CallbackRequestHandler( - [this, useThrift, taskId, currentState, maxWait]( + [this, sendThrift, taskId, currentState, maxWait]( proxygen::HTTPMessage* /*message*/, const std::vector>& /*body*/, proxygen::ResponseHandler* downstream, @@ -525,7 +549,7 @@ proxygen::RequestHandler* TaskResource::getTaskStatus( httpSrvCpuExecutor_, [this, evb, - useThrift, + sendThrift, taskId, currentState, maxWait, @@ -535,10 +559,10 @@ proxygen::RequestHandler* TaskResource::getTaskStatus( .getTaskStatus(taskId, currentState, maxWait, handlerState) .via(evb) .thenValue( - [useThrift, downstream, taskId, handlerState]( + [sendThrift, downstream, taskId, handlerState]( std::unique_ptr taskStatus) { if (!handlerState->requestExpired()) { - if (useThrift) { + if (sendThrift) { thrift::TaskStatus thriftTaskStatus; toThrift(*taskStatus, thriftTaskStatus); http::sendOkThriftResponse( diff --git a/presto-native-execution/presto_cpp/main/TaskResource.h b/presto-native-execution/presto_cpp/main/TaskResource.h index 1a1dadd5e51a5..96aa809aab9a9 100644 --- a/presto-native-execution/presto_cpp/main/TaskResource.h +++ b/presto-native-execution/presto_cpp/main/TaskResource.h @@ -76,7 +76,8 @@ class TaskResource { const protocol::TaskId&, const std::string&, const bool, - long)>& createOrUpdateFunc); + long, + const bool)>& createOrUpdateFunc); proxygen::RequestHandler* deleteTask( proxygen::HTTPMessage* message, diff --git a/presto-native-execution/presto_cpp/main/common/tests/test_json.h b/presto-native-execution/presto_cpp/main/common/tests/test_json.h index 1f1daa38a6f16..169a48534a2be 100644 --- a/presto-native-execution/presto_cpp/main/common/tests/test_json.h +++ b/presto-native-execution/presto_cpp/main/common/tests/test_json.h @@ -14,11 +14,14 @@ #pragma once #include -#include #include +#include +#include #include "presto_cpp/presto_protocol/core/presto_protocol_core.h" +namespace fs = boost::filesystem; + namespace nlohmann { // This is required avoid stack overflow when a gtest error printer is invoked. @@ -48,3 +51,19 @@ inline std::string slurp(const std::string& path) { buf << input.rdbuf(); return buf.str(); } + +inline std::string getDataPath(const std::string& dirUnderFbcode, const std::string& fileName) { + std::string currentPath = fs::current_path().c_str(); + if (boost::algorithm::ends_with(currentPath, "fbcode")) { + return currentPath + dirUnderFbcode + fileName; + } + + // CLion runs the tests from cmake-build-release/ or cmake-build-debug/ + // directory. Hard-coded json files are not copied there and test fails with + // file not found. Fixing the path so that we can trigger these tests from + // CLion. + boost::algorithm::replace_all(currentPath, "cmake-build-release/", ""); + boost::algorithm::replace_all(currentPath, "cmake-build-debug/", ""); + + return currentPath + "/data/" + fileName; +} diff --git a/presto-native-execution/presto_cpp/main/tests/CMakeLists.txt b/presto-native-execution/presto_cpp/main/tests/CMakeLists.txt index 1643ca5e17bab..74c2dc24b495f 100644 --- a/presto-native-execution/presto_cpp/main/tests/CMakeLists.txt +++ b/presto-native-execution/presto_cpp/main/tests/CMakeLists.txt @@ -22,7 +22,10 @@ add_executable( ServerOperationTest.cpp SessionPropertiesTest.cpp TaskManagerTest.cpp - QueryContextManagerTest.cpp) + QueryContextManagerTest.cpp + TaskInfoTest.cpp + TaskStatusTest.cpp + TaskUpdateRequestTest.cpp) if(DEFINED PRESTO_MEMORY_CHECKER_TYPE AND PRESTO_MEMORY_CHECKER_TYPE STREQUAL "LINUX_MEMORY_CHECKER") diff --git a/presto-native-execution/presto_cpp/main/tests/TaskInfoTest.cpp b/presto-native-execution/presto_cpp/main/tests/TaskInfoTest.cpp new file mode 100644 index 0000000000000..c54a281462a3c --- /dev/null +++ b/presto-native-execution/presto_cpp/main/tests/TaskInfoTest.cpp @@ -0,0 +1,103 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include "presto_cpp/main/thrift/ProtocolToThrift.h" +#include "presto_cpp/presto_protocol/core/Duration.h" +#include "presto_cpp/main/common/tests/test_json.h" +#include "presto_cpp/main/connectors/PrestoToVeloxConnector.h" + +using namespace facebook; +using namespace facebook::presto::protocol; + +class TaskInfoTest : public ::testing::Test {}; + +const std::string BASE_DATA_PATH = "/github/presto-trunk/presto-native-execution/presto_cpp/main/tests/data/"; + +TEST_F(TaskInfoTest, duration) { + double thrift = 0; + facebook::presto::thrift::toThrift(Duration(123, TimeUnit::MILLISECONDS), thrift); + ASSERT_EQ(thrift, 123); +} + +TEST_F(TaskInfoTest, binaryMetadataUpdates) { + std::string str = slurp(getDataPath(BASE_DATA_PATH, "MetadataUpdates.json")); + json j = json::parse(str); + registerPrestoToVeloxConnector(std::make_unique("hive")); + MetadataUpdates metadataUpdates = j; + std::unique_ptr thriftMetadataUpdates = std::make_unique(); + facebook::presto::thrift::toThrift(metadataUpdates, *thriftMetadataUpdates); + + json thriftJson = json::parse(*thriftMetadataUpdates); + ASSERT_EQ(j, thriftJson); + + presto::unregisterPrestoToVeloxConnector("hive"); +} + +TEST_F(TaskInfoTest, taskInfo) { + std::string str = slurp(getDataPath(BASE_DATA_PATH, "TaskInfo.json")); + json j = json::parse(str); + registerPrestoToVeloxConnector(std::make_unique("hive")); + TaskInfo taskInfo = j; + facebook::presto::thrift::TaskInfo thriftTaskInfo; + facebook::presto::thrift::toThrift(taskInfo, thriftTaskInfo); + + json thriftJson = json::parse(*thriftTaskInfo.metadataUpdates()->metadataUpdates()); + ASSERT_EQ(taskInfo.metadataUpdates, thriftJson); + ASSERT_EQ(thriftTaskInfo.needsPlan(), false); + ASSERT_EQ(thriftTaskInfo.outputBuffers()->buffers()->size(), 2); + ASSERT_EQ(thriftTaskInfo.outputBuffers()->buffers()[0].bufferId()->id(), 100); + ASSERT_EQ(thriftTaskInfo.outputBuffers()->buffers()[1].bufferId()->id(), 200); + ASSERT_EQ(thriftTaskInfo.stats()->blockedReasons()->count(facebook::presto::thrift::BlockedReason::WAITING_FOR_MEMORY), 1); + ASSERT_EQ(thriftTaskInfo.stats()->runtimeStats()->metrics()->size(), 2); + ASSERT_EQ(thriftTaskInfo.stats()->runtimeStats()->metrics()["test_metric1"].sum(), 123); + ASSERT_EQ(thriftTaskInfo.stats()->runtimeStats()->metrics()["test_metric2"].name(), "test_metric2"); + + presto::unregisterPrestoToVeloxConnector("hive"); +} + +TEST_F(TaskInfoTest, taskId) { + TaskId taskId = "queryId.1.2.3.4"; + facebook::presto::thrift::TaskId thriftTaskId; + facebook::presto::thrift::toThrift(taskId, thriftTaskId); + + ASSERT_EQ(thriftTaskId.stageExecutionId()->stageId()->queryId(), "queryId"); + ASSERT_EQ(thriftTaskId.stageExecutionId()->stageId()->id(), 1); + ASSERT_EQ(thriftTaskId.stageExecutionId()->id(), 2); + ASSERT_EQ(thriftTaskId.id(), 3); + ASSERT_EQ(thriftTaskId.attemptNumber(), 4); +} + + +TEST_F(TaskInfoTest, operatorStatsEmptyBlockedReason) { + std::string str = slurp(getDataPath(BASE_DATA_PATH, "OperatorStatsEmptyBlockedReason.json")); + json j = json::parse(str); + OperatorStats operatorStats = j; + facebook::presto::thrift::OperatorStats thriftOperatorStats; + facebook::presto::thrift::toThrift(operatorStats, thriftOperatorStats); + + ASSERT_EQ(thriftOperatorStats.blockedReason().has_value(), false); + ASSERT_EQ(thriftOperatorStats.blockedWall(), 80); + ASSERT_EQ(thriftOperatorStats.finishCpu(), 1000); +} + +TEST_F(TaskInfoTest, operatorStats) { + std::string str = slurp(getDataPath(BASE_DATA_PATH, "OperatorStats.json")); + json j = json::parse(str); + OperatorStats operatorStats = j; + facebook::presto::thrift::OperatorStats thriftOperatorStats; + facebook::presto::thrift::toThrift(operatorStats, thriftOperatorStats); + + ASSERT_EQ(thriftOperatorStats.blockedReason(), facebook::presto::thrift::BlockedReason::WAITING_FOR_MEMORY); +} diff --git a/presto-native-execution/presto_cpp/main/tests/TaskStatusTest.cpp b/presto-native-execution/presto_cpp/main/tests/TaskStatusTest.cpp new file mode 100644 index 0000000000000..5f375ce45c75a --- /dev/null +++ b/presto-native-execution/presto_cpp/main/tests/TaskStatusTest.cpp @@ -0,0 +1,109 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include "presto_cpp/main/thrift/ProtocolToThrift.h" +#include "presto_cpp/main/common/tests/test_json.h" + +using namespace facebook; +using namespace facebook::presto::protocol; + +class TaskStatusTest : public ::testing::Test {}; + +TEST_F(TaskStatusTest, lifeSpan) { + std::string str = R"("Group1001")"; + + json j = json::parse(str); + Lifespan lifeSpan = j; + facebook::presto::thrift::Lifespan thriftLifespan; + facebook::presto::thrift::toThrift(lifeSpan, thriftLifespan); + + ASSERT_EQ(thriftLifespan.grouped(), true); + ASSERT_EQ(thriftLifespan.groupId(), 1001); +} + +TEST_F(TaskStatusTest, errorCode) { + std::string str = R"({ + "code": 1234, + "name": "name", + "type": "INTERNAL_ERROR", + "retriable": false + })"; + + json j = json::parse(str); + ErrorCode errorCode = j; + facebook::presto::thrift::ErrorCode thriftErrorCode; + facebook::presto::thrift::toThrift(errorCode, thriftErrorCode); + + ASSERT_EQ(thriftErrorCode.code(), 1234); + ASSERT_EQ(thriftErrorCode.name(), "name"); + ASSERT_EQ(thriftErrorCode.type(), facebook::presto::thrift::ErrorType::INTERNAL_ERROR); + ASSERT_EQ(thriftErrorCode.retriable(), false); +} + +TEST_F(TaskStatusTest, executionFailureInfoOptionalFieldsEmpty) { + std::string str = R"({ + "type": "type", + "message": "message", + "suppressed": [], + "stack": [], + "errorLocation": { + "lineNumber": 1, + "columnNumber": 2 + }, + "errorCode": { + "code": 1234, + "name": "name", + "type": "INTERNAL_ERROR", + "retriable": false + }, + "remoteHost": "localhost:8080", + "errorCause": "EXCEEDS_BROADCAST_MEMORY_LIMIT" + })"; + + json j = json::parse(str); + ExecutionFailureInfo executionFailureInfo = j; + facebook::presto::thrift::ExecutionFailureInfo thriftExecutionFailureInfo; + facebook::presto::thrift::toThrift(executionFailureInfo, thriftExecutionFailureInfo); + + ASSERT_EQ(thriftExecutionFailureInfo.type(), "type"); + ASSERT_EQ(thriftExecutionFailureInfo.errorLocation()->columnNumber(), 2); + ASSERT_EQ(thriftExecutionFailureInfo.remoteHost()->host(), "localhost"); + ASSERT_EQ(thriftExecutionFailureInfo.remoteHost()->port(), 8080); + ASSERT_EQ(thriftExecutionFailureInfo.errorCode()->type(), facebook::presto::thrift::ErrorType::INTERNAL_ERROR); + ASSERT_EQ(thriftExecutionFailureInfo.errorCode()->retriable(), false); + ASSERT_EQ(thriftExecutionFailureInfo.errorCause(), facebook::presto::thrift::ErrorCause::EXCEEDS_BROADCAST_MEMORY_LIMIT); + ASSERT_EQ(thriftExecutionFailureInfo.cause(), nullptr); + ASSERT_EQ(thriftExecutionFailureInfo.suppressed()->size(), 0); +} + +TEST_F(TaskStatusTest, executionFailureInfoOptionalFieldsNonempty) { + std::string str = slurp(getDataPath("/github/presto-trunk/presto-native-execution/presto_cpp/main/tests/data/", "ExecutionFailureInfo.json")); + + json j = json::parse(str); + ExecutionFailureInfo executionFailureInfo = j; + facebook::presto::thrift::ExecutionFailureInfo thriftExecutionFailureInfo; + facebook::presto::thrift::toThrift(executionFailureInfo, thriftExecutionFailureInfo); + + ASSERT_EQ((*thriftExecutionFailureInfo.cause()).type(), "cause"); + ASSERT_EQ((*thriftExecutionFailureInfo.cause()).errorCause(), facebook::presto::thrift::ErrorCause::UNKNOWN); + ASSERT_EQ((*thriftExecutionFailureInfo.cause()).errorCode()->type(), facebook::presto::thrift::ErrorType::INSUFFICIENT_RESOURCES); + ASSERT_EQ((*thriftExecutionFailureInfo.cause()).errorCode()->retriable(), true); + ASSERT_EQ((*thriftExecutionFailureInfo.suppressed())[0].type(), "suppressed1"); + ASSERT_EQ((*thriftExecutionFailureInfo.suppressed())[0].errorCause(), facebook::presto::thrift::ErrorCause::LOW_PARTITION_COUNT); + ASSERT_EQ((*thriftExecutionFailureInfo.suppressed())[0].errorCode()->type(), facebook::presto::thrift::ErrorType::EXTERNAL); + ASSERT_EQ((*thriftExecutionFailureInfo.suppressed())[1].type(), "suppressed2"); + ASSERT_EQ((*thriftExecutionFailureInfo.suppressed())[1].errorCause(), facebook::presto::thrift::ErrorCause::EXCEEDS_BROADCAST_MEMORY_LIMIT); + ASSERT_EQ((*thriftExecutionFailureInfo.suppressed())[1].errorCode()->type(), facebook::presto::thrift::ErrorType::INTERNAL_ERROR); +} diff --git a/presto-native-execution/presto_cpp/main/tests/TaskUpdateRequestTest.cpp b/presto-native-execution/presto_cpp/main/tests/TaskUpdateRequestTest.cpp new file mode 100644 index 0000000000000..f2f5d2b8a32d3 --- /dev/null +++ b/presto-native-execution/presto_cpp/main/tests/TaskUpdateRequestTest.cpp @@ -0,0 +1,178 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include "presto_cpp/main/thrift/ProtocolToThrift.h" +#include "presto_cpp/main/common/tests/test_json.h" +#include "presto_cpp/main/connectors/PrestoToVeloxConnector.h" + +using namespace facebook; +using namespace facebook::presto::protocol; + +class TaskUpdateRequestTest : public ::testing::Test {}; + +const std::string BASE_DATA_PATH = "/github/presto-trunk/presto-native-execution/presto_cpp/main/tests/data/"; + +TEST_F(TaskUpdateRequestTest, connectorId) { + ConnectorId connectorId; + facebook::presto::thrift::ConnectorId thriftConnectorId; + thriftConnectorId.catalogName_ref() = "test"; + facebook::presto::thrift::fromThrift(thriftConnectorId, connectorId); + ASSERT_EQ(connectorId, "test"); +} + +TEST_F(TaskUpdateRequestTest, optionalField) { + ResourceEstimates resourceEstimates; + facebook::presto::thrift::ResourceEstimates thriftResourceEstimates; + thriftResourceEstimates.executionTime_ref() = 100; + thriftResourceEstimates.peakMemory_ref() = 1024 * 1024 * 1024; + facebook::presto::thrift::fromThrift(thriftResourceEstimates, resourceEstimates); + ASSERT_EQ(*resourceEstimates.executionTime, Duration(100, TimeUnit::MILLISECONDS)); + ASSERT_EQ(resourceEstimates.cpuTime, nullptr); + ASSERT_EQ(*resourceEstimates.peakMemory, DataSize(1024 * 1024 * 1024, DataUnit::BYTE)); + ASSERT_EQ(resourceEstimates.peakTaskMemory, nullptr); +} + +TEST_F(TaskUpdateRequestTest, qualifiedObjectName) { + QualifiedObjectName qualifiedObjectName; + facebook::presto::thrift::QualifiedObjectName thriftQualifiedObjectName; + thriftQualifiedObjectName.catalogName_ref() = "test_catalog"; + thriftQualifiedObjectName.schemaName_ref() = "test_schema"; + thriftQualifiedObjectName.objectName_ref() = "test_object"; + facebook::presto::thrift::fromThrift(thriftQualifiedObjectName, qualifiedObjectName); + ASSERT_EQ(qualifiedObjectName, "test_catalog.test_schema.test_object"); +} + +TEST_F(TaskUpdateRequestTest, routineCharacteristics) { + RoutineCharacteristics routineCharacteristics; + facebook::presto::thrift::RoutineCharacteristics thriftRroutineCharacteristics; + facebook::presto::thrift::Language thriftLanguage; + thriftLanguage.language_ref() = "English"; + thriftRroutineCharacteristics.language_ref() = std::move(thriftLanguage); + thriftRroutineCharacteristics.determinism_ref() = facebook::presto::thrift::Determinism::NOT_DETERMINISTIC; + thriftRroutineCharacteristics.nullCallClause_ref() = facebook::presto::thrift::NullCallClause::RETURNS_NULL_ON_NULL_INPUT; + facebook::presto::thrift::fromThrift(thriftRroutineCharacteristics, routineCharacteristics); + ASSERT_EQ((*routineCharacteristics.language).language, "English"); + ASSERT_EQ(*routineCharacteristics.determinism, Determinism::NOT_DETERMINISTIC); + ASSERT_EQ(*routineCharacteristics.nullCallClause, NullCallClause::RETURNS_NULL_ON_NULL_INPUT); +} + +TEST_F(TaskUpdateRequestTest, mapOutputBuffers) { + OutputBuffers outputBuffers; + facebook::presto::thrift::OutputBuffers thriftOutputBuffers; + thriftOutputBuffers.type_ref() = facebook::presto::thrift::BufferType::ARBITRARY; + thriftOutputBuffers.version_ref() = 1; + thriftOutputBuffers.noMoreBufferIds_ref() = true; + facebook::presto::thrift::OutputBufferId outputBufferId1; + facebook::presto::thrift::OutputBufferId outputBufferId2; + outputBufferId1.id_ref() = 1; + outputBufferId2.id_ref() = 2; + thriftOutputBuffers.buffers_ref() = { + {outputBufferId1, 10}, + {outputBufferId2, 20} + }; + + facebook::presto::thrift::fromThrift(thriftOutputBuffers, outputBuffers); + ASSERT_EQ(outputBuffers.type, BufferType::ARBITRARY); + ASSERT_EQ(outputBuffers.version, 1); + ASSERT_EQ(outputBuffers.buffers.size(), 2); + ASSERT_EQ(outputBuffers.buffers["1"], 10); + ASSERT_EQ(outputBuffers.buffers["2"], 20); +} + +TEST_F(TaskUpdateRequestTest, binarySplit) { + std::string str = slurp(getDataPath(BASE_DATA_PATH, "Split.json")); + Split split; + + registerPrestoToVeloxConnector(std::make_unique("hive")); + facebook::presto::thrift::fromThrift(str, split); + auto hiveSplit = std::dynamic_pointer_cast(split.connectorSplit); + ASSERT_EQ(split.connectorId, "hive"); + ASSERT_EQ(hiveSplit->database, "tpch"); + ASSERT_EQ(hiveSplit->nodeSelectionStrategy, NodeSelectionStrategy::NO_PREFERENCE); + + presto::unregisterPrestoToVeloxConnector("hive"); +} + +TEST_F(TaskUpdateRequestTest, binaryTableWriteInfo) { + std::string str = slurp(getDataPath(BASE_DATA_PATH, "TableWriteInfo.json")); + TableWriteInfo tableWriteInfo; + + registerPrestoToVeloxConnector(std::make_unique("hive")); + facebook::presto::thrift::fromThrift(str, tableWriteInfo); + auto hiveTableHandle = std::dynamic_pointer_cast((*tableWriteInfo.analyzeTableHandle).connectorHandle); + ASSERT_EQ(hiveTableHandle->tableName, "test_table"); + ASSERT_EQ(hiveTableHandle->analyzePartitionValues->size(), 2); + + presto::unregisterPrestoToVeloxConnector("hive"); +} + +TEST_F(TaskUpdateRequestTest, fragment) { + std::string str = slurp(getDataPath(BASE_DATA_PATH, "Fragment.thrift.base64")); + const auto strEnd = str.find_last_not_of(" \t\n\r"); + if (strEnd != std::string::npos) { + str.erase(strEnd + 1); + } + + registerPrestoToVeloxConnector(std::make_unique("hive")); + PlanFragment f = json::parse(velox::encoding::Base64::decode(str)); + + ASSERT_EQ(f.root->_type, ".AggregationNode"); + + std::shared_ptr root = + std::static_pointer_cast(f.root); + ASSERT_EQ(root->id, "211"); + ASSERT_NE(root->source, nullptr); + ASSERT_EQ(root->source->_type, ".ProjectNode"); + + std::shared_ptr proj = + std::static_pointer_cast(root->source); + ASSERT_EQ(proj->id, "233"); + ASSERT_NE(proj->source, nullptr); + ASSERT_EQ(proj->source->_type, ".TableScanNode"); + + std::shared_ptr scan = + std::static_pointer_cast(proj->source); + ASSERT_EQ(scan->id, "0"); + + presto::unregisterPrestoToVeloxConnector("hive"); +} + +TEST_F(TaskUpdateRequestTest, sessionRepresentation) { + SessionRepresentation sessionRepresentation; + facebook::presto::thrift::SessionRepresentation thriftSessionRepresentation; + std::map> thriftMap; + thriftMap["Person1"] = { + {"Name", "John Doe"}, + {"Age", "30"}, + {"City", "New York"} + }; + thriftMap["Person2"] = { + {"Name", "Jane Doe"}, + {"Age", "25"}, + {"City", "Los Angeles"} + }; + thriftMap["Person3"] = { + {"Name", "Bob Smith"}, + {"Age", "40"}, + {"City", "Chicago"} + }; + thriftSessionRepresentation.unprocessedCatalogProperties_ref() = std::move(thriftMap); + + facebook::presto::thrift::fromThrift(thriftSessionRepresentation, sessionRepresentation); + ASSERT_EQ(sessionRepresentation.unprocessedCatalogProperties.size(), 3); + ASSERT_EQ(sessionRepresentation.unprocessedCatalogProperties["Person1"]["City"], "New York"); + ASSERT_EQ(sessionRepresentation.unprocessedCatalogProperties["Person2"]["Age"], "25"); + ASSERT_EQ(sessionRepresentation.unprocessedCatalogProperties["Person3"]["Name"], "Bob Smith"); +} diff --git a/presto-native-execution/presto_cpp/main/tests/data/ExecutionFailureInfo.json b/presto-native-execution/presto_cpp/main/tests/data/ExecutionFailureInfo.json new file mode 100644 index 0000000000000..3e2f60ec1da12 --- /dev/null +++ b/presto-native-execution/presto_cpp/main/tests/data/ExecutionFailureInfo.json @@ -0,0 +1,71 @@ +{ + "type": "type", + "message": "message", + "cause": { + "type": "cause", + "message": "message", + "suppressed": [], + "stack": [], + "errorLocation": { + "lineNumber": 1, + "columnNumber": 2 + }, + "errorCode": { + "code": 1234, + "name": "name", + "type": "INSUFFICIENT_RESOURCES", + "retriable": true + }, + "remoteHost": "localhost:8080", + "errorCause": "UNKNOWN" + }, + "suppressed": [{ + "type": "suppressed1", + "message": "message", + "suppressed": [], + "stack": [], + "errorLocation": { + "lineNumber": 1, + "columnNumber": 2 + }, + "errorCode": { + "code": 1234, + "name": "name", + "type": "EXTERNAL", + "retriable": true + }, + "remoteHost": "localhost:8080", + "errorCause": "LOW_PARTITION_COUNT" + }, + { + "type": "suppressed2", + "message": "message", + "suppressed": [], + "stack": [], + "errorLocation": { + "lineNumber": 1, + "columnNumber": 2 + }, + "errorCode": { + "code": 1234, + "name": "name", + "type": "INTERNAL_ERROR", + "retriable": true + }, + "remoteHost": "localhost:8080", + "errorCause": "EXCEEDS_BROADCAST_MEMORY_LIMIT" + }], + "stack": [], + "errorLocation": { + "lineNumber": 1, + "columnNumber": 2 + }, + "errorCode": { + "code": 1234, + "name": "name", + "type": "INTERNAL_ERROR", + "retriable": false + }, + "remoteHost": "localhost:8080", + "errorCause": "EXCEEDS_BROADCAST_MEMORY_LIMIT" +} diff --git a/presto-native-execution/presto_cpp/main/tests/data/Fragment.thrift.base64 b/presto-native-execution/presto_cpp/main/tests/data/Fragment.thrift.base64 new file mode 100644 index 0000000000000..3613da5f1ca4a --- /dev/null +++ b/presto-native-execution/presto_cpp/main/tests/data/Fragment.thrift.base64 @@ -0,0 +1 @@ +eyJpZCI6IjIiLCJqc29uUmVwcmVzZW50YXRpb24iOiJ7XG4gIFwiaWRcIiA6IFwiMjExXCIsXG4gIFwibmFtZVwiIDogXCJBZ2dyZWdhdGUoUEFSVElBTClbc2VnbWVudCwgYXR0cmlidXRlZF9kc11bJGhhc2h2YWx1ZV8yM11cIixcbiAgXCJpZGVudGlmaWVyXCIgOiBcIlwiLFxuICBcImRldGFpbHNcIiA6IFwic3VtXzIxIDo9IFxcXCJwcmVzdG8uZGVmYXVsdC5zdW1cXFwiKChpbXByZXNzaW9ucykpXFxuc3VtXzIwIDo9IFxcXCJwcmVzdG8uZGVmYXVsdC5zdW1cXFwiKChldmVudF9iYXNlZF9yZXZlbnVlKSlcXG5cIixcbiAgXCJjaGlsZHJlblwiIDogWyB7XG4gICAgXCJpZFwiIDogXCIyMzNcIixcbiAgICBcIm5hbWVcIiA6IFwiU2NhblByb2plY3RcIixcbiAgICBcImlkZW50aWZpZXJcIiA6IFwiW3RhYmxlID0gVGFibGVIYW5kbGUge2Nvbm5lY3RvcklkPSdoaXZlJywgY29ubmVjdG9ySGFuZGxlPSdIaXZlVGFibGVIYW5kbGV7c2NoZW1hTmFtZT10cGNoLCB0YWJsZU5hbWU9dGVzdF9zcmVlbmlfcHJlYWdnX2J5X3VudmllcnNlX2V4cGVyaW1lbnRfZm9yX2NvbmN1cnJlbmN5LCBhbmFseXplUGFydGl0aW9uVmFsdWVzPU9wdGlvbmFsLmVtcHR5fScsIGxheW91dD0nT3B0aW9uYWxbdHBjaC50ZXN0X3NyZWVuaV9wcmVhZ2dfYnlfdW52aWVyc2VfZXhwZXJpbWVudF9mb3JfY29uY3VycmVuY3l7ZG9tYWlucz17dmFsaWRfZXhwZXJpbWVudF9yZWFkaW5nPVsgW1sxXV0gXX19XSd9XVwiLFxuICAgIFwiZGV0YWlsc1wiIDogXCIkaGFzaHZhbHVlXzIzIDo9IGNvbWJpbmVfaGFzaChjb21iaW5lX2hhc2goQklHSU5UIDAsIENPQUxFU0NFKCRvcGVyYXRvciRoYXNoX2NvZGUoc2VnbWVudCksIEJJR0lOVCAwKSksIENPQUxFU0NFKCRvcGVyYXRvciRoYXNoX2NvZGUoYXR0cmlidXRlZF9kcyksIEJJR0lOVCAwKSlcXG5MQVlPVVQ6IHRwY2gudGVzdF9zcmVlbmlfcHJlYWdnX2J5X3VudmllcnNlX2V4cGVyaW1lbnRfZm9yX2NvbmN1cnJlbmN5e2RvbWFpbnM9e3ZhbGlkX2V4cGVyaW1lbnRfcmVhZGluZz1bIFtbMV1dIF19fVxcbmltcHJlc3Npb25zIDo9IGltcHJlc3Npb25zOmJpZ2ludDoyMjpSRUdVTEFSXFxuZXZlbnRfYmFzZWRfcmV2ZW51ZSA6PSBldmVudF9iYXNlZF9yZXZlbnVlOmZsb2F0OjQ2OlJFR1VMQVJcXG5zZWdtZW50IDo9IHNlZ21lbnQ6aW50OjIwOlJFR1VMQVJcXG5hdHRyaWJ1dGVkX2RzIDo9IGF0dHJpYnV0ZWRfZHM6c3RyaW5nOi0xMzpQQVJUSVRJT05fS0VZXFxuICAgIDo6IFtbMjAxOS0xMi0wNl1dXFxudmFsaWRfZXhwZXJpbWVudF9yZWFkaW5nOmludDoxNjpSRUdVTEFSXFxuICAgIDo6IFtbMV1dXFxuZHM6c3RyaW5nOi0xNTpQQVJUSVRJT05fS0VZXFxuICAgIDo6IFtbMjAxOS0xMi0wN11dXFxuZXhwZXJpbWVudF9pZDpiaWdpbnQ6LTE0OlBBUlRJVElPTl9LRVlcXG4gICAgOjogW1szXV1cXG51bml2ZXJzZTpzdHJpbmc6LTE2OlBBUlRJVElPTl9LRVlcXG4gICAgOjogW1txcnRfZ3JvdXBfYWRzX3JhbmtpbmdfMV1dXFxuXCIsXG4gICAgXCJjaGlsZHJlblwiIDogWyBdLFxuICAgIFwicmVtb3RlU291cmNlc1wiIDogWyBdXG4gIH0gXSxcbiAgXCJyZW1vdGVTb3VyY2VzXCIgOiBbIF1cbn0iLCJvdXRwdXRUYWJsZVdyaXRlckZyYWdtZW50IjpmYWxzZSwicGFydGl0aW9uaW5nIjp7ImNvbm5lY3RvckhhbmRsZSI6eyJAdHlwZSI6IiRyZW1vdGUiLCJmdW5jdGlvbiI6IlVOS05PV04iLCJwYXJ0aXRpb25pbmciOiJTT1VSQ0UifX0sInBhcnRpdGlvbmluZ1NjaGVtZSI6eyJidWNrZXRUb1BhcnRpdGlvbiI6WzAsMSwyLDNdLCJoYXNoQ29sdW1uIjp7IkB0eXBlIjoidmFyaWFibGUiLCJuYW1lIjoiJGhhc2h2YWx1ZV8yMyIsInR5cGUiOiJiaWdpbnQifSwib3V0cHV0TGF5b3V0IjpbeyJAdHlwZSI6InZhcmlhYmxlIiwibmFtZSI6InNlZ21lbnQiLCJ0eXBlIjoiaW50ZWdlciJ9LHsiQHR5cGUiOiJ2YXJpYWJsZSIsIm5hbWUiOiJhdHRyaWJ1dGVkX2RzIiwidHlwZSI6InZhcmNoYXIifSx7IkB0eXBlIjoidmFyaWFibGUiLCJuYW1lIjoic3VtXzIxIiwidHlwZSI6ImJpZ2ludCJ9LHsiQHR5cGUiOiJ2YXJpYWJsZSIsIm5hbWUiOiJzdW1fMjAiLCJ0eXBlIjoiZG91YmxlIn0seyJAdHlwZSI6InZhcmlhYmxlIiwibmFtZSI6IiRoYXNodmFsdWVfMjMiLCJ0eXBlIjoiYmlnaW50In1dLCJwYXJ0aXRpb25pbmciOnsiYXJndW1lbnRzIjpbeyJAdHlwZSI6InZhcmlhYmxlIiwibmFtZSI6InNlZ21lbnQiLCJ0eXBlIjoiaW50ZWdlciJ9LHsiQHR5cGUiOiJ2YXJpYWJsZSIsIm5hbWUiOiJhdHRyaWJ1dGVkX2RzIiwidHlwZSI6InZhcmNoYXIifV0sImhhbmRsZSI6eyJjb25uZWN0b3JIYW5kbGUiOnsiQHR5cGUiOiIkcmVtb3RlIiwiZnVuY3Rpb24iOiJIQVNIIiwicGFydGl0aW9uaW5nIjoiRklYRUQifX19LCJyZXBsaWNhdGVOdWxsc0FuZEFueSI6ZmFsc2UsInNjYWxlV3JpdGVycyI6ZmFsc2UsImVuY29kaW5nIjoiQ09MVU1OQVIifSwicm9vdCI6eyJAdHlwZSI6Ii5BZ2dyZWdhdGlvbk5vZGUiLCJhZ2dyZWdhdGlvbnMiOnsic3VtXzIwPGRvdWJsZT4iOnsiYXJndW1lbnRzIjpbeyJAdHlwZSI6InZhcmlhYmxlIiwibmFtZSI6ImV2ZW50X2Jhc2VkX3JldmVudWUiLCJ0eXBlIjoicmVhbCJ9XSwiY2FsbCI6eyJAdHlwZSI6ImNhbGwiLCJhcmd1bWVudHMiOlt7IkB0eXBlIjoidmFyaWFibGUiLCJuYW1lIjoiZXZlbnRfYmFzZWRfcmV2ZW51ZSIsInR5cGUiOiJyZWFsIn1dLCJkaXNwbGF5TmFtZSI6InN1bSIsImZ1bmN0aW9uSGFuZGxlIjp7IkB0eXBlIjoiJHN0YXRpYyIsInNpZ25hdHVyZSI6eyJhcmd1bWVudFR5cGVzIjpbInJlYWwiXSwia2luZCI6IkFHR1JFR0FURSIsImxvbmdWYXJpYWJsZUNvbnN0cmFpbnRzIjpbXSwibmFtZSI6InByZXN0by5kZWZhdWx0LnN1bSIsInJldHVyblR5cGUiOiJyZWFsIiwidHlwZVZhcmlhYmxlQ29uc3RyYWludHMiOltdLCJ2YXJpYWJsZUFyaXR5IjpmYWxzZX19LCJyZXR1cm5UeXBlIjoiZG91YmxlIn0sImRpc3RpbmN0IjpmYWxzZSwiZnVuY3Rpb25IYW5kbGUiOnsiQHR5cGUiOiIkc3RhdGljIiwic2lnbmF0dXJlIjp7ImFyZ3VtZW50VHlwZXMiOlsicmVhbCJdLCJraW5kIjoiQUdHUkVHQVRFIiwibG9uZ1ZhcmlhYmxlQ29uc3RyYWludHMiOltdLCJuYW1lIjoicHJlc3RvLmRlZmF1bHQuc3VtIiwicmV0dXJuVHlwZSI6InJlYWwiLCJ0eXBlVmFyaWFibGVDb25zdHJhaW50cyI6W10sInZhcmlhYmxlQXJpdHkiOmZhbHNlfX19LCJzdW1fMjE8YmlnaW50PiI6eyJhcmd1bWVudHMiOlt7IkB0eXBlIjoidmFyaWFibGUiLCJuYW1lIjoiaW1wcmVzc2lvbnMiLCJ0eXBlIjoiYmlnaW50In1dLCJjYWxsIjp7IkB0eXBlIjoiY2FsbCIsImFyZ3VtZW50cyI6W3siQHR5cGUiOiJ2YXJpYWJsZSIsIm5hbWUiOiJpbXByZXNzaW9ucyIsInR5cGUiOiJiaWdpbnQifV0sImRpc3BsYXlOYW1lIjoic3VtIiwiZnVuY3Rpb25IYW5kbGUiOnsiQHR5cGUiOiIkc3RhdGljIiwic2lnbmF0dXJlIjp7ImFyZ3VtZW50VHlwZXMiOlsiYmlnaW50Il0sImtpbmQiOiJBR0dSRUdBVEUiLCJsb25nVmFyaWFibGVDb25zdHJhaW50cyI6W10sIm5hbWUiOiJwcmVzdG8uZGVmYXVsdC5zdW0iLCJyZXR1cm5UeXBlIjoiYmlnaW50IiwidHlwZVZhcmlhYmxlQ29uc3RyYWludHMiOltdLCJ2YXJpYWJsZUFyaXR5IjpmYWxzZX19LCJyZXR1cm5UeXBlIjoiYmlnaW50In0sImRpc3RpbmN0IjpmYWxzZSwiZnVuY3Rpb25IYW5kbGUiOnsiQHR5cGUiOiIkc3RhdGljIiwic2lnbmF0dXJlIjp7ImFyZ3VtZW50VHlwZXMiOlsiYmlnaW50Il0sImtpbmQiOiJBR0dSRUdBVEUiLCJsb25nVmFyaWFibGVDb25zdHJhaW50cyI6W10sIm5hbWUiOiJwcmVzdG8uZGVmYXVsdC5zdW0iLCJyZXR1cm5UeXBlIjoiYmlnaW50IiwidHlwZVZhcmlhYmxlQ29uc3RyYWludHMiOltdLCJ2YXJpYWJsZUFyaXR5IjpmYWxzZX19fX0sImdyb3VwaW5nU2V0cyI6eyJnbG9iYWxHcm91cGluZ1NldHMiOltdLCJncm91cGluZ0tleXMiOlt7IkB0eXBlIjoidmFyaWFibGUiLCJuYW1lIjoic2VnbWVudCIsInR5cGUiOiJpbnRlZ2VyIn0seyJAdHlwZSI6InZhcmlhYmxlIiwibmFtZSI6ImF0dHJpYnV0ZWRfZHMiLCJ0eXBlIjoidmFyY2hhciJ9XSwiZ3JvdXBpbmdTZXRDb3VudCI6MX0sImhhc2hWYXJpYWJsZSI6eyJAdHlwZSI6InZhcmlhYmxlIiwibmFtZSI6IiRoYXNodmFsdWVfMjMiLCJ0eXBlIjoiYmlnaW50In0sImlkIjoiMjExIiwicHJlR3JvdXBlZFZhcmlhYmxlcyI6W10sInNvdXJjZSI6eyJAdHlwZSI6Ii5Qcm9qZWN0Tm9kZSIsImxvY2FsaXR5IjoiTE9DQUwiLCJhc3NpZ25tZW50cyI6eyJhc3NpZ25tZW50cyI6eyIkaGFzaHZhbHVlXzIzPGJpZ2ludD4iOnsiQHR5cGUiOiJjYWxsIiwiYXJndW1lbnRzIjpbeyJAdHlwZSI6ImNhbGwiLCJhcmd1bWVudHMiOlt7IkB0eXBlIjoiY29uc3RhbnQiLCJ0eXBlIjoiYmlnaW50IiwidmFsdWVCbG9jayI6IkNnQUFBRXhQVGtkZlFWSlNRVmtCQUFBQUFBQUFBQUFBQUFBQSJ9LHsiQHR5cGUiOiJzcGVjaWFsIiwiYXJndW1lbnRzIjpbeyJAdHlwZSI6ImNhbGwiLCJhcmd1bWVudHMiOlt7IkB0eXBlIjoidmFyaWFibGUiLCJuYW1lIjoic2VnbWVudCIsInR5cGUiOiJpbnRlZ2VyIn1dLCJkaXNwbGF5TmFtZSI6IiRvcGVyYXRvciRoYXNoX2NvZGUiLCJmdW5jdGlvbkhhbmRsZSI6eyJAdHlwZSI6IiRzdGF0aWMiLCJzaWduYXR1cmUiOnsiYXJndW1lbnRUeXBlcyI6WyJpbnRlZ2VyIl0sImtpbmQiOiJTQ0FMQVIiLCJsb25nVmFyaWFibGVDb25zdHJhaW50cyI6W10sIm5hbWUiOiJwcmVzdG8uZGVmYXVsdC4kb3BlcmF0b3IkaGFzaF9jb2RlIiwicmV0dXJuVHlwZSI6ImJpZ2ludCIsInR5cGVWYXJpYWJsZUNvbnN0cmFpbnRzIjpbXSwidmFyaWFibGVBcml0eSI6ZmFsc2V9fSwicmV0dXJuVHlwZSI6ImJpZ2ludCJ9LHsiQHR5cGUiOiJjb25zdGFudCIsInR5cGUiOiJiaWdpbnQiLCJ2YWx1ZUJsb2NrIjoiQ2dBQUFFeFBUa2RmUVZKU1FWa0JBQUFBQUFBQUFBQUFBQUFBIn1dLCJmb3JtIjoiQ09BTEVTQ0UiLCJyZXR1cm5UeXBlIjoiYmlnaW50In1dLCJkaXNwbGF5TmFtZSI6ImNvbWJpbmVfaGFzaCIsImZ1bmN0aW9uSGFuZGxlIjp7IkB0eXBlIjoiJHN0YXRpYyIsInNpZ25hdHVyZSI6eyJhcmd1bWVudFR5cGVzIjpbImJpZ2ludCIsImJpZ2ludCJdLCJraW5kIjoiU0NBTEFSIiwibG9uZ1ZhcmlhYmxlQ29uc3RyYWludHMiOltdLCJuYW1lIjoicHJlc3RvLmRlZmF1bHQuY29tYmluZV9oYXNoIiwicmV0dXJuVHlwZSI6ImJpZ2ludCIsInR5cGVWYXJpYWJsZUNvbnN0cmFpbnRzIjpbXSwidmFyaWFibGVBcml0eSI6ZmFsc2V9fSwicmV0dXJuVHlwZSI6ImJpZ2ludCJ9LHsiQHR5cGUiOiJzcGVjaWFsIiwiYXJndW1lbnRzIjpbeyJAdHlwZSI6ImNhbGwiLCJhcmd1bWVudHMiOlt7IkB0eXBlIjoidmFyaWFibGUiLCJuYW1lIjoiYXR0cmlidXRlZF9kcyIsInR5cGUiOiJ2YXJjaGFyIn1dLCJkaXNwbGF5TmFtZSI6IiRvcGVyYXRvciRoYXNoX2NvZGUiLCJmdW5jdGlvbkhhbmRsZSI6eyJAdHlwZSI6IiRzdGF0aWMiLCJzaWduYXR1cmUiOnsiYXJndW1lbnRUeXBlcyI6WyJ2YXJjaGFyIl0sImtpbmQiOiJTQ0FMQVIiLCJsb25nVmFyaWFibGVDb25zdHJhaW50cyI6W10sIm5hbWUiOiJwcmVzdG8uZGVmYXVsdC4kb3BlcmF0b3IkaGFzaF9jb2RlIiwicmV0dXJuVHlwZSI6ImJpZ2ludCIsInR5cGVWYXJpYWJsZUNvbnN0cmFpbnRzIjpbXSwidmFyaWFibGVBcml0eSI6ZmFsc2V9fSwicmV0dXJuVHlwZSI6ImJpZ2ludCJ9LHsiQHR5cGUiOiJjb25zdGFudCIsInR5cGUiOiJiaWdpbnQiLCJ2YWx1ZUJsb2NrIjoiQ2dBQUFFeFBUa2RmUVZKU1FWa0JBQUFBQUFBQUFBQUFBQUFBIn1dLCJmb3JtIjoiQ09BTEVTQ0UiLCJyZXR1cm5UeXBlIjoiYmlnaW50In1dLCJkaXNwbGF5TmFtZSI6ImNvbWJpbmVfaGFzaCIsImZ1bmN0aW9uSGFuZGxlIjp7IkB0eXBlIjoiJHN0YXRpYyIsInNpZ25hdHVyZSI6eyJhcmd1bWVudFR5cGVzIjpbImJpZ2ludCIsImJpZ2ludCJdLCJraW5kIjoiU0NBTEFSIiwibG9uZ1ZhcmlhYmxlQ29uc3RyYWludHMiOltdLCJuYW1lIjoicHJlc3RvLmRlZmF1bHQuY29tYmluZV9oYXNoIiwicmV0dXJuVHlwZSI6ImJpZ2ludCIsInR5cGVWYXJpYWJsZUNvbnN0cmFpbnRzIjpbXSwidmFyaWFibGVBcml0eSI6ZmFsc2V9fSwicmV0dXJuVHlwZSI6ImJpZ2ludCJ9LCJhdHRyaWJ1dGVkX2RzPHZhcmNoYXI+Ijp7IkB0eXBlIjoidmFyaWFibGUiLCJuYW1lIjoiYXR0cmlidXRlZF9kcyIsInR5cGUiOiJ2YXJjaGFyIn0sImV2ZW50X2Jhc2VkX3JldmVudWU8cmVhbD4iOnsiQHR5cGUiOiJ2YXJpYWJsZSIsIm5hbWUiOiJldmVudF9iYXNlZF9yZXZlbnVlIiwidHlwZSI6InJlYWwifSwiaW1wcmVzc2lvbnM8YmlnaW50PiI6eyJAdHlwZSI6InZhcmlhYmxlIiwibmFtZSI6ImltcHJlc3Npb25zIiwidHlwZSI6ImJpZ2ludCJ9LCJzZWdtZW50PGludGVnZXI+Ijp7IkB0eXBlIjoidmFyaWFibGUiLCJuYW1lIjoic2VnbWVudCIsInR5cGUiOiJpbnRlZ2VyIn19fSwiaWQiOiIyMzMiLCJzb3VyY2UiOnsiQHR5cGUiOiIuVGFibGVTY2FuTm9kZSIsImFzc2lnbm1lbnRzIjp7ImF0dHJpYnV0ZWRfZHM8dmFyY2hhcj4iOnsiQHR5cGUiOiJoaXZlIiwiY29sdW1uVHlwZSI6IlBBUlRJVElPTl9LRVkiLCJoaXZlQ29sdW1uSW5kZXgiOi0xMywiaGl2ZVR5cGUiOiJzdHJpbmciLCJuYW1lIjoiYXR0cmlidXRlZF9kcyIsInJlcXVpcmVkU3ViZmllbGRzIjpbXSwidHlwZVNpZ25hdHVyZSI6InZhcmNoYXIifSwiZXZlbnRfYmFzZWRfcmV2ZW51ZTxyZWFsPiI6eyJAdHlwZSI6ImhpdmUiLCJjb2x1bW5UeXBlIjoiUkVHVUxBUiIsImhpdmVDb2x1bW5JbmRleCI6NDYsImhpdmVUeXBlIjoiZmxvYXQiLCJuYW1lIjoiZXZlbnRfYmFzZWRfcmV2ZW51ZSIsInJlcXVpcmVkU3ViZmllbGRzIjpbXSwidHlwZVNpZ25hdHVyZSI6InJlYWwifSwiaW1wcmVzc2lvbnM8YmlnaW50PiI6eyJAdHlwZSI6ImhpdmUiLCJjb2x1bW5UeXBlIjoiUkVHVUxBUiIsImhpdmVDb2x1bW5JbmRleCI6MjIsImhpdmVUeXBlIjoiYmlnaW50IiwibmFtZSI6ImltcHJlc3Npb25zIiwicmVxdWlyZWRTdWJmaWVsZHMiOltdLCJ0eXBlU2lnbmF0dXJlIjoiYmlnaW50In0sInNlZ21lbnQ8aW50ZWdlcj4iOnsiQHR5cGUiOiJoaXZlIiwiY29sdW1uVHlwZSI6IlJFR1VMQVIiLCJoaXZlQ29sdW1uSW5kZXgiOjIwLCJoaXZlVHlwZSI6ImludCIsIm5hbWUiOiJzZWdtZW50IiwicmVxdWlyZWRTdWJmaWVsZHMiOltdLCJ0eXBlU2lnbmF0dXJlIjoiaW50ZWdlciJ9fSwiaWQiOiIwIiwib3V0cHV0VmFyaWFibGVzIjpbeyJAdHlwZSI6InZhcmlhYmxlIiwibmFtZSI6InNlZ21lbnQiLCJ0eXBlIjoiaW50ZWdlciJ9LHsiQHR5cGUiOiJ2YXJpYWJsZSIsIm5hbWUiOiJpbXByZXNzaW9ucyIsInR5cGUiOiJiaWdpbnQifSx7IkB0eXBlIjoidmFyaWFibGUiLCJuYW1lIjoiZXZlbnRfYmFzZWRfcmV2ZW51ZSIsInR5cGUiOiJyZWFsIn0seyJAdHlwZSI6InZhcmlhYmxlIiwibmFtZSI6ImF0dHJpYnV0ZWRfZHMiLCJ0eXBlIjoidmFyY2hhciJ9XSwidGFibGUiOnsiY29ubmVjdG9ySGFuZGxlIjp7IkB0eXBlIjoiaGl2ZSIsInNjaGVtYU5hbWUiOiJ0cGNoIiwidGFibGVOYW1lIjoidGVzdF9zcmVlbmlfcHJlYWdnX2J5X3VudmllcnNlX2V4cGVyaW1lbnRfZm9yX2NvbmN1cnJlbmN5In0sImNvbm5lY3RvcklkIjoiaGl2ZSIsImNvbm5lY3RvclRhYmxlTGF5b3V0Ijp7IkB0eXBlIjoiaGl2ZSIsImRhdGFDb2x1bW5zIjpbeyJuYW1lIjoiYWRfZGlzcGxheV9mb3JtYXQiLCJ0eXBlIjoic3RyaW5nIn0seyJuYW1lIjoiYWRfbG9jYXRpb24iLCJ0eXBlIjoiaW50In0seyJuYW1lIjoiYWRfb3B0aW1pemF0aW9uX2dvYWwiLCJ0eXBlIjoiaW50In0seyJuYW1lIjoiYWRfcGl2b3RfdHlwZV9pbnQiLCJ0eXBlIjoiaW50In0seyJuYW1lIjoiYXR0cmlidXRlZF9jYW5kaWRhdGVfbWV0aG9kIiwidHlwZSI6ImludCJ9LHsibmFtZSI6ImJpZF90eXBlIiwidHlwZSI6ImludCJ9LHsibmFtZSI6ImNsaWNrX2F0dHJfd2luX2luX2RheXMiLCJ0eXBlIjoiaW50In0seyJuYW1lIjoiY291bnRyeSIsInR5cGUiOiJzdHJpbmcifSx7Im5hbWUiOiJkcGFfYnJlYWtkb3duIiwidHlwZSI6ImJpZ2ludCJ9LHsibmFtZSI6ImlzX2luc3RhbnRfYXJ0aWNsZV9ocWhpX3B1Ymxpc2hlciIsInR5cGUiOiJiaWdpbnQifSx7Im5hbWUiOiJpc19vcHRpbWl6ZWRfYWN0aW9uIiwidHlwZSI6ImludCJ9LHsibmFtZSI6ImlzX3JvYXNfZW5hYmxlZCIsInR5cGUiOiJpbnQifSx7Im5hbWUiOiJpc192aWV3X3Rocm91Z2giLCJ0eXBlIjoiaW50In0seyJuYW1lIjoib2JhX29wdF9vdXRfcmVtYXJrZXRpbmciLCJ0eXBlIjoiaW50In0seyJuYW1lIjoicGFnZV90YWIiLCJ0eXBlIjoic3RyaW5nIn0seyJuYW1lIjoic3RvcnlfdHlwZSIsInR5cGUiOiJpbnQifSx7Im5hbWUiOiJ2YWxpZF9leHBlcmltZW50X3JlYWRpbmciLCJ0eXBlIjoiaW50In0seyJuYW1lIjoidmlld190aHJvdWdoX2VuYWJsZWQiLCJ0eXBlIjoiaW50In0seyJuYW1lIjoidmVyc2lvbl9pZCIsInR5cGUiOiJiaWdpbnQifSx7Im5hbWUiOiJ2ZXJzaW9uX3R5cGUiLCJ0eXBlIjoic3RyaW5nIn0seyJuYW1lIjoic2VnbWVudCIsInR5cGUiOiJpbnQifSx7Im5hbWUiOiJwYWdlX3R5cGUiLCJ0eXBlIjoiaW50In0seyJuYW1lIjoiaW1wcmVzc2lvbnMiLCJ0eXBlIjoiYmlnaW50In0seyJuYW1lIjoic29jaWFsX2ltcHJlc3Npb25zIiwidHlwZSI6ImJpZ2ludCJ9LHsibmFtZSI6ImNsaWNrcyIsInR5cGUiOiJiaWdpbnQifSx7Im5hbWUiOiJsZWdhbF9jbGlja3MiLCJ0eXBlIjoiYmlnaW50In0seyJuYW1lIjoiY29udmVyc2lvbnMiLCJ0eXBlIjoiYmlnaW50In0seyJuYW1lIjoibmVnX2ZlZWRiYWNrcyIsInR5cGUiOiJiaWdpbnQifSx7Im5hbWUiOiJmYW5zIiwidHlwZSI6ImJpZ2ludCJ9LHsibmFtZSI6Im1pc2xlYWRpbmdzIiwidHlwZSI6ImJpZ2ludCJ9LHsibmFtZSI6Im9mZmVuc2l2ZXMiLCJ0eXBlIjoiYmlnaW50In0seyJuYW1lIjoiZnJpZW5kX2NsaWNrcyIsInR5cGUiOiJiaWdpbnQifSx7Im5hbWUiOiJyZXZlbnVlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoicXVhbGl0eV92YWx1ZSIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6InZhbHVlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiZXN0aW1hdGVkX2NsaWNrcyIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6ImZpbmFsX2VjdHJfbG9nbG9zcyIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6InF1YWxpdHlfdmFsdWVfdGVzdCIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6ImZlZWRfaW1wcmVzc2lvbnNfdG9wMiIsInR5cGUiOiJiaWdpbnQifSx7Im5hbWUiOiJmZWVkX2ltcHJlc3Npb25zX3RvcDUiLCJ0eXBlIjoiYmlnaW50In0seyJuYW1lIjoiZmluYWxfZW5mYnIiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJmaW5hbF9lbmZicl9sb2dsb3NzIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiZmVlZF9vbnNjcmVlbl9kaXNjb3VudGVkX2ltcHMiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJ3ZWlnaHRlZF9uZWdfZmVlZGJhY2tzIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiZmluYWxfcG9zdF9pbXBfZWN2ciIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6IndlaWdodGVkX2F3ZSIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6ImV2ZW50X2Jhc2VkX3JldmVudWUiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJyZXBldGl0aXZlbmVzcyIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6Im92ZXJkZWxpdmVyeV9yZXZlbnVlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoid2VpZ2h0ZWRfbmVnX2ZlZWRiYWNrc191c2QiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJpbXByZXNzaW9uX2Jhc2VkX3N1YnNpZHkiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJ0cmFja2luZ19hd2UiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJvbnNjcmVlbl9hd2VfY2xpY2tzIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoib25zY3JlZW5fYXdlX2Rpc2xpa2VzIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoib25zY3JlZW5fYXdlX3NoYXJlcyIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6Im9uc2NyZWVuX2F3ZV9saWtlcyIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6Im9uc2NyZWVuX2F3ZV9vY2xpY2tzIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiZmVlZF9vbnNjcmVlbl9pbXBzIiwidHlwZSI6ImJpZ2ludCJ9LHsibmFtZSI6ImV2ZW50X2Jhc2VkX3JldmVudWVfY2FwcGVkIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiZXN0aW1hdGVkX2V2ZW50X2Jhc2VkX3JldmVudWUiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJlc3RpbWF0ZWRfYWRzX3ZhbHVlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiZXN0aW1hdGVkX3Bvc2l0aXZlX3F1YWxpdHlfdmFsdWUiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJlc3RpbWF0ZWRfbmVnYXRpdmVfcXVhbGl0eV92YWx1ZSIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6ImFkZmluZGVyX3RvdGFsX3ZhbHVlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiZmluYWxfdG90YWxfdmFsdWUiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJhZGluZGV4ZXJfYWRmaW5kZXJfdG90YWxfdmFsdWVfZGlmZiIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6ImFkaW5kZXhlcl9maW5hbF90b3RhbF92YWx1ZV9kaWZmIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiYWRmaW5kZXJfZmluYWxfdG90YWxfdmFsdWVfZGlmZiIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6Im9yZ2FuaWNfZW5nYWdlbWVudF9sb3NzIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiYWRmaW5kZXJfcmFua19kaXZlcmdlbmNlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiZmluYWxfcmFua19kaXZlcmdlbmNlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoid2luc29yaXplZF92YWx1ZSIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6Im5lZ2F0aXZlX2V2ZW50X3F1YWxpdHlfdmFsdWUiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJjYXBwZWRfYWRzX3ZhbHVlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiYWRzX3Njb3JlX2Jhc2lzIiwidHlwZSI6Im1hcDxzdHJpbmcsZG91YmxlPiJ9LHsibmFtZSI6InRyYW5zZm9ybWVkX2Fkc19zY29yZSIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6ImJvdW5jZSIsInR5cGUiOiJiaWdpbnQifSx7Im5hbWUiOiJmaW5hbF9wb3N0X2ltcF9lY3ZyX2xvZ2xvc3MiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJsdGVfYWRzX3ZhbHVlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiaWFiX29jbGlja3MiLCJ0eXBlIjoiYmlnaW50In0seyJuYW1lIjoicG9zdF9jbGlja19xdWFsaXR5X3ZhbHVlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiYWRzX3ZhbHVlX3dpdGhfaGlnaF92YWx1ZV9lc3RpbWF0ZXMiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJpbXBfcXVhbGl0eV92YWx1ZV9yZXBldGl0aW9uIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiaW1wX3F1YWxpdHlfdmFsdWVfYXdyIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiaW1wX3F1YWxpdHlfdmFsdWVfaG9uIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoibGFuZGluZ19wYWdlX3RydWVfdmlldyIsInR5cGUiOiJiaWdpbnQifSx7Im5hbWUiOiJsYW5kaW5nX3BhZ2VfZHdlbGxpbmdfdGltZV9tcyIsInR5cGUiOiJiaWdpbnQifSx7Im5hbWUiOiJ2aWRlb19yZXZlbnVlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoidmlkZW9fY29udmVyc2lvbnMiLCJ0eXBlIjoiYmlnaW50In0seyJuYW1lIjoiYW5fcmVxdWVzdF9jb3VudCIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6ImFuX2ZpbGxfY291bnQiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJsYW5kaW5nX3BhZ2Vfc2hvcnRfdmlldyIsInR5cGUiOiJpbnQifSx7Im5hbWUiOiJkb3duX2Z1bm5lbF9ldmVudF92YWx1ZSIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6ImRvd25fZnVubmVsX2V2ZW50X2VzdGltYXRlZF9hZHNfdmFsdWUiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJkb3duX2Z1bm5lbF9ldmVudF9sdGVfYWRzX3ZhbHVlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiZG93bl9mdW5uZWxfY29udmVyc2lvbnMiLCJ0eXBlIjoiYmlnaW50In0seyJuYW1lIjoic2FuaXRpemVkX3ZhbHVlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiY2xpY2tfdGhyb3VnaF9jb252ZXJzaW9ucyIsInR5cGUiOiJiaWdpbnQifSx7Im5hbWUiOiJjb252ZXJzaW9uc19mb3JfY2FsaWJyYXRpb24iLCJ0eXBlIjoiYmlnaW50In0seyJuYW1lIjoiY2FwcGVkX2Fkc192YWx1ZV8xMHgiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJ1cHBlcl9mdW5uZWxfZXZlbnRfdmFsdWUiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJ1cHBlcl9mdW5uZWxfZXZlbnRfZXN0aW1hdGVkX2Fkc192YWx1ZSIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6InVwcGVyX2Z1bm5lbF9ldmVudF9sdGVfYWRzX3ZhbHVlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoidXBwZXJfZnVubmVsX2NvbnZlcnNpb25zIiwidHlwZSI6ImJpZ2ludCJ9LHsibmFtZSI6ImZlZWRfY2xpZW50X2ltcHJlc3Npb25zX3RvcDIiLCJ0eXBlIjoiYmlnaW50In0seyJuYW1lIjoiZmVlZF9jbGllbnRfaW1wcmVzc2lvbnNfdG9wNSIsInR5cGUiOiJiaWdpbnQifSx7Im5hbWUiOiJpYWJfc3BlY2lmaWVkX2V2ZW50IiwidHlwZSI6ImludCJ9LHsibmFtZSI6InBvc3RfY2xpY2tfZW5nYWdlbWVudCIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6ImNhcHBlZF9zYW5pdGl6ZWRfdmFsdWVfMTB4IiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiZmluYWxfcG9zdF9pbXBfZWN2cl9lbnZhbHVlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoid2luc19ub3JtX2NvbnZfdmFsdWUiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJ3ZWlnaHRlZF9zYW5pdGl6ZWRfdmFsdWUiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJ3ZWlnaHRlZF9jYXBwZWRfc2FuaXRpemVkX3ZhbHVlXzEweCIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6ImFkc192YWx1ZV9ieV9hZXRfaWQiLCJ0eXBlIjoibWFwPGludCxkb3VibGU+In0seyJuYW1lIjoiZXN0aW1hdGVkX2Fkc192YWx1ZV9ieV9hZXRfaWQiLCJ0eXBlIjoibWFwPGludCxkb3VibGU+In0seyJuYW1lIjoidmFsdWVfY2FwcGVkXzN4IiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoidmlld190aHJvdWdoX3ByZWRpY3Rpb24iLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJ2aWV3X3Rocm91Z2hfY29udmVyc2lvbnNfZm9yX2NhbGlicmF0aW9uIiwidHlwZSI6ImJpZ2ludCJ9LHsibmFtZSI6InByb3h5X2Fkc192YWx1ZV8xZCIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6InZhbHVlX3RvdGFsIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiZXZlbnRfYmFzZWRfcmV2ZW51ZV90b3RhbCIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6ImNhcHBlZF9hZHNfdmFsdWVfMTB4X3RvdGFsIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiY29udmVyc2lvbnNfdG90YWwiLCJ0eXBlIjoiYmlnaW50In0seyJuYW1lIjoiY2FwcGVkX2Fkc192YWx1ZV90b3RhbCIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6InZhbHVlX2NhcHBlZF8zeF9vbGQiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJwcm94eV9hZHNfdmFsdWVfYnlfYWV0X2lkIiwidHlwZSI6Im1hcDxpbnQsZG91YmxlPiJ9LHsibmFtZSI6InZhbHVlX2NhcHBlZF8zeF9yb2xsdXAiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJ0b3RhbF9hZHNfdmFsdWVfYnlfYWV0X2lkIiwidHlwZSI6Im1hcDxpbnQsZG91YmxlPiJ9LHsibmFtZSI6InN1YnNpZHlfdmFsdWUiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJpbXByZXNzaW9uX3F1YWxpdHlfdmFsdWVfd2l0aG91dF9zdWJzaWR5IiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiZXh0ZXJuYWxfcmV2ZW51ZSIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6InVzZXJfaXNfdHJhdmVsaW5nIiwidHlwZSI6ImJpZ2ludCJ9LHsibmFtZSI6ImNhcHBlZF9zYW5pdGl6ZWRfdmFsdWVfMTB4X3Bvc3RfMWQiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJzYW5pdGl6ZWRfdmFsdWVfcG9zdF8xZCIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6ImNhcHBlZF8xMHhfZXN0aW1hdGVkX2Fkc192YWx1ZSIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6InJlYWxpemVkX3N1YnNpZHlfYWRzX3ZhbHVlIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoicmVhbGl6ZWRfc3Vic2lkeV9hZHNfdmFsdWVfdG90YWwiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJyZWFsaXplZF9zdWJzaWR5X2Fkc192YWx1ZV9jYXBwZWRfM3giLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJ4b3V0X3JlcGV0aXRpdmUiLCJ0eXBlIjoiYmlnaW50In0seyJuYW1lIjoiYWRzX3ZhbHVlX3dpdGhfcHJveHlfdG90YWwiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJzYW5pdGl6ZWRfdmFsdWVfcm9sbHVwIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiY2FwcGVkX3Nhbml0aXplZF92YWx1ZV8xMHhfcm9sbHVwIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoic2FuaXRpemVkX3ZhbHVlXzFkX3JvbGx1cCIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6ImNhcHBlZF9zYW5pdGl6ZWRfdmFsdWVfMTB4XzFkX3JvbGx1cCIsInR5cGUiOiJmbG9hdCJ9LHsibmFtZSI6ImVzdGltYXRlZF9jYXBwZWRfYWRzX3ZhbHVlXzN4XzFkIiwidHlwZSI6ImZsb2F0In0seyJuYW1lIjoiZXN0aW1hdGVkX2NhcHBlZF9hZHNfdmFsdWVfM3hfdG90YWwiLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJhZHNfdmFsdWVfd2l0aF9wcm94eV90b3RhbF9jYXBwZWRfM3giLCJ0eXBlIjoiZmxvYXQifSx7Im5hbWUiOiJ1c2VyX2xvY19zb3VyY2VfcXVhbGl0eSIsInR5cGUiOiJiaWdpbnQifV0sImRvbWFpblByZWRpY2F0ZSI6eyJjb2x1bW5Eb21haW5zIjpbeyJjb2x1bW4iOiJ2YWxpZF9leHBlcmltZW50X3JlYWRpbmciLCJkb21haW4iOnsibnVsbEFsbG93ZWQiOmZhbHNlLCJ2YWx1ZXMiOnsiQHR5cGUiOiJzb3J0YWJsZSIsInJhbmdlcyI6W3siaGlnaCI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoiaW50ZWdlciIsInZhbHVlQmxvY2siOiJDUUFBQUVsT1ZGOUJVbEpCV1FFQUFBQUFBUUFBQUE9PSJ9LCJsb3ciOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6ImludGVnZXIiLCJ2YWx1ZUJsb2NrIjoiQ1FBQUFFbE9WRjlCVWxKQldRRUFBQUFBQVFBQUFBPT0ifX1dLCJ0eXBlIjoiaW50ZWdlciJ9fX1dfSwibGF5b3V0U3RyaW5nIjoidHBjaC50ZXN0X3NyZWVuaV9wcmVhZ2dfYnlfdW52aWVyc2VfZXhwZXJpbWVudF9mb3JfY29uY3VycmVuY3l7ZG9tYWlucz17dmFsaWRfZXhwZXJpbWVudF9yZWFkaW5nPVsgW1sxXV0gXX19IiwicGFydGl0aW9uQ29sdW1uUHJlZGljYXRlIjp7ImNvbHVtbkRvbWFpbnMiOlt7ImNvbHVtbiI6eyJAdHlwZSI6ImhpdmUiLCJjb2x1bW5UeXBlIjoiUEFSVElUSU9OX0tFWSIsImhpdmVDb2x1bW5JbmRleCI6LTEzLCJoaXZlVHlwZSI6InN0cmluZyIsIm5hbWUiOiJhdHRyaWJ1dGVkX2RzIiwicmVxdWlyZWRTdWJmaWVsZHMiOltdLCJ0eXBlU2lnbmF0dXJlIjoidmFyY2hhciJ9LCJkb21haW4iOnsibnVsbEFsbG93ZWQiOmZhbHNlLCJ2YWx1ZXMiOnsiQHR5cGUiOiJzb3J0YWJsZSIsInJhbmdlcyI6W3siaGlnaCI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEV0TVRBPSJ9LCJsb3ciOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRFdE1UQT0ifX0seyJoaWdoIjp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1URXRNVEU9In0sImxvdyI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEV0TVRFPSJ9fSx7ImhpZ2giOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRFdE1UST0ifSwibG93Ijp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1URXRNVEk9In19LHsiaGlnaCI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEV0TVRNPSJ9LCJsb3ciOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRFdE1UTT0ifX0seyJoaWdoIjp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1URXRNVFE9In0sImxvdyI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEV0TVRRPSJ9fSx7ImhpZ2giOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRFdE1UVT0ifSwibG93Ijp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1URXRNVFU9In19LHsiaGlnaCI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEV0TVRZPSJ9LCJsb3ciOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRFdE1UWT0ifX0seyJoaWdoIjp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1URXRNVGM9In0sImxvdyI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEV0TVRjPSJ9fSx7ImhpZ2giOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRFdE1UZz0ifSwibG93Ijp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1URXRNVGc9In19LHsiaGlnaCI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEV0TVRrPSJ9LCJsb3ciOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRFdE1Uaz0ifX0seyJoaWdoIjp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1URXRNakE9In0sImxvdyI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEV0TWpBPSJ9fSx7ImhpZ2giOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRFdE1qRT0ifSwibG93Ijp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1URXRNakU9In19LHsiaGlnaCI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEV0TWpJPSJ9LCJsb3ciOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRFdE1qST0ifX0seyJoaWdoIjp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1URXRNak09In0sImxvdyI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEV0TWpNPSJ9fSx7ImhpZ2giOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRFdE1qUT0ifSwibG93Ijp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1URXRNalE9In19LHsiaGlnaCI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEV0TWpVPSJ9LCJsb3ciOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRFdE1qVT0ifX0seyJoaWdoIjp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1URXRNalk9In0sImxvdyI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEV0TWpZPSJ9fSx7ImhpZ2giOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRFdE1qYz0ifSwibG93Ijp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1URXRNamM9In19LHsiaGlnaCI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEV0TWpnPSJ9LCJsb3ciOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRFdE1qZz0ifX0seyJoaWdoIjp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1URXRNams9In0sImxvdyI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEV0TWprPSJ9fSx7ImhpZ2giOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRFdE16QT0ifSwibG93Ijp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1URXRNekE9In19LHsiaGlnaCI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEl0TURFPSJ9LCJsb3ciOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRJdE1ERT0ifX0seyJoaWdoIjp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1USXRNREk9In0sImxvdyI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEl0TURJPSJ9fSx7ImhpZ2giOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRJdE1ETT0ifSwibG93Ijp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1USXRNRE09In19LHsiaGlnaCI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEl0TURRPSJ9LCJsb3ciOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRJdE1EUT0ifX0seyJoaWdoIjp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1USXRNRFU9In0sImxvdyI6eyJib3VuZCI6IkVYQUNUTFkiLCJ0eXBlIjoidmFyY2hhciIsInZhbHVlQmxvY2siOiJEZ0FBQUZaQlVrbEJRa3hGWDFkSlJGUklBUUFBQUFvQUFBQUFDZ0FBQURJd01Ua3RNVEl0TURVPSJ9fSx7ImhpZ2giOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFBb0FBQUFBQ2dBQUFESXdNVGt0TVRJdE1EWT0ifSwibG93Ijp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQW9BQUFBQUNnQUFBREl3TVRrdE1USXRNRFk9In19XSwidHlwZSI6InZhcmNoYXIifX19LHsiY29sdW1uIjp7IkB0eXBlIjoiaGl2ZSIsImNvbHVtblR5cGUiOiJQQVJUSVRJT05fS0VZIiwiaGl2ZUNvbHVtbkluZGV4IjotMTQsImhpdmVUeXBlIjoiYmlnaW50IiwibmFtZSI6ImV4cGVyaW1lbnRfaWQiLCJyZXF1aXJlZFN1YmZpZWxkcyI6W10sInR5cGVTaWduYXR1cmUiOiJiaWdpbnQifSwiZG9tYWluIjp7Im51bGxBbGxvd2VkIjpmYWxzZSwidmFsdWVzIjp7IkB0eXBlIjoic29ydGFibGUiLCJyYW5nZXMiOlt7ImhpZ2giOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6ImJpZ2ludCIsInZhbHVlQmxvY2siOiJDZ0FBQUV4UFRrZGZRVkpTUVZrQkFBQUFBQU1BQUFBQUFBQUEifSwibG93Ijp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJiaWdpbnQiLCJ2YWx1ZUJsb2NrIjoiQ2dBQUFFeFBUa2RmUVZKU1FWa0JBQUFBQUFNQUFBQUFBQUFBIn19XSwidHlwZSI6ImJpZ2ludCJ9fX0seyJjb2x1bW4iOnsiQHR5cGUiOiJoaXZlIiwiY29sdW1uVHlwZSI6IlBBUlRJVElPTl9LRVkiLCJoaXZlQ29sdW1uSW5kZXgiOi0xNiwiaGl2ZVR5cGUiOiJzdHJpbmciLCJuYW1lIjoidW5pdmVyc2UiLCJyZXF1aXJlZFN1YmZpZWxkcyI6W10sInR5cGVTaWduYXR1cmUiOiJ2YXJjaGFyIn0sImRvbWFpbiI6eyJudWxsQWxsb3dlZCI6ZmFsc2UsInZhbHVlcyI6eyJAdHlwZSI6InNvcnRhYmxlIiwicmFuZ2VzIjpbeyJoaWdoIjp7ImJvdW5kIjoiRVhBQ1RMWSIsInR5cGUiOiJ2YXJjaGFyIiwidmFsdWVCbG9jayI6IkRnQUFBRlpCVWtsQlFreEZYMWRKUkZSSUFRQUFBQmNBQUFBQUZ3QUFBSEZ5ZEY5bmNtOTFjRjloWkhOZmNtRnVhMmx1WjE4eCJ9LCJsb3ciOnsiYm91bmQiOiJFWEFDVExZIiwidHlwZSI6InZhcmNoYXIiLCJ2YWx1ZUJsb2NrIjoiRGdBQUFGWkJVa2xCUWt4RlgxZEpSRlJJQVFBQUFCY0FBQUFBRndBQUFIRnlkRjluY205MWNGOWhaSE5mY21GdWEybHVaMTh4In19XSwidHlwZSI6InZhcmNoYXIifX19XX0sInBhcnRpdGlvbkNvbHVtbnMiOlt7IkB0eXBlIjoiaGl2ZSIsImNvbHVtblR5cGUiOiJQQVJUSVRJT05fS0VZIiwiaGl2ZUNvbHVtbkluZGV4IjotMTMsImhpdmVUeXBlIjoic3RyaW5nIiwibmFtZSI6ImF0dHJpYnV0ZWRfZHMiLCJyZXF1aXJlZFN1YmZpZWxkcyI6W10sInR5cGVTaWduYXR1cmUiOiJ2YXJjaGFyIn0seyJAdHlwZSI6ImhpdmUiLCJjb2x1bW5UeXBlIjoiUEFSVElUSU9OX0tFWSIsImhpdmVDb2x1bW5JbmRleCI6LTE0LCJoaXZlVHlwZSI6ImJpZ2ludCIsIm5hbWUiOiJleHBlcmltZW50X2lkIiwicmVxdWlyZWRTdWJmaWVsZHMiOltdLCJ0eXBlU2lnbmF0dXJlIjoiYmlnaW50In0seyJAdHlwZSI6ImhpdmUiLCJjb2x1bW5UeXBlIjoiUEFSVElUSU9OX0tFWSIsImhpdmVDb2x1bW5JbmRleCI6LTE1LCJoaXZlVHlwZSI6InN0cmluZyIsIm5hbWUiOiJkcyIsInJlcXVpcmVkU3ViZmllbGRzIjpbXSwidHlwZVNpZ25hdHVyZSI6InZhcmNoYXIifSx7IkB0eXBlIjoiaGl2ZSIsImNvbHVtblR5cGUiOiJQQVJUSVRJT05fS0VZIiwiaGl2ZUNvbHVtbkluZGV4IjotMTYsImhpdmVUeXBlIjoic3RyaW5nIiwibmFtZSI6InVuaXZlcnNlIiwicmVxdWlyZWRTdWJmaWVsZHMiOltdLCJ0eXBlU2lnbmF0dXJlIjoidmFyY2hhciJ9XSwicHJlZGljYXRlQ29sdW1ucyI6eyJ2YWxpZF9leHBlcmltZW50X3JlYWRpbmciOnsiQHR5cGUiOiJoaXZlIiwiY29sdW1uVHlwZSI6IlJFR1VMQVIiLCJoaXZlQ29sdW1uSW5kZXgiOjE2LCJoaXZlVHlwZSI6ImludCIsIm5hbWUiOiJ2YWxpZF9leHBlcmltZW50X3JlYWRpbmciLCJyZXF1aXJlZFN1YmZpZWxkcyI6W10sInR5cGVTaWduYXR1cmUiOiJpbnRlZ2VyIn19LCJwdXNoZG93bkZpbHRlckVuYWJsZWQiOnRydWUsInJlbWFpbmluZ1ByZWRpY2F0ZSI6eyJAdHlwZSI6ImNvbnN0YW50IiwidHlwZSI6ImJvb2xlYW4iLCJ2YWx1ZUJsb2NrIjoiQ2dBQUFFSlpWRVZmUVZKU1FWa0JBQUFBQUFFPSJ9LCJ0YWJsZVBhdGgiOiJ0ZXN0X3BhdGgiLCJwYXJ0aWFsQWdncmVnYXRpb25zUHVzaGVkRG93biI6dHJ1ZSwiYXBwZW5kUm93TnVtYmVyIjpmYWxzZSwiZm9vdGVyU3RhdHNVbnJlbGlhYmxlIjpmYWxzZSwic2NoZW1hVGFibGVOYW1lIjp7InNjaGVtYSI6InRwY2giLCJ0YWJsZSI6InRlc3Rfc3JlZW5pX3ByZWFnZ19ieV91bnZpZXJzZV9leHBlcmltZW50X2Zvcl9jb25jdXJyZW5jeSJ9LCJ0YWJsZVBhcmFtZXRlcnMiOnsicHJlc3RvX3F1ZXJ5X2lkIjoiMjAyMDAyMTNfMDIyMzExXzAwMDAwX2MzY3p0IiwicHJlc3RvX3ZlcnNpb24iOiJ0ZXN0dmVyc2lvbiJ9fSwidHJhbnNhY3Rpb24iOnsiQHR5cGUiOiJoaXZlIiwidXVpZCI6IjhhNGQ2YzgzLTYwZWUtNDZkZS05NzE1LWJjOTE3NTU2MTlmYSJ9fX19LCJzdGVwIjoiUEFSVElBTCJ9LCJzdGFnZUV4ZWN1dGlvbkRlc2NyaXB0b3IiOnsiZ3JvdXBlZEV4ZWN1dGlvblNjYW5Ob2RlcyI6W10sInN0YWdlRXhlY3V0aW9uU3RyYXRlZ3kiOiJVTkdST1VQRURfRVhFQ1VUSU9OIiwidG90YWxMaWZlc3BhbnMiOjF9LCJzdGF0c0FuZENvc3RzIjp7ImNvc3RzIjp7fSwic3RhdHMiOnt9fSwidGFibGVTY2FuU2NoZWR1bGluZ09yZGVyIjpbIjAiXSwidmFyaWFibGVzIjpbeyJAdHlwZSI6InZhcmlhYmxlIiwibmFtZSI6IiRoYXNodmFsdWVfMjMiLCJ0eXBlIjoiYmlnaW50In0seyJAdHlwZSI6InZhcmlhYmxlIiwibmFtZSI6ImF0dHJpYnV0ZWRfZHMiLCJ0eXBlIjoidmFyY2hhciJ9LHsiQHR5cGUiOiJ2YXJpYWJsZSIsIm5hbWUiOiJldmVudF9iYXNlZF9yZXZlbnVlIiwidHlwZSI6InJlYWwifSx7IkB0eXBlIjoidmFyaWFibGUiLCJuYW1lIjoiaW1wcmVzc2lvbnMiLCJ0eXBlIjoiYmlnaW50In0seyJAdHlwZSI6InZhcmlhYmxlIiwibmFtZSI6InNlZ21lbnQiLCJ0eXBlIjoiaW50ZWdlciJ9LHsiQHR5cGUiOiJ2YXJpYWJsZSIsIm5hbWUiOiJzdW1fMjAiLCJ0eXBlIjoiZG91YmxlIn0seyJAdHlwZSI6InZhcmlhYmxlIiwibmFtZSI6InN1bV8yMSIsInR5cGUiOiJiaWdpbnQifV19 diff --git a/presto-native-execution/presto_cpp/main/tests/data/MetadataUpdates.json b/presto-native-execution/presto_cpp/main/tests/data/MetadataUpdates.json new file mode 100644 index 0000000000000..66db638753fab --- /dev/null +++ b/presto-native-execution/presto_cpp/main/tests/data/MetadataUpdates.json @@ -0,0 +1,15 @@ +{ + "connectorId": "hive", + "metadataUpdates": [ + { + "@type": "hive", + "requestId": "a1b2c3d4-e5f6-7a8b-9c0d-ef1a2b3c4d5e", + "schemaTableName": { + "schema": "test_schema", + "table": "test_table" + }, + "partitionName": "ds=2023-01-01", + "fileName": "test_file" + } + ] +} diff --git a/presto-native-execution/presto_cpp/main/tests/data/OperatorStats.json b/presto-native-execution/presto_cpp/main/tests/data/OperatorStats.json new file mode 100644 index 0000000000000..b596e4213bdfe --- /dev/null +++ b/presto-native-execution/presto_cpp/main/tests/data/OperatorStats.json @@ -0,0 +1,52 @@ +{ + "stageId": 1, + "stageExecutionId": 2, + "pipelineId": 3, + "operatorId": 4, + "planNodeId": "node-5", + "operatorType": "sort", + "totalDrivers": 10, + "isBlockedCalls": 0, + "isBlockedWall": "10ms", + "isBlockedCpu": "20ms", + "isBlockedAllocationInBytes": 0, + "addInputCalls": 100, + "addInputWall": "30ms", + "addInputCpu": "40ms", + "addInputAllocationInBytes": 1024, + "rawInputDataSizeInBytes": 1048576, + "rawInputPositions": 10000, + "inputDataSizeInBytes": 1048576, + "inputPositions": 10000, + "sumSquaredInputPositions": 100000000.0, + "getOutputCalls": 50, + "getOutputWall": "50ms", + "getOutputCpu": "60ms", + "getOutputAllocationInBytes": 512, + "outputDataSizeInBytes": 524288, + "outputPositions": 5000, + "physicalWrittenDataSizeInBytes": 524288, + "additionalCpu": "70ms", + "blockedWall": "80ms", + "finishCalls": 1, + "finishWall": "90ms", + "finishCpu": "1s", + "finishAllocationInBytes": 256, + "userMemoryReservationInBytes": 1048576, + "revocableMemoryReservationInBytes": 0, + "systemMemoryReservationInBytes": 1048576, + "peakUserMemoryReservationInBytes": 1048576, + "peakSystemMemoryReservationInBytes": 1048576, + "peakTotalMemoryReservationInBytes": 2097152, + "spilledDataSizeInBytes": 0, + "runtimeStats": {}, + "info": {}, + "dynamicFilterStats": { + "producerNodeIds": ["node-1", "node-2"] + }, + "nullJoinBuildKeyCount": 1, + "joinBuildKeyCount": 2, + "nullJoinProbeKeyCount": 3, + "joinProbeKeyCount": 4, + "blockedReason": "WAITING_FOR_MEMORY" + } diff --git a/presto-native-execution/presto_cpp/main/tests/data/OperatorStatsEmptyBlockedReason.json b/presto-native-execution/presto_cpp/main/tests/data/OperatorStatsEmptyBlockedReason.json new file mode 100644 index 0000000000000..9d788f0a59b8b --- /dev/null +++ b/presto-native-execution/presto_cpp/main/tests/data/OperatorStatsEmptyBlockedReason.json @@ -0,0 +1,51 @@ +{ + "stageId": 1, + "stageExecutionId": 2, + "pipelineId": 3, + "operatorId": 4, + "planNodeId": "node-5", + "operatorType": "sort", + "totalDrivers": 10, + "isBlockedCalls": 0, + "isBlockedWall": "10ms", + "isBlockedCpu": "20ms", + "isBlockedAllocationInBytes": 0, + "addInputCalls": 100, + "addInputWall": "30ms", + "addInputCpu": "40ms", + "addInputAllocationInBytes": 1024, + "rawInputDataSizeInBytes": 1048576, + "rawInputPositions": 10000, + "inputDataSizeInBytes": 1048576, + "inputPositions": 10000, + "sumSquaredInputPositions": 100000000.0, + "getOutputCalls": 50, + "getOutputWall": "50ms", + "getOutputCpu": "60ms", + "getOutputAllocationInBytes": 512, + "outputDataSizeInBytes": 524288, + "outputPositions": 5000, + "physicalWrittenDataSizeInBytes": 524288, + "additionalCpu": "70ms", + "blockedWall": "80ms", + "finishCalls": 1, + "finishWall": "90ms", + "finishCpu": "1s", + "finishAllocationInBytes": 256, + "userMemoryReservationInBytes": 1048576, + "revocableMemoryReservationInBytes": 0, + "systemMemoryReservationInBytes": 1048576, + "peakUserMemoryReservationInBytes": 1048576, + "peakSystemMemoryReservationInBytes": 1048576, + "peakTotalMemoryReservationInBytes": 2097152, + "spilledDataSizeInBytes": 0, + "runtimeStats": {}, + "info": {}, + "dynamicFilterStats": { + "producerNodeIds": ["node-1", "node-2"] + }, + "nullJoinBuildKeyCount": 1, + "joinBuildKeyCount": 2, + "nullJoinProbeKeyCount": 3, + "joinProbeKeyCount": 4 +} diff --git a/presto-native-execution/presto_cpp/main/tests/data/Split.json b/presto-native-execution/presto_cpp/main/tests/data/Split.json new file mode 100644 index 0000000000000..12a253142981e --- /dev/null +++ b/presto-native-execution/presto_cpp/main/tests/data/Split.json @@ -0,0 +1,70 @@ +{ + "connectorId": "hive", + "connectorSplit": { + "@type": "hive", + "nodeSelectionStrategy": "NO_PREFERENCE", + "addresses": [], + "database": "tpch", + "fileSize": 396036393, + "length": 33554432, + "partitionDataColumnCount": 148, + "cacheQuota": {"cacheQuotaScope": "TABLE"}, + "fileSplit": { + "path": "test_path", + "start": 1, + "length": 2, + "fileSize": 3, + "fileModifiedTime": 4, + "customSplitInfo": {}, + "affinitySchedulingFileSectionIndex": 5 + }, + "tableToPartitionMapping": { + "partitionSchemaDifference": {} + }, + "redundantColumnDomains": [], + "splitWeight": 1, + "partitionKeys": [ + { + "name": "attributed_ds", + "value": "2019-12-06" + }, + { + "name": "experiment_id", + "value": "3" + }, + { + "name": "ds", + "value": "2019-12-07" + }, + { + "name": "universe", + "value": "qrt_group_ads_ranking_1" + } + ], + "partitionName": "test_partition", + "partitionSchemaDifference": {}, + "path": "/test_path/test.orc", + "s3SelectPushdownEnabled": false, + "start": 301989888, + "storage": { + "location": "/test_path", + "serdeParameters": {}, + "parameters": {}, + "skewed": false, + "storageFormat": { + "inputFormat": "com.facebook.hive.orc.OrcInputFormat", + "outputFormat": "com.facebook.hive.orc.OrcOutputFormat", + "serDe": "com.facebook.hive.orc.OrcSerde" + } + }, + "table": "test_table" + }, + "lifespan": "TaskWide", + "splitContext": { + "cacheable": true + }, + "transactionHandle": { + "@type": "hive", + "uuid": "8a4d6c83-60ee-46de-9715-bc91755619fa" + } +} diff --git a/presto-native-execution/presto_cpp/main/tests/data/TableWriteInfo.json b/presto-native-execution/presto_cpp/main/tests/data/TableWriteInfo.json new file mode 100644 index 0000000000000..a9ff5cc289993 --- /dev/null +++ b/presto-native-execution/presto_cpp/main/tests/data/TableWriteInfo.json @@ -0,0 +1,18 @@ +{ + "analyzeTableHandle": { + "connectorId": "hive", + "transactionHandle": { + "@type": "hive", + "uuid": "8a4d6c83-60ee-46de-9715-bc91755619fa" + }, + "connectorHandle": { + "@type": "hive", + "schemaName": "test_schema", + "tableName": "test_table", + "analyzePartitionValues": [ + ["value1", "value2"], + ["value3", "value4"] + ] + } + } +} diff --git a/presto-native-execution/presto_cpp/main/tests/data/TaskInfo.json b/presto-native-execution/presto_cpp/main/tests/data/TaskInfo.json new file mode 100644 index 0000000000000..7a6d3cdaa3018 --- /dev/null +++ b/presto-native-execution/presto_cpp/main/tests/data/TaskInfo.json @@ -0,0 +1,145 @@ +{ + "taskId": "20221102_214710_00000_5w3xj.123.456.789.10", + "taskStatus": { + "taskInstanceIdLeastSignificantBits": 123, + "taskInstanceIdMostSignificantBits": 456, + "version": 789, + "state": "RUNNING", + "self": "http://localhost:8080/v1/task/20221102_214710_00000_5w3xj.123.456.7", + "completedDriverGroups": ["TaskWide"], + "failures": [], + "queuedPartitionedDrivers": 1, + "runningPartitionedDrivers": 2, + "outputBufferUtilization": 3, + "outputBufferOverutilized": false, + "physicalWrittenDataSizeInBytes": 4, + "memoryReservationInBytes": 5, + "systemMemoryReservationInBytes": 6, + "fullGcCount": 7, + "fullGcTimeInMillis": 8, + "peakNodeTotalMemoryReservationInBytes": 9, + "totalCpuTimeInNanos": 10, + "taskAgeInMillis": 11, + "queuedPartitionedSplitsWeight": 12, + "runningPartitionedSplitsWeight": 13 + }, + "lastHeartbeatInMillis": 123, + "outputBuffers": { + "type": "PAGES", + "state": "OPEN", + "buffers": [ + { + "bufferId": "100", + "finished": true, + "bufferedPages": 200, + "pagesSent": 300, + "pageBufferInfo": { + "partition": 400, + "bufferedPages": 500, + "bufferedBytes": 600, + "rowsAdded": 700, + "pagesAdded": 800 + } + }, + { + "bufferId": "200", + "finished": true, + "bufferedPages": 100, + "pagesSent": 200, + "pageBufferInfo": { + "partition": 300, + "bufferedPages": 400, + "bufferedBytes": 500, + "rowsAdded": 600, + "pagesAdded": 700 + } + } + ], + "canAddBuffers": true, + "canAddPages": true, + "totalBufferedBytes": 123, + "totalBufferedPages": 456, + "totalRowsSent": 789, + "totalPagesSent": 101112 + }, + "noMoreSplits": [], + "stats": { + "createTimeInMillis": 123, + "firstStartTimeInMillis": 456, + "lastStartTimeInMillis": 789, + "lastEndTimeInMillis": 123, + "endTimeInMillis": 456, + "elapsedTimeInNanos": 789, + "queuedTimeInNanos": 123, + "totalDrivers": 456, + "queuedDrivers": 789, + "runningDrivers": 123, + "blockedDrivers": 456, + "completedDrivers": 789, + "cumulativeUserMemory": 123, + "cumulativeTotalMemory": 456, + "userMemoryReservationInBytes": 123, + "revocableMemoryReservationInBytes": 456, + "systemMemoryReservationInBytes": 789, + "peakUserMemoryInBytes": 123, + "peakTotalMemoryInBytes": 456, + "peakNodeTotalMemoryInBytes": 789, + "totalScheduledTimeInNanos": 123, + "totalCpuTimeInNanos": 456, + "totalBlockedTimeInNanos": 789, + "fullyBlocked": true, + "blockedReasons": [ + "WAITING_FOR_MEMORY" + ], + "totalAllocationInBytes": 123, + "rawInputDataSizeInBytes": 456, + "rawInputPositions": 789, + "processedInputDataSizeInBytes": 123, + "processedInputPositions": 456, + "outputDataSizeInBytes": 789, + "outputPositions": 123, + "physicalWrittenDataSizeInBytes": 456, + "pipelines": [], + "queuedPartitionedDrivers": 123, + "queuedPartitionedSplitsWeight": 456, + "runningPartitionedDrivers": 789, + "runningPartitionedSplitsWeight": 123, + "fullGcCount": 456, + "fullGcTimeInMillis": 789, + "runtimeStats": { + "test_metric1": { + "name": "test_metric1", + "sum": 123, + "count": 456, + "max": 789, + "min": 123, + "unit": "NANO" + }, + "test_metric2": { + "name": "test_metric2", + "sum": 234, + "count": 567, + "max": 890, + "min": 123, + "unit": "NANO" + } + } + }, + "needsPlan": false, + "metadataUpdates": { + "connectorId": "hive", + "metadataUpdates": [ + { + "@type": "hive", + "requestId": "a1b2c3d4-e5f6-7a8b-9c0d-ef1a2b3c4d5e", + "schemaTableName": { + "schema": "test_schema", + "table": "test_table" + }, + "partitionName": "ds=2023-01-01", + "fileName": "test_file" + } + ] + }, + "nodeId": "test_node" +} diff --git a/presto-native-execution/presto_cpp/main/thrift/Makefile b/presto-native-execution/presto_cpp/main/thrift/Makefile deleted file mode 100644 index 769d05da09196..0000000000000 --- a/presto-native-execution/presto_cpp/main/thrift/Makefile +++ /dev/null @@ -1,30 +0,0 @@ -# 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. - -all: ProtocolToThrift.h ProtocolToThrift.cpp - -ProtocolToThrift.h: ProtocolToThrift-hpp.mustache presto_protocol-to-thrift-json.json - echo "// DO NOT EDIT : This file is generated by presto_protocol-to-thrift-json.py" > ProtocolToThrift.h - chevron -d presto_protocol-to-thrift-json.json ProtocolToThrift-hpp.mustache >> ProtocolToThrift.h - clang-format -style=file -i ProtocolToThrift.h - -ProtocolToThrift.cpp: ProtocolToThrift-cpp.mustache presto_protocol-to-thrift-json.json - echo "// DO NOT EDIT : This file is generated by presto_protocol-to-thrift-json.py" > ProtocolToThrift.cpp - chevron -d presto_protocol-to-thrift-json.json ProtocolToThrift-cpp.mustache >> ProtocolToThrift.cpp - clang-format -style=file -i ProtocolToThrift.cpp - -presto_protocol-to-thrift-json.json: presto_protocol-to-thrift-json.py presto_protocol-to-thrift-json.yml presto_thrift.json ../../presto_protocol/presto_protocol.json - ./presto_protocol-to-thrift-json.py presto_thrift.json ../../presto_protocol/presto_protocol.json | jq . > presto_protocol-to-thrift-json.json - -presto_thrift.json: presto_thrift.thrift ./thrift2json.py - ./thrift2json.py presto_thrift.thrift | jq . > presto_thrift.json - diff --git a/presto-native-execution/presto_cpp/main/thrift/ProtocolToThrift-cpp.mustache b/presto-native-execution/presto_cpp/main/thrift/ProtocolToThrift-cpp.mustache deleted file mode 100644 index 49776abc87ba9..0000000000000 --- a/presto-native-execution/presto_cpp/main/thrift/ProtocolToThrift-cpp.mustache +++ /dev/null @@ -1,92 +0,0 @@ -/* - * 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. - */ - -{{! Select all the comment items and expand them here }} -{{#.}} -{{#comment}} -{{comment}} -{{/comment}} -{{/.}} - -#include "presto_cpp/main/thrift/ProtocolToThrift.h" - -namespace facebook::presto { - -// These could be covered by a more general template but this way only -// conversions to supported Thrift data types can be generated. -void toThrift(const std::string& proto, std::string& thrift) { - thrift = proto; -} -void toThrift(const bool& proto, bool& thrift) { - thrift = proto; -} -void toThrift(const int32_t& proto, int32_t& thrift) { - thrift = proto; -} -void toThrift(const int64_t& proto, int64_t& thrift) { - thrift = proto; -} -void toThrift(const double& proto, double& thrift) { - thrift = proto; -} - -template -void toThrift(const std::shared_ptr

& proto, std::shared_ptr& thrift) { - if (proto) { - thrift = std::make_shared(); - toThrift(*proto, *thrift); - } -} - -template -void toThrift(const std::vector& v, std::set& s) { - S toItem; - for (const auto& fromItem : v) { - toThrift(fromItem, toItem); - s.insert(toItem); - } -} - -template -void toThrift(const std::vector

& p, std::vector& t) { - T toItem; - for (const auto& fromItem : p) { - toThrift(fromItem, toItem); - t.push_back(toItem); - } -} - -{{! Select all the items and expand either the "hinc" member or the "struct", "enum" members }} -{{#.}} -{{#cinc}} -{{&cinc}} -{{/cinc}} -{{^cinc}} -{{#struct}} -void toThrift(const protocol::{{class_name}}& proto, thrift::{{&class_name}}& thrift) { - {{#fields}} - toThrift(proto.{{proto_name}}, {{^optional}}*{{/optional}}thrift.{{field_name}}_ref()); - {{/fields}} -} -{{/struct}} -{{#enum}} -void toThrift(const protocol::{{class_name}}& proto, thrift::{{class_name}}& thrift) { - thrift = (thrift::{{class_name}})(static_cast(proto)); -} -{{/enum}} -{{/cinc}} -{{/.}} - -} - diff --git a/presto-native-execution/presto_cpp/main/thrift/ProtocolToThrift-hpp.mustache b/presto-native-execution/presto_cpp/main/thrift/ProtocolToThrift-hpp.mustache deleted file mode 100644 index 0279c3b3b64fd..0000000000000 --- a/presto-native-execution/presto_cpp/main/thrift/ProtocolToThrift-hpp.mustache +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -#pragma once - -{{! Select all the comment items and expand them here }} -{{#.}} -{{#comment}} -{{comment}} -{{/comment}} -{{/.}} - -#include "presto_cpp/main/thrift/gen-cpp2/PrestoThrift.h" -#include "presto_cpp/presto_protocol/presto_protocol.h" - -namespace facebook::presto { - -{{! Select all the items and expand either the "hinc" member or the "struct", "enum" members }} -{{#.}} -{{#hinc}} -{{&hinc}} -{{/hinc}} -{{^hinc}} -{{#struct}} -void toThrift(const protocol::{{class_name}}& proto, thrift::{{class_name}}& thrift); -{{/struct}} -{{#enum}} -void toThrift(protocol::{{class_name}}& proto, thrift::{{class_name}}& thrift); -{{/enum}} -{{/hinc}} -{{/.}} - -} // namespace facebook::presto diff --git a/presto-native-execution/presto_cpp/main/thrift/ProtocolToThrift.cpp b/presto-native-execution/presto_cpp/main/thrift/ProtocolToThrift.cpp index 6875d2fb20324..8a2f3384bb9de 100644 --- a/presto-native-execution/presto_cpp/main/thrift/ProtocolToThrift.cpp +++ b/presto-native-execution/presto_cpp/main/thrift/ProtocolToThrift.cpp @@ -13,131 +13,662 @@ * limitations under the License. */ -// This file is generated DO NOT EDIT @generated -// This file is generated DO NOT EDIT @generated - #include "presto_cpp/main/thrift/ProtocolToThrift.h" -namespace facebook::presto { - -// These could be covered by a more general template but this way only -// conversions to supported Thrift data types can be generated. -void toThrift(const std::string& proto, std::string& thrift) { - thrift = proto; -} -void toThrift(const bool& proto, bool& thrift) { - thrift = proto; -} -void toThrift(const int32_t& proto, int32_t& thrift) { - thrift = proto; -} -void toThrift(const int64_t& proto, int64_t& thrift) { - thrift = proto; -} -void toThrift(const double& proto, double& thrift) { - thrift = proto; -} - -template -void toThrift(const std::shared_ptr

& proto, std::shared_ptr& thrift) { - if (proto) { - thrift = std::make_shared(); - toThrift(*proto, *thrift); - } -} - -template -void toThrift(const std::vector& v, std::set& s) { - S toItem; - for (const auto& fromItem : v) { - toThrift(fromItem, toItem); - s.insert(toItem); - } -} - -template -void toThrift(const std::vector

& p, std::vector& t) { - T toItem; - for (const auto& fromItem : p) { - toThrift(fromItem, toItem); - t.push_back(toItem); - } -} - -void toThrift(const protocol::TaskState& proto, thrift::TaskState& thrift) { - thrift = (thrift::TaskState)((int)proto); -} -void toThrift(const protocol::ErrorType& proto, thrift::ErrorType& thrift) { - thrift = (thrift::ErrorType)((int)proto); -} -void toThrift(const protocol::Lifespan& proto, thrift::Lifespan& thrift) { - toThrift(proto.isgroup, *thrift.grouped_ref()); - toThrift(proto.groupid, *thrift.groupId_ref()); -} -void toThrift( - const protocol::ErrorLocation& proto, - thrift::ErrorLocation& thrift) { - toThrift(proto.lineNumber, *thrift.lineNumber_ref()); - toThrift(proto.columnNumber, *thrift.columnNumber_ref()); -} -void toThrift(const protocol::HostAddress& proto, thrift::HostAddress& thrift) { - std::vector parts; - folly::split(":", proto, parts); - if (parts.size() == 2) { - thrift.host_ref() = parts[0]; - thrift.port_ref() = std::stoi(parts[1]); - } -} -void toThrift(const protocol::TaskStatus& proto, thrift::TaskStatus& thrift) { - toThrift( - proto.taskInstanceIdLeastSignificantBits, - *thrift.taskInstanceIdLeastSignificantBits_ref()); - toThrift( - proto.taskInstanceIdMostSignificantBits, - *thrift.taskInstanceIdMostSignificantBits_ref()); - toThrift(proto.version, *thrift.version_ref()); - toThrift(proto.state, *thrift.state_ref()); - toThrift(proto.self, *thrift.taskName_ref()); - toThrift(proto.completedDriverGroups, *thrift.completedDriverGroups_ref()); - toThrift(proto.failures, *thrift.failures_ref()); - toThrift( - proto.queuedPartitionedDrivers, *thrift.queuedPartitionedDrivers_ref()); - toThrift( - proto.runningPartitionedDrivers, *thrift.runningPartitionedDrivers_ref()); - toThrift( - proto.outputBufferUtilization, *thrift.outputBufferUtilization_ref()); - toThrift( - proto.outputBufferOverutilized, *thrift.outputBufferOverutilized_ref()); - toThrift( - proto.physicalWrittenDataSizeInBytes, - *thrift.physicalWrittenDataSizeInBytes_ref()); - toThrift( - proto.memoryReservationInBytes, *thrift.memoryReservationInBytes_ref()); - toThrift( - proto.systemMemoryReservationInBytes, - *thrift.systemMemoryReservationInBytes_ref()); - toThrift(proto.fullGcCount, *thrift.fullGcCount_ref()); - toThrift(proto.fullGcTimeInMillis, *thrift.fullGcTimeInMillis_ref()); - toThrift( - proto.peakNodeTotalMemoryReservationInBytes, - *thrift.peakNodeTotalMemoryReservationInBytes_ref()); -} -void toThrift(const protocol::ErrorCode& proto, thrift::ErrorCode& thrift) { - toThrift(proto.code, *thrift.code_ref()); - toThrift(proto.name, *thrift.name_ref()); - toThrift(proto.type, *thrift.type_ref()); -} -void toThrift( - const protocol::ExecutionFailureInfo& proto, - thrift::ExecutionFailureInfo& thrift) { - toThrift(proto.type, *thrift.type_ref()); - toThrift(proto.message, *thrift.message_ref()); - toThrift(proto.cause, thrift.cause_ref()); - toThrift(proto.suppressed, *thrift.suppressed_ref()); - toThrift(proto.stack, *thrift.stack_ref()); - toThrift(proto.errorLocation, *thrift.errorLocation_ref()); - toThrift(proto.errorCode, *thrift.errorCode_ref()); - toThrift(proto.remoteHost, *thrift.remoteHost_ref()); -} +namespace facebook::presto::thrift { + + // Primitive types + void toThrift(const std::string& proto, std::string& thrift) { + thrift = proto; + } + void toThrift(const bool& proto, bool& thrift) { + thrift = proto; + } + void toThrift(const int32_t& proto, int32_t& thrift) { + thrift = proto; + } + void toThrift(const int64_t& proto, int64_t& thrift) { + thrift = proto; + } + void toThrift(const double& proto, double& thrift) { + thrift = proto; + } + void toThrift(const facebook::presto::protocol::Duration& duration, double& thrift) { + thrift = duration.getValue(facebook::presto::protocol::TimeUnit::MILLISECONDS); + } + + // Templates + template + void toThrift(const E& protoEnum, C& thriftEnum) { + thriftEnum = static_cast(static_cast(protoEnum)); + } + + template + void toThrift(const std::shared_ptr

& proto, std::shared_ptr& thrift) { + if (proto) { + thrift = std::make_shared(); + toThrift(*proto, *thrift); + } + } + + template + void toThrift(const std::vector& protoVector, std::set& thriftSet) { + S toItem; + for (const auto& fromItem : protoVector) { + toThrift(fromItem, toItem); + thriftSet.insert(std::move(toItem)); + } + } + + template + void toThrift(const std::vector

& protoVector, std::vector& thriftVector) { + thriftVector.reserve(protoVector.size()); + T toItem; + for (const auto& fromItem : protoVector) { + toThrift(fromItem, toItem); + thriftVector.emplace_back(std::move(toItem)); + } + } + + template + void toThrift(const std::map& protoMap, std::map& thriftMap) { + K2 toKey; + V2 toValue; + for (const auto& [fromKey, fromValue] : protoMap) { + toThrift(fromKey, toKey); + toThrift(fromValue, toValue); + thriftMap.emplace(std::move(toKey), std::move(toValue)); + } + } + + template + void toThrift(const std::shared_ptr

& proto, apache::thrift::optional_field_ref thrift) { + if (proto) { + thrift.ensure(); + toThrift(*proto, apache::thrift::can_throw(*thrift)); + } + } + + // TaskStatus types + void toThrift(const facebook::presto::protocol::TaskStatus& taskStatus, thrift::TaskStatus& thriftTaskStatus) { + toThrift(taskStatus.taskInstanceIdLeastSignificantBits, *thriftTaskStatus.taskInstanceIdLeastSignificantBits_ref()); + toThrift(taskStatus.taskInstanceIdMostSignificantBits, *thriftTaskStatus.taskInstanceIdMostSignificantBits_ref()); + toThrift(taskStatus.version, *thriftTaskStatus.version_ref()); + toThrift(taskStatus.state, *thriftTaskStatus.state_ref()); + toThrift(taskStatus.self, *thriftTaskStatus.selfUri_ref()); + toThrift(taskStatus.completedDriverGroups, *thriftTaskStatus.completedDriverGroups_ref()); + toThrift(taskStatus.failures, *thriftTaskStatus.failures_ref()); + toThrift(taskStatus.queuedPartitionedDrivers, *thriftTaskStatus.queuedPartitionedDrivers_ref()); + toThrift(taskStatus.runningPartitionedDrivers, *thriftTaskStatus.runningPartitionedDrivers_ref()); + toThrift(taskStatus.outputBufferUtilization, *thriftTaskStatus.outputBufferUtilization_ref()); + toThrift(taskStatus.outputBufferOverutilized, *thriftTaskStatus.outputBufferOverutilized_ref()); + toThrift(taskStatus.physicalWrittenDataSizeInBytes, *thriftTaskStatus.physicalWrittenDataSizeInBytes_ref()); + toThrift(taskStatus.memoryReservationInBytes, *thriftTaskStatus.memoryReservationInBytes_ref()); + toThrift(taskStatus.systemMemoryReservationInBytes, *thriftTaskStatus.systemMemoryReservationInBytes_ref()); + toThrift(taskStatus.peakNodeTotalMemoryReservationInBytes, *thriftTaskStatus.peakNodeTotalMemoryReservationInBytes_ref()); + toThrift(taskStatus.fullGcCount, *thriftTaskStatus.fullGcCount_ref()); + toThrift(taskStatus.fullGcTimeInMillis, *thriftTaskStatus.fullGcTimeInMillis_ref()); + toThrift(taskStatus.totalCpuTimeInNanos, *thriftTaskStatus.totalCpuTimeInNanos_ref()); + toThrift(taskStatus.taskAgeInMillis, *thriftTaskStatus.taskAgeInMillis_ref()); + toThrift(taskStatus.queuedPartitionedSplitsWeight, *thriftTaskStatus.queuedPartitionedSplitsWeight_ref()); + toThrift(taskStatus.runningPartitionedSplitsWeight, *thriftTaskStatus.runningPartitionedSplitsWeight_ref()); + } + + void toThrift(const facebook::presto::protocol::Lifespan& lifespan, thrift::Lifespan& thriftLifeSpan) { + toThrift(lifespan.isgroup, *thriftLifeSpan.grouped_ref()); + toThrift(lifespan.groupid, *thriftLifeSpan.groupId_ref()); + } + + void toThrift(const facebook::presto::protocol::ExecutionFailureInfo& executionFailureInfo, thrift::ExecutionFailureInfo& thriftExecutionFailureInfo) { + toThrift(executionFailureInfo.type, *thriftExecutionFailureInfo.type_ref()); + toThrift(executionFailureInfo.message, *thriftExecutionFailureInfo.message_ref()); + toThrift(executionFailureInfo.cause, thriftExecutionFailureInfo.cause_ref()); + toThrift(executionFailureInfo.suppressed, *thriftExecutionFailureInfo.suppressed_ref()); + toThrift(executionFailureInfo.stack, *thriftExecutionFailureInfo.stack_ref()); + toThrift(executionFailureInfo.errorLocation, *thriftExecutionFailureInfo.errorLocation_ref()); + toThrift(executionFailureInfo.errorCode, *thriftExecutionFailureInfo.errorCode_ref()); + toThrift(executionFailureInfo.remoteHost, *thriftExecutionFailureInfo.remoteHost_ref()); + toThrift(executionFailureInfo.errorCause, *thriftExecutionFailureInfo.errorCause_ref()); + } + + void toThrift(const facebook::presto::protocol::ErrorLocation& errorLocation, thrift::ErrorLocation& thriftErrorLocation) { + toThrift(errorLocation.lineNumber, *thriftErrorLocation.lineNumber_ref()); + toThrift(errorLocation.columnNumber, *thriftErrorLocation.columnNumber_ref()); + } + + void toThrift(const facebook::presto::protocol::ErrorCode& errorCode, thrift::ErrorCode& thriftErrorCode) { + toThrift(errorCode.code, *thriftErrorCode.code_ref()); + toThrift(errorCode.name, *thriftErrorCode.name_ref()); + toThrift(errorCode.type, *thriftErrorCode.type_ref()); + toThrift(errorCode.retriable, *thriftErrorCode.retriable_ref()); + } + + void toThrift(const facebook::presto::protocol::HostAddress& hostAddress, thrift::HostAddress& thriftHostAddress) { + std::vector parts; + folly::split(":", hostAddress, parts); + if (parts.size() == 2) { + thriftHostAddress.host_ref() = parts[0]; + thriftHostAddress.port_ref() = std::stoi(parts[1]); + } + } + + // TaskInfo types + void toThrift(const facebook::presto::protocol::TaskInfo& taskInfo, thrift::TaskInfo& thriftTaskInfo) { + toThrift(taskInfo.taskId, *thriftTaskInfo.taskId_ref()); + toThrift(taskInfo.taskStatus, *thriftTaskInfo.taskStatus_ref()); + toThrift(taskInfo.lastHeartbeatInMillis, *thriftTaskInfo.lastHeartbeatInMillis_ref()); + toThrift(taskInfo.outputBuffers, *thriftTaskInfo.outputBuffers_ref()); + toThrift(taskInfo.noMoreSplits, *thriftTaskInfo.noMoreSplits_ref()); + toThrift(taskInfo.stats, *thriftTaskInfo.stats_ref()); + toThrift(taskInfo.needsPlan, *thriftTaskInfo.needsPlan_ref()); + toThrift(taskInfo.metadataUpdates, *thriftTaskInfo.metadataUpdates_ref()); + toThrift(taskInfo.nodeId, *thriftTaskInfo.nodeId_ref()); + } + + void toThrift(const facebook::presto::protocol::MetadataUpdates& metadataUpdates, thrift::MetadataUpdatesWrapper& thriftMetadataUpdatesWrapper) { + toThrift(metadataUpdates, *thriftMetadataUpdatesWrapper.metadataUpdates_ref()); + } + + void toThrift(const facebook::presto::protocol::MetadataUpdates& metadataUpdates, std::string& thriftMetadataUpdates) { + json jsonMetadataUpdates = metadataUpdates; + std::string str = jsonMetadataUpdates.dump(); + toThrift(str, thriftMetadataUpdates); + } + + void toThrift(const facebook::presto::protocol::TaskId& taskId, thrift::TaskId& thriftTaskId) { + std::vector parts; + folly::split(".", taskId, parts); + if (parts.size() == 5) { + thriftTaskId.stageExecutionId_ref()->stageId_ref()->queryId_ref() = parts[0]; + thriftTaskId.stageExecutionId_ref()->stageId_ref()->id_ref() = std::stoi(parts[1]); + thriftTaskId.stageExecutionId_ref()->id_ref() = std::stoi(parts[2]); + thriftTaskId.id_ref() = std::stoi(parts[3]); + thriftTaskId.attemptNumber_ref() = std::stoi(parts[4]); + } + } + + void toThrift(const facebook::presto::protocol::OutputBufferInfo& outputBufferInfo, thrift::OutputBufferInfo& thriftOutputBufferInfo) { + toThrift(outputBufferInfo.type, *thriftOutputBufferInfo.type_ref()); + toThrift(outputBufferInfo.state, *thriftOutputBufferInfo.state_ref()); + toThrift(outputBufferInfo.canAddBuffers, *thriftOutputBufferInfo.canAddBuffers_ref()); + toThrift(outputBufferInfo.canAddPages, *thriftOutputBufferInfo.canAddPages_ref()); + toThrift(outputBufferInfo.totalBufferedBytes, *thriftOutputBufferInfo.totalBufferedBytes_ref()); + toThrift(outputBufferInfo.totalBufferedPages, *thriftOutputBufferInfo.totalBufferedPages_ref()); + toThrift(outputBufferInfo.totalRowsSent, *thriftOutputBufferInfo.totalRowsSent_ref()); + toThrift(outputBufferInfo.totalPagesSent, *thriftOutputBufferInfo.totalPagesSent_ref()); + toThrift(outputBufferInfo.buffers, *thriftOutputBufferInfo.buffers_ref()); + } + + void toThrift(const facebook::presto::protocol::BufferInfo& bufferInfo, thrift::BufferInfo& thriftBufferInfo) { + toThrift(std::stoi(bufferInfo.bufferId), *thriftBufferInfo.bufferId_ref()->id()); + toThrift(bufferInfo.finished, *thriftBufferInfo.finished_ref()); + toThrift(bufferInfo.bufferedPages, *thriftBufferInfo.bufferedPages_ref()); + toThrift(bufferInfo.pagesSent, *thriftBufferInfo.pagesSent_ref()); + toThrift(bufferInfo.pageBufferInfo, *thriftBufferInfo.pageBufferInfo_ref()); + } + + void toThrift(const facebook::presto::protocol::PageBufferInfo& pageBufferInfo, thrift::PageBufferInfo& thriftPageBufferInfo) { + toThrift(pageBufferInfo.partition, *thriftPageBufferInfo.partition_ref()); + toThrift(pageBufferInfo.bufferedPages, *thriftPageBufferInfo.bufferedPages_ref()); + toThrift(pageBufferInfo.bufferedBytes, *thriftPageBufferInfo.bufferedBytes_ref()); + toThrift(pageBufferInfo.rowsAdded, *thriftPageBufferInfo.rowsAdded_ref()); + toThrift(pageBufferInfo.pagesAdded, *thriftPageBufferInfo.pagesAdded_ref()); + } + + void toThrift(const facebook::presto::protocol::TaskStats& taskStats, thrift::TaskStats& thriftTaskStats) { + toThrift(taskStats.createTimeInMillis, *thriftTaskStats.createTimeInMillis_ref()); + toThrift(taskStats.firstStartTimeInMillis, *thriftTaskStats.firstStartTimeInMillis_ref()); + toThrift(taskStats.lastStartTimeInMillis, *thriftTaskStats.lastStartTimeInMillis_ref()); + toThrift(taskStats.lastEndTimeInMillis, *thriftTaskStats.lastEndTimeInMillis_ref()); + toThrift(taskStats.endTimeInMillis, *thriftTaskStats.endTimeInMillis_ref()); + toThrift(taskStats.elapsedTimeInNanos, *thriftTaskStats.elapsedTimeInNanos_ref()); + toThrift(taskStats.queuedTimeInNanos, *thriftTaskStats.queuedTimeInNanos_ref()); + toThrift(taskStats.totalDrivers, *thriftTaskStats.totalDrivers_ref()); + toThrift(taskStats.queuedDrivers, *thriftTaskStats.queuedDrivers_ref()); + toThrift(taskStats.runningDrivers, *thriftTaskStats.runningDrivers_ref()); + toThrift(taskStats.blockedDrivers, *thriftTaskStats.blockedDrivers_ref()); + toThrift(taskStats.completedDrivers, *thriftTaskStats.completedDrivers_ref()); + toThrift(taskStats.cumulativeUserMemory, *thriftTaskStats.cumulativeUserMemory_ref()); + toThrift(taskStats.cumulativeTotalMemory, *thriftTaskStats.cumulativeTotalMemory_ref()); + toThrift(taskStats.userMemoryReservationInBytes, *thriftTaskStats.userMemoryReservationInBytes_ref()); + toThrift(taskStats.revocableMemoryReservationInBytes, *thriftTaskStats.revocableMemoryReservationInBytes_ref()); + toThrift(taskStats.systemMemoryReservationInBytes, *thriftTaskStats.systemMemoryReservationInBytes_ref()); + toThrift(taskStats.peakUserMemoryInBytes, *thriftTaskStats.peakUserMemoryInBytes_ref()); + toThrift(taskStats.peakTotalMemoryInBytes, *thriftTaskStats.peakTotalMemoryInBytes_ref()); + toThrift(taskStats.peakNodeTotalMemoryInBytes, *thriftTaskStats.peakNodeTotalMemoryInBytes_ref()); + toThrift(taskStats.totalScheduledTimeInNanos, *thriftTaskStats.totalScheduledTimeInNanos_ref()); + toThrift(taskStats.totalCpuTimeInNanos, *thriftTaskStats.totalCpuTimeInNanos_ref()); + toThrift(taskStats.totalBlockedTimeInNanos, *thriftTaskStats.totalBlockedTimeInNanos_ref()); + toThrift(taskStats.fullyBlocked, *thriftTaskStats.fullyBlocked_ref()); + toThrift(taskStats.blockedReasons, *thriftTaskStats.blockedReasons_ref()); + toThrift(taskStats.totalAllocationInBytes, *thriftTaskStats.totalAllocationInBytes_ref()); + toThrift(taskStats.rawInputDataSizeInBytes, *thriftTaskStats.rawInputDataSizeInBytes_ref()); + toThrift(taskStats.rawInputPositions, *thriftTaskStats.rawInputPositions_ref()); + toThrift(taskStats.processedInputDataSizeInBytes, *thriftTaskStats.processedInputDataSizeInBytes_ref()); + toThrift(taskStats.processedInputPositions, *thriftTaskStats.processedInputPositions_ref()); + toThrift(taskStats.outputDataSizeInBytes, *thriftTaskStats.outputDataSizeInBytes_ref()); + toThrift(taskStats.outputPositions, *thriftTaskStats.outputPositions_ref()); + toThrift(taskStats.physicalWrittenDataSizeInBytes, *thriftTaskStats.physicalWrittenDataSizeInBytes_ref()); + toThrift(taskStats.pipelines, *thriftTaskStats.pipelines_ref()); + toThrift(taskStats.queuedPartitionedDrivers, *thriftTaskStats.queuedPartitionedDrivers_ref()); + toThrift(taskStats.queuedPartitionedSplitsWeight, *thriftTaskStats.queuedPartitionedSplitsWeight_ref()); + toThrift(taskStats.runningPartitionedDrivers, *thriftTaskStats.runningPartitionedDrivers_ref()); + toThrift(taskStats.runningPartitionedSplitsWeight, *thriftTaskStats.runningPartitionedSplitsWeight_ref()); + toThrift(taskStats.fullGcCount, *thriftTaskStats.fullGcCount_ref()); + toThrift(taskStats.fullGcTimeInMillis, *thriftTaskStats.fullGcTimeInMillis_ref()); + toThrift(taskStats.runtimeStats, *thriftTaskStats.runtimeStats_ref()); + } + + void toThrift(const facebook::presto::protocol::RuntimeStats& runtimeStats, thrift::RuntimeStats& thriftRuntimeStats) { + toThrift(runtimeStats, *thriftRuntimeStats.metrics_ref()); + } + + void toThrift(const facebook::presto::protocol::RuntimeMetric& runtimeMetric, thrift::RuntimeMetric& thriftRuntimeMetric) { + toThrift(runtimeMetric.name, *thriftRuntimeMetric.name_ref()); + toThrift(runtimeMetric.unit, *thriftRuntimeMetric.unit_ref()); + toThrift(runtimeMetric.sum, *thriftRuntimeMetric.sum_ref()); + toThrift(runtimeMetric.count, *thriftRuntimeMetric.count_ref()); + toThrift(runtimeMetric.max, *thriftRuntimeMetric.max_ref()); + toThrift(runtimeMetric.min, *thriftRuntimeMetric.min_ref()); + } + + void toThrift(const facebook::presto::protocol::PipelineStats& pipelineStats, thrift::PipelineStats& thriftPipelineStats) { + toThrift(pipelineStats.pipelineId, *thriftPipelineStats.pipelineId_ref()); + toThrift(pipelineStats.firstStartTimeInMillis, *thriftPipelineStats.firstStartTimeInMillis_ref()); + toThrift(pipelineStats.lastStartTimeInMillis, *thriftPipelineStats.lastStartTimeInMillis_ref()); + toThrift(pipelineStats.lastEndTimeInMillis, *thriftPipelineStats.lastEndTimeInMillis_ref()); + toThrift(pipelineStats.inputPipeline, *thriftPipelineStats.inputPipeline_ref()); + toThrift(pipelineStats.outputPipeline, *thriftPipelineStats.outputPipeline_ref()); + toThrift(pipelineStats.totalDrivers, *thriftPipelineStats.totalDrivers_ref()); + toThrift(pipelineStats.queuedDrivers, *thriftPipelineStats.queuedDrivers_ref()); + toThrift(pipelineStats.queuedPartitionedDrivers, *thriftPipelineStats.queuedPartitionedDrivers_ref()); + toThrift(pipelineStats.queuedPartitionedSplitsWeight, *thriftPipelineStats.queuedPartitionedSplitsWeight_ref()); + toThrift(pipelineStats.runningDrivers, *thriftPipelineStats.runningDrivers_ref()); + toThrift(pipelineStats.runningPartitionedDrivers, *thriftPipelineStats.runningPartitionedDrivers_ref()); + toThrift(pipelineStats.runningPartitionedSplitsWeight, *thriftPipelineStats.runningPartitionedSplitsWeight_ref()); + toThrift(pipelineStats.blockedDrivers, *thriftPipelineStats.blockedDrivers_ref()); + toThrift(pipelineStats.completedDrivers, *thriftPipelineStats.completedDrivers_ref()); + toThrift(pipelineStats.userMemoryReservationInBytes, *thriftPipelineStats.userMemoryReservationInBytes_ref()); + toThrift(pipelineStats.revocableMemoryReservationInBytes, *thriftPipelineStats.revocableMemoryReservationInBytes_ref()); + toThrift(pipelineStats.systemMemoryReservationInBytes, *thriftPipelineStats.systemMemoryReservationInBytes_ref()); + toThrift(pipelineStats.queuedTime, *thriftPipelineStats.queuedTime_ref()); + toThrift(pipelineStats.elapsedTime, *thriftPipelineStats.elapsedTime_ref()); + toThrift(pipelineStats.totalScheduledTimeInNanos, *thriftPipelineStats.totalScheduledTimeInNanos_ref()); + toThrift(pipelineStats.totalCpuTimeInNanos, *thriftPipelineStats.totalCpuTimeInNanos_ref()); + toThrift(pipelineStats.totalBlockedTimeInNanos, *thriftPipelineStats.totalBlockedTimeInNanos_ref()); + toThrift(pipelineStats.fullyBlocked, *thriftPipelineStats.fullyBlocked_ref()); + toThrift(pipelineStats.blockedReasons, *thriftPipelineStats.blockedReasons_ref()); + toThrift(pipelineStats.totalAllocationInBytes, *thriftPipelineStats.totalAllocationInBytes_ref()); + toThrift(pipelineStats.rawInputDataSizeInBytes, *thriftPipelineStats.rawInputDataSizeInBytes_ref()); + toThrift(pipelineStats.rawInputPositions, *thriftPipelineStats.rawInputPositions_ref()); + toThrift(pipelineStats.processedInputDataSizeInBytes, *thriftPipelineStats.processedInputDataSizeInBytes_ref()); + toThrift(pipelineStats.processedInputPositions, *thriftPipelineStats.processedInputPositions_ref()); + toThrift(pipelineStats.outputDataSizeInBytes, *thriftPipelineStats.outputDataSizeInBytes_ref()); + toThrift(pipelineStats.outputPositions, *thriftPipelineStats.outputPositions_ref()); + toThrift(pipelineStats.physicalWrittenDataSizeInBytes, *thriftPipelineStats.physicalWrittenDataSizeInBytes_ref()); + toThrift(pipelineStats.operatorSummaries, *thriftPipelineStats.operatorSummaries_ref()); + toThrift(pipelineStats.drivers, *thriftPipelineStats.drivers_ref()); + } + + void toThrift(const facebook::presto::protocol::DistributionSnapshot& distributionSnapshot, thrift::DistributionSnapshot& thriftDistributionSnapshot) { + toThrift(distributionSnapshot.maxError, *thriftDistributionSnapshot.maxError_ref()); + toThrift(distributionSnapshot.count, *thriftDistributionSnapshot.count_ref()); + toThrift(distributionSnapshot.total, *thriftDistributionSnapshot.total_ref()); + toThrift(distributionSnapshot.p01, *thriftDistributionSnapshot.p01_ref()); + toThrift(distributionSnapshot.p05, *thriftDistributionSnapshot.p05_ref()); + toThrift(distributionSnapshot.p10, *thriftDistributionSnapshot.p10_ref()); + toThrift(distributionSnapshot.p25, *thriftDistributionSnapshot.p25_ref()); + toThrift(distributionSnapshot.p50, *thriftDistributionSnapshot.p50_ref()); + toThrift(distributionSnapshot.p75, *thriftDistributionSnapshot.p75_ref()); + toThrift(distributionSnapshot.p90, *thriftDistributionSnapshot.p90_ref()); + toThrift(distributionSnapshot.p95, *thriftDistributionSnapshot.p95_ref()); + toThrift(distributionSnapshot.p99, *thriftDistributionSnapshot.p99_ref()); + toThrift(distributionSnapshot.min, *thriftDistributionSnapshot.min_ref()); + toThrift(distributionSnapshot.max, *thriftDistributionSnapshot.max_ref()); + toThrift(distributionSnapshot.avg, *thriftDistributionSnapshot.avg_ref()); + } + + void toThrift(const facebook::presto::protocol::PlanNodeId& planNodeId, thrift::PlanNodeId& thriftPlanNodeId) { + toThrift(planNodeId, *thriftPlanNodeId.id_ref()); + } + + void toThrift(const facebook::presto::protocol::OperatorStats& operatorStats, thrift::OperatorStats& thriftOperatorStats) { + toThrift(operatorStats.stageId, *thriftOperatorStats.stageId_ref()); + toThrift(operatorStats.stageExecutionId, *thriftOperatorStats.stageExecutionId_ref()); + toThrift(operatorStats.pipelineId, *thriftOperatorStats.pipelineId_ref()); + toThrift(operatorStats.operatorId, *thriftOperatorStats.operatorId_ref()); + toThrift(operatorStats.planNodeId, *thriftOperatorStats.planNodeId_ref()); + toThrift(operatorStats.operatorType, *thriftOperatorStats.operatorType_ref()); + toThrift(operatorStats.totalDrivers, *thriftOperatorStats.totalDrivers_ref()); + toThrift(operatorStats.isBlockedCalls, *thriftOperatorStats.isBlockedCalls_ref()); + toThrift(operatorStats.isBlockedWall, *thriftOperatorStats.isBlockedWall_ref()); + toThrift(operatorStats.isBlockedCpu, *thriftOperatorStats.isBlockedCpu_ref()); + toThrift(operatorStats.isBlockedAllocationInBytes, *thriftOperatorStats.isBlockedAllocationInBytes_ref()); + toThrift(operatorStats.addInputCalls, *thriftOperatorStats.addInputCalls_ref()); + toThrift(operatorStats.addInputWall, *thriftOperatorStats.addInputWall_ref()); + toThrift(operatorStats.addInputCpu, *thriftOperatorStats.addInputCpu_ref()); + toThrift(operatorStats.addInputAllocationInBytes, *thriftOperatorStats.addInputAllocationInBytes_ref()); + toThrift(operatorStats.rawInputDataSizeInBytes, *thriftOperatorStats.rawInputDataSizeInBytes_ref()); + toThrift(operatorStats.rawInputPositions, *thriftOperatorStats.rawInputPositions_ref()); + toThrift(operatorStats.inputDataSizeInBytes, *thriftOperatorStats.inputDataSizeInBytes_ref()); + toThrift(operatorStats.inputPositions, *thriftOperatorStats.inputPositions_ref()); + toThrift(operatorStats.sumSquaredInputPositions, *thriftOperatorStats.sumSquaredInputPositions_ref()); + toThrift(operatorStats.getOutputCalls, *thriftOperatorStats.getOutputCalls_ref()); + toThrift(operatorStats.getOutputWall, *thriftOperatorStats.getOutputWall_ref()); + toThrift(operatorStats.getOutputCpu, *thriftOperatorStats.getOutputCpu_ref()); + toThrift(operatorStats.getOutputAllocationInBytes, *thriftOperatorStats.getOutputAllocationInBytes_ref()); + toThrift(operatorStats.outputDataSizeInBytes, *thriftOperatorStats.outputDataSizeInBytes_ref()); + toThrift(operatorStats.outputPositions, *thriftOperatorStats.outputPositions_ref()); + toThrift(operatorStats.physicalWrittenDataSizeInBytes, *thriftOperatorStats.physicalWrittenDataSizeInBytes_ref()); + toThrift(operatorStats.additionalCpu, *thriftOperatorStats.additionalCpu_ref()); + toThrift(operatorStats.blockedWall, *thriftOperatorStats.blockedWall_ref()); + toThrift(operatorStats.finishCalls, *thriftOperatorStats.finishCalls_ref()); + toThrift(operatorStats.finishWall, *thriftOperatorStats.finishWall_ref()); + toThrift(operatorStats.finishCpu, *thriftOperatorStats.finishCpu_ref()); + toThrift(operatorStats.finishAllocationInBytes, *thriftOperatorStats.finishAllocationInBytes_ref()); + toThrift(operatorStats.userMemoryReservationInBytes, *thriftOperatorStats.userMemoryReservationInBytes_ref()); + toThrift(operatorStats.revocableMemoryReservationInBytes, *thriftOperatorStats.revocableMemoryReservationInBytes_ref()); + toThrift(operatorStats.systemMemoryReservationInBytes, *thriftOperatorStats.systemMemoryReservationInBytes_ref()); + toThrift(operatorStats.peakUserMemoryReservationInBytes, *thriftOperatorStats.peakUserMemoryReservationInBytes_ref()); + toThrift(operatorStats.peakSystemMemoryReservationInBytes, *thriftOperatorStats.peakSystemMemoryReservationInBytes_ref()); + toThrift(operatorStats.peakTotalMemoryReservationInBytes, *thriftOperatorStats.peakTotalMemoryReservationInBytes_ref()); + toThrift(operatorStats.spilledDataSizeInBytes, *thriftOperatorStats.spilledDataSizeInBytes_ref()); + toThrift(operatorStats.blockedReason, thriftOperatorStats.blockedReason_ref()); + // OperatorInfoUnion infoUnion (OperatorInfo is empty on cpp) + toThrift(operatorStats.runtimeStats, *thriftOperatorStats.runtimeStats_ref()); + toThrift(operatorStats.dynamicFilterStats, *thriftOperatorStats.dynamicFilterStats_ref()); + toThrift(operatorStats.nullJoinBuildKeyCount, *thriftOperatorStats.nullJoinBuildKeyCount_ref()); + toThrift(operatorStats.joinBuildKeyCount, *thriftOperatorStats.joinBuildKeyCount_ref()); + toThrift(operatorStats.nullJoinProbeKeyCount, *thriftOperatorStats.nullJoinProbeKeyCount_ref()); + toThrift(operatorStats.joinProbeKeyCount, *thriftOperatorStats.joinProbeKeyCount_ref()); + } + + void toThrift(const facebook::presto::protocol::DriverStats& driverStats, thrift::DriverStats& thriftDriverStats) { + toThrift(driverStats.lifespan, *thriftDriverStats.lifespan_ref()); + toThrift(driverStats.createTimeInMillis, *thriftDriverStats.createTimeInMillis_ref()); + toThrift(driverStats.startTimeInMillis, *thriftDriverStats.startTimeInMillis_ref()); + toThrift(driverStats.endTimeInMillis, *thriftDriverStats.endTimeInMillis_ref()); + toThrift(driverStats.queuedTime, *thriftDriverStats.queuedTime_ref()); + toThrift(driverStats.elapsedTime, *thriftDriverStats.elapsedTime_ref()); + toThrift(driverStats.userMemoryReservationInBytes, *thriftDriverStats.userMemoryReservationInBytes_ref()); + toThrift(driverStats.revocableMemoryReservationInBytes, *thriftDriverStats.revocableMemoryReservationInBytes_ref()); + toThrift(driverStats.systemMemoryReservationInBytes, *thriftDriverStats.systemMemoryReservationInBytes_ref()); + toThrift(driverStats.totalScheduledTime, *thriftDriverStats.totalScheduledTime_ref()); + toThrift(driverStats.totalCpuTime, *thriftDriverStats.totalCpuTime_ref()); + toThrift(driverStats.totalBlockedTime, *thriftDriverStats.totalBlockedTime_ref()); + toThrift(driverStats.fullyBlocked, *thriftDriverStats.fullyBlocked_ref()); + toThrift(driverStats.blockedReasons, *thriftDriverStats.blockedReasons_ref()); + toThrift(driverStats.totalAllocationInBytes, *thriftDriverStats.totalAllocationInBytes_ref()); + toThrift(driverStats.rawInputDataSizeInBytes, *thriftDriverStats.rawInputDataSizeInBytes_ref()); + toThrift(driverStats.rawInputReadTime, *thriftDriverStats.rawInputReadTime_ref()); + toThrift(driverStats.rawInputPositions, *thriftDriverStats.rawInputPositions_ref()); + toThrift(driverStats.processedInputDataSizeInBytes, *thriftDriverStats.processedInputDataSizeInBytes_ref()); + toThrift(driverStats.processedInputPositions, *thriftDriverStats.processedInputPositions_ref()); + toThrift(driverStats.outputDataSizeInBytes, *thriftDriverStats.outputDataSizeInBytes_ref()); + toThrift(driverStats.outputPositions, *thriftDriverStats.outputPositions_ref()); + toThrift(driverStats.physicalWrittenDataSizeInBytes, *thriftDriverStats.physicalWrittenDataSizeInBytes_ref()); + toThrift(driverStats.operatorStats, *thriftDriverStats.operatorStats_ref()); + } + + void toThrift(const facebook::presto::protocol::DynamicFilterStats& dynamicFilterStats, thrift::DynamicFilterStats& thriftDynamicFilterStats) { + toThrift(dynamicFilterStats.producerNodeIds, *thriftDynamicFilterStats.producerNodeIds_ref()); + } + + // TaskUpdateRequest types + void fromThrift(const std::string& thrift, std::string& proto) { + proto = thrift; + } + void fromThrift(const bool& thrift, bool& proto) { + proto = thrift; + } + void fromThrift(const int32_t& thrift, int32_t& proto) { + proto = thrift; + } + void fromThrift(const int64_t& thrift, int64_t& proto) { + proto = thrift; + } + void fromThrift(const double& thrift, double& proto) { + proto = thrift; + } + + void fromThrift(const double& thrift, facebook::presto::protocol::Duration& duration) { + duration = facebook::presto::protocol::Duration(thrift, facebook::presto::protocol::TimeUnit::MILLISECONDS); + } + + void fromThrift(const double& thrift, facebook::presto::protocol::DataSize& dataSize) { + dataSize = facebook::presto::protocol::DataSize(thrift, facebook::presto::protocol::DataUnit::BYTE); + } + + template + void fromThrift(const E& thriftEnum, C& protoEnum) { + protoEnum = static_cast(static_cast(thriftEnum)); + } + + template + void fromThrift(const apache::thrift::optional_field_ref& thrift, std::shared_ptr

& proto) { + if (thrift.has_value()) { + proto = std::make_shared

(); + fromThrift(*thrift, *proto); + } + } + + template + void fromThrift(const T& thrift, std::shared_ptr

& proto) { + proto = std::make_shared

(); + fromThrift(thrift, *proto); + } + + template + void fromThrift(const std::shared_ptr

& thrift, std::shared_ptr& proto) { + if (thrift) { + proto = std::make_shared(); + fromThrift(*thrift, *proto); + } + } + + template + void fromThrift(const std::set& thriftSet, std::vector& protoVector) { + protoVector.reserve(thriftSet.size()); + V toItem; + for (const auto& fromItem : thriftSet) { + fromThrift(fromItem, toItem); + protoVector.emplace_back(std::move(toItem)); + } + } + + template + void fromThrift(const std::vector

& thriftVector, std::vector& protoVector) { + protoVector.reserve(thriftVector.size()); + T toItem; + for (const auto& fromItem : thriftVector) { + fromThrift(fromItem, toItem); + protoVector.emplace_back(std::move(toItem)); + } + } + + template + void fromThrift(const std::map& thriftMap, std::map& protoMap) { + K2 toKey; + V2 toValue; + for (const auto& [fromKey, fromValue] : thriftMap) { + fromThrift(fromKey, toKey); + fromThrift(fromValue, toValue); + protoMap.emplace(std::move(toKey), std::move(toValue)); + } + } + + void fromThrift(const thrift::TaskUpdateRequest& thriftTaskUpdateRequest, facebook::presto::protocol::TaskUpdateRequest& taskUpdateRequest) { + fromThrift(*thriftTaskUpdateRequest.session_ref(), taskUpdateRequest.session); + fromThrift(*thriftTaskUpdateRequest.extraCredentials_ref(), taskUpdateRequest.extraCredentials); + fromThrift(thriftTaskUpdateRequest.fragment_ref(), taskUpdateRequest.fragment); + fromThrift(*thriftTaskUpdateRequest.sources_ref(), taskUpdateRequest.sources); + fromThrift(*thriftTaskUpdateRequest.outputIds_ref(), taskUpdateRequest.outputIds); + fromThrift(thriftTaskUpdateRequest.tableWriteInfo_ref(), taskUpdateRequest.tableWriteInfo); + } + + void fromThrift(const thrift::SessionRepresentation& thriftSessionRepresentation, facebook::presto::protocol::SessionRepresentation& sessionRepresentation) { + fromThrift(*thriftSessionRepresentation.queryId_ref(), sessionRepresentation.queryId); + fromThrift(thriftSessionRepresentation.transactionId_ref(), sessionRepresentation.transactionId); + fromThrift(*thriftSessionRepresentation.clientTransactionSupport_ref(), sessionRepresentation.clientTransactionSupport); + fromThrift(*thriftSessionRepresentation.user_ref(), sessionRepresentation.user); + fromThrift(thriftSessionRepresentation.principal_ref(), sessionRepresentation.principal); + fromThrift(thriftSessionRepresentation.source_ref(), sessionRepresentation.source); + fromThrift(thriftSessionRepresentation.catalog_ref(), sessionRepresentation.catalog); + fromThrift(thriftSessionRepresentation.schema_ref(), sessionRepresentation.schema); + fromThrift(thriftSessionRepresentation.traceToken_ref(), sessionRepresentation.traceToken); + fromThrift(*thriftSessionRepresentation.timeZoneKey_ref(), sessionRepresentation.timeZoneKey); + fromThrift(*thriftSessionRepresentation.locale_ref(), sessionRepresentation.locale); + fromThrift(thriftSessionRepresentation.remoteUserAddress_ref(), sessionRepresentation.remoteUserAddress); + fromThrift(thriftSessionRepresentation.userAgent_ref(), sessionRepresentation.userAgent); + fromThrift(thriftSessionRepresentation.clientInfo_ref(), sessionRepresentation.clientInfo); + fromThrift(*thriftSessionRepresentation.clientTags_ref(), sessionRepresentation.clientTags); + fromThrift(*thriftSessionRepresentation.resourceEstimates_ref(), sessionRepresentation.resourceEstimates); + fromThrift(*thriftSessionRepresentation.startTime_ref(), sessionRepresentation.startTime); + fromThrift(*thriftSessionRepresentation.systemProperties_ref(), sessionRepresentation.systemProperties); + fromThrift(*thriftSessionRepresentation.catalogProperties_ref(), sessionRepresentation.catalogProperties); + fromThrift(*thriftSessionRepresentation.unprocessedCatalogProperties_ref(), sessionRepresentation.unprocessedCatalogProperties); + fromThrift(*thriftSessionRepresentation.roles_ref(), sessionRepresentation.roles); + fromThrift(*thriftSessionRepresentation.preparedStatements_ref(), sessionRepresentation.preparedStatements); + fromThrift(*thriftSessionRepresentation.sessionFunctions_ref(), sessionRepresentation.sessionFunctions); + } + + void fromThrift(const thrift::ConnectorId& thriftConnectorId, facebook::presto::protocol::ConnectorId& connectorId) { + fromThrift(*thriftConnectorId.catalogName_ref(), connectorId); + } + + void fromThrift(const thrift::QualifiedObjectName& thriftQualifiedObjectName, facebook::presto::protocol::QualifiedObjectName& qualifiedObjectName) { + std::string qualifiedObjectNameStr = + *thriftQualifiedObjectName.catalogName_ref() + "." + + *thriftQualifiedObjectName.schemaName_ref() + "." + + *thriftQualifiedObjectName.objectName_ref(); + fromThrift(qualifiedObjectNameStr, qualifiedObjectName); + } + + void fromThrift(const thrift::SelectedRole& thriftSelectedRole, facebook::presto::protocol::SelectedRole& selectedRole) { + fromThrift(*thriftSelectedRole.type_ref(), selectedRole.type); + fromThrift(thriftSelectedRole.role_ref(), selectedRole.role); + } + + void fromThrift(const thrift::TransactionId& thriftTransactionId, facebook::presto::protocol::TransactionId& transactionId) { + fromThrift(*thriftTransactionId.uuid_ref(), transactionId); + } + + void fromThrift(const thrift::TimeZoneKey& thriftTimeZoneKey, facebook::presto::protocol::TimeZoneKey& timeZoneKey) { + fromThrift(*thriftTimeZoneKey.key_ref(), timeZoneKey); + } + + void fromThrift(const thrift::ResourceEstimates& thriftResourceEstimates, facebook::presto::protocol::ResourceEstimates& resourceEstimates) { + fromThrift(thriftResourceEstimates.executionTime_ref(), resourceEstimates.executionTime); + fromThrift(thriftResourceEstimates.cpuTime_ref(), resourceEstimates.cpuTime); + fromThrift(thriftResourceEstimates.peakMemory_ref(), resourceEstimates.peakMemory); + fromThrift(thriftResourceEstimates.peakTaskMemory_ref(), resourceEstimates.peakTaskMemory); + } + + void fromThrift(const thrift::TaskSource& thriftTaskSource, facebook::presto::protocol::TaskSource& taskSource) { + fromThrift(*thriftTaskSource.planNodeId_ref(), taskSource.planNodeId); + fromThrift(*thriftTaskSource.splits_ref(), taskSource.splits); + fromThrift(*thriftTaskSource.noMoreSplitsForLifespan_ref(), taskSource.noMoreSplitsForLifespan); + fromThrift(*thriftTaskSource.noMoreSplits_ref(), taskSource.noMoreSplits); + } + + void fromThrift(const thrift::ScheduledSplit& thriftScheduledSplit, facebook::presto::protocol::ScheduledSplit& scheduledSplit) { + fromThrift(*thriftScheduledSplit.sequenceId_ref(), scheduledSplit.sequenceId); + fromThrift(*thriftScheduledSplit.planNodeId_ref(), scheduledSplit.planNodeId); + fromThrift(*thriftScheduledSplit.split_ref(), scheduledSplit.split); + } + + void fromThrift(const thrift::OutputBuffers& thriftOutputBuffers, facebook::presto::protocol::OutputBuffers& outputBuffers) { + fromThrift(*thriftOutputBuffers.type_ref(), outputBuffers.type); + fromThrift(*thriftOutputBuffers.version_ref(), outputBuffers.version); + fromThrift(*thriftOutputBuffers.noMoreBufferIds_ref(), outputBuffers.noMoreBufferIds); + fromThrift(*thriftOutputBuffers.buffers_ref(), outputBuffers.buffers); + } + + void fromThrift(const thrift::PlanNodeId& thriftPlanNodeId, facebook::presto::protocol::PlanNodeId& planNodeId) { + fromThrift(*thriftPlanNodeId.id_ref(), planNodeId); + } + + void fromThrift(const thrift::Lifespan& thriftLifeSpan, facebook::presto::protocol::Lifespan& lifespan) { + fromThrift(*thriftLifeSpan.grouped_ref(), lifespan.isgroup); + fromThrift(*thriftLifeSpan.groupId_ref(), lifespan.groupid); + } + + void fromThrift(const thrift::OutputBufferId& thriftOutputBufferId, facebook::presto::protocol::OutputBufferId& outputBufferId) { + fromThrift(std::to_string(*thriftOutputBufferId.id_ref()), outputBufferId); + } + + void fromThrift(const thrift::SqlFunctionId& thriftSqlFunctionId, facebook::presto::protocol::SqlFunctionId& sqlFunctionId) { + fromThrift(*thriftSqlFunctionId.signature_ref(), sqlFunctionId); + } + + void fromThrift(const thrift::RoutineCharacteristics& thriftRoutineCharacteristics, facebook::presto::protocol::RoutineCharacteristics& routineCharacteristics) { + fromThrift(*thriftRoutineCharacteristics.language_ref(), routineCharacteristics.language); + fromThrift(*thriftRoutineCharacteristics.determinism_ref(), routineCharacteristics.determinism); + fromThrift(*thriftRoutineCharacteristics.nullCallClause_ref(), routineCharacteristics.nullCallClause); + } + + void fromThrift(const thrift::Parameter& thriftParameter, facebook::presto::protocol::Parameter& parameter) { + fromThrift(*thriftParameter.name_ref(), parameter.name); + fromThrift(*thriftParameter.type_ref(), parameter.type); + } + + void fromThrift(const thrift::SqlInvokedFunction& thriftSqlInvokedFunction, facebook::presto::protocol::SqlInvokedFunction& sqlInvokedFunction) { + fromThrift(*thriftSqlInvokedFunction.parameters_ref(), sqlInvokedFunction.parameters); + fromThrift(*thriftSqlInvokedFunction.description_ref(), sqlInvokedFunction.description); + fromThrift(*thriftSqlInvokedFunction.routineCharacteristics_ref(), sqlInvokedFunction.routineCharacteristics); + fromThrift(*thriftSqlInvokedFunction.body_ref(), sqlInvokedFunction.body); + fromThrift(*thriftSqlInvokedFunction.signature_ref(), sqlInvokedFunction.signature); + fromThrift(*thriftSqlInvokedFunction.variableArity_ref(), sqlInvokedFunction.variableArity); + fromThrift(*thriftSqlInvokedFunction.functionId_ref(), sqlInvokedFunction.functionId); + } + + void fromThrift(const thrift::Language& thriftLanguage, facebook::presto::protocol::Language& language) { + fromThrift(*thriftLanguage.language_ref(), language.language); + } + + void fromThrift(const thrift::TypeVariableConstraint& thriftTypeVariableConstraint, facebook::presto::protocol::TypeVariableConstraint& typeVariableConstraint) { + fromThrift(*thriftTypeVariableConstraint.name_ref(), typeVariableConstraint.name); + fromThrift(*thriftTypeVariableConstraint.comparableRequired_ref(), typeVariableConstraint.comparableRequired); + fromThrift(*thriftTypeVariableConstraint.orderableRequired_ref(), typeVariableConstraint.orderableRequired); + fromThrift(*thriftTypeVariableConstraint.variadicBound_ref(), typeVariableConstraint.variadicBound); + fromThrift(*thriftTypeVariableConstraint.nonDecimalNumericRequired_ref(), typeVariableConstraint.nonDecimalNumericRequired); + } + + void fromThrift(const thrift::LongVariableConstraint& thriftLongVariableConstraint, facebook::presto::protocol::LongVariableConstraint& longVariableConstraint) { + fromThrift(*thriftLongVariableConstraint.name_ref(), longVariableConstraint.name); + fromThrift(*thriftLongVariableConstraint.expression_ref(), longVariableConstraint.expression); + } + + void fromThrift(const thrift::Signature& thriftSignature, facebook::presto::protocol::Signature& signature) { + fromThrift(*thriftSignature.name_ref(), signature.name); + fromThrift(*thriftSignature.kind_ref(), signature.kind); + fromThrift(*thriftSignature.returnType_ref(), signature.returnType); + fromThrift(*thriftSignature.argumentTypes_ref(), signature.argumentTypes); + fromThrift(*thriftSignature.variableArity_ref(), signature.variableArity); + fromThrift(*thriftSignature.typeVariableConstraints_ref(), signature.typeVariableConstraints); + fromThrift(*thriftSignature.longVariableConstraints_ref(), signature.longVariableConstraints); + } + + void fromThrift(const thrift::TypeSignature& thriftTypeSignature, facebook::presto::protocol::TypeSignature& typeSignature) { + fromThrift(*thriftTypeSignature.signature(), typeSignature); + } + + void fromThrift(const thrift::TableWriteInfoWrapper& thriftTableWriteInfoWrapper, facebook::presto::protocol::TableWriteInfo& tableWriteInfo) { + fromThrift(*thriftTableWriteInfoWrapper.tableWriteInfo_ref(), tableWriteInfo); + } + + void fromThrift(const std::string& thriftTableWriteInfo, facebook::presto::protocol::TableWriteInfo& tableWriteInfo) { + json j = json::parse(thriftTableWriteInfo); + tableWriteInfo = j; + } + + void fromThrift(const thrift::SplitWrapper& thriftSplitWrapper, facebook::presto::protocol::Split& split) { + fromThrift(*thriftSplitWrapper.split_ref(), split); + } + + void fromThrift(const std::string& thriftSplit, facebook::presto::protocol::Split& split) { + json j = json::parse(thriftSplit); + split = j; + } } // namespace facebook::presto diff --git a/presto-native-execution/presto_cpp/main/thrift/ProtocolToThrift.h b/presto-native-execution/presto_cpp/main/thrift/ProtocolToThrift.h index ca9b32a7abc23..45a9c8edfa036 100644 --- a/presto-native-execution/presto_cpp/main/thrift/ProtocolToThrift.h +++ b/presto-native-execution/presto_cpp/main/thrift/ProtocolToThrift.h @@ -14,25 +14,76 @@ */ #pragma once -// This file is generated DO NOT EDIT @generated -// This file is generated DO NOT EDIT @generated - -#include "presto_cpp/main/thrift/gen-cpp2/PrestoThrift.h" #include "presto_cpp/presto_protocol/core/presto_protocol_core.h" +#include "presto_cpp/main/thrift/gen-cpp2/presto_thrift_types.h" + +namespace facebook::presto::thrift { + +void toThrift(const std::string& proto, std::string& thrift); +void toThrift(const bool& proto, bool& thrift); +void toThrift(const int32_t& proto, int32_t& thrift); +void toThrift(const int64_t& proto, int64_t& thrift); +void toThrift(const double& proto, double& thrift); +void toThrift(const facebook::presto::protocol::Duration& duration, double& thrift); + +void toThrift(const facebook::presto::protocol::TaskStatus& taskStatus, TaskStatus& thriftTaskStatus); +void toThrift(const facebook::presto::protocol::Lifespan& lifespan, Lifespan& thriftLifeSpan); +void toThrift(const facebook::presto::protocol::ExecutionFailureInfo& executionFailureInfo, ExecutionFailureInfo& thriftExecutionFailureInfo); +void toThrift(const facebook::presto::protocol::ErrorLocation& errorLocation, ErrorLocation& thriftErrorLocation); +void toThrift(const facebook::presto::protocol::ErrorCode& errorCode, ErrorCode& thriftErrorCode); +void toThrift(const facebook::presto::protocol::HostAddress& hostAddress, HostAddress& thriftHostAddress); + +void toThrift(const facebook::presto::protocol::TaskInfo& taskInfo, TaskInfo& thriftTaskInfo); +void toThrift(const facebook::presto::protocol::MetadataUpdates& metadataUpdates, MetadataUpdatesWrapper& thriftMetadataUpdatesWrapper); +void toThrift(const facebook::presto::protocol::MetadataUpdates& metadataUpdates, std::string& thriftMetadataUpdates); +void toThrift(const facebook::presto::protocol::TaskId& taskId, TaskId& thriftTaskId); +void toThrift(const facebook::presto::protocol::OutputBufferInfo& outputBufferInfo, OutputBufferInfo& thriftOutputBufferInfo); +void toThrift(const facebook::presto::protocol::BufferInfo& bufferInfo, BufferInfo& thriftBufferInfo); +void toThrift(const facebook::presto::protocol::PageBufferInfo& pageBufferInfo, PageBufferInfo& thriftPageBufferInfo); +void toThrift(const facebook::presto::protocol::TaskStats& taskStats, TaskStats& thriftTaskStats); +void toThrift(const facebook::presto::protocol::RuntimeStats& runtimeStats, RuntimeStats& thriftRuntimeStats); +void toThrift(const facebook::presto::protocol::RuntimeMetric& runtimeMetric, RuntimeMetric& thriftRuntimeMetric); +void toThrift(const facebook::presto::protocol::PipelineStats& pipelineStats, PipelineStats& thriftPipelineStats); +void toThrift(const facebook::presto::protocol::DistributionSnapshot& distributionSnapshot, DistributionSnapshot& thriftDistributionSnapshot); +void toThrift(const facebook::presto::protocol::PlanNodeId& planNodeId, PlanNodeId& thriftPlanNodeId); +void toThrift(const facebook::presto::protocol::OperatorStats& operatorStats, OperatorStats& thriftOperatorStats); +void toThrift(const facebook::presto::protocol::DriverStats& driverStats, DriverStats& thriftDriverStats); +void toThrift(const facebook::presto::protocol::DynamicFilterStats& dynamicFilterStats, DynamicFilterStats& thriftDynamicFilterStats); + +void fromThrift(const std::string& thrift, std::string& proto); +void fromThrift(const bool& thrift, bool& proto); +void fromThrift(const int32_t& thrift, int32_t& proto); +void fromThrift(const int64_t& thrift, int64_t& proto); +void fromThrift(const double& thrift, double& proto); +void fromThrift(const double& thrift, facebook::presto::protocol::Duration& duration); +void fromThrift(const double& thrift, facebook::presto::protocol::DataSize& dataSize); -namespace facebook::presto { - -void toThrift(protocol::TaskState& proto, thrift::TaskState& thrift); -void toThrift(protocol::ErrorType& proto, thrift::ErrorType& thrift); -void toThrift(const protocol::Lifespan& proto, thrift::Lifespan& thrift); -void toThrift( - const protocol::ErrorLocation& proto, - thrift::ErrorLocation& thrift); -void toThrift(const protocol::HostAddress& proto, thrift::HostAddress& thrift); -void toThrift(const protocol::TaskStatus& proto, thrift::TaskStatus& thrift); -void toThrift(const protocol::ErrorCode& proto, thrift::ErrorCode& thrift); -void toThrift( - const protocol::ExecutionFailureInfo& proto, - thrift::ExecutionFailureInfo& thrift); +void fromThrift(const TaskUpdateRequest& thriftTaskUpdateRequest, facebook::presto::protocol::TaskUpdateRequest& taskUpdateRequest); +void fromThrift(const SessionRepresentation& thriftSessionRepresentation, facebook::presto::protocol::SessionRepresentation& sessionRepresentation); +void fromThrift(const ConnectorId& thriftConnectorId, facebook::presto::protocol::ConnectorId& connectorId); +void fromThrift(const QualifiedObjectName& thriftQualifiedObjectName, facebook::presto::protocol::QualifiedObjectName& qualifiedObjectName); +void fromThrift(const SelectedRole& thriftSelectedRole, facebook::presto::protocol::SelectedRole& selectedRole); +void fromThrift(const TransactionId& thriftTransactionId, facebook::presto::protocol::TransactionId& transactionId); +void fromThrift(const TimeZoneKey& thriftTimeZoneKey, facebook::presto::protocol::TimeZoneKey& timeZoneKey); +void fromThrift(const ResourceEstimates& thriftResourceEstimates, facebook::presto::protocol::ResourceEstimates& resourceEstimates); +void fromThrift(const TaskSource& thriftTaskSource, facebook::presto::protocol::TaskSource& taskSource); +void fromThrift(const ScheduledSplit& thriftScheduledSplit, facebook::presto::protocol::ScheduledSplit& scheduledSplit); +void fromThrift(const OutputBuffers& thriftOutputBuffers, facebook::presto::protocol::OutputBuffers& outputBuffers); +void fromThrift(const PlanNodeId& thriftPlanNodeId, facebook::presto::protocol::PlanNodeId& planNodeId); +void fromThrift(const Lifespan& thriftLifeSpan, facebook::presto::protocol::Lifespan& lifespan); +void fromThrift(const OutputBufferId& thriftOutputBufferId, facebook::presto::protocol::OutputBufferId& outputBufferId); +void fromThrift(const SqlFunctionId& thriftSqlFunctionId, facebook::presto::protocol::SqlFunctionId& sqlFunctionId); +void fromThrift(const RoutineCharacteristics& thriftRoutineCharacteristics, facebook::presto::protocol::RoutineCharacteristics& routineCharacteristics); +void fromThrift(const Parameter& thriftParameter, facebook::presto::protocol::Parameter& parameter); +void fromThrift(const SqlInvokedFunction& thriftSqlInvokedFunction, facebook::presto::protocol::SqlInvokedFunction& sqlInvokedFunction); +void fromThrift(const Language& thriftLanguage, facebook::presto::protocol::Language& language); +void fromThrift(const TypeVariableConstraint& thriftTypeVariableConstraint, facebook::presto::protocol::TypeVariableConstraint& typeVariableConstraint); +void fromThrift(const LongVariableConstraint& thriftLongVariableConstraint, facebook::presto::protocol::LongVariableConstraint& longVariableConstraint); +void fromThrift(const Signature& thriftSignature, facebook::presto::protocol::Signature& signature); +void fromThrift(const TypeSignature& thriftTypeSignature, facebook::presto::protocol::TypeSignature& typeSignature); +void fromThrift(const TableWriteInfoWrapper& thriftTableWriteInfoWrapper, facebook::presto::protocol::TableWriteInfo& tableWriteInfo); +void fromThrift(const std::string& thriftTableWriteInfo, facebook::presto::protocol::TableWriteInfo& tableWriteInfo); +void fromThrift(const SplitWrapper& thriftSplitWrapper, facebook::presto::protocol::Split& split); +void fromThrift(const std::string& thriftSplit, facebook::presto::protocol::Split& split); } // namespace facebook::presto diff --git a/presto-native-execution/presto_cpp/main/thrift/ThriftIO.h b/presto-native-execution/presto_cpp/main/thrift/ThriftIO.h index 67c3ac31ac0c9..40c5cd3de09c9 100644 --- a/presto-native-execution/presto_cpp/main/thrift/ThriftIO.h +++ b/presto-native-execution/presto_cpp/main/thrift/ThriftIO.h @@ -16,8 +16,8 @@ #include template -void thriftRead(std::string data, std::shared_ptr& buffer) { - auto inBuf = folly::IOBuf::copyBuffer(data); +void thriftRead(const std::string& data, std::shared_ptr& buffer) { + auto inBuf = folly::IOBuf::wrapBuffer(data.data(), data.size()); apache::thrift::BinaryProtocolReader reader; reader.setInput(inBuf.get()); buffer->read(&reader); @@ -29,9 +29,5 @@ std::string thriftWrite(T& data) { apache::thrift::BinaryProtocolWriter writer; writer.setOutput(&outQueue); data.write(&writer); - - std::string str; - outQueue.appendToString(str); - - return str; + return outQueue.move()->moveToFbString().toStdString(); } diff --git a/presto-native-execution/presto_cpp/main/thrift/presto_protocol-to-thrift-json.json b/presto-native-execution/presto_cpp/main/thrift/presto_protocol-to-thrift-json.json deleted file mode 100644 index b5deb320c6bee..0000000000000 --- a/presto-native-execution/presto_cpp/main/thrift/presto_protocol-to-thrift-json.json +++ /dev/null @@ -1,400 +0,0 @@ -[ - { - "comment": "// This file is generated DO NOT EDIT @generated" - }, - { - "comment": "// This file is generated DO NOT EDIT @generated" - }, - { - "enum": true, - "class_name": "TaskState", - "elements": [ - { - "name": "PLANNED", - "tag": 0 - }, - { - "name": "RUNNING", - "tag": 1 - }, - { - "name": "FINISHED", - "tag": 2 - }, - { - "name": "CANCELED", - "tag": 3 - }, - { - "name": "ABORTED", - "tag": 4 - }, - { - "name": "FAILED", - "tag": 5 - } - ] - }, - { - "enum": true, - "class_name": "ErrorType", - "elements": [ - { - "name": "USER_ERROR", - "tag": 0 - }, - { - "name": "INTERNAL_ERROR", - "tag": 1 - }, - { - "name": "INSUFFICIENT_RESOURCES", - "tag": 2 - }, - { - "name": "EXTERNAL", - "tag": 3 - } - ] - }, - { - "struct": true, - "class_name": "Lifespan", - "fields": [ - { - "field_name": "grouped", - "field_type": "bool", - "optional": false, - "required": false, - "tag": 1, - "proto_name": "isgroup" - }, - { - "field_name": "groupId", - "field_type": "int32_t", - "optional": false, - "required": false, - "tag": 2, - "proto_name": "groupid" - } - ], - "proto_name": "Lifespan" - }, - { - "struct": true, - "class_name": "ErrorLocation", - "fields": [ - { - "field_name": "lineNumber", - "field_type": "int32_t", - "optional": false, - "required": false, - "tag": 1, - "proto_name": "lineNumber", - "convert": true - }, - { - "field_name": "columnNumber", - "field_type": "int32_t", - "optional": false, - "required": false, - "tag": 2, - "proto_name": "columnNumber", - "convert": true - } - ] - }, - { - "struct": true, - "class_name": "HostAddress", - "fields": [ - { - "field_name": "host", - "field_type": "string", - "optional": false, - "required": false, - "tag": 1, - "proto_name": "host" - }, - { - "field_name": "port", - "field_type": "int32_t", - "optional": false, - "required": false, - "tag": 2, - "proto_name": "port" - } - ], - "cinc": "void toThrift(const protocol::HostAddress& proto, thrift::HostAddress& thrift) {\n std::vector parts;\n folly::split(\":\", proto, parts);\n if (parts.size() == 2) {\n thrift.host_ref() = parts[0];\n thrift.port_ref() = std::stoi(parts[1]);\n }\n}" - }, - { - "struct": true, - "class_name": "TaskStatus", - "fields": [ - { - "field_name": "taskInstanceIdLeastSignificantBits", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 1, - "proto_name": "taskInstanceIdLeastSignificantBits", - "convert": true - }, - { - "field_name": "taskInstanceIdMostSignificantBits", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 2, - "proto_name": "taskInstanceIdMostSignificantBits", - "convert": true - }, - { - "field_name": "version", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 3, - "proto_name": "version", - "convert": true - }, - { - "field_name": "state", - "field_type": "TaskState", - "optional": false, - "required": false, - "tag": 4, - "proto_name": "state", - "convert": true - }, - { - "field_name": "taskName", - "field_type": "string", - "optional": false, - "required": false, - "tag": 5, - "proto_name": "self" - }, - { - "field_name": "completedDriverGroups", - "field_type": "std::set", - "optional": false, - "required": false, - "tag": 6, - "proto_name": "completedDriverGroups", - "convert": true - }, - { - "field_name": "failures", - "field_type": "std::list", - "optional": false, - "required": false, - "tag": 7, - "proto_name": "failures", - "convert": true - }, - { - "field_name": "queuedPartitionedDrivers", - "field_type": "int32_t", - "optional": false, - "required": false, - "tag": 8, - "proto_name": "queuedPartitionedDrivers", - "convert": true - }, - { - "field_name": "runningPartitionedDrivers", - "field_type": "int32_t", - "optional": false, - "required": false, - "tag": 9, - "proto_name": "runningPartitionedDrivers", - "convert": true - }, - { - "field_name": "outputBufferUtilization", - "field_type": "double", - "optional": false, - "required": false, - "tag": 10, - "proto_name": "outputBufferUtilization", - "convert": true - }, - { - "field_name": "outputBufferOverutilized", - "field_type": "bool", - "optional": false, - "required": false, - "tag": 11, - "proto_name": "outputBufferOverutilized", - "convert": true - }, - { - "field_name": "physicalWrittenDataSizeInBytes", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 12, - "proto_name": "physicalWrittenDataSizeInBytes", - "convert": true - }, - { - "field_name": "memoryReservationInBytes", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 13, - "proto_name": "memoryReservationInBytes", - "convert": true - }, - { - "field_name": "systemMemoryReservationInBytes", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 14, - "proto_name": "systemMemoryReservationInBytes", - "convert": true - }, - { - "field_name": "fullGcCount", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 15, - "proto_name": "fullGcCount", - "convert": true - }, - { - "field_name": "fullGcTimeInMillis", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 16, - "proto_name": "fullGcTimeInMillis", - "convert": true - }, - { - "field_name": "peakNodeTotalMemoryReservationInBytes", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 17, - "proto_name": "peakNodeTotalMemoryReservationInBytes", - "convert": true - } - ], - "proto_name": "TaskStatus" - }, - { - "struct": true, - "class_name": "ErrorCode", - "fields": [ - { - "field_name": "code", - "field_type": "int32_t", - "optional": false, - "required": false, - "tag": 1, - "proto_name": "code", - "convert": true - }, - { - "field_name": "name", - "field_type": "string", - "optional": false, - "required": false, - "tag": 2, - "proto_name": "name", - "convert": true - }, - { - "field_name": "type", - "field_type": "ErrorType", - "optional": false, - "required": false, - "tag": 3, - "proto_name": "type", - "convert": true - } - ] - }, - { - "struct": true, - "class_name": "ExecutionFailureInfo", - "fields": [ - { - "field_name": "type", - "field_type": "string", - "optional": false, - "required": false, - "tag": 1, - "proto_name": "type", - "convert": true - }, - { - "field_name": "message", - "field_type": "string", - "optional": false, - "required": false, - "tag": 2, - "proto_name": "message", - "convert": true - }, - { - "field_name": "cause", - "field_type": "ExecutionFailureInfo", - "optional": true, - "required": false, - "tag": 3, - "proto_name": "cause", - "convert": true - }, - { - "field_name": "suppressed", - "field_type": "std::list", - "optional": false, - "required": false, - "tag": 4, - "proto_name": "suppressed", - "convert": true - }, - { - "field_name": "stack", - "field_type": "std::list", - "optional": false, - "required": false, - "tag": 5, - "proto_name": "stack", - "convert": true - }, - { - "field_name": "errorLocation", - "field_type": "ErrorLocation", - "optional": false, - "required": false, - "tag": 6, - "proto_name": "errorLocation", - "convert": true - }, - { - "field_name": "errorCode", - "field_type": "ErrorCode", - "optional": false, - "required": false, - "tag": 7, - "proto_name": "errorCode", - "convert": true - }, - { - "field_name": "remoteHost", - "field_type": "HostAddress", - "optional": false, - "required": false, - "tag": 8, - "proto_name": "remoteHost", - "convert": true - } - ] - } -] diff --git a/presto-native-execution/presto_cpp/main/thrift/presto_protocol-to-thrift-json.py b/presto-native-execution/presto_cpp/main/thrift/presto_protocol-to-thrift-json.py deleted file mode 100755 index 1f71dd9b7b1c8..0000000000000 --- a/presto-native-execution/presto_cpp/main/thrift/presto_protocol-to-thrift-json.py +++ /dev/null @@ -1,136 +0,0 @@ -#!/usr/bin/env python3 -# 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. -# - -import argparse -import os -import sys - -sys.path.insert(0, "../../presto_protocol") -import util - - -def eprint(text): - print(text, file=sys.stderr) - - -def parse_args(): - parser = argparse.ArgumentParser( - description="Generate Presto protocol to/from Presto Thrift JSON data" - ) - parser.add_argument( - "-c", - "--config", - default="presto_protocol-to-thrift-json.yml", - help="config file", - ) - parser.add_argument("thrift", help="Thrift spec as JSON") - parser.add_argument("protocol", help="Presto protocol spec as JSON") - - return parser.parse_args() - - -def special_file(filename, special, thrift_item, key): - if os.path.isfile(filename): - (status, stdout, stderr) = util.run( - "../../../velox/scripts/license-header.py --header ../../../license.header --remove " - + filename - ) - thrift_item[key] = stdout - return True - - return special - - -def main(): - args = parse_args() - config = util.load_yaml(args.config) - thrift = util.load_yaml(args.thrift) - protocol = util.load_yaml(args.protocol) - - pmap = {} - for item in protocol: - if "class_name" in item: - pmap[item.class_name] = item - - comment = "// This file is generated DO NOT EDIT @" + "generated" - result = [{"comment": comment}] - - for thrift_item in thrift: - config_item = None - if "class_name" in thrift_item and thrift_item.class_name in pmap: - protocol_item = pmap[thrift_item.class_name] - - special = False - if "struct" in thrift_item: - if thrift_item.class_name in config.StructMap: - config_item = config.StructMap[thrift_item.class_name] - thrift_item["proto_name"] = config_item.class_name - special = True - - for field in thrift_item.fields: - if ( - config_item is not None - and field.field_name in config_item.fields - ): - field["proto_name"] = config_item.fields[ - field.field_name - ].field_name - else: - field["proto_name"] = field.field_name - - if "struct" in protocol_item: - thrift_field_set = {t.proto_name for t in thrift_item.fields} - protocol_field_set = {p.field_name for p in protocol_item.fields} - valid_fields = thrift_field_set.intersection(protocol_field_set) - - for field in thrift_item.fields: - if field.field_name in valid_fields: - field["convert"] = True - - if len((thrift_field_set - protocol_field_set)) != 0: - eprint( - "Missing protocol fields: " - + thrift_item.class_name - + " " - + str(thrift_field_set - protocol_field_set) - ) - - if len((protocol_field_set - thrift_field_set)) != 0: - eprint( - "Missing thrift fields: " - + thrift_item.class_name - + " " - + str(protocol_field_set - thrift_field_set) - ) - else: - hfile = "./special/" + thrift_item.class_name + ".hpp.inc" - special = special_file(hfile, special, thrift_item, "hinc") - - cfile = "./special/" + thrift_item.class_name + ".cpp.inc" - special = special_file(cfile, special, thrift_item, "cinc") - - if not special: - eprint( - "Thrift struct missing from presto_protocol: " - + thrift_item.class_name - ) - else: - eprint("Thrift item missing from presto_protocol: " + item.class_name) - - result.extend(thrift) - print(util.to_json(result)) - - -if __name__ == "__main__": - sys.exit(main()) diff --git a/presto-native-execution/presto_cpp/main/thrift/presto_protocol-to-thrift-json.yml b/presto-native-execution/presto_cpp/main/thrift/presto_protocol-to-thrift-json.yml deleted file mode 100644 index e0ba04f790023..0000000000000 --- a/presto-native-execution/presto_cpp/main/thrift/presto_protocol-to-thrift-json.yml +++ /dev/null @@ -1,23 +0,0 @@ -# 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. - -StructMap: - TaskStatus: - class_name: TaskStatus - fields: - taskName: { field_name: self } - - Lifespan: - class_name: Lifespan - fields: - grouped: { field_name: isgroup } - groupId: { field_name: groupid } diff --git a/presto-native-execution/presto_cpp/main/thrift/presto_thrift.json b/presto-native-execution/presto_cpp/main/thrift/presto_thrift.json deleted file mode 100644 index 279aa18ab0a4a..0000000000000 --- a/presto-native-execution/presto_cpp/main/thrift/presto_thrift.json +++ /dev/null @@ -1,331 +0,0 @@ -[ - { - "comment": "// This file is generated DO NOT EDIT @generated" - }, - { - "enum": true, - "class_name": "TaskState", - "elements": [ - { - "name": "PLANNED", - "tag": 0 - }, - { - "name": "RUNNING", - "tag": 1 - }, - { - "name": "FINISHED", - "tag": 2 - }, - { - "name": "CANCELED", - "tag": 3 - }, - { - "name": "ABORTED", - "tag": 4 - }, - { - "name": "FAILED", - "tag": 5 - } - ] - }, - { - "enum": true, - "class_name": "ErrorType", - "elements": [ - { - "name": "USER_ERROR", - "tag": 0 - }, - { - "name": "INTERNAL_ERROR", - "tag": 1 - }, - { - "name": "INSUFFICIENT_RESOURCES", - "tag": 2 - }, - { - "name": "EXTERNAL", - "tag": 3 - } - ] - }, - { - "struct": true, - "class_name": "Lifespan", - "fields": [ - { - "field_name": "grouped", - "field_type": "bool", - "optional": false, - "required": false, - "tag": 1 - }, - { - "field_name": "groupId", - "field_type": "int32_t", - "optional": false, - "required": false, - "tag": 2 - } - ] - }, - { - "struct": true, - "class_name": "ErrorLocation", - "fields": [ - { - "field_name": "lineNumber", - "field_type": "int32_t", - "optional": false, - "required": false, - "tag": 1 - }, - { - "field_name": "columnNumber", - "field_type": "int32_t", - "optional": false, - "required": false, - "tag": 2 - } - ] - }, - { - "struct": true, - "class_name": "HostAddress", - "fields": [ - { - "field_name": "host", - "field_type": "string", - "optional": false, - "required": false, - "tag": 1 - }, - { - "field_name": "port", - "field_type": "int32_t", - "optional": false, - "required": false, - "tag": 2 - } - ] - }, - { - "struct": true, - "class_name": "TaskStatus", - "fields": [ - { - "field_name": "taskInstanceIdLeastSignificantBits", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 1 - }, - { - "field_name": "taskInstanceIdMostSignificantBits", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 2 - }, - { - "field_name": "version", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 3 - }, - { - "field_name": "state", - "field_type": "TaskState", - "optional": false, - "required": false, - "tag": 4 - }, - { - "field_name": "taskName", - "field_type": "string", - "optional": false, - "required": false, - "tag": 5 - }, - { - "field_name": "completedDriverGroups", - "field_type": "std::set", - "optional": false, - "required": false, - "tag": 6 - }, - { - "field_name": "failures", - "field_type": "std::list", - "optional": false, - "required": false, - "tag": 7 - }, - { - "field_name": "queuedPartitionedDrivers", - "field_type": "int32_t", - "optional": false, - "required": false, - "tag": 8 - }, - { - "field_name": "runningPartitionedDrivers", - "field_type": "int32_t", - "optional": false, - "required": false, - "tag": 9 - }, - { - "field_name": "outputBufferUtilization", - "field_type": "double", - "optional": false, - "required": false, - "tag": 10 - }, - { - "field_name": "outputBufferOverutilized", - "field_type": "bool", - "optional": false, - "required": false, - "tag": 11 - }, - { - "field_name": "physicalWrittenDataSizeInBytes", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 12 - }, - { - "field_name": "memoryReservationInBytes", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 13 - }, - { - "field_name": "systemMemoryReservationInBytes", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 14 - }, - { - "field_name": "fullGcCount", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 15 - }, - { - "field_name": "fullGcTimeInMillis", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 16 - }, - { - "field_name": "peakNodeTotalMemoryReservationInBytes", - "field_type": "int64_t", - "optional": false, - "required": false, - "tag": 17 - } - ] - }, - { - "struct": true, - "class_name": "ErrorCode", - "fields": [ - { - "field_name": "code", - "field_type": "int32_t", - "optional": false, - "required": false, - "tag": 1 - }, - { - "field_name": "name", - "field_type": "string", - "optional": false, - "required": false, - "tag": 2 - }, - { - "field_name": "type", - "field_type": "ErrorType", - "optional": false, - "required": false, - "tag": 3 - } - ] - }, - { - "struct": true, - "class_name": "ExecutionFailureInfo", - "fields": [ - { - "field_name": "type", - "field_type": "string", - "optional": false, - "required": false, - "tag": 1 - }, - { - "field_name": "message", - "field_type": "string", - "optional": false, - "required": false, - "tag": 2 - }, - { - "field_name": "cause", - "field_type": "ExecutionFailureInfo", - "optional": true, - "required": false, - "tag": 3 - }, - { - "field_name": "suppressed", - "field_type": "std::list", - "optional": false, - "required": false, - "tag": 4 - }, - { - "field_name": "stack", - "field_type": "std::list", - "optional": false, - "required": false, - "tag": 5 - }, - { - "field_name": "errorLocation", - "field_type": "ErrorLocation", - "optional": false, - "required": false, - "tag": 6 - }, - { - "field_name": "errorCode", - "field_type": "ErrorCode", - "optional": false, - "required": false, - "tag": 7 - }, - { - "field_name": "remoteHost", - "field_type": "HostAddress", - "optional": false, - "required": false, - "tag": 8 - } - ] - } -] diff --git a/presto-native-execution/presto_cpp/main/thrift/presto_thrift.thrift b/presto-native-execution/presto_cpp/main/thrift/presto_thrift.thrift index b221748d430f9..e9a1fe2c6e1dd 100644 --- a/presto-native-execution/presto_cpp/main/thrift/presto_thrift.thrift +++ b/presto-native-execution/presto_cpp/main/thrift/presto_thrift.thrift @@ -22,35 +22,276 @@ enum TaskState { ABORTED = 4, FAILED = 5, } - enum ErrorType { USER_ERROR = 0, INTERNAL_ERROR = 1, INSUFFICIENT_RESOURCES = 2, EXTERNAL = 3, } - +enum ErrorCause { + UNKNOWN = 0, + LOW_PARTITION_COUNT = 1, + EXCEEDS_BROADCAST_MEMORY_LIMIT = 2, +} +enum BufferState { + OPEN = 0, + NO_MORE_BUFFERS = 1, + NO_MORE_PAGES = 2, + FLUSHING = 3, + FINISHED = 4, + FAILED = 5, +} +enum BlockedReason { + WAITING_FOR_MEMORY = 0, +} +enum RuntimeUnit { + NONE = 0, + NANO = 1, + BYTE = 2, +} +enum JoinType { + INNER = 0, + PROBE_OUTER = 1, + LOOKUP_OUTER = 2, + FULL_OUTER = 3, +} +enum Type { + ROLE = 0, + ALL = 1, + NONE = 2, +} +enum Determinism { + DETERMINISTIC = 0, + NOT_DETERMINISTIC = 1, +} +enum NullCallClause { + RETURNS_NULL_ON_NULL_INPUT = 0, + CALLED_ON_NULL_INPUT = 1, +} +enum FunctionKind { + SCALAR = 0, + AGGREGATE = 1, + WINDOW = 2, +} +enum BufferType { + PARTITIONED = 0, + BROADCAST = 1, + ARBITRARY = 2, + DISCARDING = 3, + SPOOLING = 4, +} +struct SplitWrapper { + 1: string split; +} +struct TableWriteInfoWrapper { + 1: string tableWriteInfo; +} +struct MetadataUpdatesWrapper { + 1: string metadataUpdates; +} struct Lifespan { 1: bool grouped; 2: i32 groupId; } - struct ErrorLocation { 1: i32 lineNumber; 2: i32 columnNumber; } - struct HostAddress { 1: string host; 2: i32 port; } - +struct StageId { + 1: string queryId; + 2: i32 id; +} +struct OutputBufferId { + 1: i32 id; +} +struct PageBufferInfo { + 1: i32 partition; + 2: i64 bufferedPages; + 3: i64 bufferedBytes; + 4: i64 rowsAdded; + 5: i64 pagesAdded; +} +struct PlanNodeId { + 1: string id; +} +struct DistributionSnapshot { + 1: double maxError; + 2: double count; + 3: double total; + 4: i64 p01; + 5: i64 p05; + 6: i64 p10; + 7: i64 p25; + 8: i64 p50; + 9: i64 p75; + 10: i64 p90; + 11: i64 p95; + 12: i64 p99; + 13: i64 min; + 14: i64 max; + 15: double avg; +} +struct RuntimeStats { + 1: map metrics; +} +struct ExchangeClientStatus { + 1: i64 bufferedBytes; + 2: i64 maxBufferedBytes; + 3: i64 averageBytesPerRequest; + 4: i64 successfulRequestsCount; + 5: i32 bufferedPages; + 6: bool noMoreLocations; + 7: list pageBufferClientStatuses; +} +struct PageBufferClientStatus { + 1: string uri; + 2: string state; + 3: i64 lastUpdate; + 4: i64 rowsReceived; + 5: i32 pagesReceived; + 6: i64 rowsRejected; + 7: i32 pagesRejected; + 8: i32 requestsScheduled; + 9: i32 requestsCompleted; + 10: i32 requestsFailed; + 11: string httpRequestState; +} +struct LocalExchangeBufferInfo { + 1: i64 bufferedBytes; + 2: i32 bufferedPages; +} +struct TableFinishInfo { + 1: string serializedConnectorOutputMetadata; + 2: bool jsonLengthLimitExceeded; + 3: double statisticsWallTime; + 4: double statisticsCpuTime; +} +struct SplitOperatorInfo { + 1: map splitInfoMap; +} +struct HashCollisionsInfo { + 1: double weightedSumSquaredHashCollisions; + 2: double weightedHashCollisions; + 3: double weightedExpectedHashCollisions; +} +struct PartitionedOutputInfo { + 1: i64 rowsAdded; + 2: i64 pagesAdded; + 3: i64 outputBufferPeakMemoryUsage; +} +struct WindowInfo { + 1: list windowInfos; +} +struct DriverWindowInfo { + 1: double sumSquaredDifferencesPositionsOfIndex; + 2: double sumSquaredDifferencesSizeOfIndex; + 3: double sumSquaredDifferencesSizeInPartition; + 4: i64 totalPartitionsCount; + 5: i64 totalRowsCount; + 6: i64 numberOfIndexes; +} +struct TableWriterInfo { + 1: i64 pageSinkPeakMemoryUsage; + 2: double statisticsWallTime; + 3: double statisticsCpuTime; + 4: double validationCpuTime; +} +struct TableWriterMergeInfo { + 1: double statisticsWallTime; + 2: double statisticsCpuTime; +} +struct DynamicFilterStats { + 1: set producerNodeIds; +} +struct DriverStats { + 1: Lifespan lifespan; + 2: i64 createTimeInMillis; + 3: i64 startTimeInMillis; + 4: i64 endTimeInMillis; + 5: double queuedTime; + 6: double elapsedTime; + 7: i64 userMemoryReservationInBytes; + 8: i64 revocableMemoryReservationInBytes; + 9: i64 systemMemoryReservationInBytes; + 10: double totalScheduledTime; + 11: double totalCpuTime; + 12: double totalBlockedTime; + 13: bool fullyBlocked; + 14: set blockedReasons; + 15: i64 totalAllocationInBytes; + 16: i64 rawInputDataSizeInBytes; + 17: double rawInputReadTime; + 18: i64 rawInputPositions; + 19: i64 processedInputDataSizeInBytes; + 20: i64 processedInputPositions; + 21: i64 outputDataSizeInBytes; + 22: i64 outputPositions; + 23: i64 physicalWrittenDataSizeInBytes; + 24: list operatorStats; +} +struct TransactionId { + 1: string uuid; +} +struct TimeZoneKey { + 1: string id; + 2: i16 key; +} +struct ResourceEstimates { + 1: optional double executionTime; + 2: optional double cpuTime; + 3: optional double peakMemory; + 4: optional double peakTaskMemory; +} +struct ConnectorId { + 1: string catalogName; +} +struct SqlFunctionId { + 1: string signature; +} +struct TypeSignature { + 1: string signature; + 2: bool ignore; +} +struct Language { + 1: string language; +} +struct QualifiedObjectName { + 1: string catalogName; + 2: string schemaName; + 3: string objectName; +} +struct TypeVariableConstraint { + 1: string name; + 2: bool comparableRequired; + 3: bool orderableRequired; + 4: string variadicBound; + 5: bool nonDecimalNumericRequired; +} +struct LongVariableConstraint { + 1: string name; + 2: string expression; +} +struct TaskSource { + 1: PlanNodeId planNodeId; + 2: set splits; + 3: set noMoreSplitsForLifespan; + 4: bool noMoreSplits; +} +struct ScheduledSplit { + 1: i64 sequenceId; + 2: PlanNodeId planNodeId; + 3: SplitWrapper split; +} struct TaskStatus { 1: i64 taskInstanceIdLeastSignificantBits; 2: i64 taskInstanceIdMostSignificantBits; 3: i64 version; 4: TaskState state; - 5: string taskName; + 5: string selfUri; 6: set completedDriverGroups; 7: list failures; 8: i32 queuedPartitionedDrivers; @@ -63,23 +304,291 @@ struct TaskStatus { 15: i64 fullGcCount; 16: i64 fullGcTimeInMillis; 17: i64 peakNodeTotalMemoryReservationInBytes; + 18: i64 totalCpuTimeInNanos; + 19: i64 taskAgeInMillis; + 20: i64 queuedPartitionedSplitsWeight; + 21: i64 runningPartitionedSplitsWeight; } - struct ErrorCode { 1: i32 code; 2: string name; 3: ErrorType type; + 4: bool retriable; +} +struct StageExecutionId { + 1: StageId stageId; + 2: i32 id; +} +struct OutputBufferInfo { + 1: string type; + 2: BufferState state; + 3: list buffers; + 4: bool canAddBuffers; + 5: bool canAddPages; + 6: i64 totalBufferedBytes; + 7: i64 totalBufferedPages; + 8: i64 totalRowsSent; + 9: i64 totalPagesSent; +} +struct BufferInfo { + 1: OutputBufferId bufferId; + 2: bool finished; + 3: i32 bufferedPages; + 4: i64 pagesSent; + 5: PageBufferInfo pageBufferInfo; +} +struct TaskStats { + 1: i64 createTimeInMillis; + 2: i64 firstStartTimeInMillis; + 3: i64 lastStartTimeInMillis; + 4: i64 lastEndTimeInMillis; + 5: i64 endTimeInMillis; + 6: i64 elapsedTimeInNanos; + 7: i64 queuedTimeInNanos; + 8: i32 totalDrivers; + 9: i32 queuedDrivers; + 10: i32 runningDrivers; + 11: i32 blockedDrivers; + 12: i32 completedDrivers; + 13: double cumulativeUserMemory; + 14: double cumulativeTotalMemory; + 15: i64 userMemoryReservationInBytes; + 16: i64 revocableMemoryReservationInBytes; + 17: i64 systemMemoryReservationInBytes; + 18: i64 peakUserMemoryInBytes; + 19: i64 peakTotalMemoryInBytes; + 20: i64 peakNodeTotalMemoryInBytes; + 21: i64 totalScheduledTimeInNanos; + 22: i64 totalCpuTimeInNanos; + 23: i64 totalBlockedTimeInNanos; + 24: bool fullyBlocked; + 25: set blockedReasons; + 26: i64 totalAllocationInBytes; + 27: i64 rawInputDataSizeInBytes; + 28: i64 rawInputPositions; + 29: i64 processedInputDataSizeInBytes; + 30: i64 processedInputPositions; + 31: i64 outputDataSizeInBytes; + 32: i64 outputPositions; + 33: i64 physicalWrittenDataSizeInBytes; + 34: list pipelines; + 35: i32 queuedPartitionedDrivers; + 36: i64 queuedPartitionedSplitsWeight; + 37: i32 runningPartitionedDrivers; + 38: i64 runningPartitionedSplitsWeight; + 39: i32 fullGcCount; + 40: i64 fullGcTimeInMillis; + 41: RuntimeStats runtimeStats; +} +struct PipelineStats { + 1: i32 pipelineId; + 2: i64 firstStartTimeInMillis; + 3: i64 lastStartTimeInMillis; + 4: i64 lastEndTimeInMillis; + 5: bool inputPipeline; + 6: bool outputPipeline; + 7: i32 totalDrivers; + 8: i32 queuedDrivers; + 9: i32 queuedPartitionedDrivers; + 10: i64 queuedPartitionedSplitsWeight; + 11: i32 runningDrivers; + 12: i32 runningPartitionedDrivers; + 13: i64 runningPartitionedSplitsWeight; + 14: i32 blockedDrivers; + 15: i32 completedDrivers; + 16: i64 userMemoryReservationInBytes; + 17: i64 revocableMemoryReservationInBytes; + 18: i64 systemMemoryReservationInBytes; + 19: DistributionSnapshot queuedTime; + 20: DistributionSnapshot elapsedTime; + 21: i64 totalScheduledTimeInNanos; + 22: i64 totalCpuTimeInNanos; + 23: i64 totalBlockedTimeInNanos; + 24: bool fullyBlocked; + 25: set blockedReasons; + 26: i64 totalAllocationInBytes; + 27: i64 rawInputDataSizeInBytes; + 28: i64 rawInputPositions; + 29: i64 processedInputDataSizeInBytes; + 30: i64 processedInputPositions; + 31: i64 outputDataSizeInBytes; + 32: i64 outputPositions; + 33: i64 physicalWrittenDataSizeInBytes; + 34: list operatorSummaries; + 35: list drivers; +} +struct RuntimeMetric { + 1: string name; + 2: i64 sum; + 3: i64 count; + 4: i64 max; + 5: i64 min; + 6: RuntimeUnit unit; +} +struct JoinOperatorInfo { + 1: JoinType joinType; + 2: list logHistogramProbes; + 3: list logHistogramOutput; + 4: optional i64 lookupSourcePositions; +} +struct SessionRepresentation { + 1: string queryId; + 2: optional TransactionId transactionId; + 3: bool clientTransactionSupport; + 4: string user; + 5: optional string principal; + 6: optional string source; + 7: optional string catalog; + 8: optional string schema; + 9: optional string traceToken; + 10: TimeZoneKey timeZoneKey; + 11: string locale; + 12: optional string remoteUserAddress; + 13: optional string userAgent; + 14: optional string clientInfo; + 15: set clientTags; + 16: ResourceEstimates resourceEstimates; + 17: i64 startTime; + 18: map systemProperties; + 19: map> catalogProperties; + 20: map> unprocessedCatalogProperties; + 21: map roles; + 22: map preparedStatements; + 23: map sessionFunctions; +} +struct SelectedRole { + 1: Type type; + 2: optional string role; +} +struct Parameter { + 1: string name; + 2: TypeSignature type; +} +struct RoutineCharacteristics { + 1: Language language; + 2: Determinism determinism; + 3: NullCallClause nullCallClause; +} +struct Signature { + 1: QualifiedObjectName name; + 2: FunctionKind kind; + 3: TypeSignature returnType; + 4: list argumentTypes; + 5: bool variableArity; + 6: list typeVariableConstraints; + 7: list longVariableConstraints; +} +struct OutputBuffers { + 1: BufferType type; + 2: i64 version; + 3: bool noMoreBufferIds; + 4: map buffers; +} +struct TaskUpdateRequest { + 1: SessionRepresentation session; + 2: map extraCredentials; + 3: optional binary fragment; + 4: list sources; + 5: OutputBuffers outputIds; + 6: optional TableWriteInfoWrapper tableWriteInfo; } - struct ExecutionFailureInfo { 1: string type; 2: string message; - 3: optional ExecutionFailureInfo cause (cpp.ref_type = "shared"); + 3: optional ExecutionFailureInfo cause (cpp.ref_type="shared", drift.recursive_reference=true); 4: list suppressed; 5: list stack; 6: ErrorLocation errorLocation; 7: ErrorCode errorCode; 8: HostAddress remoteHost; + 9: ErrorCause errorCause; +} +struct TaskId { + 1: StageExecutionId stageExecutionId; + 2: i32 id; + 3: i32 attemptNumber; +} +union OperatorInfoUnion { + 1: ExchangeClientStatus exchangeClientStatus; + 2: LocalExchangeBufferInfo localExchangeBufferInfo; + 3: TableFinishInfo tableFinishInfo; + 4: SplitOperatorInfo splitOperatorInfo; + 5: HashCollisionsInfo hashCollisionsInfo; + 6: PartitionedOutputInfo partitionedOutputInfo; + 7: JoinOperatorInfo joinOperatorInfo; + 8: WindowInfo windowInfo; + 9: TableWriterInfo tableWriterInfo; + 10: TableWriterMergeInfo tableWriterMergeInfo; +} +struct SqlInvokedFunction { + 1: list parameters; + 2: string description; + 3: RoutineCharacteristics routineCharacteristics; + 4: string body; + 5: bool variableArity; + 6: Signature signature; + 7: SqlFunctionId functionId; +} +struct TaskInfo { + 1: TaskId taskId; + 2: TaskStatus taskStatus; + 3: i64 lastHeartbeatInMillis; + 4: OutputBufferInfo outputBuffers; + 5: set noMoreSplits; + 6: TaskStats stats; + 7: bool needsPlan; + 8: MetadataUpdatesWrapper metadataUpdates; + 9: string nodeId; +} +struct OperatorStats { + 1: i32 stageId; + 2: i32 stageExecutionId; + 3: i32 pipelineId; + 4: i32 operatorId; + 5: PlanNodeId planNodeId; + 6: string operatorType; + 7: i64 totalDrivers; + 8: i64 addInputCalls; + 9: double addInputWall; + 10: double addInputCpu; + 11: i64 addInputAllocationInBytes; + 12: i64 rawInputDataSizeInBytes; + 13: i64 rawInputPositions; + 14: i64 inputDataSizeInBytes; + 15: i64 inputPositions; + 16: double sumSquaredInputPositions; + 17: i64 getOutputCalls; + 18: double getOutputWall; + 19: double getOutputCpu; + 20: i64 getOutputAllocationInBytes; + 21: i64 outputDataSizeInBytes; + 22: i64 outputPositions; + 23: i64 physicalWrittenDataSizeInBytes; + 24: double additionalCpu; + 25: double blockedWall; + 26: i64 finishCalls; + 27: double finishWall; + 28: double finishCpu; + 29: i64 finishAllocationInBytes; + 30: i64 userMemoryReservationInBytes; + 31: i64 revocableMemoryReservationInBytes; + 32: i64 systemMemoryReservationInBytes; + 33: i64 peakUserMemoryReservationInBytes; + 34: i64 peakSystemMemoryReservationInBytes; + 35: i64 peakTotalMemoryReservationInBytes; + 36: i64 spilledDataSizeInBytes; + 37: RuntimeStats runtimeStats; + 38: optional BlockedReason blockedReason; + 39: OperatorInfoUnion infoUnion; + 40: i64 nullJoinBuildKeyCount; + 41: i64 joinBuildKeyCount; + 42: i64 nullJoinProbeKeyCount; + 43: i64 joinProbeKeyCount; + 44: DynamicFilterStats dynamicFilterStats; + 45: i64 isBlockedCalls; + 46: double isBlockedWall; + 47: double isBlockedCpu; + 48: i64 isBlockedAllocationInBytes; } service PrestoThrift { diff --git a/presto-native-execution/presto_cpp/main/thrift/thrift2json.py b/presto-native-execution/presto_cpp/main/thrift/thrift2json.py deleted file mode 100755 index 1d29293de8b0a..0000000000000 --- a/presto-native-execution/presto_cpp/main/thrift/thrift2json.py +++ /dev/null @@ -1,111 +0,0 @@ -#!/Library/Developer/CommandLineTools/usr/bin/python3 -# 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. - -import json -import sys - -from ptsd_jbroll import ast -from ptsd_jbroll.loader import Loader - - -def eprint(text): - print(text, file=sys.stderr) - - -def devnull(string): - pass - - -class attrdict(dict): - __getattr__ = dict.__getitem__ - __setattr__ = dict.__setitem__ - - -loader = Loader(sys.argv[1], logger=devnull) - -typeMap = { - "byte": "byte", - "i16": "int16_t", - "i32": "int32_t", - "i64": "int64_t", - "bool": "bool", - "double": "double", - "string": "string", -} - - -def str_type(t): - if isinstance( - t, (ast.Byte, ast.I16, ast.I32, ast.I64, ast.Bool, ast.Double, ast.String) - ): - return typeMap[str(t)] - elif isinstance(t, ast.Identifier): - return t.value - elif isinstance(t, ast.Map): - return "std::map<%s, %s>" % (str_type(t.key_type), str_type(t.value_type)) - elif isinstance(t, ast.List): - return "std::list<%s>" % str_type(t.value_type) - elif isinstance(t, ast.Set): - return "std::set<%s>" % str_type(t.value_type) - - return str(t) - - -def enum(n): - v = {} - v["enum"] = True - v["class_name"] = n.name.value - v["elements"] = [] - for value in n.values: - v["elements"].append(dict(name=value.name.value, tag=value.tag)) - - return v - - -def struct(n): - v = {} - v["struct"] = True - v["class_name"] = n.name.value - v["fields"] = [] - for field in n.fields: - v["fields"].append( - dict( - field_name=field.name.value, - field_type=str_type(field.type), - optional=(field.required == "optional"), - required=(field.required == "required"), - tag=field.tag, - ) - ) - - return v - - -def items(module): - i = [] - for node in module.values(): - if not isinstance(node, ast.Node): - continue - if isinstance(node, ast.Enum): - i.append(enum(node)) - elif isinstance(node, ast.Struct): - i.append(struct(node)) - - return i - - -everything = [{"comment": "// This file is generated DO NOT EDIT @" + "generated"}] -for module in loader.modules.values(): - everything.extend(items(module)) - -print(json.dumps(everything)) diff --git a/presto-native-execution/presto_cpp/presto_protocol/core/DataSize.h b/presto-native-execution/presto_cpp/presto_protocol/core/DataSize.h index c35ada05ce4fa..127fbac0fb0e8 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/core/DataSize.h +++ b/presto-native-execution/presto_cpp/presto_protocol/core/DataSize.h @@ -51,6 +51,10 @@ class DataSize { std::string dataUnitToString(DataUnit dataUnit) const; + bool operator==(const DataSize& other) const { + return value_ == other.value_ && dataUnit_ == other.dataUnit_; + } + private: double value_ = 0; DataUnit dataUnit_ = DataUnit::BYTE; diff --git a/presto-native-execution/presto_cpp/presto_protocol/core/Duration.h b/presto-native-execution/presto_cpp/presto_protocol/core/Duration.h index 5755b67138391..e0aae14020320 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/core/Duration.h +++ b/presto-native-execution/presto_cpp/presto_protocol/core/Duration.h @@ -86,6 +86,10 @@ class Duration { } } + bool operator==(const Duration& other) const { + return value_ == other.value_ && timeUnit_ == other.timeUnit_; + } + private: double value_ = 0; TimeUnit timeUnit_ = TimeUnit::SECONDS; diff --git a/presto-native-execution/presto_cpp/presto_protocol/tests/PlanFragmentTest.cpp b/presto-native-execution/presto_cpp/presto_protocol/tests/PlanFragmentTest.cpp index 791e542af1134..48da464e3f4b4 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/tests/PlanFragmentTest.cpp +++ b/presto-native-execution/presto_cpp/presto_protocol/tests/PlanFragmentTest.cpp @@ -22,23 +22,6 @@ namespace fs = boost::filesystem; using namespace facebook::presto; namespace { -std::string getDataPath(const std::string& fileName) { - std::string currentPath = fs::current_path().c_str(); - if (boost::algorithm::ends_with(currentPath, "fbcode")) { - return currentPath + - "/github/presto-trunk/presto-native-execution/presto_cpp/presto_protocol/tests/data/" + - fileName; - } - - // CLion runs the tests from cmake-build-release/ or cmake-build-debug/ - // directory. Hard-coded json files are not copied there and test fails with - // file not found. Fixing the path so that we can trigger these tests from - // CLion. - boost::algorithm::replace_all(currentPath, "cmake-build-release/", ""); - boost::algorithm::replace_all(currentPath, "cmake-build-debug/", ""); - - return currentPath + "/data/" + fileName; -} template void testJsonRoundTrip(const std::string& str) { @@ -50,7 +33,7 @@ void testJsonRoundTrip(const std::string& str) { template void testJsonRoundTripFile(const std::string& filename) { - testJsonRoundTrip(slurp(getDataPath(filename))); + testJsonRoundTrip(slurp(getDataPath("/github/presto-trunk/presto-native-execution/presto_cpp/presto_protocol/tests/data/", filename))); } } // namespace