diff --git a/src/clients/meta/MetaClient.cpp b/src/clients/meta/MetaClient.cpp index 8cafe3d59e1..ba7ec449329 100644 --- a/src/clients/meta/MetaClient.cpp +++ b/src/clients/meta/MetaClient.cpp @@ -185,8 +185,8 @@ bool MetaClient::loadData() { return false; } - if (!loadFulltextClients()) { - LOG(ERROR) << "Load fulltext services Failed"; + if (!loadGlobalServiceClients()) { + LOG(ERROR) << "Load global services Failed"; return false; } @@ -519,15 +519,15 @@ bool MetaClient::loadListeners(GraphSpaceID spaceId, std::shared_ptr> MetaClient::reportTaskFinish( return fut; } -folly::Future> MetaClient::signInFTService( - cpp2::FTServiceType type, const std::vector& clients) { - cpp2::SignInFTServiceReq req; +folly::Future> MetaClient::signInService( + const cpp2::ExternalServiceType& type, const std::vector& clients) { + cpp2::SignInServiceReq req; req.type_ref() = type; req.clients_ref() = clients; folly::Promise> promise; auto future = promise.getFuture(); getResponse( std::move(req), - [](auto client, auto request) { return client->future_signInFTService(request); }, + [](auto client, auto request) { return client->future_signInService(request); }, [](cpp2::ExecResp&& resp) -> bool { return resp.get_code() == nebula::cpp2::ErrorCode::SUCCEEDED; }, @@ -3295,13 +3295,14 @@ folly::Future> MetaClient::signInFTService( return future; } -folly::Future> MetaClient::signOutFTService() { - cpp2::SignOutFTServiceReq req; +folly::Future> MetaClient::signOutService(const cpp2::ExternalServiceType& type) { + cpp2::SignOutServiceReq req; + req.type_ref() = type; folly::Promise> promise; auto future = promise.getFuture(); getResponse( std::move(req), - [](auto client, auto request) { return client->future_signOutFTService(request); }, + [](auto client, auto request) { return client->future_signOutService(request); }, [](cpp2::ExecResp&& resp) -> bool { return resp.get_code() == nebula::cpp2::ErrorCode::SUCCEEDED; }, @@ -3310,25 +3311,36 @@ folly::Future> MetaClient::signOutFTService() { return future; } -folly::Future>> MetaClient::listFTClients() { - cpp2::ListFTClientsReq req; - folly::Promise>> promise; +folly::Future> MetaClient::listServiceClients( + const cpp2::ExternalServiceType& type) { + cpp2::ListServiceClientsReq req; + req.type_ref() = type; + folly::Promise> promise; auto future = promise.getFuture(); getResponse( std::move(req), - [](auto client, auto request) { return client->future_listFTClients(request); }, - [](cpp2::ListFTClientsResp&& resp) -> decltype(auto) { + [](auto client, auto request) { return client->future_listServiceClients(request); }, + [](cpp2::ListServiceClientsResp&& resp) -> decltype(auto) { return std::move(resp).get_clients(); }, std::move(promise)); return future; } -StatusOr> MetaClient::getFTClientsFromCache() { +StatusOr> MetaClient::getServiceClientsFromCache( + const cpp2::ExternalServiceType& type) { if (!ready_) { return Status::Error("Not ready!"); } - return fulltextClientList_; + + folly::RWSpinLock::ReadHolder holder(localCacheLock_); + if (type == cpp2::ExternalServiceType::ELASTICSEARCH) { + auto sIter = serviceClientList_.find(type); + if (sIter != serviceClientList_.end()) { + return sIter->second; + } + } + return Status::Error("Service not found!"); } folly::Future> MetaClient::createFTIndex(const std::string& name, diff --git a/src/clients/meta/MetaClient.h b/src/clients/meta/MetaClient.h index 5f1d5022c63..38ae54353ff 100644 --- a/src/clients/meta/MetaClient.h +++ b/src/clients/meta/MetaClient.h @@ -72,6 +72,10 @@ using Indexes = std::unordered_map>; using Listeners = std::unordered_map>>; +// Get services +using ServiceClientsList = + std::unordered_map>; + struct SpaceInfoCache { cpp2::SpaceDesc spaceDesc_; PartsAlloc partsAlloc_; @@ -144,9 +148,6 @@ using UserPasswordMap = std::unordered_map; using MetaConfigMap = std::unordered_map, cpp2::ConfigItem>; -// get fulltext services -using FulltextClientsList = std::vector; - using FTIndexMap = std::unordered_map; using SessionMap = std::unordered_map; @@ -447,15 +448,17 @@ class MetaClient { StatusOr> getListenerHostTypeBySpacePartType(GraphSpaceID spaceId, PartitionID partId); - // Operations for fulltext services - folly::Future> signInFTService(cpp2::FTServiceType type, - const std::vector& clients); + // Operations for services + folly::Future> signInService(const cpp2::ExternalServiceType& type, + const std::vector& clients); - folly::Future> signOutFTService(); + folly::Future> signOutService(const cpp2::ExternalServiceType& type); - folly::Future>> listFTClients(); + folly::Future> listServiceClients( + const cpp2::ExternalServiceType& type); - StatusOr> getFTClientsFromCache(); + StatusOr> getServiceClientsFromCache( + const cpp2::ExternalServiceType& type); // Operations for fulltext index. @@ -682,7 +685,7 @@ class MetaClient { bool loadListeners(GraphSpaceID spaceId, std::shared_ptr cache); - bool loadFulltextClients(); + bool loadGlobalServiceClients(); bool loadFulltextIndexes(); @@ -815,7 +818,9 @@ class MetaClient { NameIndexMap tagNameIndexMap_; NameIndexMap edgeNameIndexMap_; - FulltextClientsList fulltextClientList_; + + // Global service client + ServiceClientsList serviceClientList_; FTIndexMap fulltextIndexMap_; mutable folly::RWSpinLock localCacheLock_; diff --git a/src/clients/storage/InternalStorageClient.cpp b/src/clients/storage/InternalStorageClient.cpp index c51bd6dd4aa..ad2a7c58960 100644 --- a/src/clients/storage/InternalStorageClient.cpp +++ b/src/clients/storage/InternalStorageClient.cpp @@ -88,7 +88,8 @@ void InternalStorageClient::chainAddEdges(cpp2::AddEdgesRequest& directReq, auto partId = directReq.get_parts().begin()->first; auto optLeader = getLeader(directReq.get_space_id(), partId); if (!optLeader.ok()) { - LOG(WARNING) << folly::sformat("failed to get leader, space {}, part {}", spaceId, partId); + LOG(WARNING) << folly::sformat("failed to get leader, space {}, part {}", spaceId, partId) + << optLeader.status(); p.setValue(::nebula::cpp2::ErrorCode::E_SPACE_NOT_FOUND); return; } @@ -131,5 +132,47 @@ cpp2::ChainAddEdgesRequest InternalStorageClient::makeChainAddReq(const cpp2::Ad return ret; } +void InternalStorageClient::chainDeleteEdges(cpp2::DeleteEdgesRequest& req, + const std::string& txnId, + TermID termId, + folly::Promise&& p, + folly::EventBase* evb) { + auto spaceId = req.get_space_id(); + auto partId = req.get_parts().begin()->first; + auto optLeader = getLeader(req.get_space_id(), partId); + if (!optLeader.ok()) { + LOG(WARNING) << folly::sformat("failed to get leader, space {}, part {}", spaceId, partId) + << optLeader.status(); + p.setValue(::nebula::cpp2::ErrorCode::E_SPACE_NOT_FOUND); + return; + } + HostAddr& leader = optLeader.value(); + leader.port += kInternalPortOffset; + VLOG(2) << "leader host: " << leader; + + cpp2::ChainDeleteEdgesRequest chainReq; + chainReq.space_id_ref() = req.get_space_id(); + chainReq.parts_ref() = req.get_parts(); + chainReq.txn_id_ref() = txnId; + chainReq.term_ref() = termId; + auto resp = getResponse( + evb, + std::make_pair(leader, chainReq), + [](cpp2::InternalStorageServiceAsyncClient* client, const cpp2::ChainDeleteEdgesRequest& r) { + return client->future_chainDeleteEdges(r); + }); + + std::move(resp).thenTry([=, p = std::move(p)](auto&& t) mutable { + auto code = getErrorCode(t); + if (code == nebula::cpp2::ErrorCode::E_LEADER_CHANGED) { + std::this_thread::sleep_for(std::chrono::milliseconds(500)); + chainDeleteEdges(req, txnId, termId, std::move(p)); + } else { + p.setValue(code); + } + return; + }); +} + } // namespace storage } // namespace nebula diff --git a/src/clients/storage/InternalStorageClient.h b/src/clients/storage/InternalStorageClient.h index f34311f605a..6c56559f847 100644 --- a/src/clients/storage/InternalStorageClient.h +++ b/src/clients/storage/InternalStorageClient.h @@ -48,6 +48,12 @@ class InternalStorageClient folly::Promise<::nebula::cpp2::ErrorCode>&& p, folly::EventBase* evb = nullptr); + virtual void chainDeleteEdges(cpp2::DeleteEdgesRequest& req, + const std::string& txnId, + TermID termId, + folly::Promise<::nebula::cpp2::ErrorCode>&& p, + folly::EventBase* evb = nullptr); + private: cpp2::ChainAddEdgesRequest makeChainAddReq(const cpp2::AddEdgesRequest& req, TermID termId, diff --git a/src/clients/storage/StorageClient.cpp b/src/clients/storage/StorageClient.cpp index 61381598225..6c1a81f5a8d 100644 --- a/src/clients/storage/StorageClient.cpp +++ b/src/clients/storage/StorageClient.cpp @@ -268,8 +268,10 @@ StorageRpcRespFuture StorageClient::deleteEdges( return collectResponse(param.evb, std::move(requests), - [](ThriftClientType* client, const cpp2::DeleteEdgesRequest& r) { - return client->future_deleteEdges(r); + [useToss = param.useExperimentalFeature]( + ThriftClientType* client, const cpp2::DeleteEdgesRequest& r) { + return useToss ? client->future_chainDeleteEdges(r) + : client->future_deleteEdges(r); }); } diff --git a/src/common/meta/SchemaManager.h b/src/common/meta/SchemaManager.h index 23fa8c68dec..cdf7335633f 100644 --- a/src/common/meta/SchemaManager.h +++ b/src/common/meta/SchemaManager.h @@ -82,7 +82,8 @@ class SchemaManager { // get all latest version of all edge schema virtual StatusOr getAllLatestVerEdgeSchema(GraphSpaceID space) = 0; - virtual StatusOr> getFTClients() = 0; + virtual StatusOr> getServiceClients( + cpp2::ExternalServiceType type) = 0; // Get the TagID or EdgeType by the name. // The first one is a bool which is used to distinguish the type. diff --git a/src/common/meta/ServerBasedSchemaManager.cpp b/src/common/meta/ServerBasedSchemaManager.cpp index 50139d0a25c..e86b8651c28 100644 --- a/src/common/meta/ServerBasedSchemaManager.cpp +++ b/src/common/meta/ServerBasedSchemaManager.cpp @@ -147,13 +147,14 @@ StatusOr ServerBasedSchemaManager::getAllLatestVerEdgeSchema(GraphSp return metaClient_->getAllLatestVerEdgeSchemaFromCache(space); } -StatusOr> ServerBasedSchemaManager::getFTClients() { - auto ret = metaClient_->getFTClientsFromCache(); +StatusOr> +ServerBasedSchemaManager::getServiceClients(meta::cpp2::ExternalServiceType type) { + auto ret = metaClient_->getServiceClientsFromCache(type); if (!ret.ok()) { return ret.status(); } if (ret.value().empty()) { - return Status::Error("fulltext client list is empty"); + return Status::Error("Service list is empty"); } return std::move(ret).value(); } diff --git a/src/common/meta/ServerBasedSchemaManager.h b/src/common/meta/ServerBasedSchemaManager.h index 7b11424fc4c..75002effba1 100644 --- a/src/common/meta/ServerBasedSchemaManager.h +++ b/src/common/meta/ServerBasedSchemaManager.h @@ -68,7 +68,8 @@ class ServerBasedSchemaManager : public SchemaManager { // get all latest version of all edges StatusOr getAllLatestVerEdgeSchema(GraphSpaceID space) override; - StatusOr> getFTClients() override; + StatusOr> getServiceClients( + cpp2::ExternalServiceType type) override; StatusOr> getFTIndex( GraphSpaceID spaceId, int32_t schemaId) override; diff --git a/src/common/utils/MetaKeyUtils.cpp b/src/common/utils/MetaKeyUtils.cpp index 294374b2222..a45ad90c002 100644 --- a/src/common/utils/MetaKeyUtils.cpp +++ b/src/common/utils/MetaKeyUtils.cpp @@ -27,7 +27,7 @@ static const std::unordered_map> syste {"configs", {"__configs__", true}}, {"groups", {"__groups__", true}}, {"zones", {"__zones__", true}}, - {"ft_service", {"__ft_service__", false}}, + {"services", {"__services__", false}}, {"sessions", {"__sessions__", true}}}; // SystemInfo will always be backed up @@ -89,7 +89,7 @@ static const std::string kBalancePlanTable = tableMaps.at("balance_plan").fir static const std::string kLocalIdTable = tableMaps.at("local_id").first; // NOLINT const std::string kFTIndexTable = tableMaps.at("ft_index").first; // NOLINT -const std::string kFTServiceTable = systemTableMaps.at("ft_service").first; // NOLINT +const std::string kServicesTable = systemTableMaps.at("services").first; // NOLINT const std::string kSessionsTable = systemTableMaps.at("sessions").first; // NOLINT const std::string kIdKey = systemInfoMaps.at("autoIncrementId").first; // NOLINT @@ -1155,27 +1155,33 @@ const std::string& MetaKeyUtils::statsKeyPrefix() { return kStatsTable; } -std::string MetaKeyUtils::fulltextServiceKey() { +std::string MetaKeyUtils::serviceKey(const meta::cpp2::ExternalServiceType& type) { std::string key; - key.reserve(kFTServiceTable.size()); - key.append(kFTServiceTable.data(), kFTServiceTable.size()); + key.reserve(kServicesTable.size() + sizeof(meta::cpp2::ExternalServiceType)); + key.append(kServicesTable.data(), kServicesTable.size()) + .append(reinterpret_cast(&type), sizeof(meta::cpp2::ExternalServiceType)); return key; } -std::string MetaKeyUtils::fulltextServiceVal(meta::cpp2::FTServiceType type, - const std::vector& clients) { - std::string val, cval; - apache::thrift::CompactSerializer::serialize(clients, &cval); - val.reserve(sizeof(meta::cpp2::FTServiceType) + cval.size()); - val.append(reinterpret_cast(&type), sizeof(meta::cpp2::FTServiceType)).append(cval); +const std::string& MetaKeyUtils::servicePrefix() { + return kServicesTable; +} + +meta::cpp2::ExternalServiceType MetaKeyUtils::parseServiceType(folly::StringPiece rawData) { + auto offset = kServicesTable.size(); + return *reinterpret_cast(rawData.data() + offset); +} + +std::string MetaKeyUtils::serviceVal(const std::vector& clients) { + std::string val; + apache::thrift::CompactSerializer::serialize(clients, &val); return val; } -std::vector MetaKeyUtils::parseFTClients(folly::StringPiece rawData) { - std::vector clients; - int32_t offset = sizeof(meta::cpp2::FTServiceType); - auto clientsRaw = rawData.subpiece(offset, rawData.size() - offset); - apache::thrift::CompactSerializer::deserialize(clientsRaw, clients); +std::vector MetaKeyUtils::parseServiceClients( + folly::StringPiece rawData) { + std::vector clients; + apache::thrift::CompactSerializer::deserialize(rawData, clients); return clients; } diff --git a/src/common/utils/MetaKeyUtils.h b/src/common/utils/MetaKeyUtils.h index bad713be151..5d463da6c53 100644 --- a/src/common/utils/MetaKeyUtils.h +++ b/src/common/utils/MetaKeyUtils.h @@ -346,12 +346,15 @@ class MetaKeyUtils final { static GraphSpaceID parseStatsSpace(folly::StringPiece rawData); - static std::string fulltextServiceKey(); + static std::string serviceKey(const meta::cpp2::ExternalServiceType& type); - static std::string fulltextServiceVal(meta::cpp2::FTServiceType type, - const std::vector& clients); + static std::string serviceVal(const std::vector& client); - static std::vector parseFTClients(folly::StringPiece rawData); + static const std::string& servicePrefix(); + + static meta::cpp2::ExternalServiceType parseServiceType(folly::StringPiece rawData); + + static std::vector parseServiceClients(folly::StringPiece rawData); static const std::string& sessionPrefix(); diff --git a/src/common/utils/Types.h b/src/common/utils/Types.h index 88e74106edc..e365fb4270e 100644 --- a/src/common/utils/Types.h +++ b/src/common/utils/Types.h @@ -19,6 +19,8 @@ enum class NebulaKeyType : uint32_t { kOperation = 0x00000005, kKeyValue = 0x00000006, kVertex = 0x00000007, + kPrime = 0x00000008, // used in TOSS, if we write a lock succeed + kDoublePrime = 0x00000009, // used in TOSS, if we get RPC back from remote. }; enum class NebulaSystemKeyType : uint32_t { diff --git a/src/daemons/GraphDaemon.cpp b/src/daemons/GraphDaemon.cpp index 8ef9389cf7f..0b76f558120 100644 --- a/src/daemons/GraphDaemon.cpp +++ b/src/daemons/GraphDaemon.cpp @@ -10,7 +10,6 @@ #include #include "common/base/Base.h" -#include "common/base/SignalHandler.h" #include "common/fs/FileUtils.h" #include "common/network/NetworkUtils.h" #include "common/process/ProcessUtils.h" @@ -18,6 +17,7 @@ #include "common/time/TimezoneInfo.h" #include "daemons/SetupLogging.h" #include "graph/service/GraphFlags.h" +#include "graph/service/GraphServer.h" #include "graph/service/GraphService.h" #include "graph/stats/GraphStats.h" #include "version/Version.h" @@ -30,12 +30,9 @@ using nebula::fs::FileUtils; using nebula::graph::GraphService; using nebula::network::NetworkUtils; -static std::unique_ptr gServer; - static void signalHandler(int sig); static Status setupSignalHandler(); static void printHelp(const char *prog); -static void setupThreadManager(); #if defined(__x86_64__) extern Status setupBreakpad(); #endif @@ -43,6 +40,8 @@ extern Status setupBreakpad(); DECLARE_string(flagfile); DECLARE_bool(containerized); +std::unique_ptr gServer; + int main(int argc, char *argv[]) { google::SetVersionString(nebula::versionString()); if (argc == 1) { @@ -154,30 +153,6 @@ int main(int argc, char *argv[]) { } LOG(INFO) << "Number of worker threads: " << FLAGS_num_worker_threads; - auto threadFactory = std::make_shared("graph-netio"); - auto ioThreadPool = std::make_shared(FLAGS_num_netio_threads, - std::move(threadFactory)); - gServer = std::make_unique(); - gServer->setIOThreadPool(ioThreadPool); - - auto interface = std::make_shared(); - status = interface->init(ioThreadPool, localhost); - if (!status.ok()) { - LOG(ERROR) << status; - return EXIT_FAILURE; - } - - gServer->setPort(localhost.port); - gServer->setInterface(std::move(interface)); - gServer->setReusePort(FLAGS_reuse_port); - gServer->setIdleTimeout(std::chrono::seconds(FLAGS_client_idle_timeout_secs)); - gServer->setNumAcceptThreads(FLAGS_num_accept_threads); - gServer->setListenBacklog(FLAGS_listen_backlog); - if (FLAGS_enable_ssl || FLAGS_enable_graph_ssl) { - gServer->setSSLConfig(nebula::sslContextConfig()); - } - setupThreadManager(); - // Setup the signal handlers status = setupSignalHandler(); if (!status.ok()) { @@ -185,16 +160,15 @@ int main(int argc, char *argv[]) { return EXIT_FAILURE; } - FLOG_INFO("Starting nebula-graphd on %s:%d\n", localhost.host.c_str(), localhost.port); - try { - gServer->serve(); // Blocking wait until shut down via gServer->stop() - } catch (const std::exception &e) { - FLOG_ERROR("Exception thrown while starting the RPC server: %s", e.what()); + gServer = std::make_unique(localhost); + + if (!gServer->start()) { + LOG(ERROR) << "The graph server start failed"; return EXIT_FAILURE; } - FLOG_INFO("nebula-graphd on %s:%d has been stopped", localhost.host.c_str(), localhost.port); - + gServer->waitUntilStop(); + LOG(INFO) << "The graph Daemon stopped"; return EXIT_SUCCESS; } @@ -209,7 +183,7 @@ void signalHandler(int sig) { case SIGINT: case SIGTERM: FLOG_INFO("Signal %d(%s) received, stopping this server", sig, ::strsignal(sig)); - gServer->stop(); + gServer->notifyStop(); break; default: FLOG_ERROR("Signal %d(%s) received but ignored", sig, ::strsignal(sig)); @@ -219,13 +193,3 @@ void signalHandler(int sig) { void printHelp(const char *prog) { fprintf(stderr, "%s --flagfile \n", prog); } - -void setupThreadManager() { - int numThreads = - FLAGS_num_worker_threads > 0 ? FLAGS_num_worker_threads : gServer->getNumIOWorkerThreads(); - std::shared_ptr threadManager( - PriorityThreadManager::newPriorityThreadManager(numThreads)); - threadManager->setNamePrefix("executor"); - threadManager->start(); - gServer->setThreadManager(threadManager); -} diff --git a/src/graph/executor/CMakeLists.txt b/src/graph/executor/CMakeLists.txt index d05dd595738..4ed8baca7c6 100644 --- a/src/graph/executor/CMakeLists.txt +++ b/src/graph/executor/CMakeLists.txt @@ -71,9 +71,9 @@ nebula_add_library( admin/IngestExecutor.cpp admin/ConfigExecutor.cpp admin/ZoneExecutor.cpp - admin/ShowTSClientsExecutor.cpp - admin/SignInTSServiceExecutor.cpp - admin/SignOutTSServiceExecutor.cpp + admin/ShowServiceClientsExecutor.cpp + admin/SignInServiceExecutor.cpp + admin/SignOutServiceExecutor.cpp admin/SessionExecutor.cpp admin/ShowQueriesExecutor.cpp admin/KillQueryExecutor.cpp diff --git a/src/graph/executor/Executor.cpp b/src/graph/executor/Executor.cpp index 1a254c1063c..4e1dfee089b 100644 --- a/src/graph/executor/Executor.cpp +++ b/src/graph/executor/Executor.cpp @@ -39,10 +39,10 @@ #include "graph/executor/admin/ShowHostsExecutor.h" #include "graph/executor/admin/ShowMetaLeaderExecutor.h" #include "graph/executor/admin/ShowQueriesExecutor.h" +#include "graph/executor/admin/ShowServiceClientsExecutor.h" #include "graph/executor/admin/ShowStatsExecutor.h" -#include "graph/executor/admin/ShowTSClientsExecutor.h" -#include "graph/executor/admin/SignInTSServiceExecutor.h" -#include "graph/executor/admin/SignOutTSServiceExecutor.h" +#include "graph/executor/admin/SignInServiceExecutor.h" +#include "graph/executor/admin/SignOutServiceExecutor.h" #include "graph/executor/admin/SnapshotExecutor.h" #include "graph/executor/admin/SpaceExecutor.h" #include "graph/executor/admin/SubmitJobExecutor.h" @@ -155,16 +155,25 @@ Executor *Executor::makeExecutor(const PlanNode *node, // static Executor *Executor::makeExecutor(QueryContext *qctx, const PlanNode *node) { auto pool = qctx->objPool(); + auto &spaceName = qctx->rctx() ? qctx->rctx()->session()->spaceName() : ""; switch (node->kind()) { case PlanNode::Kind::kPassThrough: { return pool->add(new PassThroughExecutor(node, qctx)); } case PlanNode::Kind::kAggregate: { stats::StatsManager::addValue(kNumAggregateExecutors); + if (FLAGS_enable_space_level_metrics && spaceName != "") { + stats::StatsManager::addValue( + stats::StatsManager::counterWithLabels(kNumAggregateExecutors, {{"space", spaceName}})); + } return pool->add(new AggregateExecutor(node, qctx)); } case PlanNode::Kind::kSort: { stats::StatsManager::addValue(kNumSortExecutors); + if (FLAGS_enable_space_level_metrics && spaceName != "") { + stats::StatsManager::addValue( + stats::StatsManager::counterWithLabels(kNumSortExecutors, {{"space", spaceName}})); + } return pool->add(new SortExecutor(node, qctx)); } case PlanNode::Kind::kTopN: { @@ -208,6 +217,10 @@ Executor *Executor::makeExecutor(QueryContext *qctx, const PlanNode *node) { case PlanNode::Kind::kTagIndexPrefixScan: case PlanNode::Kind::kTagIndexRangeScan: { stats::StatsManager::addValue(kNumIndexScanExecutors); + if (FLAGS_enable_space_level_metrics && spaceName != "") { + stats::StatsManager::addValue( + stats::StatsManager::counterWithLabels(kNumIndexScanExecutors, {{"space", spaceName}})); + } return pool->add(new IndexScanExecutor(node, qctx)); } case PlanNode::Kind::kStart: { @@ -489,17 +502,17 @@ Executor *Executor::makeExecutor(QueryContext *qctx, const PlanNode *node) { case PlanNode::Kind::kShowStats: { return pool->add(new ShowStatsExecutor(node, qctx)); } - case PlanNode::Kind::kShowTSClients: { - return pool->add(new ShowTSClientsExecutor(node, qctx)); + case PlanNode::Kind::kShowServiceClients: { + return pool->add(new ShowServiceClientsExecutor(node, qctx)); } case PlanNode::Kind::kShowFTIndexes: { return pool->add(new ShowFTIndexesExecutor(node, qctx)); } - case PlanNode::Kind::kSignInTSService: { - return pool->add(new SignInTSServiceExecutor(node, qctx)); + case PlanNode::Kind::kSignInService: { + return pool->add(new SignInServiceExecutor(node, qctx)); } - case PlanNode::Kind::kSignOutTSService: { - return pool->add(new SignOutTSServiceExecutor(node, qctx)); + case PlanNode::Kind::kSignOutService: { + return pool->add(new SignOutServiceExecutor(node, qctx)); } case PlanNode::Kind::kDownload: { return pool->add(new DownloadExecutor(node, qctx)); @@ -594,7 +607,12 @@ Status Executor::close() { Status Executor::checkMemoryWatermark() { if (node_->isQueryNode() && MemoryUtils::kHitMemoryHighWatermark.load()) { - stats::StatsManager::addValue(kNumOomExecutors); + stats::StatsManager::addValue(kNumQueriesHitMemoryWatermark); + auto &spaceName = qctx()->rctx() ? qctx()->rctx()->session()->spaceName() : ""; + if (FLAGS_enable_space_level_metrics && spaceName != "") { + stats::StatsManager::addValue(stats::StatsManager::counterWithLabels( + kNumQueriesHitMemoryWatermark, {{"space", spaceName}})); + } return Status::Error("Used memory hits the high watermark(%lf) of total system memory.", FLAGS_system_memory_high_watermark_ratio); } diff --git a/src/graph/executor/admin/ShowServiceClientsExecutor.cpp b/src/graph/executor/admin/ShowServiceClientsExecutor.cpp new file mode 100644 index 00000000000..af9748e60d1 --- /dev/null +++ b/src/graph/executor/admin/ShowServiceClientsExecutor.cpp @@ -0,0 +1,48 @@ +/* Copyright (c) 2020 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "graph/executor/admin/ShowServiceClientsExecutor.h" + +#include + +#include "graph/context/QueryContext.h" +#include "graph/planner/plan/Admin.h" +#include "graph/service/PermissionManager.h" +#include "interface/gen-cpp2/meta_types.h" + +namespace nebula { +namespace graph { + +folly::Future ShowServiceClientsExecutor::execute() { + SCOPED_TIMER(&execTime_); + return showServiceClients(); +} + +folly::Future ShowServiceClientsExecutor::showServiceClients() { + auto *siNode = asNode(node()); + auto type = siNode->type(); + + return qctx()->getMetaClient()->listServiceClients(type).via(runner()).thenValue( + [this](auto &&resp) { + if (!resp.ok()) { + LOG(ERROR) << resp.status(); + return resp.status(); + } + auto values = std::move(resp).value(); + DataSet v({"Type", "Host", "Port"}); + for (const auto &value : values) { + for (const auto &client : value.second) { + nebula::Row r({apache::thrift::util::enumNameSafe(value.first), + client.host.host, + client.host.port}); + v.emplace_back(std::move(r)); + } + } + return finish(std::move(v)); + }); +} + +} // namespace graph +} // namespace nebula diff --git a/src/graph/executor/admin/ShowServiceClientsExecutor.h b/src/graph/executor/admin/ShowServiceClientsExecutor.h new file mode 100644 index 00000000000..e524954c9d3 --- /dev/null +++ b/src/graph/executor/admin/ShowServiceClientsExecutor.h @@ -0,0 +1,28 @@ +/* Copyright (c) 2020 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#ifndef GRAPH_EXECUTOR_ADMIN_SHOW_SERVICE_CLIENTS_EXECUTOR_H_ +#define GRAPH_EXECUTOR_ADMIN_SHOW_SERVICE_CLIENTS_EXECUTOR_H_ + +#include "graph/executor/Executor.h" + +namespace nebula { +namespace graph { + +class ShowServiceClientsExecutor final : public Executor { + public: + ShowServiceClientsExecutor(const PlanNode *node, QueryContext *qctx) + : Executor("ShowServiceClientsExecutor", node, qctx) {} + + folly::Future execute() override; + + private: + folly::Future showServiceClients(); +}; + +} // namespace graph +} // namespace nebula + +#endif // GRAPH_EXECUTOR_ADMIN_SHOW_SERVICE_CLIENTS_EXECUTOR_H_ diff --git a/src/graph/executor/admin/ShowTSClientsExecutor.cpp b/src/graph/executor/admin/ShowTSClientsExecutor.cpp deleted file mode 100644 index f5981a6c1f3..00000000000 --- a/src/graph/executor/admin/ShowTSClientsExecutor.cpp +++ /dev/null @@ -1,41 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License. - */ - -#include "graph/executor/admin/ShowTSClientsExecutor.h" - -#include "graph/context/QueryContext.h" -#include "graph/planner/plan/Admin.h" -#include "graph/service/PermissionManager.h" -#include "interface/gen-cpp2/meta_types.h" - -namespace nebula { -namespace graph { - -folly::Future ShowTSClientsExecutor::execute() { - SCOPED_TIMER(&execTime_); - return showTSClients(); -} - -folly::Future ShowTSClientsExecutor::showTSClients() { - return qctx()->getMetaClient()->listFTClients().via(runner()).thenValue([this](auto &&resp) { - if (!resp.ok()) { - LOG(ERROR) << resp.status(); - return resp.status(); - } - auto value = std::move(resp).value(); - DataSet v({"Host", "Port", "Connection type"}); - for (const auto &client : value) { - nebula::Row r; - r.values.emplace_back(client.host.host); - r.values.emplace_back(client.host.port); - r.values.emplace_back(client.conn_type_ref().has_value() ? *client.get_conn_type() : "http"); - v.emplace_back(std::move(r)); - } - return finish(std::move(v)); - }); -} - -} // namespace graph -} // namespace nebula diff --git a/src/graph/executor/admin/ShowTSClientsExecutor.h b/src/graph/executor/admin/ShowTSClientsExecutor.h deleted file mode 100644 index 56734694ce3..00000000000 --- a/src/graph/executor/admin/ShowTSClientsExecutor.h +++ /dev/null @@ -1,28 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License. - */ - -#ifndef GRAPH_EXECUTOR_ADMIN_SHOW_TS_CLIENTS_EXECUTOR_H_ -#define GRAPH_EXECUTOR_ADMIN_SHOW_TS_CLIENTS_EXECUTOR_H_ - -#include "graph/executor/Executor.h" - -namespace nebula { -namespace graph { - -class ShowTSClientsExecutor final : public Executor { - public: - ShowTSClientsExecutor(const PlanNode *node, QueryContext *qctx) - : Executor("ShowTSClientsExecutor", node, qctx) {} - - folly::Future execute() override; - - private: - folly::Future showTSClients(); -}; - -} // namespace graph -} // namespace nebula - -#endif // GRAPH_EXECUTOR_ADMIN_SHOW_TS_CLIENTS_EXECUTOR_H_ diff --git a/src/graph/executor/admin/SignInTSServiceExecutor.cpp b/src/graph/executor/admin/SignInServiceExecutor.cpp similarity index 57% rename from src/graph/executor/admin/SignInTSServiceExecutor.cpp rename to src/graph/executor/admin/SignInServiceExecutor.cpp index e98ebe0cee8..f83bb068314 100644 --- a/src/graph/executor/admin/SignInTSServiceExecutor.cpp +++ b/src/graph/executor/admin/SignInServiceExecutor.cpp @@ -3,29 +3,30 @@ * This source code is licensed under Apache 2.0 License. */ -#include "graph/executor/admin/SignInTSServiceExecutor.h" +#include "graph/executor/admin/SignInServiceExecutor.h" #include "graph/planner/plan/Admin.h" namespace nebula { namespace graph { -folly::Future SignInTSServiceExecutor::execute() { +folly::Future SignInServiceExecutor::execute() { SCOPED_TIMER(&execTime_); - return signInTSService(); + return signInService(); } -folly::Future SignInTSServiceExecutor::signInTSService() { - auto *siNode = asNode(node()); +folly::Future SignInServiceExecutor::signInService() { + auto *siNode = asNode(node()); + auto type = siNode->type(); return qctx() ->getMetaClient() - ->signInFTService(siNode->type(), siNode->clients()) + ->signInService(type, siNode->clients()) .via(runner()) .thenValue([this](StatusOr resp) { SCOPED_TIMER(&execTime_); NG_RETURN_IF_ERROR(resp); if (!resp.value()) { - return Status::Error("Sign in text service failed!"); + return Status::Error("Sign in service failed!"); } return Status::OK(); }); diff --git a/src/graph/executor/admin/SignInServiceExecutor.h b/src/graph/executor/admin/SignInServiceExecutor.h new file mode 100644 index 00000000000..83d4fd22ed9 --- /dev/null +++ b/src/graph/executor/admin/SignInServiceExecutor.h @@ -0,0 +1,28 @@ +/* Copyright (c) 2020 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#ifndef GRAPH_EXECUTOR_ADMIN_SIGNINSERVICEEXECUTOR_H_ +#define GRAPH_EXECUTOR_ADMIN_SIGNINSERVICEEXECUTOR_H_ + +#include "graph/executor/Executor.h" + +namespace nebula { +namespace graph { + +class SignInServiceExecutor final : public Executor { + public: + SignInServiceExecutor(const PlanNode *node, QueryContext *qctx) + : Executor("SignInServiceExecutor", node, qctx) {} + + folly::Future execute() override; + + private: + folly::Future signInService(); +}; + +} // namespace graph +} // namespace nebula + +#endif // GRAPH_EXECUTOR_ADMIN_SIGNINSERVICEEXECUTOR_H_ diff --git a/src/graph/executor/admin/SignInTSServiceExecutor.h b/src/graph/executor/admin/SignInTSServiceExecutor.h deleted file mode 100644 index 6567501ce38..00000000000 --- a/src/graph/executor/admin/SignInTSServiceExecutor.h +++ /dev/null @@ -1,28 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License. - */ - -#ifndef GRAPH_EXECUTOR_ADMIN_SIGNINTSSERVICEEXECUTOR_H_ -#define GRAPH_EXECUTOR_ADMIN_SIGNINTSSERVICEEXECUTOR_H_ - -#include "graph/executor/Executor.h" - -namespace nebula { -namespace graph { - -class SignInTSServiceExecutor final : public Executor { - public: - SignInTSServiceExecutor(const PlanNode *node, QueryContext *qctx) - : Executor("SignInTSServiceExecutor", node, qctx) {} - - folly::Future execute() override; - - private: - folly::Future signInTSService(); -}; - -} // namespace graph -} // namespace nebula - -#endif // GRAPH_EXECUTOR_ADMIN_SIGNINTSSERVICEEXECUTOR_H_ diff --git a/src/graph/executor/admin/SignOutTSServiceExecutor.cpp b/src/graph/executor/admin/SignOutServiceExecutor.cpp similarity index 51% rename from src/graph/executor/admin/SignOutTSServiceExecutor.cpp rename to src/graph/executor/admin/SignOutServiceExecutor.cpp index 64f395f3967..3c6f81c8969 100644 --- a/src/graph/executor/admin/SignOutTSServiceExecutor.cpp +++ b/src/graph/executor/admin/SignOutServiceExecutor.cpp @@ -3,25 +3,28 @@ * This source code is licensed under Apache 2.0 License. */ -#include "graph/executor/admin/SignOutTSServiceExecutor.h" +#include "graph/executor/admin/SignOutServiceExecutor.h" #include "graph/planner/plan/Admin.h" namespace nebula { namespace graph { -folly::Future SignOutTSServiceExecutor::execute() { +folly::Future SignOutServiceExecutor::execute() { SCOPED_TIMER(&execTime_); - return signOutTSService(); + return signOutService(); } -folly::Future SignOutTSServiceExecutor::signOutTSService() { - return qctx()->getMetaClient()->signOutFTService().via(runner()).thenValue( +folly::Future SignOutServiceExecutor::signOutService() { + auto *siNode = asNode(node()); + auto type = siNode->type(); + + return qctx()->getMetaClient()->signOutService(type).via(runner()).thenValue( [this](StatusOr resp) { SCOPED_TIMER(&execTime_); NG_RETURN_IF_ERROR(resp); if (!resp.value()) { - return Status::Error("Sign out text service failed!"); + return Status::Error("Sign out service failed!"); } return Status::OK(); }); diff --git a/src/graph/executor/admin/SignOutTSServiceExecutor.h b/src/graph/executor/admin/SignOutServiceExecutor.h similarity index 68% rename from src/graph/executor/admin/SignOutTSServiceExecutor.h rename to src/graph/executor/admin/SignOutServiceExecutor.h index e03f2dde8a6..1e1ded6de35 100644 --- a/src/graph/executor/admin/SignOutTSServiceExecutor.h +++ b/src/graph/executor/admin/SignOutServiceExecutor.h @@ -11,15 +11,15 @@ namespace nebula { namespace graph { -class SignOutTSServiceExecutor final : public Executor { +class SignOutServiceExecutor final : public Executor { public: - SignOutTSServiceExecutor(const PlanNode *node, QueryContext *qctx) - : Executor("SignInTSServiceExecutor", node, qctx) {} + SignOutServiceExecutor(const PlanNode *node, QueryContext *qctx) + : Executor("SignOutServiceExecutor", node, qctx) {} folly::Future execute() override; private: - folly::Future signOutTSService(); + folly::Future signOutService(); }; } // namespace graph diff --git a/src/graph/executor/mutate/DeleteExecutor.cpp b/src/graph/executor/mutate/DeleteExecutor.cpp index 3cb1c670c0b..bcee155e38e 100644 --- a/src/graph/executor/mutate/DeleteExecutor.cpp +++ b/src/graph/executor/mutate/DeleteExecutor.cpp @@ -9,6 +9,7 @@ #include "graph/context/QueryContext.h" #include "graph/executor/mutate/DeleteExecutor.h" #include "graph/planner/plan/Mutate.h" +#include "graph/service/GraphFlags.h" #include "graph/util/SchemaUtil.h" using nebula::storage::StorageClient; @@ -208,6 +209,7 @@ folly::Future DeleteEdgesExecutor::deleteEdges() { auto plan = qctx()->plan(); StorageClient::CommonRequestParam param( spaceId, qctx()->rctx()->session()->id(), plan->id(), plan->isProfileEnabled()); + param.useExperimentalFeature = FLAGS_enable_experimental_feature; return qctx() ->getStorageClient() ->deleteEdges(param, std::move(edgeKeys)) diff --git a/src/graph/optimizer/rule/IndexScanRule.cpp b/src/graph/optimizer/rule/IndexScanRule.cpp index f1ae0ea24af..5ad02a164f4 100644 --- a/src/graph/optimizer/rule/IndexScanRule.cpp +++ b/src/graph/optimizer/rule/IndexScanRule.cpp @@ -4,6 +4,7 @@ */ #include "graph/optimizer/rule/IndexScanRule.h" + #include "graph/optimizer/OptContext.h" #include "graph/optimizer/OptGroup.h" #include "graph/optimizer/OptRule.h" diff --git a/src/graph/planner/plan/Admin.h b/src/graph/planner/plan/Admin.h index 3785031a7c0..1496d6a2ae5 100644 --- a/src/graph/planner/plan/Admin.h +++ b/src/graph/planner/plan/Admin.h @@ -1149,49 +1149,75 @@ class ShowStats final : public SingleDependencyNode { : SingleDependencyNode(qctx, Kind::kShowStats, input) {} }; -class ShowTSClients final : public SingleDependencyNode { +class ShowServiceClients final : public SingleDependencyNode { public: - static ShowTSClients* make(QueryContext* qctx, PlanNode* input) { - return qctx->objPool()->add(new ShowTSClients(qctx, input)); + static ShowServiceClients* make(QueryContext* qctx, + PlanNode* input, + meta::cpp2::ExternalServiceType type) { + return qctx->objPool()->add(new ShowServiceClients(qctx, input, type)); + } + + meta::cpp2::ExternalServiceType type() const { + return type_; } private: - ShowTSClients(QueryContext* qctx, PlanNode* input) - : SingleDependencyNode(qctx, Kind::kShowTSClients, input) {} + ShowServiceClients(QueryContext* qctx, PlanNode* input, meta::cpp2::ExternalServiceType type) + : SingleDependencyNode(qctx, Kind::kShowServiceClients, input), type_(type) {} + + private: + meta::cpp2::ExternalServiceType type_; }; -class SignInTSService final : public SingleDependencyNode { +class SignInService final : public SingleDependencyNode { public: - static SignInTSService* make(QueryContext* qctx, - PlanNode* input, - std::vector clients) { - return qctx->objPool()->add(new SignInTSService(qctx, input, std::move(clients))); + static SignInService* make(QueryContext* qctx, + PlanNode* input, + std::vector clients, + meta::cpp2::ExternalServiceType type) { + return qctx->objPool()->add(new SignInService(qctx, input, std::move(clients), type)); } - const std::vector& clients() const { + const std::vector& clients() const { return clients_; } - meta::cpp2::FTServiceType type() const { - return meta::cpp2::FTServiceType::ELASTICSEARCH; + meta::cpp2::ExternalServiceType type() const { + return type_; } private: - SignInTSService(QueryContext* qctx, PlanNode* input, std::vector clients) - : SingleDependencyNode(qctx, Kind::kSignInTSService, input), clients_(std::move(clients)) {} + SignInService(QueryContext* qctx, + PlanNode* input, + std::vector clients, + meta::cpp2::ExternalServiceType type) + : SingleDependencyNode(qctx, Kind::kSignInService, input), + clients_(std::move(clients)), + type_(type) {} - std::vector clients_; + private: + std::vector clients_; + meta::cpp2::ExternalServiceType type_; }; -class SignOutTSService final : public SingleDependencyNode { +class SignOutService final : public SingleDependencyNode { public: - static SignOutTSService* make(QueryContext* qctx, PlanNode* input) { - return qctx->objPool()->add(new SignOutTSService(qctx, input)); + static SignOutService* make(QueryContext* qctx, + PlanNode* input, + meta::cpp2::ExternalServiceType type) { + return qctx->objPool()->add(new SignOutService(qctx, input, type)); + } + + meta::cpp2::ExternalServiceType type() const { + return type_; } private: - SignOutTSService(QueryContext* qctx, PlanNode* input) - : SingleDependencyNode(qctx, Kind::kSignOutTSService, input) {} + SignOutService(QueryContext* qctx, PlanNode* input, meta::cpp2::ExternalServiceType type) + : SingleDependencyNode(qctx, Kind::kSignOutService, input), type_(type) {} + + private: + meta::cpp2::ExternalServiceType type_; }; class ShowSessions final : public SingleInputNode { diff --git a/src/graph/planner/plan/PlanNode.cpp b/src/graph/planner/plan/PlanNode.cpp index 75964f3565f..abe09aaa9b1 100644 --- a/src/graph/planner/plan/PlanNode.cpp +++ b/src/graph/planner/plan/PlanNode.cpp @@ -265,15 +265,15 @@ const char* PlanNode::toString(PlanNode::Kind kind) { return "ShowListener"; case Kind::kShowStats: return "ShowStats"; - // text search - case Kind::kShowTSClients: - return "ShowTSClients"; + // service search + case Kind::kShowServiceClients: + return "ShowServiceClients"; case Kind::kShowFTIndexes: return "ShowFTIndexes"; - case Kind::kSignInTSService: - return "SignInTSService"; - case Kind::kSignOutTSService: - return "SignOutTSService"; + case Kind::kSignInService: + return "SignInService"; + case Kind::kSignOutService: + return "SignOutService"; case Kind::kDownload: return "Download"; case Kind::kIngest: diff --git a/src/graph/planner/plan/PlanNode.h b/src/graph/planner/plan/PlanNode.h index fd42c3439f9..0d54422c0ca 100644 --- a/src/graph/planner/plan/PlanNode.h +++ b/src/graph/planner/plan/PlanNode.h @@ -168,11 +168,11 @@ class PlanNode { kRemoveListener, kShowListener, - // text service related - kShowTSClients, + // service related + kShowServiceClients, kShowFTIndexes, - kSignInTSService, - kSignOutTSService, + kSignInService, + kSignOutService, kDownload, kIngest, kShowSessions, diff --git a/src/graph/service/CMakeLists.txt b/src/graph/service/CMakeLists.txt index c63f1535069..2ff0c6406dc 100644 --- a/src/graph/service/CMakeLists.txt +++ b/src/graph/service/CMakeLists.txt @@ -10,6 +10,7 @@ nebula_add_library( nebula_add_library( service_obj OBJECT GraphService.cpp + GraphServer.cpp ) nebula_add_library( diff --git a/src/graph/service/GraphServer.cpp b/src/graph/service/GraphServer.cpp new file mode 100644 index 00000000000..25e6ee6cc5e --- /dev/null +++ b/src/graph/service/GraphServer.cpp @@ -0,0 +1,105 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ +#include "GraphServer.h" + +#include +#include + +#include "graph/service/GraphFlags.h" +#include "graph/service/GraphService.h" +namespace nebula { +namespace graph { + +GraphServer::GraphServer(HostAddr localHost) : localHost_(std::move(localHost)) {} + +GraphServer::~GraphServer() { + stop(); +} + +bool GraphServer::start() { + auto threadFactory = std::make_shared("graph-netio"); + auto ioThreadPool = std::make_shared(FLAGS_num_netio_threads, + std::move(threadFactory)); + int numThreads = FLAGS_num_worker_threads > 0 ? FLAGS_num_worker_threads + : thriftServer_->getNumIOWorkerThreads(); + std::shared_ptr threadManager( + PriorityThreadManager::newPriorityThreadManager(numThreads)); + threadManager->setNamePrefix("executor"); + threadManager->start(); + + thriftServer_ = std::make_unique(); + thriftServer_->setIOThreadPool(ioThreadPool); + + auto interface = std::make_shared(); + auto status = interface->init(ioThreadPool, localHost_); + if (!status.ok()) { + LOG(ERROR) << status; + return false; + } + + graphThread_ = std::make_unique([&] { + thriftServer_->setPort(localHost_.port); + thriftServer_->setInterface(std::move(interface)); + thriftServer_->setReusePort(FLAGS_reuse_port); + thriftServer_->setIdleTimeout(std::chrono::seconds(FLAGS_client_idle_timeout_secs)); + thriftServer_->setNumAcceptThreads(FLAGS_num_accept_threads); + thriftServer_->setListenBacklog(FLAGS_listen_backlog); + if (FLAGS_enable_ssl || FLAGS_enable_graph_ssl) { + thriftServer_->setSSLConfig(nebula::sslContextConfig()); + } + thriftServer_->setThreadManager(threadManager); + + serverStatus_.store(STATUS_RUNNING); + FLOG_INFO("Starting nebula-graphd on %s:%d\n", localHost_.host.c_str(), localHost_.port); + try { + thriftServer_->serve(); // Blocking wait until shut down via thriftServer_->stop() + } catch (const std::exception &e) { + FLOG_ERROR("Exception thrown while starting the graph RPC server: %s", e.what()); + } + serverStatus_.store(STATUS_STOPPED); + FLOG_INFO("nebula-graphd on %s:%d has been stopped", localHost_.host.c_str(), localHost_.port); + }); + + while (serverStatus_ == STATUS_UNINITIALIZED) { + std::this_thread::sleep_for(std::chrono::microseconds(100)); + } + return true; +} + +void GraphServer::waitUntilStop() { + { + std::unique_lock lkStop(muStop_); + cvStop_.wait(lkStop, [&] { return serverStatus_ != STATUS_RUNNING; }); + } + + thriftServer_->stop(); + + graphThread_->join(); +} + +void GraphServer::notifyStop() { + std::unique_lock lkStop(muStop_); + if (serverStatus_ == STATUS_RUNNING) { + serverStatus_ = STATUS_STOPPED; + cvStop_.notify_one(); + } +} + +void GraphServer::stop() { + if (serverStatus_.load() == ServiceStatus::STATUS_STOPPED) { + LOG(INFO) << "The graph server has been stopped"; + return; + } + + ServiceStatus serverExpected = ServiceStatus::STATUS_RUNNING; + serverStatus_.compare_exchange_strong(serverExpected, STATUS_STOPPED); + + if (thriftServer_) { + thriftServer_->stop(); + } +} + +} // namespace graph +} // namespace nebula diff --git a/src/graph/service/GraphServer.h b/src/graph/service/GraphServer.h new file mode 100644 index 00000000000..671c0720605 --- /dev/null +++ b/src/graph/service/GraphServer.h @@ -0,0 +1,46 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ +#include + +#include +#include +#include + +#include "common/base/Base.h" +#include "common/base/SignalHandler.h" +#include "common/network/NetworkUtils.h" +namespace nebula { +namespace graph { +class GraphServer { + public: + explicit GraphServer(HostAddr localHost); + + ~GraphServer(); + + // Return false if failed. + bool start(); + + void stop(); + + // used for signal handler to set an internal stop flag + void notifyStop(); + + void waitUntilStop(); + + private: + HostAddr localHost_; + + std::shared_ptr ioThreadPool_; + std::shared_ptr workers_; + std::unique_ptr thriftServer_; + std::unique_ptr graphThread_; + + enum ServiceStatus : uint8_t { STATUS_UNINITIALIZED = 0, STATUS_RUNNING = 1, STATUS_STOPPED = 2 }; + std::atomic serverStatus_{STATUS_UNINITIALIZED}; + std::mutex muStop_; + std::condition_variable cvStop_; +}; +} // namespace graph +} // namespace nebula diff --git a/src/graph/service/GraphService.cpp b/src/graph/service/GraphService.cpp index 73f903526e6..7229b548ddb 100644 --- a/src/graph/service/GraphService.cpp +++ b/src/graph/service/GraphService.cpp @@ -134,8 +134,6 @@ folly::Future GraphService::future_executeWithParameter( ctx->setRunner(getThreadManager()); ctx->setSessionMgr(sessionManager_.get()); auto future = ctx->future(); - stats::StatsManager::addValue(kNumQueries); - stats::StatsManager::addValue(kNumActiveQueries); // When the sessionId is 0, it means the clients to ping the connection is ok if (sessionId == 0) { ctx->resp().errorCode = ErrorCode::E_SESSION_INVALID; @@ -161,14 +159,22 @@ folly::Future GraphService::future_executeWithParameter( return ctx->finish(); } stats::StatsManager::addValue(kNumQueries); + stats::StatsManager::addValue(kNumActiveQueries); if (FLAGS_enable_space_level_metrics && sessionPtr->space().name != "") { stats::StatsManager::addValue(stats::StatsManager::counterWithLabels( kNumQueries, {{"space", sessionPtr->space().name}})); + stats::StatsManager::addValue(stats::StatsManager::counterWithLabels( + kNumActiveQueries, {{"space", sessionPtr->space().name}})); } + auto& spaceName = sessionPtr->space().name; ctx->setSession(std::move(sessionPtr)); ctx->setParameterMap(parameterMap); queryEngine_->execute(std::move(ctx)); stats::StatsManager::decValue(kNumActiveQueries); + if (FLAGS_enable_space_level_metrics && spaceName != "") { + stats::StatsManager::decValue( + stats::StatsManager::counterWithLabels(kNumActiveQueries, {{"space", spaceName}})); + } }; sessionManager_->findSession(sessionId, getThreadManager()).thenValue(std::move(cb)); return future; diff --git a/src/graph/service/PermissionCheck.cpp b/src/graph/service/PermissionCheck.cpp index 0e6c2e9bbca..e94c571bba4 100644 --- a/src/graph/service/PermissionCheck.cpp +++ b/src/graph/service/PermissionCheck.cpp @@ -70,8 +70,8 @@ Status PermissionCheck::permissionCheck(ClientSession *session, case Sentence::Kind::kGetConfig: case Sentence::Kind::kIngest: case Sentence::Kind::kDownload: - case Sentence::Kind::kSignOutTSService: - case Sentence::Kind::kSignInTSService: { + case Sentence::Kind::kSignOutService: + case Sentence::Kind::kSignInService: { return PermissionManager::canWriteSpace(session); } case Sentence::Kind::kCreateTag: @@ -184,7 +184,7 @@ Status PermissionCheck::permissionCheck(ClientSession *session, case Sentence::Kind::kDescribeUser: case Sentence::Kind::kShowUsers: case Sentence::Kind::kShowSnapshots: - case Sentence::Kind::kShowTSClients: + case Sentence::Kind::kShowServiceClients: case Sentence::Kind::kShowSessions: { /** * Only GOD role can be show. @@ -192,7 +192,7 @@ Status PermissionCheck::permissionCheck(ClientSession *session, if (session->isGod()) { return Status::OK(); } else { - return Status::PermissionError("No permission to show users/snapshots/textClients"); + return Status::PermissionError("No permission to show users/snapshots/serviceClients"); } } case Sentence::Kind::kChangePassword: { diff --git a/src/graph/service/QueryInstance.cpp b/src/graph/service/QueryInstance.cpp index 09d6769f90d..aa56c9e58d5 100644 --- a/src/graph/service/QueryInstance.cpp +++ b/src/graph/service/QueryInstance.cpp @@ -64,6 +64,7 @@ void QueryInstance::execute() { Status QueryInstance::validateAndOptimize() { auto *rctx = qctx()->rctx(); + auto &spaceName = rctx->session()->space().name; VLOG(1) << "Parsing query: " << rctx->query(); auto result = GQLParser(qctx()).parse(rctx->query()); NG_RETURN_IF_ERROR(result); @@ -71,13 +72,25 @@ Status QueryInstance::validateAndOptimize() { if (sentence_->kind() == Sentence::Kind::kSequential) { size_t num = static_cast(sentence_.get())->numSentences(); stats::StatsManager::addValue(kNumSentences, num); + if (FLAGS_enable_space_level_metrics && spaceName != "") { + stats::StatsManager::addValue( + stats::StatsManager::counterWithLabels(kNumSentences, {{"space", spaceName}}), num); + } } else { stats::StatsManager::addValue(kNumSentences); + if (FLAGS_enable_space_level_metrics && spaceName != "") { + stats::StatsManager::addValue( + stats::StatsManager::counterWithLabels(kNumSentences, {{"space", spaceName}})); + } } NG_RETURN_IF_ERROR(Validator::validate(sentence_.get(), qctx())); NG_RETURN_IF_ERROR(findBestPlan()); stats::StatsManager::addValue(kOptimizerLatencyUs, *(qctx_->plan()->optimizeTimeInUs())); + if (FLAGS_enable_space_level_metrics && spaceName != "") { + stats::StatsManager::addValue( + stats::StatsManager::histoWithLabels(kOptimizerLatencyUs, {{"space", spaceName}})); + } return Status::OK(); } @@ -117,6 +130,7 @@ void QueryInstance::onFinish() { void QueryInstance::onError(Status status) { LOG(ERROR) << status; auto *rctx = qctx()->rctx(); + auto &spaceName = rctx->session()->space().name; switch (status.code()) { case Status::Code::kOk: rctx->resp().errorCode = ErrorCode::SUCCEEDED; @@ -135,6 +149,10 @@ void QueryInstance::onError(Status status) { break; case Status::Code::kLeaderChanged: stats::StatsManager::addValue(kNumQueryErrorsLeaderChanges); + if (FLAGS_enable_space_level_metrics && spaceName != "") { + stats::StatsManager::addValue(stats::StatsManager::counterWithLabels( + kNumQueryErrorsLeaderChanges, {{"space", spaceName}})); + } [[fallthrough]]; case Status::Code::kBalanced: case Status::Code::kEdgeNotFound: @@ -157,7 +175,6 @@ void QueryInstance::onError(Status status) { rctx->resp().errorCode = ErrorCode::E_EXECUTION_ERROR; break; } - auto &spaceName = rctx->session()->space().name; rctx->resp().spaceName = std::make_unique(spaceName); rctx->resp().errorMsg = std::make_unique(status.toString()); auto latency = rctx->duration().elapsedInUSec(); diff --git a/src/graph/session/ClientSession.cpp b/src/graph/session/ClientSession.cpp index d812ee586a8..1e39ff0c49c 100644 --- a/src/graph/session/ClientSession.cpp +++ b/src/graph/session/ClientSession.cpp @@ -78,6 +78,10 @@ void ClientSession::markQueryKilled(nebula::ExecutionPlanID epId) { } context->second->markKilled(); stats::StatsManager::addValue(kNumKilledQueries); + if (FLAGS_enable_space_level_metrics && space_.name != "") { + stats::StatsManager::addValue( + stats::StatsManager::counterWithLabels(kNumKilledQueries, {{"space", space_.name}})); + } VLOG(1) << "Mark query killed in local cache, epId: " << epId; auto query = session_.queries_ref()->find(epId); @@ -95,6 +99,11 @@ void ClientSession::markAllQueryKilled() { session_.queries_ref()->clear(); } stats::StatsManager::addValue(kNumKilledQueries, contexts_.size()); + if (FLAGS_enable_space_level_metrics && space_.name != "") { + stats::StatsManager::addValue( + stats::StatsManager::counterWithLabels(kNumKilledQueries, {{"space", space_.name}}), + contexts_.size()); + } } } // namespace graph } // namespace nebula diff --git a/src/graph/stats/GraphStats.cpp b/src/graph/stats/GraphStats.cpp index adab4a84352..f03cd46885b 100644 --- a/src/graph/stats/GraphStats.cpp +++ b/src/graph/stats/GraphStats.cpp @@ -26,13 +26,13 @@ stats::CounterId kNumSentences; stats::CounterId kQueryLatencyUs; stats::CounterId kSlowQueryLatencyUs; stats::CounterId kNumKilledQueries; +stats::CounterId kNumQueriesHitMemoryWatermark; stats::CounterId kOptimizerLatencyUs; stats::CounterId kNumAggregateExecutors; stats::CounterId kNumSortExecutors; stats::CounterId kNumIndexScanExecutors; -stats::CounterId kNumOomExecutors; stats::CounterId kNumOpenedSessions; stats::CounterId kNumAuthFailedSessions; @@ -54,6 +54,8 @@ void initGraphStats() { kSlowQueryLatencyUs = stats::StatsManager::registerHisto( "slow_query_latency_us", 1000, 0, 2000, "avg, p75, p95, p99, p999"); kNumKilledQueries = stats::StatsManager::registerStats("num_killed_queries", "rate, sum"); + kNumQueriesHitMemoryWatermark = + stats::StatsManager::registerStats("num_queries_hit_memory_watermark", "rate, sum"); kOptimizerLatencyUs = stats::StatsManager::registerHisto( "optimizer_latency_us", 1000, 0, 2000, "avg, p75, p95, p99, p999"); @@ -63,7 +65,6 @@ void initGraphStats() { kNumSortExecutors = stats::StatsManager::registerStats("num_sort_executors", "rate, sum"); kNumIndexScanExecutors = stats::StatsManager::registerStats("num_indexscan_executors", "rate, sum"); - kNumOomExecutors = stats::StatsManager::registerStats("num_oom_executors", "rate, sum"); kNumOpenedSessions = stats::StatsManager::registerStats("num_opened_sessions", "rate, sum"); kNumAuthFailedSessions = diff --git a/src/graph/stats/GraphStats.h b/src/graph/stats/GraphStats.h index 70bb6a1efd1..2d89e60e4c5 100644 --- a/src/graph/stats/GraphStats.h +++ b/src/graph/stats/GraphStats.h @@ -23,6 +23,7 @@ extern stats::CounterId kNumSentences; extern stats::CounterId kQueryLatencyUs; extern stats::CounterId kSlowQueryLatencyUs; extern stats::CounterId kNumKilledQueries; +extern stats::CounterId kNumQueriesHitMemoryWatermark; extern stats::CounterId kOptimizerLatencyUs; @@ -30,7 +31,6 @@ extern stats::CounterId kOptimizerLatencyUs; extern stats::CounterId kNumAggregateExecutors; extern stats::CounterId kNumSortExecutors; extern stats::CounterId kNumIndexScanExecutors; -extern stats::CounterId kNumOomExecutors; // Server client traffic // extern stats::CounterId kReceivedBytes; diff --git a/src/graph/util/FTIndexUtils.cpp b/src/graph/util/FTIndexUtils.cpp index 82d97cee8f4..3960661c948 100644 --- a/src/graph/util/FTIndexUtils.cpp +++ b/src/graph/util/FTIndexUtils.cpp @@ -28,7 +28,7 @@ bool FTIndexUtils::needTextSearch(const Expression* expr) { StatusOr> FTIndexUtils::getTSClients( meta::MetaClient* client) { - auto tcs = client->getFTClientsFromCache(); + auto tcs = client->getServiceClientsFromCache(meta::cpp2::ExternalServiceType::ELASTICSEARCH); if (!tcs.ok()) { return tcs.status(); } diff --git a/src/graph/validator/AdminValidator.cpp b/src/graph/validator/AdminValidator.cpp index 5afcdb2a925..7e5358de7be 100644 --- a/src/graph/validator/AdminValidator.cpp +++ b/src/graph/validator/AdminValidator.cpp @@ -531,39 +531,44 @@ Status ShowStatusValidator::toPlan() { return Status::OK(); } -Status ShowTSClientsValidator::validateImpl() { +Status ShowServiceClientsValidator::validateImpl() { return Status::OK(); } -Status ShowTSClientsValidator::toPlan() { - auto *doNode = ShowTSClients::make(qctx_, nullptr); +Status ShowServiceClientsValidator::toPlan() { + auto sentence = static_cast(sentence_); + auto type = sentence->getType(); + auto *doNode = ShowServiceClients::make(qctx_, nullptr, type); root_ = doNode; tail_ = root_; return Status::OK(); } -Status SignInTSServiceValidator::validateImpl() { +Status SignInServiceValidator::validateImpl() { return Status::OK(); } -Status SignInTSServiceValidator::toPlan() { - auto sentence = static_cast(sentence_); - std::vector clients; +Status SignInServiceValidator::toPlan() { + auto sentence = static_cast(sentence_); + std::vector clients; if (sentence->clients() != nullptr) { clients = sentence->clients()->clients(); } - auto *node = SignInTSService::make(qctx_, nullptr, std::move(clients)); + auto type = sentence->getType(); + auto *node = SignInService::make(qctx_, nullptr, std::move(clients), type); root_ = node; tail_ = root_; return Status::OK(); } -Status SignOutTSServiceValidator::validateImpl() { +Status SignOutServiceValidator::validateImpl() { return Status::OK(); } -Status SignOutTSServiceValidator::toPlan() { - auto *node = SignOutTSService::make(qctx_, nullptr); +Status SignOutServiceValidator::toPlan() { + auto sentence = static_cast(sentence_); + auto type = sentence->getType(); + auto *node = SignOutService::make(qctx_, nullptr, type); root_ = node; tail_ = root_; return Status::OK(); diff --git a/src/graph/validator/AdminValidator.h b/src/graph/validator/AdminValidator.h index 93d5d17ac09..03f8c0d1c42 100644 --- a/src/graph/validator/AdminValidator.h +++ b/src/graph/validator/AdminValidator.h @@ -25,7 +25,8 @@ class CreateSpaceValidator final : public Validator { Status toPlan() override; - bool checkTSIndex(const std::vector& clients, const std::string& index); + bool checkTSIndex(const std::vector& clients, + const std::string& index); private: meta::cpp2::SpaceDesc spaceDesc_; @@ -316,9 +317,10 @@ class ShowStatusValidator final : public Validator { Status toPlan() override; }; -class ShowTSClientsValidator final : public Validator { +class ShowServiceClientsValidator final : public Validator { public: - ShowTSClientsValidator(Sentence* sentence, QueryContext* context) : Validator(sentence, context) { + ShowServiceClientsValidator(Sentence* sentence, QueryContext* context) + : Validator(sentence, context) { setNoSpaceRequired(); } @@ -328,10 +330,9 @@ class ShowTSClientsValidator final : public Validator { Status toPlan() override; }; -class SignInTSServiceValidator final : public Validator { +class SignInServiceValidator final : public Validator { public: - SignInTSServiceValidator(Sentence* sentence, QueryContext* context) - : Validator(sentence, context) { + SignInServiceValidator(Sentence* sentence, QueryContext* context) : Validator(sentence, context) { setNoSpaceRequired(); } @@ -341,9 +342,9 @@ class SignInTSServiceValidator final : public Validator { Status toPlan() override; }; -class SignOutTSServiceValidator final : public Validator { +class SignOutServiceValidator final : public Validator { public: - SignOutTSServiceValidator(Sentence* sentence, QueryContext* context) + SignOutServiceValidator(Sentence* sentence, QueryContext* context) : Validator(sentence, context) { setNoSpaceRequired(); } diff --git a/src/graph/validator/Validator.cpp b/src/graph/validator/Validator.cpp index f38965f4b73..7c4e9da9495 100644 --- a/src/graph/validator/Validator.cpp +++ b/src/graph/validator/Validator.cpp @@ -228,14 +228,14 @@ std::unique_ptr Validator::makeValidator(Sentence* sentence, QueryCon return std::make_unique(sentence, context); case Sentence::Kind::kShowStats: return std::make_unique(sentence, context); - case Sentence::Kind::kShowTSClients: - return std::make_unique(sentence, context); + case Sentence::Kind::kShowServiceClients: + return std::make_unique(sentence, context); case Sentence::Kind::kShowFTIndexes: return std::make_unique(sentence, context); - case Sentence::Kind::kSignInTSService: - return std::make_unique(sentence, context); - case Sentence::Kind::kSignOutTSService: - return std::make_unique(sentence, context); + case Sentence::Kind::kSignInService: + return std::make_unique(sentence, context); + case Sentence::Kind::kSignOutService: + return std::make_unique(sentence, context); case Sentence::Kind::kDownload: return std::make_unique(sentence, context); case Sentence::Kind::kIngest: diff --git a/src/graph/validator/test/MockSchemaManager.cpp b/src/graph/validator/test/MockSchemaManager.cpp index d6cb0e62045..6236d73212e 100644 --- a/src/graph/validator/test/MockSchemaManager.cpp +++ b/src/graph/validator/test/MockSchemaManager.cpp @@ -167,7 +167,8 @@ StatusOr> MockSchemaManager::getAllEdge(GraphSpaceID) { return edgeNames; } -StatusOr> MockSchemaManager::getFTClients() { +StatusOr> MockSchemaManager::getServiceClients( + meta::cpp2::ExternalServiceType) { return Status::Error("Not implemented"); } diff --git a/src/graph/validator/test/MockSchemaManager.h b/src/graph/validator/test/MockSchemaManager.h index 350153067d8..be2e904d80f 100644 --- a/src/graph/validator/test/MockSchemaManager.h +++ b/src/graph/validator/test/MockSchemaManager.h @@ -108,7 +108,8 @@ class MockSchemaManager final : public nebula::meta::SchemaManager { return allLatestVerEdgeSchemas; } - StatusOr> getFTClients() override; + StatusOr> getServiceClients( + meta::cpp2::ExternalServiceType type) override; StatusOr getPartsNum(GraphSpaceID) override { LOG(FATAL) << "Unimplemented."; diff --git a/src/interface/common.thrift b/src/interface/common.thrift index 14dd6e96531..66cef5e00e9 100644 --- a/src/interface/common.thrift +++ b/src/interface/common.thrift @@ -321,6 +321,7 @@ enum ErrorCode { E_KEY_NOT_FOUND = -17, E_USER_NOT_FOUND = -18, E_STATS_NOT_FOUND = -19, + E_SERVICE_NOT_FOUND = -20, // backup failed E_BACKUP_FAILED = -24, diff --git a/src/interface/meta.thrift b/src/interface/meta.thrift index 10ac522975c..4e1c42997a7 100644 --- a/src/interface/meta.thrift +++ b/src/interface/meta.thrift @@ -1006,32 +1006,35 @@ struct RestoreMetaReq { 2: list hosts, } -enum FTServiceType { +enum ExternalServiceType { ELASTICSEARCH = 0x01, } (cpp.enum_strict) -struct FTClient { +struct ServiceClient { 1: required common.HostAddr host, 2: optional binary user, 3: optional binary pwd, 4: optional binary conn_type, } -struct SignInFTServiceReq { - 1: FTServiceType type, - 2: list clients, +struct SignInServiceReq { + 1: ExternalServiceType type, + 2: list clients, } -struct SignOutFTServiceReq { +struct SignOutServiceReq { + 1: ExternalServiceType type, } -struct ListFTClientsReq { +struct ListServiceClientsReq { + 1: ExternalServiceType type, } -struct ListFTClientsResp { +struct ListServiceClientsResp { 1: common.ErrorCode code, 2: common.HostAddr leader, - 3: list clients, + 3: map> + (cpp.template = "std::unordered_map") clients, } struct FTIndex { @@ -1260,9 +1263,9 @@ service MetaService { ListListenerResp listListener(1: ListListenerReq req); GetStatsResp getStats(1: GetStatsReq req); - ExecResp signInFTService(1: SignInFTServiceReq req); - ExecResp signOutFTService(1: SignOutFTServiceReq req); - ListFTClientsResp listFTClients(1: ListFTClientsReq req); + ExecResp signInService(1: SignInServiceReq req); + ExecResp signOutService(1: SignOutServiceReq req); + ListServiceClientsResp listServiceClients(1: ListServiceClientsReq req); ExecResp createFTIndex(1: CreateFTIndexReq req); ExecResp dropFTIndex(1: DropFTIndexReq req); diff --git a/src/interface/storage.thrift b/src/interface/storage.thrift index 2fee5c79ab3..eca97a22203 100644 --- a/src/interface/storage.thrift +++ b/src/interface/storage.thrift @@ -684,6 +684,7 @@ service GraphStorageService { UpdateResponse chainUpdateEdge(1: UpdateEdgeRequest req); ExecResponse chainAddEdges(1: AddEdgesRequest req); + ExecResponse chainDeleteEdges(1: DeleteEdgesRequest req); KVGetResponse get(1: KVGetRequest req); ExecResponse put(1: KVPutRequest req); @@ -854,17 +855,6 @@ service StorageAdminService { // ////////////////////////////////////////////////////////// -// transaction request -struct InternalTxnRequest { - 1: i64 txn_id, - 2: map term_of_parts, - 3: optional AddEdgesRequest add_edge_req, - 4: optional UpdateEdgeRequest upd_edge_req, - 5: optional map>( - cpp.template = "std::unordered_map") edge_ver, -} - - struct ChainAddEdgesRequest { 1: common.GraphSpaceID space_id, // partId => edges @@ -875,7 +865,6 @@ struct ChainAddEdgesRequest { 3: list prop_names, // if true, when edge already exists, do nothing 4: bool if_not_exists, - // 5: map term_of_parts, 5: i64 term 6: optional i64 edge_version // 6: optional map>( @@ -891,7 +880,17 @@ struct ChainUpdateEdgeRequest { 5: required list parts, } +struct ChainDeleteEdgesRequest { + 1: common.GraphSpaceID space_id, + // partId => edgeKeys + 2: map> + (cpp.template = "std::unordered_map") parts, + 3: binary txn_id + 4: i64 term, +} + service InternalStorageService { ExecResponse chainAddEdges(1: ChainAddEdgesRequest req); UpdateResponse chainUpdateEdge(1: ChainUpdateEdgeRequest req); + ExecResponse chainDeleteEdges(1: ChainDeleteEdgesRequest req); } diff --git a/src/kvstore/Part.cpp b/src/kvstore/Part.cpp index ec10f7c9ed7..87f64f96bf5 100644 --- a/src/kvstore/Part.cpp +++ b/src/kvstore/Part.cpp @@ -252,7 +252,7 @@ std::tuple Part::commitLogs( // Make the number of values are an even number DCHECK_EQ((kvs.size() + 1) / 2, kvs.size() / 2); for (size_t i = 0; i < kvs.size(); i += 2) { - VLOG(1) << "OP_MULTI_PUT " << folly::hexlify(kvs[i]) + VLOG(2) << "OP_MULTI_PUT " << folly::hexlify(kvs[i]) << ", val = " << folly::hexlify(kvs[i + 1]); auto code = batch->put(kvs[i], kvs[i + 1]); if (code != nebula::cpp2::ErrorCode::SUCCEEDED) { @@ -295,7 +295,7 @@ std::tuple Part::commitLogs( case OP_BATCH_WRITE: { auto data = decodeBatchValue(log); for (auto& op : data) { - VLOG(1) << "OP_BATCH_WRITE: " << folly::hexlify(op.second.first) + VLOG(2) << "OP_BATCH_WRITE: " << folly::hexlify(op.second.first) << ", val=" << folly::hexlify(op.second.second); auto code = nebula::cpp2::ErrorCode::SUCCEEDED; if (op.first == BatchLogType::OP_BATCH_PUT) { diff --git a/src/kvstore/plugins/elasticsearch/ESListener.cpp b/src/kvstore/plugins/elasticsearch/ESListener.cpp index 75b7918eb40..3f40ed8d9bc 100644 --- a/src/kvstore/plugins/elasticsearch/ESListener.cpp +++ b/src/kvstore/plugins/elasticsearch/ESListener.cpp @@ -20,7 +20,7 @@ void ESListener::init() { } vIdLen_ = vRet.value(); - auto cRet = schemaMan_->getFTClients(); + auto cRet = schemaMan_->getServiceClients(meta::cpp2::ExternalServiceType::ELASTICSEARCH); if (!cRet.ok() || cRet.value().empty()) { LOG(FATAL) << "elasticsearch clients error"; } diff --git a/src/kvstore/raftex/RaftPart.cpp b/src/kvstore/raftex/RaftPart.cpp index f8571c775b4..1b845a89f40 100644 --- a/src/kvstore/raftex/RaftPart.cpp +++ b/src/kvstore/raftex/RaftPart.cpp @@ -2026,7 +2026,6 @@ void RaftPart::checkRemoteListeners(const std::set& expected) { } } } - bool RaftPart::leaseValid() { std::lock_guard g(raftLock_); if (hosts_.empty()) { diff --git a/src/meta/CMakeLists.txt b/src/meta/CMakeLists.txt index c3051cb4068..54873ae9c70 100644 --- a/src/meta/CMakeLists.txt +++ b/src/meta/CMakeLists.txt @@ -35,7 +35,7 @@ nebula_add_library( processors/index/DropEdgeIndexProcessor.cpp processors/index/GetEdgeIndexProcessor.cpp processors/index/ListEdgeIndexesProcessor.cpp - processors/index/FTServiceProcessor.cpp + processors/service/ServiceProcessor.cpp processors/index/FTIndexProcessor.cpp processors/kv/GetProcessor.cpp processors/kv/MultiGetProcessor.cpp diff --git a/src/meta/MetaServiceHandler.cpp b/src/meta/MetaServiceHandler.cpp index ba29d149291..997d38ed894 100644 --- a/src/meta/MetaServiceHandler.cpp +++ b/src/meta/MetaServiceHandler.cpp @@ -25,7 +25,6 @@ #include "meta/processors/index/DropEdgeIndexProcessor.h" #include "meta/processors/index/DropTagIndexProcessor.h" #include "meta/processors/index/FTIndexProcessor.h" -#include "meta/processors/index/FTServiceProcessor.h" #include "meta/processors/index/GetEdgeIndexProcessor.h" #include "meta/processors/index/GetTagIndexProcessor.h" #include "meta/processors/index/ListEdgeIndexesProcessor.h" @@ -61,6 +60,7 @@ #include "meta/processors/schema/GetTagProcessor.h" #include "meta/processors/schema/ListEdgesProcessor.h" #include "meta/processors/schema/ListTagsProcessor.h" +#include "meta/processors/service/ServiceProcessor.h" #include "meta/processors/session/SessionManagerProcessor.h" #include "meta/processors/user/AuthenticationProcessor.h" #include "meta/processors/zone/AddHostsIntoZoneProcessor.h" @@ -301,21 +301,21 @@ folly::Future MetaServiceHandler::future_listEdgeInde RETURN_FUTURE(processor); } -folly::Future MetaServiceHandler::future_signInFTService( - const cpp2::SignInFTServiceReq& req) { - auto* processor = SignInFTServiceProcessor::instance(kvstore_); +folly::Future MetaServiceHandler::future_signInService( + const cpp2::SignInServiceReq& req) { + auto* processor = SignInServiceProcessor::instance(kvstore_); RETURN_FUTURE(processor); } -folly::Future MetaServiceHandler::future_signOutFTService( - const cpp2::SignOutFTServiceReq& req) { - auto* processor = SignOutFTServiceProcessor::instance(kvstore_); +folly::Future MetaServiceHandler::future_signOutService( + const cpp2::SignOutServiceReq& req) { + auto* processor = SignOutServiceProcessor::instance(kvstore_); RETURN_FUTURE(processor); } -folly::Future MetaServiceHandler::future_listFTClients( - const cpp2::ListFTClientsReq& req) { - auto* processor = ListFTClientsProcessor::instance(kvstore_); +folly::Future MetaServiceHandler::future_listServiceClients( + const cpp2::ListServiceClientsReq& req) { + auto* processor = ListServiceClientsProcessor::instance(kvstore_); RETURN_FUTURE(processor); } diff --git a/src/meta/MetaServiceHandler.h b/src/meta/MetaServiceHandler.h index 84fbade153b..4991bf731bc 100644 --- a/src/meta/MetaServiceHandler.h +++ b/src/meta/MetaServiceHandler.h @@ -122,14 +122,12 @@ class MetaServiceHandler final : public cpp2::MetaServiceSvIf { folly::Future future_listEdgeIndexStatus( const cpp2::ListIndexStatusReq& req) override; - folly::Future future_signInFTService( - const cpp2::SignInFTServiceReq& req) override; + folly::Future future_signInService(const cpp2::SignInServiceReq& req) override; - folly::Future future_signOutFTService( - const cpp2::SignOutFTServiceReq& req) override; + folly::Future future_signOutService(const cpp2::SignOutServiceReq& req) override; - folly::Future future_listFTClients( - const cpp2::ListFTClientsReq& req) override; + folly::Future future_listServiceClients( + const cpp2::ListServiceClientsReq& req) override; folly::Future future_createFTIndex(const cpp2::CreateFTIndexReq& req) override; diff --git a/src/meta/processors/Common.h b/src/meta/processors/Common.h index ef160578cf6..353fddacc85 100644 --- a/src/meta/processors/Common.h +++ b/src/meta/processors/Common.h @@ -28,7 +28,7 @@ class LockUtils { GENERATE_LOCK(edge); GENERATE_LOCK(tagIndex); GENERATE_LOCK(edgeIndex); - GENERATE_LOCK(fulltextServices); + GENERATE_LOCK(service); GENERATE_LOCK(fulltextIndex); GENERATE_LOCK(user); GENERATE_LOCK(config); diff --git a/src/meta/processors/admin/HBProcessor.cpp b/src/meta/processors/admin/HBProcessor.cpp index 2012e42e251..7e78aba9732 100644 --- a/src/meta/processors/admin/HBProcessor.cpp +++ b/src/meta/processors/admin/HBProcessor.cpp @@ -10,8 +10,6 @@ #include "meta/KVBasedClusterIdMan.h" #include "meta/MetaVersionMan.h" -DEFINE_bool(hosts_whitelist_enabled, true, "Automatically receive the heartbeat report"); - namespace nebula { namespace meta { @@ -36,7 +34,7 @@ void HBProcessor::process(const cpp2::HBReq& req) { << ", role = " << apache::thrift::util::enumNameSafe(role); if (role == cpp2::HostRole::STORAGE) { - if (!FLAGS_hosts_whitelist_enabled && !ActiveHostsMan::machineRegisted(kvstore_, host)) { + if (!ActiveHostsMan::machineRegisted(kvstore_, host)) { LOG(ERROR) << "Machine " << host << " is not registed"; handleErrorCode(nebula::cpp2::ErrorCode::E_MACHINE_NOT_FOUND); onFinished(); diff --git a/src/meta/processors/admin/HBProcessor.h b/src/meta/processors/admin/HBProcessor.h index 4974f781c77..4fde1b03bac 100644 --- a/src/meta/processors/admin/HBProcessor.h +++ b/src/meta/processors/admin/HBProcessor.h @@ -11,8 +11,6 @@ #include "common/stats/StatsManager.h" #include "meta/processors/BaseProcessor.h" -DECLARE_bool(hosts_whitelist_enabled); - namespace nebula { namespace meta { diff --git a/src/meta/processors/index/FTServiceProcessor.cpp b/src/meta/processors/index/FTServiceProcessor.cpp deleted file mode 100644 index af3fbaf720b..00000000000 --- a/src/meta/processors/index/FTServiceProcessor.cpp +++ /dev/null @@ -1,80 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License. - */ - -#include "meta/processors/index/FTServiceProcessor.h" - -namespace nebula { -namespace meta { - -void SignInFTServiceProcessor::process(const cpp2::SignInFTServiceReq& req) { - folly::SharedMutex::WriteHolder wHolder(LockUtils::fulltextServicesLock()); - auto serviceKey = MetaKeyUtils::fulltextServiceKey(); - auto ret = doGet(serviceKey); - if (nebula::ok(ret)) { - LOG(ERROR) << "Fulltext already exists."; - handleErrorCode(nebula::cpp2::ErrorCode::E_EXISTED); - onFinished(); - return; - } else { - auto retCode = nebula::error(ret); - if (retCode != nebula::cpp2::ErrorCode::E_KEY_NOT_FOUND) { - LOG(ERROR) << "Sign in fulltext failed, error: " - << apache::thrift::util::enumNameSafe(retCode); - handleErrorCode(retCode); - onFinished(); - return; - } - } - - std::vector data; - data.emplace_back(std::move(serviceKey), - MetaKeyUtils::fulltextServiceVal(req.get_type(), req.get_clients())); - doSyncPutAndUpdate(std::move(data)); -} - -void SignOutFTServiceProcessor::process(const cpp2::SignOutFTServiceReq&) { - folly::SharedMutex::WriteHolder wHolder(LockUtils::fulltextServicesLock()); - auto serviceKey = MetaKeyUtils::fulltextServiceKey(); - auto ret = doGet(serviceKey); - if (!nebula::ok(ret)) { - auto retCode = nebula::error(ret); - if (retCode == nebula::cpp2::ErrorCode::E_KEY_NOT_FOUND) { - LOG(ERROR) << "Sign out fulltext failed, Fulltext not exists."; - } else { - LOG(ERROR) << "Sign out fulltext failed, error: " - << apache::thrift::util::enumNameSafe(retCode); - } - handleErrorCode(retCode); - onFinished(); - return; - } - - doSyncMultiRemoveAndUpdate({std::move(serviceKey)}); -} - -void ListFTClientsProcessor::process(const cpp2::ListFTClientsReq&) { - folly::SharedMutex::ReadHolder rHolder(LockUtils::fulltextServicesLock()); - const auto& prefix = MetaKeyUtils::fulltextServiceKey(); - auto iterRet = doPrefix(prefix); - if (!nebula::ok(iterRet)) { - auto retCode = nebula::error(iterRet); - LOG(ERROR) << "List fulltext failed, error: " << apache::thrift::util::enumNameSafe(retCode); - handleErrorCode(retCode); - onFinished(); - return; - } - - auto iter = nebula::value(iterRet).get(); - std::vector clients; - if (iter->valid()) { - clients = MetaKeyUtils::parseFTClients(iter->val()); - } - resp_.clients_ref() = std::move(clients); - handleErrorCode(nebula::cpp2::ErrorCode::SUCCEEDED); - onFinished(); -} - -} // namespace meta -} // namespace nebula diff --git a/src/meta/processors/index/FTServiceProcessor.h b/src/meta/processors/index/FTServiceProcessor.h deleted file mode 100644 index b6a2a49ac90..00000000000 --- a/src/meta/processors/index/FTServiceProcessor.h +++ /dev/null @@ -1,54 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License. - */ -#ifndef META_FTISERVICEPROCESSOR_H_ -#define META_FTISERVICEPROCESSOR_H_ - -#include "meta/processors/BaseProcessor.h" - -namespace nebula { -namespace meta { - -class SignInFTServiceProcessor : public BaseProcessor { - public: - static SignInFTServiceProcessor* instance(kvstore::KVStore* kvstore) { - return new SignInFTServiceProcessor(kvstore); - } - - void process(const cpp2::SignInFTServiceReq& req); - - private: - explicit SignInFTServiceProcessor(kvstore::KVStore* kvstore) - : BaseProcessor(kvstore) {} -}; - -class SignOutFTServiceProcessor : public BaseProcessor { - public: - static SignOutFTServiceProcessor* instance(kvstore::KVStore* kvstore) { - return new SignOutFTServiceProcessor(kvstore); - } - - void process(const cpp2::SignOutFTServiceReq& req); - - private: - explicit SignOutFTServiceProcessor(kvstore::KVStore* kvstore) - : BaseProcessor(kvstore) {} -}; - -class ListFTClientsProcessor : public BaseProcessor { - public: - static ListFTClientsProcessor* instance(kvstore::KVStore* kvstore) { - return new ListFTClientsProcessor(kvstore); - } - - void process(const cpp2::ListFTClientsReq& req); - - private: - explicit ListFTClientsProcessor(kvstore::KVStore* kvstore) - : BaseProcessor(kvstore) {} -}; -} // namespace meta -} // namespace nebula - -#endif // META_FTISERVICEPROCESSOR_H_ diff --git a/src/meta/processors/service/ServiceProcessor.cpp b/src/meta/processors/service/ServiceProcessor.cpp new file mode 100644 index 00000000000..bff6ba7aa30 --- /dev/null +++ b/src/meta/processors/service/ServiceProcessor.cpp @@ -0,0 +1,85 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "meta/processors/service/ServiceProcessor.h" + +namespace nebula { +namespace meta { + +void SignInServiceProcessor::process(const cpp2::SignInServiceReq& req) { + folly::SharedMutex::WriteHolder wHolder(LockUtils::serviceLock()); + auto type = req.get_type(); + + auto serviceKey = MetaKeyUtils::serviceKey(type); + auto ret = doGet(serviceKey); + if (nebula::ok(ret)) { + LOG(ERROR) << "Service already exists."; + handleErrorCode(nebula::cpp2::ErrorCode::E_EXISTED); + onFinished(); + return; + } else { + auto retCode = nebula::error(ret); + if (retCode != nebula::cpp2::ErrorCode::E_KEY_NOT_FOUND) { + LOG(ERROR) << "Sign in service failed, error: " + << apache::thrift::util::enumNameSafe(retCode); + handleErrorCode(retCode); + onFinished(); + return; + } + } + + std::vector data; + data.emplace_back(std::move(serviceKey), MetaKeyUtils::serviceVal(req.get_clients())); + doSyncPutAndUpdate(std::move(data)); +} + +void SignOutServiceProcessor::process(const cpp2::SignOutServiceReq& req) { + folly::SharedMutex::WriteHolder wHolder(LockUtils::serviceLock()); + auto type = req.get_type(); + + auto serviceKey = MetaKeyUtils::serviceKey(type); + auto ret = doGet(serviceKey); + if (!nebula::ok(ret)) { + auto retCode = nebula::error(ret); + if (retCode == nebula::cpp2::ErrorCode::E_KEY_NOT_FOUND) { + LOG(ERROR) << "Sign out service failed, service not exists."; + } else { + LOG(ERROR) << "Sign out service failed, error: " + << apache::thrift::util::enumNameSafe(retCode); + } + handleErrorCode(retCode); + onFinished(); + return; + } + + doSyncMultiRemoveAndUpdate({std::move(serviceKey)}); +} + +void ListServiceClientsProcessor::process(const cpp2::ListServiceClientsReq& req) { + folly::SharedMutex::ReadHolder rHolder(LockUtils::serviceLock()); + auto type = req.get_type(); + + std::unordered_map> serviceClients; + const auto& serviceKey = MetaKeyUtils::serviceKey(type); + auto ret = doGet(serviceKey); + if (!nebula::ok(ret) && nebula::error(ret) != nebula::cpp2::ErrorCode::E_KEY_NOT_FOUND) { + auto retCode = nebula::error(ret); + LOG(ERROR) << "List service failed, error: " << apache::thrift::util::enumNameSafe(retCode); + handleErrorCode(retCode); + onFinished(); + return; + } + + if (nebula::ok(ret)) { + serviceClients.emplace(type, MetaKeyUtils::parseServiceClients(nebula::value(ret))); + } + + resp_.clients_ref() = std::move(serviceClients); + handleErrorCode(nebula::cpp2::ErrorCode::SUCCEEDED); + onFinished(); +} + +} // namespace meta +} // namespace nebula diff --git a/src/meta/processors/service/ServiceProcessor.h b/src/meta/processors/service/ServiceProcessor.h new file mode 100644 index 00000000000..a424381579e --- /dev/null +++ b/src/meta/processors/service/ServiceProcessor.h @@ -0,0 +1,56 @@ +/* Copyright (c) 2020 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#ifndef META_SERVICEPROCESSOR_H_ +#define META_SERVICEPROCESSOR_H_ + +#include "meta/processors/BaseProcessor.h" + +namespace nebula { +namespace meta { + +class SignInServiceProcessor : public BaseProcessor { + public: + static SignInServiceProcessor* instance(kvstore::KVStore* kvstore) { + return new SignInServiceProcessor(kvstore); + } + + void process(const cpp2::SignInServiceReq& req); + + private: + explicit SignInServiceProcessor(kvstore::KVStore* kvstore) + : BaseProcessor(kvstore) {} +}; + +class SignOutServiceProcessor : public BaseProcessor { + public: + static SignOutServiceProcessor* instance(kvstore::KVStore* kvstore) { + return new SignOutServiceProcessor(kvstore); + } + + void process(const cpp2::SignOutServiceReq& req); + + private: + explicit SignOutServiceProcessor(kvstore::KVStore* kvstore) + : BaseProcessor(kvstore) {} +}; + +class ListServiceClientsProcessor : public BaseProcessor { + public: + static ListServiceClientsProcessor* instance(kvstore::KVStore* kvstore) { + return new ListServiceClientsProcessor(kvstore); + } + + void process(const cpp2::ListServiceClientsReq& req); + + private: + explicit ListServiceClientsProcessor(kvstore::KVStore* kvstore) + : BaseProcessor(kvstore) {} +}; + +} // namespace meta +} // namespace nebula + +#endif // META_SERVICEPROCESSOR_H_ diff --git a/src/meta/test/HBProcessorTest.cpp b/src/meta/test/HBProcessorTest.cpp index 0773294711a..bbd619d579a 100644 --- a/src/meta/test/HBProcessorTest.cpp +++ b/src/meta/test/HBProcessorTest.cpp @@ -15,7 +15,6 @@ namespace nebula { namespace meta { TEST(HBProcessorTest, HBTest) { - FLAGS_hosts_whitelist_enabled = false; fs::TempDir rootPath("/tmp/HBTest.XXXXXX"); std::unique_ptr kv(MockCluster::initMetaKV(rootPath.path())); diff --git a/src/meta/test/MetaClientTest.cpp b/src/meta/test/MetaClientTest.cpp index 26a22b3abef..dc7315aa395 100644 --- a/src/meta/test/MetaClientTest.cpp +++ b/src/meta/test/MetaClientTest.cpp @@ -1069,8 +1069,8 @@ TEST(MetaClientTest, FTServiceTest) { std::vector hosts = {{"0", 0}, {"1", 1}, {"2", 2}, {"3", 3}}; TestUtils::registerHB(kv, hosts); - std::vector clients; - cpp2::FTClient c1, c2; + std::vector clients; + cpp2::ServiceClient c1, c2; c1.host_ref() = {"0", 0}; c1.user_ref() = "u1"; c1.pwd_ref() = "pwd"; @@ -1079,21 +1079,21 @@ TEST(MetaClientTest, FTServiceTest) { c2.user_ref() = "u2"; clients.emplace_back(c2); { - cpp2::FTServiceType type = cpp2::FTServiceType::ELASTICSEARCH; - auto result = client->signInFTService(type, clients).get(); + cpp2::ExternalServiceType type = cpp2::ExternalServiceType::ELASTICSEARCH; + auto result = client->signInService(type, clients).get(); ASSERT_TRUE(result.ok()); } { - auto result = client->listFTClients().get(); + auto result = client->listServiceClients(cpp2::ExternalServiceType::ELASTICSEARCH).get(); ASSERT_TRUE(result.ok()); - ASSERT_EQ(clients, result.value()); + ASSERT_EQ(clients, result.value()[cpp2::ExternalServiceType::ELASTICSEARCH]); } { - auto result = client->signOutFTService().get(); + auto result = client->signOutService(cpp2::ExternalServiceType::ELASTICSEARCH).get(); ASSERT_TRUE(result.ok()); } { - auto result = client->listFTClients().get(); + auto result = client->listServiceClients(cpp2::ExternalServiceType::ELASTICSEARCH).get(); ASSERT_TRUE(result.ok()); ASSERT_TRUE(result.value().empty()); } diff --git a/src/meta/test/ProcessorTest.cpp b/src/meta/test/ProcessorTest.cpp index 6fd6090c6f2..b942e98afdd 100644 --- a/src/meta/test/ProcessorTest.cpp +++ b/src/meta/test/ProcessorTest.cpp @@ -2681,7 +2681,6 @@ TEST(ProcessorTest, TagIdAndEdgeTypeInSpaceRangeTest) { } TEST(ProcessorTest, HostsTest) { - FLAGS_hosts_whitelist_enabled = false; fs::TempDir rootPath("/tmp/HostsTest.XXXXXX"); auto kv = MockCluster::initMetaKV(rootPath.path()); { @@ -2883,7 +2882,6 @@ TEST(ProcessorTest, HostsTest) { } TEST(ProcessorTest, AddHostsIntoNewZoneTest) { - FLAGS_hosts_whitelist_enabled = false; fs::TempDir rootPath("/tmp/AddHostsIntoZoneTest.XXXXXX"); auto kv = MockCluster::initMetaKV(rootPath.path()); { @@ -2985,7 +2983,6 @@ TEST(ProcessorTest, AddHostsIntoNewZoneTest) { } TEST(ProcessorTest, AddHostsIntoZoneTest) { - FLAGS_hosts_whitelist_enabled = false; fs::TempDir rootPath("/tmp/AddHostsIntoZoneTest.XXXXXX"); auto kv = MockCluster::initMetaKV(rootPath.path()); { @@ -3156,7 +3153,6 @@ TEST(ProcessorTest, AddHostsIntoZoneTest) { } TEST(ProcessorTest, DropHostsTest) { - FLAGS_hosts_whitelist_enabled = false; fs::TempDir rootPath("/tmp/DropHostsTest.XXXXXX"); auto kv = MockCluster::initMetaKV(rootPath.path()); { diff --git a/src/mock/AdHocSchemaManager.cpp b/src/mock/AdHocSchemaManager.cpp index bca76e50021..7256fcf4d48 100644 --- a/src/mock/AdHocSchemaManager.cpp +++ b/src/mock/AdHocSchemaManager.cpp @@ -299,12 +299,14 @@ StatusOr AdHocSchemaManager::getAllLatestVerEdgeSchema(GraphSpaceID return edgesSchema; } -StatusOr> AdHocSchemaManager::getFTClients() { - return ftClients_; +StatusOr> AdHocSchemaManager::getServiceClients( + nebula::meta::cpp2::ExternalServiceType) { + return serviceClients_; } -void AdHocSchemaManager::addFTClient(const nebula::meta::cpp2::FTClient& client) { - ftClients_.emplace_back(client); +void AdHocSchemaManager::addServiceClient(const nebula::meta::cpp2::ServiceClient& client) { + serviceClients_.emplace_back(client); } + } // namespace mock } // namespace nebula diff --git a/src/mock/AdHocSchemaManager.h b/src/mock/AdHocSchemaManager.h index 35d9a2f7b03..07c9989e606 100644 --- a/src/mock/AdHocSchemaManager.h +++ b/src/mock/AdHocSchemaManager.h @@ -98,9 +98,10 @@ class AdHocSchemaManager final : public nebula::meta::SchemaManager { EdgeType edge, SchemaVer ver); - StatusOr> getFTClients() override; + StatusOr> getServiceClients( + nebula::meta::cpp2::ExternalServiceType type) override; - void addFTClient(const nebula::meta::cpp2::FTClient& client); + void addServiceClient(const nebula::meta::cpp2::ServiceClient& client); StatusOr> getFTIndex(GraphSpaceID, int32_t) override { @@ -143,7 +144,7 @@ class AdHocSchemaManager final : public nebula::meta::SchemaManager { std::shared_ptr>> edgeSchemasInMap_; - std::vector ftClients_; + std::vector serviceClients_; int32_t partNum_; }; diff --git a/src/mock/MockCluster.cpp b/src/mock/MockCluster.cpp index 76b27e5e9af..e2c2732e9ba 100644 --- a/src/mock/MockCluster.cpp +++ b/src/mock/MockCluster.cpp @@ -134,7 +134,7 @@ void MockCluster::initStorageKV(const char* dataPath, SchemaVer schemaVerCount, bool hasProp, bool hasListener, - const std::vector& clients, + const std::vector& clients, bool needCffBuilder) { FLAGS_heartbeat_interval_secs = 1; const std::vector parts{1, 2, 3, 4, 5, 6}; @@ -169,7 +169,8 @@ void MockCluster::initStorageKV(const char* dataPath, if (clients.empty()) { LOG(FATAL) << "full text client list is empty"; } - ret = metaClient_->signInFTService(meta::cpp2::FTServiceType::ELASTICSEARCH, clients).get(); + ret = + metaClient_->signInService(meta::cpp2::ExternalServiceType::ELASTICSEARCH, clients).get(); if (!ret.ok()) { LOG(FATAL) << "full text client sign in failed"; } diff --git a/src/mock/MockCluster.h b/src/mock/MockCluster.h index 749594b3646..1e898df72a3 100644 --- a/src/mock/MockCluster.h +++ b/src/mock/MockCluster.h @@ -79,7 +79,7 @@ class MockCluster { SchemaVer schemaVerCount = 1, bool hasProp = true, bool hasListener = false, - const std::vector& clients = {}, + const std::vector& clients = {}, bool needCffBuilder = false); std::shared_ptr getWorkers(); diff --git a/src/mock/MockData.cpp b/src/mock/MockData.cpp index 6553dd07da5..78811aead2f 100644 --- a/src/mock/MockData.cpp +++ b/src/mock/MockData.cpp @@ -744,7 +744,7 @@ std::vector MockData::mockPlayerVerticeIds() { return ret; } -std::vector MockData::mockEdges(bool upper) { +std::vector MockData::mockEdges(bool upper, bool hasInEdges) { std::vector ret; // Use serve data, positive edgeType is 101, reverse edgeType is -101 for (auto& serve : serves_) { @@ -788,7 +788,9 @@ std::vector MockData::mockEdges(bool upper) { positiveEdge.props_ = std::move(props); auto reverseData = getReverseEdge(positiveEdge); ret.emplace_back(std::move(positiveEdge)); - ret.emplace_back(std::move(reverseData)); + if (hasInEdges) { + ret.emplace_back(std::move(reverseData)); + } } return ret; } @@ -947,11 +949,13 @@ nebula::storage::cpp2::DeleteVerticesRequest MockData::mockDeleteVerticesReq(int return req; } -nebula::storage::cpp2::AddEdgesRequest MockData::mockAddEdgesReq(bool upper, int32_t parts) { +nebula::storage::cpp2::AddEdgesRequest MockData::mockAddEdgesReq(bool upper, + int32_t parts, + bool hasInEdges) { nebula::storage::cpp2::AddEdgesRequest req; req.space_id_ref() = 1; req.if_not_exists_ref() = true; - auto retRecs = mockEdges(upper); + auto retRecs = mockEdges(upper, hasInEdges); for (auto& rec : retRecs) { nebula::storage::cpp2::NewEdge newEdge; nebula::storage::cpp2::EdgeKey edgeKey; diff --git a/src/mock/MockData.h b/src/mock/MockData.h index 80899aad8e5..d96f87ab864 100644 --- a/src/mock/MockData.h +++ b/src/mock/MockData.h @@ -115,7 +115,8 @@ class MockData { static std::vector> mockPlayerIndexKeys(bool upper = false); // generate serve edge - static std::vector mockEdges(bool upper = false); + // param: includeInEdges, if the return set has both out and in edges + static std::vector mockEdges(bool upper = false, bool includeInEdges = true); static std::vector> mockServeIndexKeys(); @@ -169,7 +170,8 @@ class MockData { int32_t parts = 6); static nebula::storage::cpp2::AddEdgesRequest mockAddEdgesReq(bool upper = false, - int32_t parts = 6); + int32_t parts = 6, + bool hasInEdges = true); static nebula::storage::cpp2::DeleteVerticesRequest mockDeleteVerticesReq(int32_t parts = 6); diff --git a/src/parser/AdminSentences.cpp b/src/parser/AdminSentences.cpp index e3fe023ff14..69d33bd8c3b 100644 --- a/src/parser/AdminSentences.cpp +++ b/src/parser/AdminSentences.cpp @@ -346,14 +346,26 @@ std::string ShowStatsSentence::toString() const { return folly::stringPrintf("SHOW STATS"); } -std::string ShowTSClientsSentence::toString() const { - return "SHOW TEXT SEARCH CLIENTS"; +std::string ShowServiceClientsSentence::toString() const { + switch (type_) { + case meta::cpp2::ExternalServiceType::ELASTICSEARCH: + return "SHOW TEXT SEARCH CLIENTS"; + default: + LOG(FATAL) << "Unknown service type " << static_cast(type_); + } } -std::string SignInTextServiceSentence::toString() const { +std::string SignInServiceSentence::toString() const { std::string buf; buf.reserve(256); - buf += "SIGN IN TEXT SERVICE "; + switch (type_) { + case meta::cpp2::ExternalServiceType::ELASTICSEARCH: + buf += "SIGN IN TEXT SERVICE "; + break; + default: + LOG(FATAL) << "Unknown service type " << static_cast(type_); + } + for (auto &client : clients_->clients()) { buf += "("; buf += client.get_host().host; @@ -385,8 +397,13 @@ std::string SignInTextServiceSentence::toString() const { return buf; } -std::string SignOutTextServiceSentence::toString() const { - return "SIGN OUT TEXT SERVICE"; +std::string SignOutServiceSentence::toString() const { + switch (type_) { + case meta::cpp2::ExternalServiceType::ELASTICSEARCH: + return "SIGN OUT TEXT SERVICE"; + default: + LOG(FATAL) << "Unknown service type " << static_cast(type_); + } } std::string ShowSessionsSentence::toString() const { diff --git a/src/parser/AdminSentences.h b/src/parser/AdminSentences.h index 812fe6f4d1e..dc6d1653449 100644 --- a/src/parser/AdminSentences.h +++ b/src/parser/AdminSentences.h @@ -680,16 +680,16 @@ class ShowStatsSentence final : public Sentence { std::string toString() const override; }; -class TSClientList final { +class ServiceClientList final { public: - void addClient(nebula::meta::cpp2::FTClient* client) { + void addClient(nebula::meta::cpp2::ServiceClient* client) { clients_.emplace_back(client); } std::string toString() const; - std::vector clients() const { - std::vector result; + std::vector clients() const { + std::vector result; result.reserve(clients_.size()); for (auto& client : clients_) { result.emplace_back(*client); @@ -698,41 +698,63 @@ class TSClientList final { } private: - std::vector> clients_; + std::vector> clients_; }; -class ShowTSClientsSentence final : public Sentence { +class ShowServiceClientsSentence final : public Sentence { public: - ShowTSClientsSentence() { - kind_ = Kind::kShowTSClients; + explicit ShowServiceClientsSentence(const meta::cpp2::ExternalServiceType& type) : type_(type) { + kind_ = Kind::kShowServiceClients; } + std::string toString() const override; + + meta::cpp2::ExternalServiceType getType() { + return type_; + } + + private: + meta::cpp2::ExternalServiceType type_; }; -class SignInTextServiceSentence final : public Sentence { +class SignInServiceSentence final : public Sentence { public: - explicit SignInTextServiceSentence(TSClientList* clients) { - kind_ = Kind::kSignInTSService; + explicit SignInServiceSentence(const meta::cpp2::ExternalServiceType& type, + ServiceClientList* clients) + : type_(type) { + kind_ = Kind::kSignInService; clients_.reset(clients); } std::string toString() const override; - TSClientList* clients() const { + ServiceClientList* clients() const { return clients_.get(); } + meta::cpp2::ExternalServiceType getType() { + return type_; + } + private: - std::unique_ptr clients_; + std::unique_ptr clients_; + meta::cpp2::ExternalServiceType type_; }; -class SignOutTextServiceSentence final : public Sentence { +class SignOutServiceSentence final : public Sentence { public: - SignOutTextServiceSentence() { - kind_ = Kind::kSignOutTSService; + explicit SignOutServiceSentence(const meta::cpp2::ExternalServiceType& type) : type_(type) { + kind_ = Kind::kSignOutService; } std::string toString() const override; + + meta::cpp2::ExternalServiceType getType() { + return type_; + } + + private: + meta::cpp2::ExternalServiceType type_; }; class ShowSessionsSentence final : public Sentence { diff --git a/src/parser/Sentence.h b/src/parser/Sentence.h index 17995bfba3f..563cb20e82a 100644 --- a/src/parser/Sentence.h +++ b/src/parser/Sentence.h @@ -79,7 +79,7 @@ class Sentence { kShowGroups, kShowZones, kShowStats, - kShowTSClients, + kShowServiceClients, kShowFTIndexes, kDescribeUser, kDeleteVertices, @@ -124,8 +124,8 @@ class Sentence { kAddListener, kRemoveListener, kShowListener, - kSignInTSService, - kSignOutTSService, + kSignInService, + kSignOutService, kCreateFTIndex, kDropFTIndex, kShowSessions, diff --git a/src/parser/parser.yy b/src/parser/parser.yy index 51732359e69..e79d77d5697 100644 --- a/src/parser/parser.yy +++ b/src/parser/parser.yy @@ -150,8 +150,8 @@ static constexpr size_t kCommentLengthLimit = 256; nebula::TextSearchArgument *text_search_argument; nebula::TextSearchArgument *base_text_search_argument; nebula::TextSearchArgument *fuzzy_text_search_argument; - nebula::meta::cpp2::FTClient *text_search_client_item; - nebula::TSClientList *text_search_client_list; + nebula::meta::cpp2::ServiceClient *service_client_item; + nebula::ServiceClientList *service_client_list; nebula::QueryUniqueIdentifier *query_unique_identifier; } @@ -333,8 +333,8 @@ static constexpr size_t kCommentLengthLimit = 256; %type text_search_argument %type base_text_search_argument %type fuzzy_text_search_argument -%type text_search_client_item -%type text_search_client_list +%type service_client_item +%type service_client_list %type legal_integer unary_integer rank port job_concurrency @@ -395,7 +395,7 @@ static constexpr size_t kCommentLengthLimit = 256; %type seq_sentences %type explain_sentence %type sentences -%type sign_in_text_search_service_sentence sign_out_text_search_service_sentence +%type sign_in_service_sentence sign_out_service_sentence %type opt_if_not_exists %type opt_if_exists @@ -1917,26 +1917,26 @@ match_limit ; -text_search_client_item +service_client_item : L_PAREN host_item R_PAREN { - $$ = new nebula::meta::cpp2::FTClient(); + $$ = new nebula::meta::cpp2::ServiceClient(); $$->host_ref() = *$2; delete $2; } | L_PAREN host_item COMMA KW_HTTP R_PAREN { - $$ = new nebula::meta::cpp2::FTClient(); + $$ = new nebula::meta::cpp2::ServiceClient(); $$->host_ref() = *$2; $$->conn_type_ref() = "http"; delete $2; } | L_PAREN host_item COMMA KW_HTTPS R_PAREN { - $$ = new nebula::meta::cpp2::FTClient(); + $$ = new nebula::meta::cpp2::ServiceClient(); $$->host_ref() = *$2; $$->conn_type_ref() = "https"; delete $2; } | L_PAREN host_item COMMA STRING COMMA STRING R_PAREN { - $$ = new nebula::meta::cpp2::FTClient(); + $$ = new nebula::meta::cpp2::ServiceClient(); $$->host_ref() = *$2; $$->user_ref() = *$4; $$->pwd_ref() = *$6; @@ -1945,7 +1945,7 @@ text_search_client_item delete $6; } | L_PAREN host_item COMMA KW_HTTP COMMA STRING COMMA STRING R_PAREN { - $$ = new nebula::meta::cpp2::FTClient(); + $$ = new nebula::meta::cpp2::ServiceClient(); $$->host_ref() = *$2; $$->user_ref() = *$6; $$->pwd_ref() = *$8; @@ -1955,7 +1955,7 @@ text_search_client_item delete $8; } | L_PAREN host_item COMMA KW_HTTPS COMMA STRING COMMA STRING R_PAREN { - $$ = new nebula::meta::cpp2::FTClient(); + $$ = new nebula::meta::cpp2::ServiceClient(); $$->host_ref() = *$2; $$->user_ref() = *$6; $$->pwd_ref() = *$8; @@ -1966,29 +1966,29 @@ text_search_client_item } ; -text_search_client_list - : text_search_client_item { - $$ = new TSClientList(); +service_client_list + : service_client_item { + $$ = new ServiceClientList(); $$->addClient($1); } - | text_search_client_list COMMA text_search_client_item { + | service_client_list COMMA service_client_item { $$ = $1; $$->addClient($3); } - | text_search_client_list COMMA { + | service_client_list COMMA { $$ = $1; } ; -sign_in_text_search_service_sentence - : KW_SIGN KW_IN KW_TEXT KW_SERVICE text_search_client_list { - $$ = new SignInTextServiceSentence($5); +sign_in_service_sentence + : KW_SIGN KW_IN KW_TEXT KW_SERVICE service_client_list { + $$ = new SignInServiceSentence(meta::cpp2::ExternalServiceType::ELASTICSEARCH, $5); } ; -sign_out_text_search_service_sentence +sign_out_service_sentence : KW_SIGN KW_OUT KW_TEXT KW_SERVICE { - $$ = new SignOutTextServiceSentence(); + $$ = new SignOutServiceSentence(meta::cpp2::ExternalServiceType::ELASTICSEARCH); } ; @@ -3419,7 +3419,7 @@ show_sentence $$ = new ShowStatsSentence(); } | KW_SHOW KW_TEXT KW_SEARCH KW_CLIENTS { - $$ = new ShowTSClientsSentence(); + $$ = new ShowServiceClientsSentence(meta::cpp2::ExternalServiceType::ELASTICSEARCH); } | KW_SHOW KW_FULLTEXT KW_INDEXES { $$ = new ShowFTIndexesSentence(); @@ -3902,8 +3902,8 @@ maintain_sentence | list_listener_sentence { $$ = $1; } | create_snapshot_sentence { $$ = $1; } | drop_snapshot_sentence { $$ = $1; } - | sign_in_text_search_service_sentence { $$ = $1; } - | sign_out_text_search_service_sentence { $$ = $1; } + | sign_in_service_sentence { $$ = $1; } + | sign_out_service_sentence { $$ = $1; } ; sentence diff --git a/src/storage/CMakeLists.txt b/src/storage/CMakeLists.txt index 5f7d027e739..ddf106cae08 100644 --- a/src/storage/CMakeLists.txt +++ b/src/storage/CMakeLists.txt @@ -71,17 +71,22 @@ nebula_add_library( storage_transaction_executor OBJECT transaction/TransactionManager.cpp transaction/ConsistUtil.cpp - transaction/ChainUpdateEdgeProcessorLocal.cpp - transaction/ChainUpdateEdgeProcessorRemote.cpp + transaction/ChainUpdateEdgeLocalProcessor.cpp + transaction/ChainUpdateEdgeRemoteProcessor.cpp transaction/ChainResumeProcessor.cpp transaction/ChainAddEdgesGroupProcessor.cpp - transaction/ChainAddEdgesProcessorLocal.cpp - transaction/ChainAddEdgesProcessorRemote.cpp + transaction/ChainAddEdgesLocalProcessor.cpp + transaction/ChainAddEdgesRemoteProcessor.cpp transaction/ResumeAddEdgeProcessor.cpp transaction/ResumeAddEdgeRemoteProcessor.cpp transaction/ResumeUpdateProcessor.cpp transaction/ResumeUpdateRemoteProcessor.cpp transaction/ChainProcessorFactory.cpp + transaction/ChainDeleteEdgesGroupProcessor.cpp + transaction/ChainDeleteEdgesLocalProcessor.cpp + transaction/ChainDeleteEdgesRemoteProcessor.cpp + transaction/ChainDeleteEdgesResumeProcessor.cpp + transaction/ChainDeleteEdgesResumeRemoteProcessor.cpp ) nebula_add_library( diff --git a/src/storage/CommonUtils.h b/src/storage/CommonUtils.h index fba3ea6f6b5..2826dc654d1 100644 --- a/src/storage/CommonUtils.h +++ b/src/storage/CommonUtils.h @@ -241,6 +241,10 @@ struct RuntimeContext { // used for update bool insert_ = false; + // some times, one line is filter out but still return (has edge) + // and some time, this line is just removed from the return result + bool filterInvalidResultOut = false; + ResultStatus resultStat_{ResultStatus::NORMAL}; }; diff --git a/src/storage/GraphStorageServiceHandler.cpp b/src/storage/GraphStorageServiceHandler.cpp index 579d0fd2a68..bebabfd1987 100644 --- a/src/storage/GraphStorageServiceHandler.cpp +++ b/src/storage/GraphStorageServiceHandler.cpp @@ -21,7 +21,8 @@ #include "storage/query/ScanEdgeProcessor.h" #include "storage/query/ScanVertexProcessor.h" #include "storage/transaction/ChainAddEdgesGroupProcessor.h" -#include "storage/transaction/ChainUpdateEdgeProcessorLocal.h" +#include "storage/transaction/ChainDeleteEdgesGroupProcessor.h" +#include "storage/transaction/ChainUpdateEdgeLocalProcessor.h" #define RETURN_FUTURE(processor) \ auto f = processor->getFuture(); \ @@ -112,7 +113,7 @@ folly::Future GraphStorageServiceHandler::future_updateEdg folly::Future GraphStorageServiceHandler::future_chainUpdateEdge( const cpp2::UpdateEdgeRequest& req) { - auto* proc = ChainUpdateEdgeProcessorLocal::instance(env_); + auto* proc = ChainUpdateEdgeLocalProcessor::instance(env_); RETURN_FUTURE(proc); } @@ -160,6 +161,12 @@ folly::Future GraphStorageServiceHandler::future_chainAddEdg RETURN_FUTURE(processor); } +folly::Future GraphStorageServiceHandler::future_chainDeleteEdges( + const cpp2::DeleteEdgesRequest& req) { + auto* processor = ChainDeleteEdgesGroupProcessor::instance(env_); + RETURN_FUTURE(processor); +} + folly::Future GraphStorageServiceHandler::future_put( const cpp2::KVPutRequest& req) { auto* processor = PutProcessor::instance(env_); diff --git a/src/storage/GraphStorageServiceHandler.h b/src/storage/GraphStorageServiceHandler.h index d4e806d9bc5..9c4b7a2e898 100644 --- a/src/storage/GraphStorageServiceHandler.h +++ b/src/storage/GraphStorageServiceHandler.h @@ -57,6 +57,9 @@ class GraphStorageServiceHandler final : public cpp2::GraphStorageServiceSvIf { folly::Future future_scanVertex(const cpp2::ScanVertexRequest& req) override; + folly::Future future_chainDeleteEdges( + const cpp2::DeleteEdgesRequest& req) override; + folly::Future future_scanEdge(const cpp2::ScanEdgeRequest& req) override; folly::Future future_getUUID(const cpp2::GetUUIDReq& req) override; diff --git a/src/storage/InternalStorageServiceHandler.cpp b/src/storage/InternalStorageServiceHandler.cpp index e15e431b734..29b1cf2cc6b 100644 --- a/src/storage/InternalStorageServiceHandler.cpp +++ b/src/storage/InternalStorageServiceHandler.cpp @@ -5,8 +5,9 @@ #include "storage/InternalStorageServiceHandler.h" -#include "storage/transaction/ChainAddEdgesProcessorRemote.h" -#include "storage/transaction/ChainUpdateEdgeProcessorRemote.h" +#include "storage/transaction/ChainAddEdgesRemoteProcessor.h" +#include "storage/transaction/ChainDeleteEdgesRemoteProcessor.h" +#include "storage/transaction/ChainUpdateEdgeRemoteProcessor.h" #define RETURN_FUTURE(processor) \ auto f = processor->getFuture(); \ @@ -20,13 +21,19 @@ InternalStorageServiceHandler::InternalStorageServiceHandler(StorageEnv* env) : folly::Future InternalStorageServiceHandler::future_chainAddEdges( const cpp2::ChainAddEdgesRequest& req) { - auto* processor = ChainAddEdgesProcessorRemote::instance(env_); + auto* processor = ChainAddEdgesRemoteProcessor::instance(env_); RETURN_FUTURE(processor); } folly::Future InternalStorageServiceHandler::future_chainUpdateEdge( const cpp2::ChainUpdateEdgeRequest& req) { - auto* processor = ChainUpdateEdgeProcessorRemote::instance(env_); + auto* processor = ChainUpdateEdgeRemoteProcessor::instance(env_); + RETURN_FUTURE(processor); +} + +folly::Future InternalStorageServiceHandler::future_chainDeleteEdges( + const cpp2::ChainDeleteEdgesRequest& req) { + auto* processor = ChainDeleteEdgesRemoteProcessor::instance(env_); RETURN_FUTURE(processor); } diff --git a/src/storage/InternalStorageServiceHandler.h b/src/storage/InternalStorageServiceHandler.h index c5a0bbd6379..01407c3b204 100644 --- a/src/storage/InternalStorageServiceHandler.h +++ b/src/storage/InternalStorageServiceHandler.h @@ -27,6 +27,9 @@ class InternalStorageServiceHandler final : public cpp2::InternalStorageServiceS folly::Future future_chainUpdateEdge( const cpp2::ChainUpdateEdgeRequest& p_req); + folly::Future future_chainDeleteEdges( + const cpp2::ChainDeleteEdgesRequest& p_req); + private: StorageEnv* env_{nullptr}; }; diff --git a/src/storage/exec/FilterNode.h b/src/storage/exec/FilterNode.h index b70df4d389b..d8d24dd88ae 100644 --- a/src/storage/exec/FilterNode.h +++ b/src/storage/exec/FilterNode.h @@ -14,6 +14,11 @@ namespace nebula { namespace storage { +enum class FilterMode { + TAG_AND_EDGE = 0, + TAG_ONLY = 1, +}; + /* FilterNode will receive the result from upstream, check whether tag data or edge data could pass the expression filter. FilterNode can only accept one upstream @@ -59,27 +64,48 @@ class FilterNode : public IterateNode { return nebula::cpp2::ErrorCode::SUCCEEDED; } + void setFilterMode(FilterMode mode) { + mode_ = mode; + } + private: - // return true when the value iter points to a value which can filter bool check() override { - if (filterExp_ != nullptr) { - expCtx_->reset(this->reader(), this->key().str()); - // result is false when filter out - auto result = filterExp_->eval(*expCtx_); - // NULL is always false - auto ret = result.toBool(); - if (ret.isBool() && ret.getBool()) { - return true; - } - return false; + if (filterExp_ == nullptr) { + return true; } - return true; + switch (mode_) { + case FilterMode::TAG_AND_EDGE: + return checkTagAndEdge(); + case FilterMode::TAG_ONLY: + return checkTagOnly(); + default: + return checkTagAndEdge(); + } + } + + bool checkTagOnly() { + auto result = filterExp_->eval(*expCtx_); + // NULL is always false + auto ret = result.toBool(); + return ret.isBool() && ret.getBool(); + } + + // return true when the value iter points to a value which can filter + bool checkTagAndEdge() { + expCtx_->reset(this->reader(), this->key().str()); + // result is false when filter out + auto result = filterExp_->eval(*expCtx_); + // NULL is always false + auto ret = result.toBool(); + return ret.isBool() && ret.getBool(); } private: RuntimeContext* context_; StorageExpressionContext* expCtx_; Expression* filterExp_; + FilterMode mode_{FilterMode::TAG_AND_EDGE}; + int32_t callCheck{0}; }; } // namespace storage diff --git a/src/storage/exec/GetNeighborsNode.h b/src/storage/exec/GetNeighborsNode.h index a983a9c66b7..5a5c17f9dab 100644 --- a/src/storage/exec/GetNeighborsNode.h +++ b/src/storage/exec/GetNeighborsNode.h @@ -82,7 +82,12 @@ class GetNeighborsNode : public QueryNode { row[1].setList(agg->mutableResult().moveList()); } - resultDataSet_->rows.emplace_back(std::move(row)); + // only set filterInvalidResultOut = true in TagOnly mode + // so if it it an edge, this test is always true + if (!context_->filterInvalidResultOut || context_->resultStat_ == ResultStatus::NORMAL) { + resultDataSet_->rows.emplace_back(std::move(row)); + } + return nebula::cpp2::ErrorCode::SUCCEEDED; } @@ -90,6 +95,9 @@ class GetNeighborsNode : public QueryNode { GetNeighborsNode() = default; virtual nebula::cpp2::ErrorCode iterateEdges(std::vector& row) { + if (edgeContext_->propContexts_.empty()) { + return nebula::cpp2::ErrorCode::SUCCEEDED; + } int64_t edgeRowCount = 0; nebula::List list; for (; upstream_->valid(); upstream_->next(), ++edgeRowCount) { diff --git a/src/storage/exec/MultiTagNode.h b/src/storage/exec/MultiTagNode.h new file mode 100644 index 00000000000..ba2ba3ade92 --- /dev/null +++ b/src/storage/exec/MultiTagNode.h @@ -0,0 +1,118 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ +#pragma once + +#include "common/base/Base.h" +#include "storage/context/StorageExpressionContext.h" +#include "storage/exec/EdgeNode.h" +#include "storage/exec/StorageIterator.h" +#include "storage/exec/TagNode.h" + +namespace nebula { +namespace storage { + +// MultiTagNode is a replacement of HashJoinNode +// in execution of "go over" +// if Graph don't pass any Edge prop +class MultiTagNode : public IterateNode { + public: + using RelNode::doExecute; + + MultiTagNode(RuntimeContext* context, + const std::vector& tagNodes, + StorageExpressionContext* expCtx) + : context_(context), tagNodes_(tagNodes), expCtx_(expCtx) { + IterateNode::name_ = "MultiTagNode"; + } + + nebula::cpp2::ErrorCode doExecute(PartitionID partId, const VertexID& vId) override { + auto ret = RelNode::doExecute(partId, vId); + if (ret != nebula::cpp2::ErrorCode::SUCCEEDED) { + return ret; + } + + if (expCtx_ != nullptr) { + expCtx_->clear(); + } + result_.setList(nebula::List()); + auto& result = result_.mutableList(); + if (context_->resultStat_ == ResultStatus::ILLEGAL_DATA) { + return nebula::cpp2::ErrorCode::E_INVALID_DATA; + } + + // add result of each tag node to tagResult + for (auto* tagNode : tagNodes_) { + if (context_->isPlanKilled()) { + return nebula::cpp2::ErrorCode::E_PLAN_IS_KILLED; + } + ret = tagNode->collectTagPropsIfValid( + [&result](const std::vector*) -> nebula::cpp2::ErrorCode { + result.values.emplace_back(Value()); + return nebula::cpp2::ErrorCode::SUCCEEDED; + }, + [this, &result, tagNode]( + folly::StringPiece key, + RowReader* reader, + const std::vector* props) -> nebula::cpp2::ErrorCode { + nebula::List list; + list.reserve(props->size()); + const auto& tagName = tagNode->getTagName(); + for (const auto& prop : *props) { + VLOG(2) << "Collect prop " << prop.name_; + auto value = QueryUtils::readVertexProp( + key, context_->vIdLen(), context_->isIntId(), reader, prop); + if (!value.ok()) { + return nebula::cpp2::ErrorCode::E_TAG_PROP_NOT_FOUND; + } + if (prop.filtered_ && expCtx_ != nullptr) { + expCtx_->setTagProp(tagName, prop.name_, value.value()); + } + if (prop.returned_) { + list.emplace_back(std::move(value).value()); + } + } + result.values.emplace_back(std::move(list)); + return nebula::cpp2::ErrorCode::SUCCEEDED; + }); + if (ret != nebula::cpp2::ErrorCode::SUCCEEDED) { + return ret; + } + } + + return nebula::cpp2::ErrorCode::SUCCEEDED; + } + + bool valid() const override { + auto ret = tagNodes_.back()->valid(); + return ret; + } + + void next() override { + tagNodes_.back()->next(); + } + + folly::StringPiece key() const override { + LOG(FATAL) << "not allowed to do this"; + return ""; + } + + folly::StringPiece val() const override { + LOG(FATAL) << "not allowed to do this"; + return ""; + } + + RowReader* reader() const override { + LOG(FATAL) << "not allowed to do this"; + return nullptr; + } + + private: + RuntimeContext* context_; + std::vector tagNodes_; + StorageExpressionContext* expCtx_; +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/mutate/AddEdgesProcessor.h b/src/storage/mutate/AddEdgesProcessor.h index b8f75b6caec..cec28b69e3c 100644 --- a/src/storage/mutate/AddEdgesProcessor.h +++ b/src/storage/mutate/AddEdgesProcessor.h @@ -19,7 +19,7 @@ extern ProcessorCounters kAddEdgesCounters; class AddEdgesProcessor : public BaseProcessor { friend class TransactionManager; - friend class ChainAddEdgesProcessorLocal; + friend class ChainAddEdgesLocalProcessor; public: static AddEdgesProcessor* instance(StorageEnv* env, diff --git a/src/storage/mutate/DeleteEdgesProcessor.cpp b/src/storage/mutate/DeleteEdgesProcessor.cpp index ac6071436cd..0761fe8e7ca 100644 --- a/src/storage/mutate/DeleteEdgesProcessor.cpp +++ b/src/storage/mutate/DeleteEdgesProcessor.cpp @@ -77,8 +77,22 @@ void DeleteEdgesProcessor::process(const cpp2::DeleteEdgesRequest& req) { handleAsync(spaceId_, partId, code); continue; } - doRemove(spaceId_, partId, std::move(keys)); - stats::StatsManager::addValue(kNumEdgesDeleted, keys.size()); + + HookFuncPara para; + if (tossHookFunc_) { + para.keys.emplace(&keys); + (*tossHookFunc_)(para); + } + if (para.result) { + env_->kvstore_->asyncAppendBatch( + spaceId_, + partId, + std::move(para.result.value()), + [partId, this](nebula::cpp2::ErrorCode rc) { handleAsync(spaceId_, partId, rc); }); + } else { + doRemove(spaceId_, partId, std::move(keys)); + stats::StatsManager::addValue(kNumEdgesDeleted, keys.size()); + } } } else { for (auto& part : partEdges) { @@ -198,6 +212,11 @@ ErrorOr DeleteEdgesProcessor::deleteEdges( } } + if (tossHookFunc_) { + HookFuncPara para; + para.batch.emplace(batchHolder.get()); + (*tossHookFunc_)(para); + } return encodeBatchValue(batchHolder->getBatch()); } diff --git a/src/storage/mutate/DeleteEdgesProcessor.h b/src/storage/mutate/DeleteEdgesProcessor.h index 2950a8f63ae..273399bd1e7 100644 --- a/src/storage/mutate/DeleteEdgesProcessor.h +++ b/src/storage/mutate/DeleteEdgesProcessor.h @@ -9,6 +9,7 @@ #include "common/base/Base.h" #include "kvstore/LogEncoder.h" #include "storage/BaseProcessor.h" +#include "storage/transaction/ConsistTypes.h" namespace nebula { namespace storage { @@ -24,6 +25,11 @@ class DeleteEdgesProcessor : public BaseProcessor { void process(const cpp2::DeleteEdgesRequest& req); + using HookFunction = std::function; + void setHookFunc(HookFunction func) { + tossHookFunc_ = func; + } + private: DeleteEdgesProcessor(StorageEnv* env, const ProcessorCounters* counters) : BaseProcessor(env, counters) {} @@ -34,6 +40,11 @@ class DeleteEdgesProcessor : public BaseProcessor { private: GraphSpaceID spaceId_; std::vector> indexes_; + + protected: + // TOSS use this hook function to append some delete operation + // or may append some put operation + std::optional tossHookFunc_; }; } // namespace storage diff --git a/src/storage/query/GetNeighborsProcessor.cpp b/src/storage/query/GetNeighborsProcessor.cpp index 42e65f8d033..017e60d6d78 100644 --- a/src/storage/query/GetNeighborsProcessor.cpp +++ b/src/storage/query/GetNeighborsProcessor.cpp @@ -11,6 +11,7 @@ #include "storage/exec/FilterNode.h" #include "storage/exec/GetNeighborsNode.h" #include "storage/exec/HashJoinNode.h" +#include "storage/exec/MultiTagNode.h" #include "storage/exec/TagNode.h" namespace nebula { @@ -79,6 +80,7 @@ void GetNeighborsProcessor::runInSingleThread(const cpp2::GetNeighborsRequest& r auto plan = buildPlan(&contexts_.front(), &expCtxs_.front(), &resultDataSet_, limit, random); std::unordered_set failedParts; for (const auto& partEntry : req.get_parts()) { + contexts_.front().resultStat_ = ResultStatus::NORMAL; auto partId = partEntry.first; for (const auto& row : partEntry.second) { CHECK_GE(row.values.size(), 1); @@ -184,24 +186,32 @@ StoragePlan GetNeighborsProcessor::buildPlan(RuntimeContext* context, bool random) { /* The StoragePlan looks like this: - +--------+---------+ - | GetNeighborsNode | - +--------+---------+ - | - +--------+---------+ - | AggregateNode | - +--------+---------+ - | - +--------+---------+ - | FilterNode | - +--------+---------+ - | - +--------+---------+ - +-->+ HashJoinNode +<----+ - | +------------------+ | - +--------+---------+ +---------+--------+ - | TagNodes | | EdgeNodes | - +------------------+ +------------------+ + +------------------+ or, if there is no edge: + | GetNeighborsNode | + +--------+---------+ +-----------------+ + | |GetNeighborsNode | + +--------+---------+ +--------+--------+ + | AggregateNode | | + +--------+---------+ +------+------+ + | |AggregateNode| + +--------+---------+ +------+------+ + | FilterNode | | + +--------+---------+ +-----+----+ + | |FilterNode| + +--------+---------+ +-----+----+ + +-->+ HashJoinNode +<----+ | + | +------------------+ | +------+-----+ ++--------+---------+ +---------+--------+ |HashJoinNode| +| TagNodes | | EdgeNodes | +------+-----+ ++------------------+ +------------------+ | + +------+-----+ + |MultiTagNode| + +------+-----+ + | + +----+---+ + |TagNodes| + +--------+ + */ StoragePlan plan; std::vector tags; @@ -217,23 +227,39 @@ StoragePlan GetNeighborsProcessor::buildPlan(RuntimeContext* context, plan.addNode(std::move(edge)); } - auto hashJoin = - std::make_unique(context, tags, edges, &tagContext_, &edgeContext_, expCtx); - for (auto* tag : tags) { - hashJoin->addDependency(tag); - } - for (auto* edge : edges) { - hashJoin->addDependency(edge); + IterateNode* upstream = nullptr; + IterateNode* join = nullptr; + if (!edges.empty()) { + auto hashJoin = + std::make_unique(context, tags, edges, &tagContext_, &edgeContext_, expCtx); + for (auto* tag : tags) { + hashJoin->addDependency(tag); + } + for (auto* edge : edges) { + hashJoin->addDependency(edge); + } + join = hashJoin.get(); + upstream = hashJoin.get(); + plan.addNode(std::move(hashJoin)); + } else { + context->filterInvalidResultOut = true; + auto groupNode = std::make_unique(context, tags, expCtx); + for (auto* tag : tags) { + groupNode->addDependency(tag); + } + join = groupNode.get(); + upstream = groupNode.get(); + plan.addNode(std::move(groupNode)); } - IterateNode* join = hashJoin.get(); - IterateNode* upstream = hashJoin.get(); - plan.addNode(std::move(hashJoin)); if (filter_) { auto filter = std::make_unique>(context, upstream, expCtx, filter_->clone()); filter->addDependency(upstream); upstream = filter.get(); + if (edges.empty()) { + filter.get()->setFilterMode(FilterMode::TAG_ONLY); + } plan.addNode(std::move(filter)); } @@ -302,8 +328,7 @@ nebula::cpp2::ErrorCode GetNeighborsProcessor::buildTagContext(const cpp2::Trave // If the list is not given, no prop will be returned. return nebula::cpp2::ErrorCode::SUCCEEDED; } - auto returnProps = - (*req.vertex_props_ref()).empty() ? buildAllTagProps() : *req.vertex_props_ref(); + auto returnProps = *req.vertex_props_ref(); auto ret = handleVertexProps(returnProps); if (ret != nebula::cpp2::ErrorCode::SUCCEEDED) { @@ -320,8 +345,7 @@ nebula::cpp2::ErrorCode GetNeighborsProcessor::buildEdgeContext(const cpp2::Trav // If the list is not given, no prop will be returned. return nebula::cpp2::ErrorCode::SUCCEEDED; } - auto returnProps = (*req.edge_props_ref()).empty() ? buildAllEdgeProps(*req.edge_direction_ref()) - : *req.edge_props_ref(); + auto returnProps = *req.edge_props_ref(); auto ret = handleEdgeProps(returnProps); if (ret != nebula::cpp2::ErrorCode::SUCCEEDED) { return ret; diff --git a/src/storage/test/CMakeLists.txt b/src/storage/test/CMakeLists.txt index 5371ad3492c..65b89befe2a 100644 --- a/src/storage/test/CMakeLists.txt +++ b/src/storage/test/CMakeLists.txt @@ -741,6 +741,21 @@ nebula_add_executable( gtest ) +nebula_add_test( + NAME + chain_delete_edge_test + SOURCES + ChainDeleteEdgesTest.cpp + OBJECTS + ${storage_test_deps} + LIBRARIES + ${ROCKSDB_LIBRARIES} + ${THRIFT_LIBRARIES} + ${PROXYGEN_LIBRARIES} + wangle + gtest +) + nebula_add_executable( NAME storage_index_write_bm diff --git a/src/storage/test/ChainAddEdgesTest.cpp b/src/storage/test/ChainAddEdgesTest.cpp index 8f693d4d4bc..3881e0cc671 100644 --- a/src/storage/test/ChainAddEdgesTest.cpp +++ b/src/storage/test/ChainAddEdgesTest.cpp @@ -18,7 +18,7 @@ #include "storage/test/ChainTestUtils.h" #include "storage/test/TestUtils.h" #include "storage/transaction/ChainAddEdgesGroupProcessor.h" -#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ChainAddEdgesLocalProcessor.h" #include "storage/transaction/ConsistUtil.h" namespace nebula { @@ -34,11 +34,11 @@ TEST(ChainAddEdgesTest, TestUtilsTest) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, fackTerm); - auto* processor = new FakeChainAddEdgesProcessorLocal(env); + auto* processor = new FakeChainAddEdgesLocalProcessor(env); processor->rcPrepareLocal = nebula::cpp2::ErrorCode::SUCCEEDED; processor->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; @@ -63,10 +63,10 @@ TEST(ChainAddEdgesTest, prepareLocalSucceedTest) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, fackTerm); - auto* proc = new FakeChainAddEdgesProcessorLocal(env); + auto* proc = new FakeChainAddEdgesLocalProcessor(env); proc->rcProcessRemote = nebula::cpp2::ErrorCode::E_RPC_FAILURE; @@ -91,10 +91,10 @@ TEST(ChainAddEdgesTest, processRemoteSucceededTest) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); - auto* proc = new FakeChainAddEdgesProcessorLocal(env); + auto* proc = new FakeChainAddEdgesLocalProcessor(env); MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, fackTerm); proc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; @@ -122,11 +122,11 @@ TEST(ChainAddEdgesTest, processRemoteFailedTest) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, fackTerm); - auto* proc = new FakeChainAddEdgesProcessorLocal(env); + auto* proc = new FakeChainAddEdgesLocalProcessor(env); proc->rcProcessRemote = nebula::cpp2::ErrorCode::E_OUTDATED_TERM; LOG(INFO) << "Build AddEdgesRequest..."; @@ -144,6 +144,8 @@ TEST(ChainAddEdgesTest, processRemoteFailedTest) { // prime key should be deleted EXPECT_EQ(0, numOfKey(req, util.genPrime, env)); EXPECT_EQ(0, numOfKey(req, util.genDoublePrime, env)); + + // env->txnMan_->stop(); } TEST(ChainAddEdgesTest, processRemoteUnknownTest) { @@ -151,11 +153,11 @@ TEST(ChainAddEdgesTest, processRemoteUnknownTest) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, fackTerm); - auto* proc = new FakeChainAddEdgesProcessorLocal(env); + auto* proc = new FakeChainAddEdgesLocalProcessor(env); proc->rcProcessRemote = nebula::cpp2::ErrorCode::E_RPC_FAILURE; @@ -182,12 +184,12 @@ TEST(ChainAddEdgesTest, processRemoteTest) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, fackTerm); - auto* proc = new FakeChainAddEdgesProcessorLocal(env); + auto* proc = new FakeChainAddEdgesLocalProcessor(env); LOG(INFO) << "Build AddEdgesRequest..."; cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, 1); diff --git a/src/storage/test/ChainDeleteEdgesTest.cpp b/src/storage/test/ChainDeleteEdgesTest.cpp new file mode 100644 index 00000000000..91ef7a00597 --- /dev/null +++ b/src/storage/test/ChainDeleteEdgesTest.cpp @@ -0,0 +1,521 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include +#include +#include +#include +#include +#include +#include + +#include "common/fs/TempDir.h" +#include "mock/MockCluster.h" +#include "mock/MockData.h" +#include "storage/CommonUtils.h" +#include "storage/StorageFlags.h" +#include "storage/test/ChainTestUtils.h" +#include "storage/test/TestUtils.h" +#include "storage/transaction/ChainDeleteEdgesGroupProcessor.h" +#include "storage/transaction/ChainDeleteEdgesLocalProcessor.h" +#include "storage/transaction/ConsistUtil.h" + +namespace nebula { +namespace storage { + +constexpr int32_t mockSpaceId = 1; +constexpr int32_t mockPartNum = 1; +constexpr int32_t gTerm = 1; + +class GlobalCluster { + public: + static mock::MockCluster* get() { + static mock::MockCluster cluster; + static fs::TempDir rootPath("/tmp/DeleteEdgesTest.XXXXXX"); + static bool init = false; + if (!init) { + cluster.initStorageKV(rootPath.path()); + init = true; + } + return &cluster; + } +}; + +// class FakeChainDeleteEdgesProcessor; +class FakeChainDeleteEdgesProcessor : public ChainDeleteEdgesLocalProcessor { + public: + explicit FakeChainDeleteEdgesProcessor(StorageEnv* env); + folly::SemiFuture prepareLocal() override; + folly::SemiFuture processRemote(Code code) override; + folly::SemiFuture processLocal(Code code) override; + + cpp2::DeleteEdgesRequest makeDelRequest(cpp2::AddEdgesRequest, + int32_t limit = std::numeric_limits::max()); + + public: + folly::Optional rcPrepareLocal; + folly::Optional rcProcessRemote; + folly::Optional rcProcessLocal; +}; + +// make sure test utils works +TEST(ChainDeleteEdgesTest, TestUtilsTest) { + fs::TempDir rootPath("/tmp/DeleteEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefault(); + env->metaClient_ = mClient.get(); + MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, gTerm); + + auto* proc = new FakeChainDeleteEdgesProcessor(env); + + proc->rcPrepareLocal = nebula::cpp2::ErrorCode::SUCCEEDED; + proc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; + proc->rcProcessLocal = nebula::cpp2::ErrorCode::SUCCEEDED; + + LOG(INFO) << "Build DeleteEdgesReq..."; + auto req = mock::MockData::mockDeleteEdgesReq(mockPartNum); + + LOG(INFO) << "Run DeleteEdgesReq..."; + auto fut = proc->getFuture(); + proc->process(req); + auto resp = std::move(fut).get(); + // EXPECT_EQ(0, resp.result.failed_parts.size()); + + LOG(INFO) << "Check data in kv store..."; + // sleep(1); + // The number of data in serve is 334 + // checkAddEdgesData(req, env, 0, 0); +} + +// delete a not exist edge +TEST(ChainDeleteEdgesTest, Test2) { + fs::TempDir rootPath("/tmp/DeleteEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefault(); + env->metaClient_ = mClient.get(); + MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, gTerm); + + auto* proc = new FakeChainDeleteEdgesProcessor(env); + + // proc->rcPrepareLocal = nebula::cpp2::ErrorCode::SUCCEEDED; + proc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; + // proc->rcProcessLocal = nebula::cpp2::ErrorCode::SUCCEEDED; + + LOG(INFO) << "Build DeleteEdgesReq..."; + auto req = mock::MockData::mockDeleteEdgesReq(mockPartNum); + + LOG(INFO) << "Run DeleteEdgesReq..."; + LOG(INFO) << "spaceId = " << req.get_space_id(); + auto fut = proc->getFuture(); + proc->process(req); + auto resp = std::move(fut).get(); + + // we need this sleep to ensure processor deleted before transaction manager + std::this_thread::sleep_for(std::chrono::milliseconds(300)); +} + +// add some edges, then delete it, all phase succeed +TEST(ChainDeleteEdgesTest, Test3) { + fs::TempDir rootPath("/tmp/DeleteEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefault(); + env->metaClient_ = mClient.get(); + MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, gTerm); + + auto* addProc = new FakeChainAddEdgesLocalProcessor(env); + addProc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; + + bool upperPropVal = false; + int32_t partNum = 1; + bool hasInEdges = false; + auto addReq = mock::MockData::mockAddEdgesReq(upperPropVal, partNum, hasInEdges); + + LOG(INFO) << "Run FakeChainAddEdgesLocalProcessor..."; + auto fut = addProc->getFuture(); + addProc->process(addReq); + auto resp = std::move(fut).get(); + + ChainTestUtils util; + auto edgeKeys = util.genEdgeKeys(addReq, util.genKey); + auto num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + EXPECT_EQ(num, 167); + LOG(INFO) << "after add(), edge num = " << num; + + auto* delProc = new FakeChainDeleteEdgesProcessor(env); + auto delReq = delProc->makeDelRequest(addReq); + // delProc->rcPrepareLocal = nebula::cpp2::ErrorCode::SUCCEEDED; + delProc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; + // delProc->rcProcessLocal = nebula::cpp2::ErrorCode::SUCCEEDED; + + LOG(INFO) << "Build DeleteEdgesReq..."; + // auto req = mock::MockData::mockDeleteEdgesReq(mockPartNum); + + LOG(INFO) << "Run DeleteEdgesReq..."; + auto futDel = delProc->getFuture(); + delProc->process(delReq); + std::move(futDel).get(); + + num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + LOG(INFO) << "after del(), edge num = " << num; + EXPECT_EQ(num, 0); + std::this_thread::sleep_for(std::chrono::milliseconds(300)); +} + +// add some edges, then delete one of them, all phase succeed +TEST(ChainDeleteEdgesTest, Test4) { + fs::TempDir rootPath("/tmp/DeleteEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefault(); + env->metaClient_ = mClient.get(); + MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, gTerm); + + auto* addProc = new FakeChainAddEdgesLocalProcessor(env); + addProc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; + + bool upperPropVal = false; + int32_t partNum = 1; + bool hasInEdges = false; + auto addReq = mock::MockData::mockAddEdgesReq(upperPropVal, partNum, hasInEdges); + + LOG(INFO) << "Run FakeChainAddEdgesLocalProcessor..."; + auto fut = addProc->getFuture(); + addProc->process(addReq); + auto resp = std::move(fut).get(); + + ChainTestUtils util; + auto edgeKeys = util.genEdgeKeys(addReq, util.genKey); + auto num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + EXPECT_EQ(num, 167); + LOG(INFO) << "after add(), edge num = " << num; + + auto* delProc = new FakeChainDeleteEdgesProcessor(env); + int32_t limit = 1; + auto delReq = delProc->makeDelRequest(addReq, limit); + // delProc->rcPrepareLocal = nebula::cpp2::ErrorCode::SUCCEEDED; + delProc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; + // delProc->rcProcessLocal = nebula::cpp2::ErrorCode::SUCCEEDED; + + LOG(INFO) << "Build DeleteEdgesReq..."; + // auto req = mock::MockData::mockDeleteEdgesReq(mockPartNum); + + LOG(INFO) << "Run DeleteEdgesReq..."; + auto futDel = delProc->getFuture(); + delProc->process(delReq); + std::move(futDel).get(); + + num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + LOG(INFO) << "after del(), edge num = " << num; + EXPECT_EQ(num, 166); + std::this_thread::sleep_for(std::chrono::milliseconds(300)); +} + +// add some edges, then delete one of them, not execute local commit +TEST(ChainDeleteEdgesTest, Test5) { + fs::TempDir rootPath("/tmp/DeleteEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefault(); + env->metaClient_ = mClient.get(); + MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, gTerm); + + auto* addProc = new FakeChainAddEdgesLocalProcessor(env); + addProc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; + + bool upperPropVal = false; + int32_t partNum = 1; + bool hasInEdges = false; + auto addReq = mock::MockData::mockAddEdgesReq(upperPropVal, partNum, hasInEdges); + + LOG(INFO) << "Run FakeChainAddEdgesLocalProcessor..."; + auto fut = addProc->getFuture(); + addProc->process(addReq); + auto resp = std::move(fut).get(); + + ChainTestUtils util; + auto edgeKeys = util.genEdgeKeys(addReq, util.genKey); + auto num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + EXPECT_EQ(num, 167); + LOG(INFO) << "after add(), edge num = " << num; + + auto* delProc = new FakeChainDeleteEdgesProcessor(env); + auto delReq = delProc->makeDelRequest(addReq); + // delProc->rcPrepareLocal = nebula::cpp2::ErrorCode::SUCCEEDED; + delProc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; + delProc->rcProcessLocal = nebula::cpp2::ErrorCode::SUCCEEDED; + + LOG(INFO) << "Build DeleteEdgesReq..."; + // auto req = mock::MockData::mockDeleteEdgesReq(mockPartNum); + + LOG(INFO) << "Run DeleteEdgesReq..."; + auto futDel = delProc->getFuture(); + delProc->process(delReq); + std::move(futDel).get(); + + num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + LOG(INFO) << "after del(), edge num = " << num; + EXPECT_EQ(num, 167); + + env->txnMan_->scanAll(); + auto* iClient = FakeInternalStorageClient::instance(env, nebula::cpp2::ErrorCode::SUCCEEDED); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + ChainResumeProcessor resumeProc(env); + resumeProc.process(); + num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + EXPECT_EQ(num, 0); + std::this_thread::sleep_for(std::chrono::milliseconds(300)); + + delete iClient; +} + +// add some edges, then delete all of them, not execute local commit +TEST(ChainDeleteEdgesTest, Test6) { + fs::TempDir rootPath("/tmp/DeleteEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefault(); + env->metaClient_ = mClient.get(); + MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, gTerm); + + auto* addProc = new FakeChainAddEdgesLocalProcessor(env); + addProc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; + + bool upperPropVal = false; + int32_t partNum = 1; + bool hasInEdges = false; + auto addReq = mock::MockData::mockAddEdgesReq(upperPropVal, partNum, hasInEdges); + + LOG(INFO) << "Run FakeChainAddEdgesLocalProcessor..."; + auto fut = addProc->getFuture(); + addProc->process(addReq); + auto resp = std::move(fut).get(); + + ChainTestUtils util; + auto edgeKeys = util.genEdgeKeys(addReq, util.genKey); + auto num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + EXPECT_EQ(num, 167); + LOG(INFO) << "after add(), edge num = " << num; + + auto* delProc = new FakeChainDeleteEdgesProcessor(env); + auto delReq = delProc->makeDelRequest(addReq); + delProc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; + delProc->rcProcessLocal = nebula::cpp2::ErrorCode::SUCCEEDED; + + LOG(INFO) << "Run DeleteEdgesReq..."; + auto futDel = delProc->getFuture(); + delProc->process(delReq); + std::move(futDel).get(); + + num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + LOG(INFO) << "after del(), edge num = " << num; + EXPECT_EQ(num, 167); + + env->txnMan_->scanAll(); + auto* iClient = FakeInternalStorageClient::instance(env, nebula::cpp2::ErrorCode::SUCCEEDED); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + ChainResumeProcessor resumeProc(env); + resumeProc.process(); + num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + EXPECT_EQ(num, 0); + std::this_thread::sleep_for(std::chrono::milliseconds(300)); + + delete iClient; +} + +// add some edges, delete one of them, rpc failure +TEST(ChainDeleteEdgesTest, Test7) { + fs::TempDir rootPath("/tmp/DeleteEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefault(); + env->metaClient_ = mClient.get(); + MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, gTerm); + + auto* addProc = new FakeChainAddEdgesLocalProcessor(env); + addProc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; + + bool upperPropVal = false; + int32_t partNum = 1; + bool hasInEdges = false; + auto addReq = mock::MockData::mockAddEdgesReq(upperPropVal, partNum, hasInEdges); + + LOG(INFO) << "Run FakeChainAddEdgesLocalProcessor..."; + auto fut = addProc->getFuture(); + addProc->process(addReq); + auto resp = std::move(fut).get(); + + ChainTestUtils util; + auto edgeKeys = util.genEdgeKeys(addReq, util.genKey); + auto num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + EXPECT_EQ(num, 167); + LOG(INFO) << "after add(), edge num = " << num; + + auto* delProc = new FakeChainDeleteEdgesProcessor(env); + int32_t limit = 1; + auto delReq = delProc->makeDelRequest(addReq, limit); + delProc->rcProcessRemote = nebula::cpp2::ErrorCode::E_RPC_FAILURE; + + LOG(INFO) << "Run DeleteEdgesReq..."; + auto futDel = delProc->getFuture(); + delProc->process(delReq); + std::move(futDel).get(); + + num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + LOG(INFO) << "after del(), edge num = " << num; + EXPECT_EQ(num, 166); + + env->txnMan_->scanAll(); + auto* iClient = FakeInternalStorageClient::instance(env, nebula::cpp2::ErrorCode::SUCCEEDED); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + ChainResumeProcessor resumeProc(env); + resumeProc.process(); + LOG(INFO) << "after recover()"; + num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + EXPECT_EQ(num, 166); + std::this_thread::sleep_for(std::chrono::milliseconds(300)); + + delete iClient; +} + +// add some edges, then one all of them, rpc failure +TEST(ChainDeleteEdgesTest, Test8) { + fs::TempDir rootPath("/tmp/DeleteEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefault(); + env->metaClient_ = mClient.get(); + MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, gTerm); + + auto* addProc = new FakeChainAddEdgesLocalProcessor(env); + addProc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; + + bool upperPropVal = false; + int32_t partNum = 1; + bool hasInEdges = false; + auto addReq = mock::MockData::mockAddEdgesReq(upperPropVal, partNum, hasInEdges); + + LOG(INFO) << "Run FakeChainAddEdgesLocalProcessor..."; + auto fut = addProc->getFuture(); + addProc->process(addReq); + auto resp = std::move(fut).get(); + + ChainTestUtils util; + auto edgeKeys = util.genEdgeKeys(addReq, util.genKey); + auto num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + EXPECT_EQ(num, 167); + LOG(INFO) << "after add(), edge num = " << num; + + auto* delProc = new FakeChainDeleteEdgesProcessor(env); + int32_t limit = num; + auto delReq = delProc->makeDelRequest(addReq, limit); + delProc->rcProcessRemote = nebula::cpp2::ErrorCode::E_RPC_FAILURE; + + LOG(INFO) << "Run DeleteEdgesReq..."; + auto futDel = delProc->getFuture(); + delProc->process(delReq); + std::move(futDel).get(); + + num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + LOG(INFO) << "after del(), edge num = " << num; + EXPECT_EQ(num, 0); + + env->txnMan_->scanAll(); + auto* iClient = FakeInternalStorageClient::instance(env, nebula::cpp2::ErrorCode::SUCCEEDED); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + ChainResumeProcessor resumeProc(env); + resumeProc.process(); + num = util.checkNumOfKey(env, mockSpaceId, edgeKeys); + EXPECT_EQ(num, 0); + std::this_thread::sleep_for(std::chrono::milliseconds(300)); + + delete iClient; +} + +} // namespace storage +} // namespace nebula + +int main(int argc, char** argv) { + FLAGS_trace_toss = true; + + testing::InitGoogleTest(&argc, argv); + folly::init(&argc, &argv, false); + google::SetStderrLogging(google::INFO); + return RUN_ALL_TESTS(); +} + +namespace nebula { +namespace storage { + +FakeChainDeleteEdgesProcessor::FakeChainDeleteEdgesProcessor(StorageEnv* env) + : ChainDeleteEdgesLocalProcessor(env) { + spaceVidLen_ = 32; +} + +folly::SemiFuture FakeChainDeleteEdgesProcessor::prepareLocal() { + LOG(INFO) << "FakeChainDeleteEdgesProcessor::" << __func__ << "()"; + if (rcPrepareLocal) { + LOG(INFO) << "Fake return " << apache::thrift::util::enumNameSafe(*rcPrepareLocal); + return *rcPrepareLocal; + } + LOG(INFO) << "forward to ChainDeleteEdgesLocalProcessor::prepareLocal()"; + return ChainDeleteEdgesLocalProcessor::prepareLocal(); +} + +folly::SemiFuture FakeChainDeleteEdgesProcessor::processRemote(Code code) { + LOG(INFO) << "FakeChainDeleteEdgesProcessor::" << __func__ << "()"; + if (rcProcessRemote) { + LOG(INFO) << "processRemote() fake return " + << apache::thrift::util::enumNameSafe(*rcProcessRemote); + return *rcProcessRemote; + } + LOG(INFO) << "forward to ChainDeleteEdgesLocalProcessor::processRemote()"; + return ChainDeleteEdgesLocalProcessor::processRemote(code); +} + +folly::SemiFuture FakeChainDeleteEdgesProcessor::processLocal(Code code) { + LOG(INFO) << "FakeChainDeleteEdgesProcessor::" << __func__ << "()"; + if (rcProcessLocal) { + LOG(INFO) << "Fake return " << apache::thrift::util::enumNameSafe(*rcProcessLocal); + return *rcProcessLocal; + } + LOG(INFO) << "forward to ChainDeleteEdgesLocalProcessor::processLocal()"; + return ChainDeleteEdgesLocalProcessor::processLocal(code); +} + +// make DeleteEdgesRequest according to an AddEdgesRequest +cpp2::DeleteEdgesRequest FakeChainDeleteEdgesProcessor::makeDelRequest(cpp2::AddEdgesRequest req, + int32_t limit) { + cpp2::DeleteEdgesRequest ret; + int32_t num = 0; + // ret.set_space_id(req.get_space_id()); + ret.space_id_ref() = req.get_space_id(); + for (auto& partAndEdges : req.get_parts()) { + auto partId = partAndEdges.first; + for (auto& newEdge : partAndEdges.second) { + ret.parts_ref().value()[partId].emplace_back(newEdge.get_key()); + if (++num == limit) { + break; + } + } + if (num == limit) { + break; + } + } + return ret; +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/test/ChainResumeEdgeTest.cpp b/src/storage/test/ChainResumeEdgeTest.cpp index 61e305d0fbd..9c985a8462d 100644 --- a/src/storage/test/ChainResumeEdgeTest.cpp +++ b/src/storage/test/ChainResumeEdgeTest.cpp @@ -19,7 +19,7 @@ #include "storage/test/QueryTestUtils.h" #include "storage/test/TestUtils.h" #include "storage/transaction/ChainAddEdgesGroupProcessor.h" -#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ChainAddEdgesLocalProcessor.h" #include "storage/transaction/ChainResumeProcessor.h" #include "storage/transaction/ConsistUtil.h" @@ -27,7 +27,7 @@ namespace nebula { namespace storage { constexpr int32_t mockSpaceId = 1; -constexpr int32_t mockPartNum = 6; +constexpr int32_t mockPartNum = 1; constexpr int32_t mockSpaceVidLen = 32; ChainTestUtils gTestUtil; @@ -47,16 +47,16 @@ TEST(ChainResumeEdgesTest, resumeTest1) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); - auto* proc = new FakeChainAddEdgesProcessorLocal(env); + auto* proc = new FakeChainAddEdgesLocalProcessor(env); proc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; proc->rcProcessLocal = nebula::cpp2::ErrorCode::SUCCEEDED; LOG(INFO) << "Build AddEdgesRequest..."; - cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, 1); + cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, mockPartNum); auto fut = proc->getFuture(); proc->process(req); @@ -66,7 +66,9 @@ TEST(ChainResumeEdgesTest, resumeTest1) { EXPECT_EQ(334, numOfKey(req, gTestUtil.genPrime, env)); EXPECT_EQ(0, numOfKey(req, gTestUtil.genDoublePrime, env)); - env->txnMan_->scanPrimes(1, 1); + for (int32_t i = 1; i <= mockPartNum; ++i) { + env->txnMan_->scanPrimes(1, i); + } auto* iClient = FakeInternalStorageClient::instance(env); FakeInternalStorageClient::hookInternalStorageClient(env, iClient); @@ -76,6 +78,8 @@ TEST(ChainResumeEdgesTest, resumeTest1) { EXPECT_EQ(334, numOfKey(req, gTestUtil.genKey, env)); EXPECT_EQ(0, numOfKey(req, gTestUtil.genPrime, env)); EXPECT_EQ(0, numOfKey(req, gTestUtil.genDoublePrime, env)); + + delete iClient; } /** @@ -92,16 +96,16 @@ TEST(ChainResumeEdgesTest, resumeTest2) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); - auto* proc = new FakeChainAddEdgesProcessorLocal(env); + auto* proc = new FakeChainAddEdgesLocalProcessor(env); proc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; proc->rcProcessLocal = nebula::cpp2::ErrorCode::SUCCEEDED; LOG(INFO) << "Build AddEdgesRequest..."; - cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, 1); + cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, mockPartNum); LOG(INFO) << "Test AddEdgesProcessor..."; auto fut = proc->getFuture(); @@ -122,26 +126,28 @@ TEST(ChainResumeEdgesTest, resumeTest2) { EXPECT_EQ(0, numOfKey(req, util.genKey, env)); EXPECT_EQ(334, numOfKey(req, util.genPrime, env)); EXPECT_EQ(0, numOfKey(req, util.genDoublePrime, env)); + + delete iClient; } /** - * @brief resumePrimeTest3 (resume insert prime outdated) + * @brief resumeTest3 (resume insert prime outdated) */ -TEST(ChainResumeEdgesTest, resumePrimeTest3) { +TEST(ChainResumeEdgesTest, resumeTest3) { fs::TempDir rootPath("/tmp/AddEdgesTest.XXXXXX"); mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); - auto* proc = new FakeChainAddEdgesProcessorLocal(env); + auto* proc = new FakeChainAddEdgesLocalProcessor(env); proc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; proc->rcProcessLocal = nebula::cpp2::ErrorCode::SUCCEEDED; LOG(INFO) << "Build AddEdgesRequest..."; - cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, 1); + cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, mockPartNum); LOG(INFO) << "Test AddEdgesProcessor..."; auto fut = proc->getFuture(); @@ -157,7 +163,11 @@ TEST(ChainResumeEdgesTest, resumePrimeTest3) { auto error = nebula::cpp2::ErrorCode::E_RPC_FAILURE; auto* iClient = FakeInternalStorageClient::instance(env, error); FakeInternalStorageClient::hookInternalStorageClient(env, iClient); - env->txnMan_->scanPrimes(1, 1); + + for (auto i = 1; i <= mockPartNum; ++i) { + env->txnMan_->scanPrimes(1, i); + } + ChainResumeProcessor resumeProc(env); resumeProc.process(); @@ -165,6 +175,8 @@ TEST(ChainResumeEdgesTest, resumePrimeTest3) { EXPECT_EQ(334, numOfKey(req, util.genKey, env)); EXPECT_EQ(0, numOfKey(req, util.genPrime, env)); EXPECT_EQ(334, numOfKey(req, util.genDoublePrime, env)); + + delete iClient; } /** @@ -181,10 +193,10 @@ TEST(ChainResumeEdgesTest, resumeTest4) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); - auto* proc = new FakeChainAddEdgesProcessorLocal(env); + auto* proc = new FakeChainAddEdgesLocalProcessor(env); proc->rcProcessRemote = nebula::cpp2::ErrorCode::E_RPC_FAILURE; @@ -212,6 +224,8 @@ TEST(ChainResumeEdgesTest, resumeTest4) { EXPECT_EQ(334, numOfKey(req, gTestUtil.genKey, env)); EXPECT_EQ(0, numOfKey(req, gTestUtil.genPrime, env)); EXPECT_EQ(334, numOfKey(req, gTestUtil.genDoublePrime, env)); + + delete iClient; } /** @@ -222,10 +236,10 @@ TEST(ChainResumeEdgesTest, resumeTest5) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); - auto* proc = new FakeChainAddEdgesProcessorLocal(env); + auto* proc = new FakeChainAddEdgesLocalProcessor(env); proc->rcProcessRemote = nebula::cpp2::ErrorCode::E_RPC_FAILURE; @@ -252,6 +266,8 @@ TEST(ChainResumeEdgesTest, resumeTest5) { EXPECT_EQ(334, numOfKey(req, util.genKey, env)); EXPECT_EQ(0, numOfKey(req, util.genPrime, env)); EXPECT_EQ(334, numOfKey(req, util.genDoublePrime, env)); + + delete iClient; } /** @@ -262,10 +278,10 @@ TEST(ChainResumeEdgesTest, resumeTest6) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); - auto* proc = new FakeChainAddEdgesProcessorLocal(env); + auto* proc = new FakeChainAddEdgesLocalProcessor(env); proc->rcProcessRemote = nebula::cpp2::ErrorCode::E_RPC_FAILURE; @@ -285,13 +301,19 @@ TEST(ChainResumeEdgesTest, resumeTest6) { auto* iClient = FakeInternalStorageClient::instance(env); FakeInternalStorageClient::hookInternalStorageClient(env, iClient); - env->txnMan_->scanPrimes(1, 1); + + for (auto i = 1; i <= mockPartNum; ++i) { + env->txnMan_->scanPrimes(1, i); + } + ChainResumeProcessor resumeProc(env); resumeProc.process(); EXPECT_EQ(334, numOfKey(req, util.genKey, env)); EXPECT_EQ(0, numOfKey(req, util.genPrime, env)); EXPECT_EQ(0, numOfKey(req, util.genDoublePrime, env)); + + delete iClient; } // resume an update left prime, check resume succeeded @@ -300,10 +322,12 @@ TEST(ChainUpdateEdgeTest, resumeTest7) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); - auto parts = cluster.getTotalParts(); + // auto parts = cluster.getTotalParts(); + auto parts = mockPartNum; + LOG(INFO) << "total parts: " << parts; EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); LOG(INFO) << "Test UpdateEdgeRequest..."; @@ -326,13 +350,19 @@ TEST(ChainUpdateEdgeTest, resumeTest7) { auto* iClient = FakeInternalStorageClient::instance(env); FakeInternalStorageClient::hookInternalStorageClient(env, iClient); - env->txnMan_->scanPrimes(1, 1); + + for (auto i = 1; i <= mockPartNum; ++i) { + env->txnMan_->scanPrimes(1, i); + } + ChainResumeProcessor resumeProc(env); resumeProc.process(); EXPECT_TRUE(helper.edgeExist(env, req)); EXPECT_FALSE(helper.primeExist(env, req)); EXPECT_FALSE(helper.doublePrimeExist(env, req)); + + delete iClient; } // resume an update left prime, resume failed @@ -341,10 +371,11 @@ TEST(ChainUpdateEdgeTest, resumeTest8) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); - auto parts = cluster.getTotalParts(); + // auto parts = cluster.getTotalParts(); + auto parts = mockPartNum; EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); LOG(INFO) << "Test UpdateEdgeRequest..."; @@ -373,6 +404,8 @@ TEST(ChainUpdateEdgeTest, resumeTest8) { EXPECT_TRUE(helper.edgeExist(env, req)); EXPECT_TRUE(helper.primeExist(env, req)); EXPECT_FALSE(helper.doublePrimeExist(env, req)); + + delete iClient; } // resume an update left prime, resume outdated @@ -381,10 +414,11 @@ TEST(ChainUpdateEdgeTest, resumeTest9) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); - auto parts = cluster.getTotalParts(); + // auto parts = cluster.getTotalParts(); + auto parts = mockPartNum; EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); LOG(INFO) << "Test UpdateEdgeRequest..."; @@ -414,6 +448,8 @@ TEST(ChainUpdateEdgeTest, resumeTest9) { EXPECT_TRUE(helper.edgeExist(env, req)); EXPECT_FALSE(helper.primeExist(env, req)); EXPECT_TRUE(helper.doublePrimeExist(env, req)); + + delete iClient; } // resume an update left prime, check resume succeeded @@ -422,10 +458,11 @@ TEST(ChainUpdateEdgeTest, resumeTest10) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); - auto parts = cluster.getTotalParts(); + // auto parts = cluster.getTotalParts(); + auto parts = mockPartNum; EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); LOG(INFO) << "Test UpdateEdgeRequest..."; @@ -452,6 +489,8 @@ TEST(ChainUpdateEdgeTest, resumeTest10) { EXPECT_TRUE(helper.edgeExist(env, req)); EXPECT_FALSE(helper.primeExist(env, req)); EXPECT_FALSE(helper.doublePrimeExist(env, req)); + + delete iClient; } // resume an update left prime, resume failed @@ -460,10 +499,11 @@ TEST(ChainUpdateEdgeTest, resumeTest11) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); - auto parts = cluster.getTotalParts(); + // auto parts = cluster.getTotalParts(); + auto parts = mockPartNum; EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); LOG(INFO) << "Test UpdateEdgeRequest..."; @@ -491,6 +531,8 @@ TEST(ChainUpdateEdgeTest, resumeTest11) { EXPECT_TRUE(helper.edgeExist(env, req)); EXPECT_FALSE(helper.primeExist(env, req)); EXPECT_TRUE(helper.doublePrimeExist(env, req)); + + delete iClient; } // resume an update left prime, resume outdated @@ -499,10 +541,11 @@ TEST(ChainUpdateEdgeTest, resumeTest12) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); - auto parts = cluster.getTotalParts(); + // auto parts = cluster.getTotalParts(); + auto parts = mockPartNum; EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); LOG(INFO) << "Test UpdateEdgeRequest..."; @@ -530,6 +573,8 @@ TEST(ChainUpdateEdgeTest, resumeTest12) { EXPECT_TRUE(helper.edgeExist(env, req)); EXPECT_FALSE(helper.primeExist(env, req)); EXPECT_TRUE(helper.doublePrimeExist(env, req)); + + delete iClient; } } // namespace storage } // namespace nebula diff --git a/src/storage/test/ChainTestUtils.h b/src/storage/test/ChainTestUtils.h index 13da0bf8473..d94f30b2a74 100644 --- a/src/storage/test/ChainTestUtils.h +++ b/src/storage/test/ChainTestUtils.h @@ -7,8 +7,8 @@ #include "storage/CommonUtils.h" #include "storage/transaction/ChainResumeProcessor.h" -#include "storage/transaction/ChainUpdateEdgeProcessorLocal.h" -#include "storage/transaction/ChainUpdateEdgeProcessorRemote.h" +#include "storage/transaction/ChainUpdateEdgeLocalProcessor.h" +#include "storage/transaction/ChainUpdateEdgeRemoteProcessor.h" namespace nebula { namespace storage { @@ -36,6 +36,38 @@ class ChainTestUtils { }; } + std::vector genEdgeKeys(const cpp2::AddEdgesRequest& req, KeyGenerator gen) { + std::vector ret; + for (auto& partAndEdges : *req.parts_ref()) { + auto partId = partAndEdges.first; + auto& edgeVec = partAndEdges.second; + for (auto& edge : edgeVec) { + auto key = gen(partId, edge); + ret.emplace_back(std::move(key)); + } + } + return ret; + } + + // return the actual num of keys in nebula store. + int32_t checkNumOfKey(StorageEnv* env, + GraphSpaceID spaceId, + const std::vector& keys) { + int32_t ret = 0; + + std::unique_ptr iter; + for (auto& key : keys) { + iter.reset(); + auto partId = NebulaKeyUtils::getPart(key); + auto rc = env->kvstore_->prefix(spaceId, partId, key, &iter); + if (rc == Code::SUCCEEDED && iter && iter->valid()) { + ++ret; + } + } + + return ret; + } + public: int32_t spaceVidLen_{32}; KeyGenerator genKey; @@ -57,6 +89,7 @@ int numOfKey(const cpp2::AddEdgesRequest& req, KeyGenerator gen, StorageEnv* env std::unique_ptr iter; EXPECT_EQ(Code::SUCCEEDED, env->kvstore_->prefix(spaceId, partId, key, &iter)); if (iter && iter->valid()) { + // LOG(INFO) << "key = " << key; ++numOfEdges; } else { // LOG(INFO) << "key: " << key << " not exist"; @@ -82,7 +115,7 @@ bool keyExist(StorageEnv* env, GraphSpaceID spaceId, PartitionID partId, std::st return rc == Code::SUCCEEDED; } -class FakeChainAddEdgesProcessorLocal : public ChainAddEdgesProcessorLocal { +class FakeChainAddEdgesLocalProcessor : public ChainAddEdgesLocalProcessor { FRIEND_TEST(ChainAddEdgesTest, prepareLocalSucceededTest); FRIEND_TEST(ChainAddEdgesTest, processRemoteSucceededTest); FRIEND_TEST(ChainAddEdgesTest, processRemoteFailedTest); @@ -90,44 +123,44 @@ class FakeChainAddEdgesProcessorLocal : public ChainAddEdgesProcessorLocal { // all the above will test succeeded path of process local // the failed path of process local will be tested in resume test public: - explicit FakeChainAddEdgesProcessorLocal(StorageEnv* env) : ChainAddEdgesProcessorLocal(env) { + explicit FakeChainAddEdgesLocalProcessor(StorageEnv* env) : ChainAddEdgesLocalProcessor(env) { spaceVidLen_ = 32; } folly::SemiFuture prepareLocal() override { - LOG(INFO) << "FakeChainAddEdgesProcessorLocal::" << __func__ << "()"; + LOG(INFO) << "FakeChainAddEdgesLocalProcessor::" << __func__ << "()"; if (rcPrepareLocal) { LOG(INFO) << "Fake return " << apache::thrift::util::enumNameSafe(*rcPrepareLocal); return *rcPrepareLocal; } - LOG(INFO) << "forward to ChainAddEdgesProcessorLocal::prepareLocal()"; - return ChainAddEdgesProcessorLocal::prepareLocal(); + LOG(INFO) << "forward to ChainAddEdgesLocalProcessor::prepareLocal()"; + return ChainAddEdgesLocalProcessor::prepareLocal(); } folly::SemiFuture processRemote(Code code) override { - LOG(INFO) << "FakeChainAddEdgesProcessorLocal::" << __func__ << "()"; + LOG(INFO) << "FakeChainAddEdgesLocalProcessor::" << __func__ << "()"; if (rcProcessRemote) { LOG(INFO) << "processRemote() fake return " << apache::thrift::util::enumNameSafe(*rcProcessRemote); LOG_IF(FATAL, code != Code::SUCCEEDED) << "cheat must base on truth"; return *rcProcessRemote; } - LOG(INFO) << "forward to ChainAddEdgesProcessorLocal::processRemote()"; - return ChainAddEdgesProcessorLocal::processRemote(code); + LOG(INFO) << "forward to ChainAddEdgesLocalProcessor::processRemote()"; + return ChainAddEdgesLocalProcessor::processRemote(code); } folly::SemiFuture processLocal(Code code) override { - LOG(INFO) << "FakeChainAddEdgesProcessorLocal::" << __func__ << "()"; + LOG(INFO) << "FakeChainAddEdgesLocalProcessor::" << __func__ << "()"; if (rcProcessLocal) { LOG(INFO) << "Fake return " << apache::thrift::util::enumNameSafe(*rcProcessLocal); return *rcProcessLocal; } - LOG(INFO) << "forward to ChainAddEdgesProcessorLocal::processLocal()"; - return ChainAddEdgesProcessorLocal::processLocal(code); + LOG(INFO) << "forward to ChainAddEdgesLocalProcessor::processLocal()"; + return ChainAddEdgesLocalProcessor::processLocal(code); } cpp2::AddEdgesRequest reverseRequestForward(const cpp2::AddEdgesRequest& req) { - return ChainAddEdgesProcessorLocal::reverseRequest(req); + return ChainAddEdgesLocalProcessor::reverseRequest(req); } folly::Optional rcPrepareLocal; @@ -137,9 +170,9 @@ class FakeChainAddEdgesProcessorLocal : public ChainAddEdgesProcessorLocal { folly::Optional rcProcessLocal; }; -class FakeChainUpdateProcessor : public ChainUpdateEdgeProcessorLocal { +class FakeChainUpdateProcessor : public ChainUpdateEdgeLocalProcessor { public: - explicit FakeChainUpdateProcessor(StorageEnv* env) : ChainUpdateEdgeProcessorLocal(env) { + explicit FakeChainUpdateProcessor(StorageEnv* env) : ChainUpdateEdgeLocalProcessor(env) { spaceVidLen_ = 32; } @@ -149,8 +182,8 @@ class FakeChainUpdateProcessor : public ChainUpdateEdgeProcessorLocal { LOG(INFO) << "Fake return " << apache::thrift::util::enumNameSafe(*rcPrepareLocal); return *rcPrepareLocal; } - LOG(INFO) << "forward to ChainUpdateEdgeProcessorLocal::prepareLocal()"; - return ChainUpdateEdgeProcessorLocal::prepareLocal(); + LOG(INFO) << "forward to ChainUpdateEdgeLocalProcessor::prepareLocal()"; + return ChainUpdateEdgeLocalProcessor::prepareLocal(); } folly::SemiFuture processRemote(Code code) override { @@ -161,8 +194,8 @@ class FakeChainUpdateProcessor : public ChainUpdateEdgeProcessorLocal { LOG_IF(FATAL, code != Code::SUCCEEDED) << "cheat must base on truth"; return *rcProcessRemote; } - LOG(INFO) << "forward to ChainUpdateEdgeProcessorLocal::processRemote()"; - return ChainUpdateEdgeProcessorLocal::processRemote(code); + LOG(INFO) << "forward to ChainUpdateEdgeLocalProcessor::processRemote()"; + return ChainUpdateEdgeLocalProcessor::processRemote(code); } folly::SemiFuture processLocal(Code code) override { @@ -172,8 +205,8 @@ class FakeChainUpdateProcessor : public ChainUpdateEdgeProcessorLocal { << apache::thrift::util::enumNameSafe(*rcProcessLocal); return *rcProcessLocal; } - LOG(INFO) << "forward to ChainUpdateEdgeProcessorLocal::processLocal()"; - return ChainUpdateEdgeProcessorLocal::processLocal(code); + LOG(INFO) << "forward to ChainUpdateEdgeLocalProcessor::processLocal()"; + return ChainUpdateEdgeLocalProcessor::processLocal(code); } void wrapAddUnfinishedEdge(ResumeType type) { @@ -216,7 +249,7 @@ class MetaClientTestUpdater { pCache->termOfPartition_[partId] = termId; } - static std::unique_ptr makeDefaultMetaClient() { + static std::unique_ptr makeDefault() { auto exec = std::make_shared(3); std::vector addrs(1); meta::MetaClientOptions options; @@ -254,7 +287,7 @@ class FakeInternalStorageClient : public InternalStorageClient { chainReq.update_edge_request_ref() = req; chainReq.term_ref() = termOfSrc; - auto* proc = ChainUpdateEdgeProcessorRemote::instance(env_); + auto* proc = ChainUpdateEdgeRemoteProcessor::instance(env_); auto f = proc->getFuture(); proc->process(chainReq); auto resp = std::move(f).get(); @@ -280,9 +313,23 @@ class FakeInternalStorageClient : public InternalStorageClient { UNUSED(evb); } + void chainDeleteEdges(cpp2::DeleteEdgesRequest& req, + const std::string& txnId, + TermID termId, + folly::Promise<::nebula::cpp2::ErrorCode>&& p, + folly::EventBase* evb = nullptr) override { + UNUSED(req); + UNUSED(txnId); + UNUSED(termId); + p.setValue(code_); + UNUSED(evb); + } + static FakeInternalStorageClient* instance(StorageEnv* env, Code fakeCode = Code::SUCCEEDED) { auto pool = std::make_shared(3); return new FakeInternalStorageClient(env, pool, fakeCode); + // static FakeInternalStorageClient client(env, pool, fakeCode); + // return &client; } static void hookInternalStorageClient(StorageEnv* env, InternalStorageClient* client) { @@ -351,7 +398,7 @@ struct ChainUpdateEdgeTestHelper { } cpp2::UpdateEdgeRequest reverseRequest(StorageEnv* env, const cpp2::UpdateEdgeRequest& req) { - ChainUpdateEdgeProcessorLocal proc(env); + ChainUpdateEdgeLocalProcessor proc(env); return proc.reverseRequest(req); } @@ -481,12 +528,12 @@ struct ChainUpdateEdgeTestHelper { // public: // explicit ChainResumeProcessorTestHelper(ChainResumeProcessor* proc) : proc_(proc) {} -// void setAddEdgeProc(ChainAddEdgesProcessorLocal* proc) { +// void setAddEdgeProc(ChainAddEdgesLocalProcessor* proc) { // proc_->addProc = proc; // } // // setUpdProc -// void setUpdProc(ChainUpdateEdgeProcessorLocal* proc) { +// void setUpdProc(ChainUpdateEdgeLocalProcessor* proc) { // proc_->updProc = proc; // } diff --git a/src/storage/test/ChainUpdateEdgeTest.cpp b/src/storage/test/ChainUpdateEdgeTest.cpp index 048bd0fb8b0..6249dac0bdf 100644 --- a/src/storage/test/ChainUpdateEdgeTest.cpp +++ b/src/storage/test/ChainUpdateEdgeTest.cpp @@ -20,9 +20,9 @@ #include "storage/test/QueryTestUtils.h" #include "storage/test/TestUtils.h" #include "storage/transaction/ChainAddEdgesGroupProcessor.h" -#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ChainAddEdgesLocalProcessor.h" #include "storage/transaction/ChainResumeProcessor.h" -#include "storage/transaction/ChainUpdateEdgeProcessorRemote.h" +#include "storage/transaction/ChainUpdateEdgeRemoteProcessor.h" #include "storage/transaction/ConsistUtil.h" namespace nebula { @@ -42,7 +42,7 @@ TEST(ChainUpdateEdgeTest, updateTest1) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); auto parts = cluster.getTotalParts(); @@ -73,7 +73,7 @@ TEST(ChainUpdateEdgeTest, updateTest2) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); auto parts = cluster.getTotalParts(); @@ -105,7 +105,7 @@ TEST(ChainUpdateEdgeTest, updateTest3) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); auto parts = cluster.getTotalParts(); @@ -134,7 +134,7 @@ TEST(ChainUpdateEdgeTest, updateTest4) { mock::MockCluster cluster; cluster.initStorageKV(rootPath.path()); auto* env = cluster.storageEnv_.get(); - auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + auto mClient = MetaClientTestUpdater::makeDefault(); env->metaClient_ = mClient.get(); auto parts = cluster.getTotalParts(); diff --git a/src/storage/test/ElasticSearchBulkInsertTest.cpp b/src/storage/test/ElasticSearchBulkInsertTest.cpp index cb7f3ae9741..94197abe449 100644 --- a/src/storage/test/ElasticSearchBulkInsertTest.cpp +++ b/src/storage/test/ElasticSearchBulkInsertTest.cpp @@ -167,7 +167,7 @@ class ElasticSearchBasicTest : public ::testing::Test { TEST_F(ElasticSearchBasicTest, SimpleTest) { FLAGS_heartbeat_interval_secs = 1; - meta::cpp2::FTClient ftClient; + meta::cpp2::ServiceClient ftClient; ftClient.host_ref() = HostAddr("127.0.0.1", esPort_); const nebula::ClusterID kClusterId = 10; mock::MockCluster cluster; diff --git a/src/storage/test/GetNeighborsTest.cpp b/src/storage/test/GetNeighborsTest.cpp index 3bcd3115442..cd9c5eb6b9c 100644 --- a/src/storage/test/GetNeighborsTest.cpp +++ b/src/storage/test/GetNeighborsTest.cpp @@ -49,6 +49,28 @@ TEST(GetNeighborsTest, PropertyTest) { // vId, stat, player, serve, expr QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 1, 5); } + { + LOG(INFO) << "OneOutEdgeMultiProperty"; + std::vector vertices = {"Tim Duncan"}; + std::vector over = {serve}; + std::vector>> tags; + std::vector>> edges; + tags.emplace_back(player, std::vector{"name", "age", "avgScore"}); + bool retNoneProps = false; + auto req = QueryTestUtils::buildRequest(totalParts, vertices, over, tags, edges, retNoneProps); + + auto* processor = GetNeighborsProcessor::instance(env, nullptr, threadPool.get()); + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + + ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); + // vId, stat, player, expr + size_t expectRowCount = 1; + size_t expectColumnCount = 4; + QueryTestUtils::checkResponse( + *resp.vertices_ref(), vertices, over, tags, edges, expectRowCount, expectColumnCount); + } { LOG(INFO) << "OneOutEdgeKeyInProperty"; std::vector vertices = {"Tim Duncan"}; @@ -164,8 +186,10 @@ TEST(GetNeighborsTest, PropertyTest) { auto resp = std::move(fut).get(); ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); - // vId, stat, player, serve, teammate, expr - QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 1, 6); + // vId, stat, player, expr + size_t expectColumnCount = 4; + QueryTestUtils::checkResponse( + *resp.vertices_ref(), vertices, over, tags, edges, 1, expectColumnCount); } { LOG(INFO) << "InEdgeReturnAllProperty"; @@ -183,8 +207,10 @@ TEST(GetNeighborsTest, PropertyTest) { auto resp = std::move(fut).get(); ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); - // vId, stat, player, - teammate, - serve, expr - QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 1, 6); + // vId, stat, player, expr + size_t expectColumnCount = 4; + QueryTestUtils::checkResponse( + *resp.vertices_ref(), vertices, over, tags, edges, 1, expectColumnCount); } { LOG(INFO) << "InOutEdgeReturnAllProperty"; @@ -203,7 +229,9 @@ TEST(GetNeighborsTest, PropertyTest) { ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); // vId, stat, player, - teammate, - serve, serve, teammate, expr - QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 1, 8); + size_t expectColumnCount = 4; + QueryTestUtils::checkResponse( + *resp.vertices_ref(), vertices, over, tags, edges, 1, expectColumnCount); } { LOG(INFO) << "InEdgeReturnAllProperty"; @@ -222,7 +250,9 @@ TEST(GetNeighborsTest, PropertyTest) { ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); // vId, stat, player, - teammate, - serve, serve, teammate, expr - QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 1, 8); + size_t expectColumnCount = 4; + QueryTestUtils::checkResponse( + *resp.vertices_ref(), vertices, over, tags, edges, 1, expectColumnCount); } { LOG(INFO) << "Nullable"; @@ -385,6 +415,26 @@ TEST(GetNeighborsTest, GoFromMultiVerticesTest) { // vId, stat, player, serve, expr QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 2, 5); } + { + LOG(INFO) << "NoOutEdgeMultiProperty"; + std::vector vertices = {"Tim Duncan", "Tony Parker"}; + std::vector over = {serve}; + std::vector>> tags; + std::vector>> edges; + tags.emplace_back(player, std::vector{"name", "age", "avgScore"}); + auto req = QueryTestUtils::buildRequest(totalParts, vertices, over, tags, edges); + + auto* processor = GetNeighborsProcessor::instance(env, nullptr, threadPool.get()); + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + + ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); + // vId, stat, player, serve, expr + size_t expectColumnCount = 4; + QueryTestUtils::checkResponse( + *resp.vertices_ref(), vertices, over, tags, edges, 2, expectColumnCount); + } { LOG(INFO) << "OneInEdgeMultiProperty"; std::vector vertices = {"Spurs", "Rockets"}; @@ -935,6 +985,7 @@ TEST(GetNeighborsTest, TtlTest) { TagID player = 1; TagID team = 2; EdgeType serve = 101; + EdgeType teammate = 102; { LOG(INFO) << "OutEdgeReturnAllProperty"; @@ -988,9 +1039,10 @@ TEST(GetNeighborsTest, TtlTest) { auto resp = std::move(fut).get(); ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); - // vId, stat, player, team, general tag, - teammate, - serve, + serve, + - // teammate, expr - QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 1, 10); + // vId, stat, expr + size_t expectColsNum = 3; + QueryTestUtils::checkResponse( + *resp.vertices_ref(), vertices, over, tags, edges, 1, expectColsNum); } sleep(FLAGS_mock_ttl_duration + 1); { @@ -1048,7 +1100,7 @@ TEST(GetNeighborsTest, TtlTest) { { LOG(INFO) << "GoFromPlayerOverAll"; std::vector vertices = {"Tim Duncan"}; - std::vector over = {}; + std::vector over = {serve, teammate}; std::vector>> tags; std::vector>> edges; auto req = QueryTestUtils::buildRequest(totalParts, vertices, over, tags, edges); @@ -1060,20 +1112,15 @@ TEST(GetNeighborsTest, TtlTest) { auto resp = std::move(fut).get(); ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); - // vId, stat, player, team, general tag, - teammate, - serve, + serve, + - // teammate, expr + // vId, stat, expr ASSERT_EQ(1, (*resp.vertices_ref()).rows.size()); - ASSERT_EQ(10, (*resp.vertices_ref()).rows[0].values.size()); + ASSERT_EQ(3, (*resp.vertices_ref()).rows[0].values.size()); + for (auto& s : resp.vertices_ref().value().colNames) { + LOG(INFO) << "colName: " << s; + } ASSERT_EQ("Tim Duncan", (*resp.vertices_ref()).rows[0].values[0].getStr()); ASSERT_TRUE((*resp.vertices_ref()).rows[0].values[1].empty()); // stat - ASSERT_TRUE((*resp.vertices_ref()).rows[0].values[2].empty()); // player expired - ASSERT_TRUE((*resp.vertices_ref()).rows[0].values[3].empty()); // team not exists - ASSERT_TRUE((*resp.vertices_ref()).rows[0].values[4].empty()); // general tag not exists - ASSERT_TRUE((*resp.vertices_ref()).rows[0].values[5].isList()); // - teammate valid - ASSERT_TRUE((*resp.vertices_ref()).rows[0].values[6].empty()); // - serve expired - ASSERT_TRUE((*resp.vertices_ref()).rows[0].values[7].empty()); // + serve expired - ASSERT_TRUE((*resp.vertices_ref()).rows[0].values[8].isList()); // + teammate valid - ASSERT_TRUE((*resp.vertices_ref()).rows[0].values[9].empty()); // expr + ASSERT_TRUE((*resp.vertices_ref()).rows[0].values[2].empty()); // expr } FLAGS_mock_ttl_col = false; } @@ -1267,9 +1314,8 @@ TEST(GetNeighborsTest, GoOverAllTest) { auto resp = std::move(fut).get(); ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); - // vId, stat, player, team, general tag, - teammate, - serve, + serve, + - // teammate, expr - QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 1, 10); + // vId, stat, expr + QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 1, 3); } { LOG(INFO) << "GoFromTeamOverAll"; @@ -1286,9 +1332,10 @@ TEST(GetNeighborsTest, GoOverAllTest) { auto resp = std::move(fut).get(); ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); - // vId, stat, player, team, general tag, - teammate, - serve, + serve, + - // teammate, expr - QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 1, 10); + // vId, stat, expr + size_t expectColCnt = 3; + QueryTestUtils::checkResponse( + *resp.vertices_ref(), vertices, over, tags, edges, 1, expectColCnt); } { LOG(INFO) << "GoFromPlayerOverInEdge"; @@ -1305,8 +1352,8 @@ TEST(GetNeighborsTest, GoOverAllTest) { auto resp = std::move(fut).get(); ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); - // vId, stat, player, team, general tag, - serve, - teammate, expr - QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 1, 8); + // vId, stat, expr + QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 1, 3); } { LOG(INFO) << "GoFromPlayerOverOutEdge"; @@ -1323,8 +1370,8 @@ TEST(GetNeighborsTest, GoOverAllTest) { auto resp = std::move(fut).get(); ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); - // vId, stat, player, team, general tag, + serve, + teammate, expr - QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 1, 8); + // vId, stat, expr + QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 1, 3); } { LOG(INFO) << "GoFromMultiPlayerOverAll"; @@ -1341,9 +1388,8 @@ TEST(GetNeighborsTest, GoOverAllTest) { auto resp = std::move(fut).get(); ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); - // vId, stat, player, team, general tag, - teammate, - serve, + serve, + - // teammate, expr - QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 3, 10); + // vId, stat, _expr + QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 3, 3); } { LOG(INFO) << "GoFromMultiTeamOverAll"; @@ -1360,9 +1406,8 @@ TEST(GetNeighborsTest, GoOverAllTest) { auto resp = std::move(fut).get(); ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); - // vId, stat, player, team, general tag, - teammate, - serve, + serve, + - // teammate, expr - QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 3, 10); + // vId, stat, _expr + QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 3, 3); } } @@ -1391,9 +1436,93 @@ TEST(GetNeighborsTest, MultiVersionTest) { auto resp = std::move(fut).get(); ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); - // vId, stat, player, team, general tag, - teammate, - serve, + serve, + - // teammate, expr - QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 1, 10); + // vId, stat, player, team + QueryTestUtils::checkResponse(*resp.vertices_ref(), vertices, over, tags, edges, 1, 3); + } +} + +TEST(GetNeighborsTest, MultiTagNodeTest) { + fs::TempDir rootPath("/tmp/GetNeighborsTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto totalParts = cluster.getTotalParts(); + ASSERT_EQ(true, QueryTestUtils::mockVertexData(env, totalParts)); + ASSERT_EQ(true, QueryTestUtils::mockEdgeData(env, totalParts)); + auto threadPool = std::make_shared(4); + + TagID player = 1; + // TagID team = 2; + EdgeType serve = 101; + + { + LOG(INFO) << "negative no edge prop RelExp "; + std::vector vertices = {"Tracy McGrady"}; + std::vector over = {serve}; + std::vector>> tags; + std::vector>> edges; + tags.emplace_back(player, std::vector{"name", "age", "avgScore"}); + auto req = QueryTestUtils::buildRequest(totalParts, vertices, over, tags, edges); + + { + const auto& exp = *RelationalExpression::makeGT( + pool, + SourcePropertyExpression::make(pool, folly::to(player), "avgScore"), + ConstantExpression::make(pool, Value(9999))); + req.traverse_spec_ref()->filter_ref() = Expression::encode(exp); + } + + auto* processor = GetNeighborsProcessor::instance(env, nullptr, threadPool.get()); + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + + EXPECT_EQ(0, (*resp.result_ref()).failed_parts.size()); + // vId, stat, player, serve, expr + nebula::DataSet expected; + expected.colNames = {kVid, "_stats", "_tag:1:name:age:avgScore", "_expr"}; + // nebula::Row row({"Tracy McGrady", Value()}); + nebula::Row row({""}); + expected.rows.emplace_back(std::move(row)); + EXPECT_EQ(expected.colNames, resp.vertices_ref().value().colNames); + EXPECT_TRUE(resp.vertices_ref().value().rows.empty()); + } + { + LOG(INFO) << "no edge prop RelExp multi result"; + std::vector vertices = { + "Tracy McGrady", "Dwight Howard", "Shaquille O'Neal", "Yao Ming"}; + std::vector over = {serve}; + std::vector>> tags; + std::vector>> edges; + tags.emplace_back(player, std::vector{"name", "age", "avgScore"}); + auto req = QueryTestUtils::buildRequest(totalParts, vertices, over, tags, edges); + + { + const auto& exp = *RelationalExpression::makeGT( + pool, + SourcePropertyExpression::make(pool, folly::to(player), "avgScore"), + ConstantExpression::make(pool, Value(19))); + req.traverse_spec_ref()->filter_ref() = Expression::encode(exp); + } + + auto* processor = GetNeighborsProcessor::instance(env, nullptr, threadPool.get()); + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + + ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); + // vId, stat, player, serve, expr + nebula::DataSet expected; + expected.colNames = {kVid, "_stats", "_tag:1:name:age:avgScore", "_expr"}; + nebula::Row row1( + {"Shaquille O'Neal", Value(), nebula::List({"Shaquille O'Neal", 48, 23.7}), Value()}); + nebula::Row row2( + {"Tracy McGrady", Value(), nebula::List({"Tracy McGrady", 41, 19.6}), Value()}); + expected.rows.emplace_back(std::move(row1)); + expected.rows.emplace_back(std::move(row2)); + ASSERT_EQ(expected.colNames, resp.vertices_ref().value().colNames); + ASSERT_EQ(expected.rows, resp.vertices_ref().value().rows); + ASSERT_EQ(expected, *resp.vertices_ref()); } } @@ -1453,6 +1582,39 @@ TEST(GetNeighborsTest, FilterTest) { expected.rows.emplace_back(std::move(row)); ASSERT_EQ(expected, *resp.vertices_ref()); } + { + LOG(INFO) << "positive no edge prop RelExp"; + std::vector vertices = {"Tracy McGrady"}; + std::vector over = {serve}; + std::vector>> tags; + std::vector>> edges; + tags.emplace_back(player, std::vector{"name", "age", "avgScore"}); + auto req = QueryTestUtils::buildRequest(totalParts, vertices, over, tags, edges); + + { + const auto& exp = *RelationalExpression::makeGT( + pool, + SourcePropertyExpression::make(pool, folly::to(player), "avgScore"), + ConstantExpression::make(pool, Value(18))); + // (*req.traverse_spec_ref()).set_filter(Expression::encode(exp)); + req.traverse_spec_ref()->filter_ref() = Expression::encode(exp); + } + + auto* processor = GetNeighborsProcessor::instance(env, nullptr, threadPool.get()); + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + + ASSERT_EQ(0, (*resp.result_ref()).failed_parts.size()); + // vId, stat, player, serve, expr + nebula::DataSet expected; + expected.colNames = {kVid, "_stats", "_tag:1:name:age:avgScore", "_expr"}; + nebula::Row row({"Tracy McGrady", Value(), nebula::List({"Tracy McGrady", 41, 19.6}), Value()}); + expected.rows.emplace_back(std::move(row)); + EXPECT_EQ(expected.colNames, resp.vertices_ref().value().colNames); + EXPECT_EQ(expected.rows, resp.vertices_ref().value().rows); + EXPECT_EQ(expected, *resp.vertices_ref()); + } { LOG(INFO) << "ArithExpression"; std::vector vertices = {"Tracy McGrady"}; diff --git a/src/storage/transaction/ChainAddEdgesGroupProcessor.cpp b/src/storage/transaction/ChainAddEdgesGroupProcessor.cpp index 56323288c25..eb6c11f64c5 100644 --- a/src/storage/transaction/ChainAddEdgesGroupProcessor.cpp +++ b/src/storage/transaction/ChainAddEdgesGroupProcessor.cpp @@ -7,7 +7,7 @@ #include "storage/StorageFlags.h" #include "storage/mutate/AddEdgesProcessor.h" -#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ChainAddEdgesLocalProcessor.h" #include "storage/transaction/ConsistUtil.h" #include "storage/transaction/TransactionManager.h" @@ -23,7 +23,7 @@ void ChainAddEdgesGroupProcessor::process(const cpp2::AddEdgesRequest& req) { auto delegateProcess = [&](auto& item) { auto localPartId = item.first.first; - auto* proc = ChainAddEdgesProcessorLocal::instance(env_); + auto* proc = ChainAddEdgesLocalProcessor::instance(env_); proc->setRemotePartId(item.first.second); proc->getFuture().thenValue([=](auto&& resp) { auto code = resp.get_result().get_failed_parts().empty() diff --git a/src/storage/transaction/ChainAddEdgesProcessorLocal.cpp b/src/storage/transaction/ChainAddEdgesLocalProcessor.cpp similarity index 82% rename from src/storage/transaction/ChainAddEdgesProcessorLocal.cpp rename to src/storage/transaction/ChainAddEdgesLocalProcessor.cpp index 72e983302f1..0a3eed149db 100644 --- a/src/storage/transaction/ChainAddEdgesProcessorLocal.cpp +++ b/src/storage/transaction/ChainAddEdgesLocalProcessor.cpp @@ -3,7 +3,7 @@ * This source code is licensed under Apache 2.0 License. */ -#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ChainAddEdgesLocalProcessor.h" #include @@ -17,7 +17,7 @@ namespace nebula { namespace storage { -void ChainAddEdgesProcessorLocal::process(const cpp2::AddEdgesRequest& req) { +void ChainAddEdgesLocalProcessor::process(const cpp2::AddEdgesRequest& req) { if (!prepareRequest(req)) { finish(); return; @@ -31,7 +31,7 @@ void ChainAddEdgesProcessorLocal::process(const cpp2::AddEdgesRequest& req) { * 2. set mem lock * 3. write edge prime(key = edge prime, val = ) */ -folly::SemiFuture ChainAddEdgesProcessorLocal::prepareLocal() { +folly::SemiFuture ChainAddEdgesLocalProcessor::prepareLocal() { if (FLAGS_trace_toss) { uuid_ = ConsistUtil::strUUID(); readableEdgeDesc_ = makeReadableEdge(req_); @@ -73,7 +73,7 @@ folly::SemiFuture ChainAddEdgesProcessorLocal::prepareLocal() { return std::move(fut); } -folly::SemiFuture ChainAddEdgesProcessorLocal::processRemote(Code code) { +folly::SemiFuture ChainAddEdgesLocalProcessor::processRemote(Code code) { VLOG(1) << uuid_ << " prepareLocal(), code = " << apache::thrift::util::enumNameSafe(code); if (code != Code::SUCCEEDED) { return code; @@ -86,7 +86,7 @@ folly::SemiFuture ChainAddEdgesProcessorLocal::processRemote(Code code) { return std::move(fut); } -folly::SemiFuture ChainAddEdgesProcessorLocal::processLocal(Code code) { +folly::SemiFuture ChainAddEdgesLocalProcessor::processLocal(Code code) { if (FLAGS_trace_toss) { VLOG(1) << uuid_ << " processRemote(), code = " << apache::thrift::util::enumNameSafe(code); } @@ -103,9 +103,10 @@ folly::SemiFuture ChainAddEdgesProcessorLocal::processLocal(Code code) { code_ = code; } - if (!checkTerm(req_)) { - LOG(WARNING) << "E_OUTDATED_TERM"; - code_ = Code::E_OUTDATED_TERM; + auto currTerm = env_->txnMan_->getTerm(spaceId_, localPartId_); + if (currTerm.first != term_) { + LOG(WARNING) << "E_LEADER_CHANGED during prepare and commit local"; + code_ = Code::E_LEADER_CHANGED; } if (code == Code::E_RPC_FAILURE) { @@ -124,17 +125,17 @@ folly::SemiFuture ChainAddEdgesProcessorLocal::processLocal(Code code) { return code_; } -void ChainAddEdgesProcessorLocal::addUnfinishedEdge(ResumeType type) { +void ChainAddEdgesLocalProcessor::addUnfinishedEdge(ResumeType type) { if (lk_ != nullptr) { lk_->forceUnlock(); } - auto keys = sEdgeKey(req_); + auto keys = toStrKeys(req_); for (auto& key : keys) { env_->txnMan_->addPrime(spaceId_, key, type); } } -bool ChainAddEdgesProcessorLocal::prepareRequest(const cpp2::AddEdgesRequest& req) { +bool ChainAddEdgesLocalProcessor::prepareRequest(const cpp2::AddEdgesRequest& req) { CHECK_EQ(req.get_parts().size(), 1); req_ = req; spaceId_ = req_.get_space_id(); @@ -147,12 +148,12 @@ bool ChainAddEdgesProcessorLocal::prepareRequest(const cpp2::AddEdgesRequest& re } localPartId_ = req.get_parts().begin()->first; replaceNullWithDefaultValue(req_); - auto part = env_->kvstore_->part(spaceId_, localPartId_); - if (!nebula::ok(part)) { - pushResultCode(nebula::error(part), localPartId_); + + std::tie(term_, code_) = env_->txnMan_->getTerm(spaceId_, localPartId_); + if (code_ != Code::SUCCEEDED) { + LOG(INFO) << "get term failed"; return false; } - restrictTerm_ = (nebula::value(part))->termId(); auto vidLen = env_->schemaMan_->getSpaceVidLen(spaceId_); if (!vidLen.ok()) { @@ -165,7 +166,7 @@ bool ChainAddEdgesProcessorLocal::prepareRequest(const cpp2::AddEdgesRequest& re return true; } -folly::SemiFuture ChainAddEdgesProcessorLocal::forwardToDelegateProcessor() { +folly::SemiFuture ChainAddEdgesLocalProcessor::forwardToDelegateProcessor() { auto* proc = AddEdgesProcessor::instance(env_, nullptr); proc->consistOp_ = [&](kvstore::BatchHolder& a, std::vector* b) { callbackOfChainOp(a, b); @@ -198,7 +199,7 @@ folly::SemiFuture ChainAddEdgesProcessorLocal::forwardToDelegateProcessor( return std::move(fut); } -Code ChainAddEdgesProcessorLocal::extractRpcError(const cpp2::ExecResponse& resp) { +Code ChainAddEdgesLocalProcessor::extractRpcError(const cpp2::ExecResponse& resp) { Code ret = Code::SUCCEEDED; auto& respComn = resp.get_result(); for (auto& part : respComn.get_failed_parts()) { @@ -207,7 +208,7 @@ Code ChainAddEdgesProcessorLocal::extractRpcError(const cpp2::ExecResponse& resp return ret; } -void ChainAddEdgesProcessorLocal::doRpc(folly::Promise&& promise, +void ChainAddEdgesLocalProcessor::doRpc(folly::Promise&& promise, cpp2::AddEdgesRequest&& req, int retry) noexcept { if (retry > retryLimit_) { @@ -217,7 +218,7 @@ void ChainAddEdgesProcessorLocal::doRpc(folly::Promise&& promise, auto* iClient = env_->txnMan_->getInternalClient(); folly::Promise p; auto f = p.getFuture(); - iClient->chainAddEdges(req, restrictTerm_, edgeVer_, std::move(p)); + iClient->chainAddEdges(req, term_, edgeVer_, std::move(p)); std::move(f).thenTry([=, p = std::move(promise)](auto&& t) mutable { auto code = t.hasValue() ? t.value() : Code::E_RPC_FAILURE; @@ -233,7 +234,7 @@ void ChainAddEdgesProcessorLocal::doRpc(folly::Promise&& promise, }); } -void ChainAddEdgesProcessorLocal::callbackOfChainOp(kvstore::BatchHolder& batch, +void ChainAddEdgesLocalProcessor::callbackOfChainOp(kvstore::BatchHolder& batch, std::vector* pData) { if (pData != nullptr) { for (auto& kv : *pData) { @@ -248,7 +249,7 @@ void ChainAddEdgesProcessorLocal::callbackOfChainOp(kvstore::BatchHolder& batch, } } -folly::SemiFuture ChainAddEdgesProcessorLocal::abort() { +folly::SemiFuture ChainAddEdgesLocalProcessor::abort() { if (kvErased_.empty()) { return Code::SUCCEEDED; } @@ -279,7 +280,7 @@ folly::SemiFuture ChainAddEdgesProcessorLocal::abort() { return std::move(fut); } -std::vector ChainAddEdgesProcessorLocal::makePrime() { +std::vector ChainAddEdgesLocalProcessor::makePrime() { std::vector ret; for (auto& edge : req_.get_parts().begin()->second) { auto key = ConsistUtil::primeKey(spaceVidLen_, localPartId_, edge.get_key()); @@ -294,7 +295,7 @@ std::vector ChainAddEdgesProcessorLocal::makePrime() { return ret; } -std::vector ChainAddEdgesProcessorLocal::makeDoublePrime() { +std::vector ChainAddEdgesLocalProcessor::makeDoublePrime() { std::vector ret; for (auto& edge : req_.get_parts().begin()->second) { auto key = ConsistUtil::doublePrime(spaceVidLen_, localPartId_, edge.get_key()); @@ -309,7 +310,7 @@ std::vector ChainAddEdgesProcessorLocal::makeDoublePrime() { return ret; } -void ChainAddEdgesProcessorLocal::erasePrime() { +void ChainAddEdgesLocalProcessor::erasePrime() { auto fn = [&](const cpp2::NewEdge& edge) { auto key = ConsistUtil::primeKey(spaceVidLen_, localPartId_, edge.get_key()); return key; @@ -319,17 +320,7 @@ void ChainAddEdgesProcessorLocal::erasePrime() { } } -void ChainAddEdgesProcessorLocal::eraseDoublePrime() { - auto fn = [&](const cpp2::NewEdge& edge) { - auto key = ConsistUtil::doublePrime(spaceVidLen_, localPartId_, edge.get_key()); - return key; - }; - for (auto& edge : req_.get_parts().begin()->second) { - kvErased_.push_back(fn(edge)); - } -} - -bool ChainAddEdgesProcessorLocal::lockEdges(const cpp2::AddEdgesRequest& req) { +bool ChainAddEdgesLocalProcessor::lockEdges(const cpp2::AddEdgesRequest& req) { auto partId = req.get_parts().begin()->first; auto* lockCore = env_->txnMan_->getLockCore(req.get_space_id(), partId); if (!lockCore) { @@ -344,40 +335,7 @@ bool ChainAddEdgesProcessorLocal::lockEdges(const cpp2::AddEdgesRequest& req) { return lk_->isLocked(); } -// we need to check term at both remote phase and local commit -bool ChainAddEdgesProcessorLocal::checkTerm(const cpp2::AddEdgesRequest& req) { - auto space = req.get_space_id(); - auto partId = req.get_parts().begin()->first; - - auto part = env_->kvstore_->part(space, partId); - if (!nebula::ok(part)) { - pushResultCode(nebula::error(part), localPartId_); - return false; - } - auto curTerm = (nebula::value(part))->termId(); - if (restrictTerm_ != curTerm) { - VLOG(1) << folly::sformat( - "check term failed, restrictTerm_={}, currTerm={}", restrictTerm_, curTerm); - return false; - } - return true; -} - -// check if current edge is not newer than the one trying to resume. -// this function only take effect in resume mode -bool ChainAddEdgesProcessorLocal::checkVersion(const cpp2::AddEdgesRequest& req) { - auto part = req.get_parts().begin()->first; - auto sKeys = sEdgeKey(req); - auto currVer = ConsistUtil::getMultiEdgeVers(env_->kvstore_, spaceId_, part, sKeys); - for (auto i = 0U; i != currVer.size(); ++i) { - if (currVer[i] < resumedEdgeVer_) { - return false; - } - } - return true; -} - -std::vector ChainAddEdgesProcessorLocal::sEdgeKey(const cpp2::AddEdgesRequest& req) { +std::vector ChainAddEdgesLocalProcessor::toStrKeys(const cpp2::AddEdgesRequest& req) { std::vector ret; for (auto& edgesOfPart : req.get_parts()) { auto partId = edgesOfPart.first; @@ -388,7 +346,7 @@ std::vector ChainAddEdgesProcessorLocal::sEdgeKey(const cpp2::AddEd return ret; } -cpp2::AddEdgesRequest ChainAddEdgesProcessorLocal::reverseRequest( +cpp2::AddEdgesRequest ChainAddEdgesLocalProcessor::reverseRequest( const cpp2::AddEdgesRequest& req) { cpp2::AddEdgesRequest reversedRequest; for (auto& edgesOfPart : *req.parts_ref()) { @@ -398,20 +356,20 @@ cpp2::AddEdgesRequest ChainAddEdgesProcessorLocal::reverseRequest( ConsistUtil::reverseEdgeKeyInplace(*newEdgeRef.key_ref()); } } - reversedRequest.space_id_ref() = (req.get_space_id()); - reversedRequest.prop_names_ref() = (req.get_prop_names()); - reversedRequest.if_not_exists_ref() = (req.get_if_not_exists()); + reversedRequest.space_id_ref() = req.get_space_id(); + reversedRequest.prop_names_ref() = req.get_prop_names(); + reversedRequest.if_not_exists_ref() = req.get_if_not_exists(); return reversedRequest; } -void ChainAddEdgesProcessorLocal::finish() { +void ChainAddEdgesLocalProcessor::finish() { VLOG(1) << uuid_ << " commitLocal(), code_ = " << apache::thrift::util::enumNameSafe(code_); pushResultCode(code_, localPartId_); finished_.setValue(code_); onFinished(); } -cpp2::AddEdgesRequest ChainAddEdgesProcessorLocal::makeSingleEdgeRequest( +cpp2::AddEdgesRequest ChainAddEdgesLocalProcessor::makeSingleEdgeRequest( PartitionID partId, const cpp2::NewEdge& edge) { cpp2::AddEdgesRequest req; req.space_id_ref() = (req_.get_space_id()); @@ -425,7 +383,7 @@ cpp2::AddEdgesRequest ChainAddEdgesProcessorLocal::makeSingleEdgeRequest( return req; } -int64_t ChainAddEdgesProcessorLocal::toInt(const ::nebula::Value& val) { +int64_t ChainAddEdgesLocalProcessor::toInt(const ::nebula::Value& val) { if (spaceVidType_ == nebula::cpp2::PropertyType::FIXED_STRING) { auto str = val.toString(); if (str.size() < 3) { @@ -439,7 +397,7 @@ int64_t ChainAddEdgesProcessorLocal::toInt(const ::nebula::Value& val) { return 0; } -std::string ChainAddEdgesProcessorLocal::makeReadableEdge(const cpp2::AddEdgesRequest& req) { +std::string ChainAddEdgesLocalProcessor::makeReadableEdge(const cpp2::AddEdgesRequest& req) { if (req.get_parts().size() != 1) { LOG(INFO) << req.get_parts().size(); return ""; @@ -461,6 +419,16 @@ std::string ChainAddEdgesProcessorLocal::makeReadableEdge(const cpp2::AddEdgesRe return oss.str(); } +void ChainAddEdgesLocalProcessor::eraseDoublePrime() { + auto fn = [&](const cpp2::NewEdge& edge) { + auto key = ConsistUtil::doublePrime(spaceVidLen_, localPartId_, edge.get_key()); + return key; + }; + for (auto& edge : req_.get_parts().begin()->second) { + kvErased_.push_back(fn(edge)); + } +} + /*** consider the following case: * * create edge known(kdate datetime default datetime(), degree int); @@ -473,7 +441,7 @@ std::string ChainAddEdgesProcessorLocal::makeReadableEdge(const cpp2::AddEdgesRe * that's why we need to replace the inconsistency prone value * at the moment the request comes * */ -void ChainAddEdgesProcessorLocal::replaceNullWithDefaultValue(cpp2::AddEdgesRequest& req) { +void ChainAddEdgesLocalProcessor::replaceNullWithDefaultValue(cpp2::AddEdgesRequest& req) { auto& edgesOfPart = *req.parts_ref(); if (edgesOfPart.empty()) { return; diff --git a/src/storage/transaction/ChainAddEdgesProcessorLocal.h b/src/storage/transaction/ChainAddEdgesLocalProcessor.h similarity index 89% rename from src/storage/transaction/ChainAddEdgesProcessorLocal.h rename to src/storage/transaction/ChainAddEdgesLocalProcessor.h index 68333e7d210..06695e29677 100644 --- a/src/storage/transaction/ChainAddEdgesProcessorLocal.h +++ b/src/storage/transaction/ChainAddEdgesLocalProcessor.h @@ -14,15 +14,15 @@ namespace nebula { namespace storage { -class ChainAddEdgesProcessorLocal : public BaseProcessor, +class ChainAddEdgesLocalProcessor : public BaseProcessor, public ChainBaseProcessor { friend class ChainResumeProcessorTestHelper; // for test friendly public: - static ChainAddEdgesProcessorLocal* instance(StorageEnv* env) { - return new ChainAddEdgesProcessorLocal(env); + static ChainAddEdgesLocalProcessor* instance(StorageEnv* env) { + return new ChainAddEdgesLocalProcessor(env); } - virtual ~ChainAddEdgesProcessorLocal() = default; + virtual ~ChainAddEdgesLocalProcessor() = default; virtual void process(const cpp2::AddEdgesRequest& req); @@ -39,7 +39,7 @@ class ChainAddEdgesProcessorLocal : public BaseProcessor, void finish() override; protected: - explicit ChainAddEdgesProcessorLocal(StorageEnv* env) : BaseProcessor(env) {} + explicit ChainAddEdgesLocalProcessor(StorageEnv* env) : BaseProcessor(env) {} bool prepareRequest(const cpp2::AddEdgesRequest& req); @@ -53,10 +53,6 @@ class ChainAddEdgesProcessorLocal : public BaseProcessor, bool lockEdges(const cpp2::AddEdgesRequest& req); - bool checkTerm(const cpp2::AddEdgesRequest& req); - - bool checkVersion(const cpp2::AddEdgesRequest& req); - /** * @brief This is a call back function, to let AddEdgesProcessor so some * addition thing for chain operation @@ -68,7 +64,7 @@ class ChainAddEdgesProcessorLocal : public BaseProcessor, /** * @brief helper function to generate string form of keys of request */ - std::vector sEdgeKey(const cpp2::AddEdgesRequest& req); + std::vector toStrKeys(const cpp2::AddEdgesRequest& req); /** * @brief normally, the prime/double prime keys will be deleted at AddEdgeProcessor @@ -134,8 +130,9 @@ class ChainAddEdgesProcessorLocal : public BaseProcessor, cpp2::AddEdgesRequest req_; std::unique_ptr lk_{nullptr}; int retryLimit_{10}; - // need to restrict all the phase in the same term. - TermID restrictTerm_{-1}; + // term at prepareLocal, not allowed to change during execution + TermID term_{-1}; + // set to true when prime insert succeed // in processLocal(), we check this to determine if need to do abort() bool primeInserted_{false}; @@ -145,6 +142,7 @@ class ChainAddEdgesProcessorLocal : public BaseProcessor, folly::Optional edgeVer_{folly::none}; int64_t resumedEdgeVer_{-1}; + // for debug / trace purpose std::string uuid_; // for debug, edge "100"->"101" will print like 2231303022->2231303122 diff --git a/src/storage/transaction/ChainAddEdgesProcessorRemote.cpp b/src/storage/transaction/ChainAddEdgesRemoteProcessor.cpp similarity index 59% rename from src/storage/transaction/ChainAddEdgesProcessorRemote.cpp rename to src/storage/transaction/ChainAddEdgesRemoteProcessor.cpp index df2973c17cc..94dfce48417 100644 --- a/src/storage/transaction/ChainAddEdgesProcessorRemote.cpp +++ b/src/storage/transaction/ChainAddEdgesRemoteProcessor.cpp @@ -3,7 +3,7 @@ * This source code is licensed under Apache 2.0 License. */ -#include "storage/transaction/ChainAddEdgesProcessorRemote.h" +#include "storage/transaction/ChainAddEdgesRemoteProcessor.h" #include "storage/mutate/AddEdgesProcessor.h" #include "storage/transaction/ConsistUtil.h" @@ -12,22 +12,22 @@ namespace nebula { namespace storage { -void ChainAddEdgesProcessorRemote::process(const cpp2::ChainAddEdgesRequest& req) { - if (FLAGS_trace_toss) { - uuid_ = ConsistUtil::strUUID(); - } - VLOG(1) << uuid_ << ConsistUtil::dumpParts(req.get_parts()); - auto partId = req.get_parts().begin()->first; +void ChainAddEdgesRemoteProcessor::process(const cpp2::ChainAddEdgesRequest& req) { + uuid_ = ConsistUtil::strUUID(); + auto spaceId = req.get_space_id(); + auto edgeKey = req.get_parts().begin()->second.back().key(); + auto localPartId = NebulaKeyUtils::getPart(edgeKey->dst_ref()->getStr()); + auto localTerm = req.get_term(); + auto remotePartId = req.get_parts().begin()->first; auto code = nebula::cpp2::ErrorCode::SUCCEEDED; do { - if (!checkTerm(req)) { - LOG(WARNING) << uuid_ << " invalid term, incoming part " << partId + if (!env_->txnMan_->checkTermFromCache(spaceId, localPartId, localTerm)) { + LOG(WARNING) << uuid_ << " invalid term, incoming part " << remotePartId << ", term = " << req.get_term(); code = nebula::cpp2::ErrorCode::E_OUTDATED_TERM; break; } - auto spaceId = req.get_space_id(); auto vIdLen = env_->metaClient_->getSpaceVidLen(spaceId); if (!vIdLen.ok()) { code = Code::E_INVALID_SPACEVIDLEN; @@ -45,19 +45,14 @@ void ChainAddEdgesProcessorRemote::process(const cpp2::ChainAddEdgesRequest& req LOG(INFO) << uuid_ << ", key = " << folly::hexlify(key); } } - forwardRequest(req); + commit(req); } else { - pushResultCode(code, partId); + pushResultCode(code, remotePartId); onFinished(); } } -bool ChainAddEdgesProcessorRemote::checkTerm(const cpp2::ChainAddEdgesRequest& req) { - auto partId = req.get_parts().begin()->first; - return env_->txnMan_->checkTerm(req.get_space_id(), partId, req.get_term()); -} - -void ChainAddEdgesProcessorRemote::forwardRequest(const cpp2::ChainAddEdgesRequest& req) { +void ChainAddEdgesRemoteProcessor::commit(const cpp2::ChainAddEdgesRequest& req) { auto spaceId = req.get_space_id(); auto* proc = AddEdgesProcessor::instance(env_); proc->getFuture().thenValue([=](auto&& resp) { @@ -73,25 +68,7 @@ void ChainAddEdgesProcessorRemote::forwardRequest(const cpp2::ChainAddEdgesReque proc->process(ConsistUtil::toAddEdgesRequest(req)); } -bool ChainAddEdgesProcessorRemote::checkVersion(const cpp2::ChainAddEdgesRequest& req) { - if (!req.edge_version_ref()) { - return true; - } - auto spaceId = req.get_space_id(); - auto partId = req.get_parts().begin()->first; - auto strEdgeKeys = getStrEdgeKeys(req); - auto currVer = ConsistUtil::getMultiEdgeVers(env_->kvstore_, spaceId, partId, strEdgeKeys); - auto edgeVer = *req.edge_version_ref(); - for (auto i = 0U; i != currVer.size(); ++i) { - if (currVer[i] > edgeVer) { - LOG(WARNING) << "currVer[i]=" << currVer[i] << ", edgeVer=" << edgeVer; - return false; - } - } - return true; -} - -std::vector ChainAddEdgesProcessorRemote::getStrEdgeKeys( +std::vector ChainAddEdgesRemoteProcessor::getStrEdgeKeys( const cpp2::ChainAddEdgesRequest& req) { std::vector ret; for (auto& edgesOfPart : req.get_parts()) { diff --git a/src/storage/transaction/ChainAddEdgesProcessorRemote.h b/src/storage/transaction/ChainAddEdgesRemoteProcessor.h similarity index 58% rename from src/storage/transaction/ChainAddEdgesProcessorRemote.h rename to src/storage/transaction/ChainAddEdgesRemoteProcessor.h index 6f67f202bb7..56c8c2bf6fb 100644 --- a/src/storage/transaction/ChainAddEdgesProcessorRemote.h +++ b/src/storage/transaction/ChainAddEdgesRemoteProcessor.h @@ -11,22 +11,18 @@ namespace nebula { namespace storage { -class ChainAddEdgesProcessorRemote : public BaseProcessor { +class ChainAddEdgesRemoteProcessor : public BaseProcessor { public: - static ChainAddEdgesProcessorRemote* instance(StorageEnv* env) { - return new ChainAddEdgesProcessorRemote(env); + static ChainAddEdgesRemoteProcessor* instance(StorageEnv* env) { + return new ChainAddEdgesRemoteProcessor(env); } void process(const cpp2::ChainAddEdgesRequest& req); private: - explicit ChainAddEdgesProcessorRemote(StorageEnv* env) : BaseProcessor(env) {} + explicit ChainAddEdgesRemoteProcessor(StorageEnv* env) : BaseProcessor(env) {} - bool checkTerm(const cpp2::ChainAddEdgesRequest& req); - - bool checkVersion(const cpp2::ChainAddEdgesRequest& req); - - void forwardRequest(const cpp2::ChainAddEdgesRequest& req); + void commit(const cpp2::ChainAddEdgesRequest& req); std::vector getStrEdgeKeys(const cpp2::ChainAddEdgesRequest& req); diff --git a/src/storage/transaction/ChainDeleteEdgesGroupProcessor.cpp b/src/storage/transaction/ChainDeleteEdgesGroupProcessor.cpp new file mode 100644 index 00000000000..760505a41cb --- /dev/null +++ b/src/storage/transaction/ChainDeleteEdgesGroupProcessor.cpp @@ -0,0 +1,75 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "storage/transaction/ChainDeleteEdgesGroupProcessor.h" + +#include "storage/StorageFlags.h" +#include "storage/transaction/ChainDeleteEdgesLocalProcessor.h" +#include "storage/transaction/ConsistUtil.h" +#include "storage/transaction/TransactionManager.h" + +namespace nebula { +namespace storage { +using ChainID = std::pair; +using SplitedRequest = std::unordered_map; + +void ChainDeleteEdgesGroupProcessor::process(const cpp2::DeleteEdgesRequest& req) { + auto spaceId = req.get_space_id(); + auto localPartId = req.get_parts().begin()->first; + auto stSplitRequest = splitRequest(req); + if (!stSplitRequest.ok()) { + // TODO(liuyu): change this when error code done + pushResultCode(Code::E_PART_NOT_FOUND, localPartId); + onFinished(); + } + + SplitedRequest splitedRequest = stSplitRequest.value(); + + callingNum_ = splitedRequest.size(); + + auto fnSplit = [&](auto& request) { + auto* proc = ChainDeleteEdgesLocalProcessor::instance(env_); + proc->getFuture().thenValue([=](auto&& resp) { + auto code = resp.get_result().get_failed_parts().empty() + ? nebula::cpp2::ErrorCode::SUCCEEDED + : resp.get_result().get_failed_parts().begin()->get_code(); + handleAsync(spaceId, localPartId, code); + }); + proc->process(request.second); + }; + + std::for_each(splitedRequest.begin(), splitedRequest.end(), fnSplit); +} + +StatusOr ChainDeleteEdgesGroupProcessor::splitRequest( + const cpp2::DeleteEdgesRequest& req) { + SplitedRequest ret; + auto numOfPart = env_->metaClient_->partsNum(req.get_space_id()); + if (!numOfPart.ok()) { + return numOfPart.status(); + } + auto partNum = numOfPart.value(); + + for (auto& onePart : req.get_parts()) { + auto localPartId = onePart.first; + for (auto& edgeKey : onePart.second) { + auto& remoteVid = edgeKey.get_dst().getStr(); + auto remotePartId = env_->metaClient_->partId(partNum, remoteVid); + auto key = std::make_pair(localPartId, remotePartId); + if (ret.count(key) == 0) { + ret[key].space_id_ref() = req.get_space_id(); + if (req.common_ref()) { + ret[key].common_ref() = req.common_ref().value(); + } + } + ret[key].parts_ref().value()[localPartId].emplace_back(edgeKey); + } + } + + return ret; +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainDeleteEdgesGroupProcessor.h b/src/storage/transaction/ChainDeleteEdgesGroupProcessor.h new file mode 100644 index 00000000000..ea195b700fc --- /dev/null +++ b/src/storage/transaction/ChainDeleteEdgesGroupProcessor.h @@ -0,0 +1,34 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#pragma once + +#include "storage/BaseProcessor.h" +#include "storage/transaction/ChainBaseProcessor.h" +#include "storage/transaction/ConsistUtil.h" +#include "storage/transaction/TransactionManager.h" + +namespace nebula { +namespace storage { + +class ChainDeleteEdgesGroupProcessor : public BaseProcessor { + public: + static ChainDeleteEdgesGroupProcessor* instance(StorageEnv* env) { + return new ChainDeleteEdgesGroupProcessor(env); + } + + void process(const cpp2::DeleteEdgesRequest& req); + + protected: + explicit ChainDeleteEdgesGroupProcessor(StorageEnv* env) + : BaseProcessor(env) {} + + using ChainID = std::pair; + using SplitedRequest = std::unordered_map; + StatusOr splitRequest(const cpp2::DeleteEdgesRequest& src); +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainDeleteEdgesLocalProcessor.cpp b/src/storage/transaction/ChainDeleteEdgesLocalProcessor.cpp new file mode 100644 index 00000000000..f9c9e1951b8 --- /dev/null +++ b/src/storage/transaction/ChainDeleteEdgesLocalProcessor.cpp @@ -0,0 +1,360 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "storage/transaction/ChainDeleteEdgesLocalProcessor.h" + +#include + +#include "common/utils/DefaultValueContext.h" +#include "kvstore/Part.h" +#include "storage/StorageFlags.h" +#include "storage/mutate/DeleteEdgesProcessor.h" +#include "storage/transaction/ConsistUtil.h" +#include "storage/transaction/TransactionManager.h" + +namespace nebula { +namespace storage { + +void ChainDeleteEdgesLocalProcessor::process(const cpp2::DeleteEdgesRequest& req) { + auto rc = checkRequest(req); + if (rc != Code::SUCCEEDED) { + pushResultCode(rc, localPartId_); + finish(); + return; + } + env_->txnMan_->addChainTask(this); +} + +folly::SemiFuture ChainDeleteEdgesLocalProcessor::prepareLocal() { + txnId_ = ConsistUtil::strUUID(); + VLOG(1) << txnId_ << " prepareLocal(): " << DeleteEdgesRequestHelper::explain(req_); + + if (!lockEdges(req_)) { + return Code::E_WRITE_WRITE_CONFLICT; + } + + primes_ = makePrime(req_); + + std::vector primes(primes_); + + auto [pro, fut] = folly::makePromiseContract(); + env_->kvstore_->asyncMultiPut( + spaceId_, localPartId_, std::move(primes), [p = std::move(pro), this](auto rc) mutable { + if (rc == nebula::cpp2::ErrorCode::SUCCEEDED) { + setPrime_ = true; + } else { + LOG(WARNING) << txnId_ << "kvstore err: " << apache::thrift::util::enumNameSafe(rc); + } + + p.setValue(rc); + }); + return std::move(fut); +} + +folly::SemiFuture ChainDeleteEdgesLocalProcessor::processRemote(Code code) { + VLOG(1) << txnId_ << " prepareLocal(), code = " << apache::thrift::util::enumNameSafe(code); + if (code != Code::SUCCEEDED) { + return code; + } + DCHECK_EQ(req_.get_parts().size(), 1); + auto reversedRequest = reverseRequest(req_); + DCHECK_EQ(reversedRequest.get_parts().size(), 1); + auto [pro, fut] = folly::makePromiseContract(); + doRpc(std::move(pro), std::move(reversedRequest)); + return std::move(fut); +} + +folly::SemiFuture ChainDeleteEdgesLocalProcessor::processLocal(Code code) { + VLOG(1) << txnId_ << " processRemote(), code = " << apache::thrift::util::enumNameSafe(code); + + bool remoteFailed{false}; + if (code == Code::SUCCEEDED) { + // do nothing + } else if (code == Code::E_RPC_FAILURE) { + code_ = Code::SUCCEEDED; + } else { + code_ = code; + remoteFailed = true; + } + + auto [currTerm, suc] = env_->txnMan_->getTerm(spaceId_, localPartId_); + if (currTerm != term_) { + LOG(WARNING) << "E_LEADER_CHANGED during prepare and commit local"; + code_ = Code::E_LEADER_CHANGED; + } + + if (code == Code::E_RPC_FAILURE) { + for (auto& kv : primes_) { + auto key = + ConsistUtil::doublePrimeTable().append(kv.first.substr(ConsistUtil::primeTable().size())); + setDoublePrime_ = true; + doublePrimes_.emplace_back(key, kv.second); + } + reportFailed(ResumeType::RESUME_REMOTE); + } + + if (code_ == Code::SUCCEEDED) { + return commitLocal(); + } else { + if (setPrime_ && remoteFailed) { + return abort(); + } + } + + return code_; +} + +void ChainDeleteEdgesLocalProcessor::reportFailed(ResumeType type) { + if (lk_ != nullptr) { + lk_->forceUnlock(); + } + for (auto& edgesOfPart : req_.get_parts()) { + auto partId = edgesOfPart.first; + for (auto& key : edgesOfPart.second) { + auto strKey = ConsistUtil::edgeKey(spaceVidLen_, partId, key); + env_->txnMan_->addPrime(spaceId_, strKey, type); + } + } +} + +std::vector ChainDeleteEdgesLocalProcessor::makePrime( + const cpp2::DeleteEdgesRequest& req) { + std::vector ret; + std::vector requests; + + for (auto& partOfKeys : req.get_parts()) { + auto partId = partOfKeys.first; + for (auto& key : partOfKeys.second) { + requests.emplace_back(); + requests.back().space_id_ref() = req_.get_space_id(); + std::unordered_map> parts; + parts[partId].emplace_back(key); + requests.back().parts_ref() = parts; + requests.back().common_ref().copy_from(req_.common_ref()); + } + } + + for (auto& singleReq : requests) { + std::string val; + apache::thrift::CompactSerializer::serialize(singleReq, &val); + val += ConsistUtil::deleteIdentifier(); + auto partId = singleReq.get_parts().begin()->first; + auto& edgeKey = singleReq.get_parts().begin()->second.back(); + auto key = ConsistUtil::primeTable(); + key += ConsistUtil::edgeKey(spaceVidLen_, partId, edgeKey); + ret.emplace_back(std::make_pair(key, val)); + } + return ret; +} + +Code ChainDeleteEdgesLocalProcessor::checkRequest(const cpp2::DeleteEdgesRequest& req) { + CHECK_EQ(req.get_parts().size(), 1); + req_ = req; + DCHECK(!req_.get_parts().empty()); + spaceId_ = req_.get_space_id(); + + auto vidType = env_->metaClient_->getSpaceVidType(spaceId_); + if (!vidType.ok()) { + LOG(WARNING) << "can't get vidType, spaceId_ = " << spaceId_; + return Code::E_SPACE_NOT_FOUND; + } else { + spaceVidType_ = vidType.value(); + } + localPartId_ = req.get_parts().begin()->first; + auto part = env_->kvstore_->part(spaceId_, localPartId_); + if (!nebula::ok(part)) { + pushResultCode(nebula::error(part), localPartId_); + return Code::E_SPACE_NOT_FOUND; + } + auto stPartNum = env_->metaClient_->partsNum(spaceId_); + if (!stPartNum.ok()) { + pushResultCode(nebula::error(part), localPartId_); + return Code::E_PART_NOT_FOUND; + } + + auto& oneEdgeKey = req.get_parts().begin()->second.front(); + auto& remoteVid = oneEdgeKey.get_dst().getStr(); + remotePartId_ = env_->metaClient_->partId(stPartNum.value(), remoteVid); + + term_ = (nebula::value(part))->termId(); + + auto vidLen = env_->schemaMan_->getSpaceVidLen(spaceId_); + if (!vidLen.ok()) { + LOG(ERROR) << "getSpaceVidLen failed, spaceId_: " << spaceId_ + << ", status: " << vidLen.status(); + return Code::E_INVALID_SPACEVIDLEN; + } + spaceVidLen_ = vidLen.value(); + return Code::SUCCEEDED; +} + +folly::SemiFuture ChainDeleteEdgesLocalProcessor::commitLocal() { + auto* proc = DeleteEdgesProcessor::instance(env_, nullptr); + auto fn = std::bind(&ChainDeleteEdgesLocalProcessor::hookFunc, this, std::placeholders::_1); + proc->setHookFunc(fn); + + auto futProc = proc->getFuture(); + auto [pro, fut] = folly::makePromiseContract(); + std::move(futProc).thenValue([&, p = std::move(pro)](auto&& resp) mutable { + auto rc = ConsistUtil::getErrorCode(resp); + VLOG(1) << txnId_ << " commitLocal() " << apache::thrift::util::enumNameSafe(rc); + if (rc == Code::SUCCEEDED) { + // do nothing + } else { + reportFailed(ResumeType::RESUME_CHAIN); + } + p.setValue(rc); + }); + proc->process(req_); + return std::move(fut); +} + +void ChainDeleteEdgesLocalProcessor::doRpc(folly::Promise&& promise, + cpp2::DeleteEdgesRequest&& req, + int retry) noexcept { + if (retry > retryLimit_) { + promise.setValue(Code::E_LEADER_CHANGED); + return; + } + auto* iClient = env_->txnMan_->getInternalClient(); + folly::Promise p; + auto f = p.getFuture(); + iClient->chainDeleteEdges(req, txnId_, term_, std::move(p)); + + std::move(f).thenTry([=, p = std::move(promise)](auto&& t) mutable { + auto code = t.hasValue() ? t.value() : Code::E_RPC_FAILURE; + switch (code) { + case Code::E_LEADER_CHANGED: + doRpc(std::move(p), std::move(req), ++retry); + break; + default: + p.setValue(code); + break; + } + return code; + }); +} + +/** + * @brief input para may be varies according to if the edge has index + * if yes, DeleteEdgeProcessor will use batch, + * else it will use a simple vector of keys + * |-------------------|--------------------------|------------------------------------| + * | | input keys | input a batch | + * |-------------------|--------------------------|------------------------------------| + * | double prime (N) | del edge, prime keys | bat.remove(prime) | + * | double prime (Y) | transform to batchHolder | bat.remove(prime) & put(double p.) | + */ +void ChainDeleteEdgesLocalProcessor::hookFunc(HookFuncPara& para) { + std::string ret; + + if (setDoublePrime_) { + if (para.keys) { + kvstore::BatchHolder bat; + for (auto& edgeKey : *para.keys.value()) { + bat.remove(std::string(edgeKey)); + } + for (auto& kv : primes_) { + bat.remove(std::string(kv.first)); + } + for (auto& kv : doublePrimes_) { + bat.put(std::string(kv.first), std::string(kv.second)); + } + para.result.emplace(kvstore::encodeBatchValue(bat.getBatch())); + } else if (para.batch) { + for (auto& kv : primes_) { + para.batch.value()->remove(std::string(kv.first)); + } + for (auto& kv : doublePrimes_) { + para.batch.value()->put(std::string(kv.first), std::string(kv.second)); + } + } else { + LOG(ERROR) << "not supposed runs here"; + } + } else { // there is no double prime + if (para.keys) { + for (auto& kv : primes_) { + para.keys.value()->emplace_back(kv.first); + } + } else if (para.batch) { + for (auto& kv : primes_) { + para.batch.value()->remove(std::string(kv.first)); + } + } else { + LOG(ERROR) << "not supposed runs here"; + } + } +} + +folly::SemiFuture ChainDeleteEdgesLocalProcessor::abort() { + if (setPrime_) { + return Code::SUCCEEDED; + } + + std::vector keyRemoved; + for (auto& key : primes_) { + keyRemoved.emplace_back(key.first); + } + + auto [pro, fut] = folly::makePromiseContract(); + env_->kvstore_->asyncMultiRemove( + req_.get_space_id(), + localPartId_, + std::move(keyRemoved), + [p = std::move(pro), this](auto rc) mutable { + VLOG(1) << txnId_ << " abort()=" << apache::thrift::util::enumNameSafe(rc); + if (rc == Code::SUCCEEDED) { + // do nothing + } else { + reportFailed(ResumeType::RESUME_CHAIN); + } + p.setValue(rc); + }); + return std::move(fut); +} + +bool ChainDeleteEdgesLocalProcessor::lockEdges(const cpp2::DeleteEdgesRequest& req) { + auto* lockCore = env_->txnMan_->getLockCore(req.get_space_id(), localPartId_); + if (!lockCore) { + VLOG(1) << txnId_ << "get lock failed."; + return false; + } + + std::vector keys; + for (auto& key : req.get_parts().begin()->second) { + auto eKey = ConsistUtil::edgeKey(spaceVidLen_, localPartId_, key); + keys.emplace_back(std::move(eKey)); + } + bool dedup = true; + lk_ = std::make_unique(lockCore, keys, dedup); + if (!lk_->isLocked()) { + VLOG(1) << txnId_ << " conflict " << ConsistUtil::readableKey(spaceVidLen_, lk_->conflictKey()); + } + return lk_->isLocked(); +} + +cpp2::DeleteEdgesRequest ChainDeleteEdgesLocalProcessor::reverseRequest( + const cpp2::DeleteEdgesRequest& req) { + cpp2::DeleteEdgesRequest reversedRequest; + reversedRequest.space_id_ref() = req.get_space_id(); + reversedRequest.common_ref().copy_from(req.common_ref()); + for (auto& keysOfPart : *req.parts_ref()) { + for (auto& edgeKey : keysOfPart.second) { + auto rEdgeKey = ConsistUtil::reverseEdgeKey(edgeKey); + (*reversedRequest.parts_ref())[remotePartId_].emplace_back(rEdgeKey); + } + } + return reversedRequest; +} + +void ChainDeleteEdgesLocalProcessor::finish() { + VLOG(1) << txnId_ << " commitLocal(), code_ = " << apache::thrift::util::enumNameSafe(code_); + pushResultCode(code_, localPartId_); + finished_.setValue(code_); + onFinished(); +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainDeleteEdgesLocalProcessor.h b/src/storage/transaction/ChainDeleteEdgesLocalProcessor.h new file mode 100644 index 00000000000..2c4f467b3d3 --- /dev/null +++ b/src/storage/transaction/ChainDeleteEdgesLocalProcessor.h @@ -0,0 +1,118 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#pragma once + +#include "interface/gen-cpp2/common_types.h" +#include "interface/gen-cpp2/storage_types.h" +#include "kvstore/LogEncoder.h" +#include "storage/BaseProcessor.h" +#include "storage/transaction/ConsistUtil.h" +#include "storage/transaction/TransactionManager.h" + +namespace nebula { +namespace storage { + +class ChainDeleteEdgesLocalProcessor : public BaseProcessor, + public ChainBaseProcessor { + friend class ChainResumeProcessorTestHelper; // for test friendly + public: + static ChainDeleteEdgesLocalProcessor* instance(StorageEnv* env) { + return new ChainDeleteEdgesLocalProcessor(env); + } + + virtual ~ChainDeleteEdgesLocalProcessor() = default; + + virtual void process(const cpp2::DeleteEdgesRequest& req); + + folly::SemiFuture prepareLocal() override; + + folly::SemiFuture processRemote(Code code) override; + + folly::SemiFuture processLocal(Code code) override; + + void finish() override; + + protected: + explicit ChainDeleteEdgesLocalProcessor(StorageEnv* env) + : BaseProcessor(env) {} + + Code checkRequest(const cpp2::DeleteEdgesRequest& req); + + void doRpc(folly::Promise&& pro, cpp2::DeleteEdgesRequest&& req, int retry = 0) noexcept; + + bool lockEdges(const cpp2::DeleteEdgesRequest& req); + + /** + * @brief This is a hook function, inject to DeleteEdgesProcessor, + * called before DeleteEdgesProcessor ready to commit something + */ + void hookFunc(HookFuncPara& para); + + /** + * @brief if remote side explicit reported faild, called this + */ + folly::SemiFuture abort(); + + /** + * @brief call DeleteEdgesProcessor to do the real thing + */ + folly::SemiFuture commitLocal(); + + std::vector makePrime(const cpp2::DeleteEdgesRequest& req); + + /** + * @brief generate reversed request of the incoming req. + */ + cpp2::DeleteEdgesRequest reverseRequest(const cpp2::DeleteEdgesRequest& req); + + /** + * @brief wrapper function to get error code from ExecResponse + */ + Code extractRpcError(const cpp2::ExecResponse& resp); + + /** + * @brief if any operation failed or can not determined(RPC error) + * call this to leave a record in transaction manager + * the record can be scanned by the background resume thread + * then will do fail over logic + */ + void reportFailed(ResumeType type); + + protected: + GraphSpaceID spaceId_; + PartitionID localPartId_; + PartitionID remotePartId_; + cpp2::DeleteEdgesRequest req_; + std::unique_ptr lk_{nullptr}; + int retryLimit_{10}; + /** + * @brief this is the term when prepare called, + * and must be kept during the whole execution + * if not, will return OUT_OF_TERM ERROR + */ + TermID term_{-1}; + + // set to true when prime insert succeed + // in processLocal(), we check this to determine if need to do abort() + bool setPrime_{false}; + + bool setDoublePrime_{false}; + + std::vector primes_; + + std::vector doublePrimes_; + + std::string txnId_; + + ::nebula::cpp2::PropertyType spaceVidType_{::nebula::cpp2::PropertyType::UNKNOWN}; + + // for debug, edge "100"->"101" will print like 2231303022->2231303122 + // which is hard to recognize. Transform to human readable format + std::string readableEdgeDesc_; +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainDeleteEdgesRemoteProcessor.cpp b/src/storage/transaction/ChainDeleteEdgesRemoteProcessor.cpp new file mode 100644 index 00000000000..0311019f23f --- /dev/null +++ b/src/storage/transaction/ChainDeleteEdgesRemoteProcessor.cpp @@ -0,0 +1,72 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "storage/transaction/ChainDeleteEdgesRemoteProcessor.h" + +#include "storage/StorageFlags.h" +#include "storage/mutate/DeleteEdgesProcessor.h" +#include "storage/transaction/ConsistUtil.h" +#include "storage/transaction/TransactionManager.h" + +namespace nebula { +namespace storage { + +void ChainDeleteEdgesRemoteProcessor::process(const cpp2::ChainDeleteEdgesRequest& chianReq) { + txnId_ = chianReq.get_txn_id(); + cpp2::DeleteEdgesRequest req = DeleteEdgesRequestHelper::toDeleteEdgesRequest(chianReq); + auto term = chianReq.get_term(); + txnId_ = chianReq.get_txn_id(); + auto partId = req.get_parts().begin()->first; + auto code = nebula::cpp2::ErrorCode::SUCCEEDED; + do { + auto spaceId = req.get_space_id(); + if (!env_->txnMan_->checkTermFromCache(spaceId, partId, term)) { + LOG(WARNING) << txnId_ << "outdate term, incoming part " << partId << ", term = " << term; + code = nebula::cpp2::ErrorCode::E_OUTDATED_TERM; + break; + } + + auto vIdLen = env_->metaClient_->getSpaceVidLen(spaceId); + if (!vIdLen.ok()) { + code = Code::E_INVALID_SPACEVIDLEN; + break; + } else { + spaceVidLen_ = vIdLen.value(); + } + } while (0); + + if (code == nebula::cpp2::ErrorCode::SUCCEEDED) { + if (FLAGS_trace_toss) { + // need to do this after set spaceVidLen_ + auto keys = ConsistUtil::toStrKeys(req, spaceVidLen_); + for (auto& key : keys) { + VLOG(1) << txnId_ << ", key = " << folly::hexlify(key); + } + } + commit(req); + } else { + pushResultCode(code, partId); + onFinished(); + } +} + +void ChainDeleteEdgesRemoteProcessor::commit(const cpp2::DeleteEdgesRequest& req) { + auto spaceId = req.get_space_id(); + auto* proc = DeleteEdgesProcessor::instance(env_); + proc->getFuture().thenValue([=](auto&& resp) { + Code rc = Code::SUCCEEDED; + for (auto& part : resp.get_result().get_failed_parts()) { + rc = part.code; + handleErrorCode(part.code, spaceId, part.get_part_id()); + } + VLOG(1) << txnId_ << " " << apache::thrift::util::enumNameSafe(rc); + this->result_ = resp.get_result(); + this->onFinished(); + }); + proc->process(req); +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainDeleteEdgesRemoteProcessor.h b/src/storage/transaction/ChainDeleteEdgesRemoteProcessor.h new file mode 100644 index 00000000000..1c142b6ac59 --- /dev/null +++ b/src/storage/transaction/ChainDeleteEdgesRemoteProcessor.h @@ -0,0 +1,35 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#pragma once + +#include "storage/BaseProcessor.h" +#include "storage/transaction/ChainBaseProcessor.h" + +namespace nebula { +namespace storage { + +class ChainDeleteEdgesRemoteProcessor : public BaseProcessor { + public: + static ChainDeleteEdgesRemoteProcessor* instance(StorageEnv* env) { + return new ChainDeleteEdgesRemoteProcessor(env); + } + + void process(const cpp2::ChainDeleteEdgesRequest& req); + + private: + explicit ChainDeleteEdgesRemoteProcessor(StorageEnv* env) + : BaseProcessor(env) {} + + void commit(const cpp2::DeleteEdgesRequest& req); + + cpp2::DeleteEdgesRequest toDeleteEdgesRequest(const cpp2::ChainDeleteEdgesRequest& req); + + private: + std::string txnId_; +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainDeleteEdgesResumeProcessor.cpp b/src/storage/transaction/ChainDeleteEdgesResumeProcessor.cpp new file mode 100644 index 00000000000..19698798cff --- /dev/null +++ b/src/storage/transaction/ChainDeleteEdgesResumeProcessor.cpp @@ -0,0 +1,59 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "storage/transaction/ChainDeleteEdgesResumeProcessor.h" + +#include +#include + +#include "storage/StorageFlags.h" + +namespace nebula { +namespace storage { + +ChainDeleteEdgesResumeProcessor::ChainDeleteEdgesResumeProcessor(StorageEnv* env, + const std::string& val) + : ChainDeleteEdgesLocalProcessor(env) { + req_ = DeleteEdgesRequestHelper::parseDeleteEdgesRequest(val); + + VLOG(1) << "explain req_: " << DeleteEdgesRequestHelper::explain(req_); +} + +folly::SemiFuture ChainDeleteEdgesResumeProcessor::prepareLocal() { + code_ = checkRequest(req_); + primes_ = makePrime(req_); + setPrime_ = true; + return code_; +} + +folly::SemiFuture ChainDeleteEdgesResumeProcessor::processRemote(Code code) { + VLOG(1) << txnId_ << " prepareLocal() " << apache::thrift::util::enumNameSafe(code); + return ChainDeleteEdgesLocalProcessor::processRemote(code); +} + +folly::SemiFuture ChainDeleteEdgesResumeProcessor::processLocal(Code code) { + VLOG(1) << txnId_ << " processRemote() " << apache::thrift::util::enumNameSafe(code); + setErrorCode(code); + + if (code == Code::E_RPC_FAILURE) { + for (auto& kv : primes_) { + auto key = + ConsistUtil::doublePrimeTable().append(kv.first.substr(ConsistUtil::primeTable().size())); + doublePrimes_.emplace_back(key, kv.second); + } + } + + if (code == Code::E_RPC_FAILURE || code == Code::SUCCEEDED) { + // if there are something wrong other than rpc failure + // we need to keep the resume retry(by not remove those prime key) + code_ = commitLocal().get(); + return code_; + } + + return code_; +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainDeleteEdgesResumeProcessor.h b/src/storage/transaction/ChainDeleteEdgesResumeProcessor.h new file mode 100644 index 00000000000..d7ff4060260 --- /dev/null +++ b/src/storage/transaction/ChainDeleteEdgesResumeProcessor.h @@ -0,0 +1,32 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#pragma once + +#include "storage/transaction/ChainDeleteEdgesLocalProcessor.h" + +namespace nebula { +namespace storage { + +class ChainDeleteEdgesResumeProcessor : public ChainDeleteEdgesLocalProcessor { + public: + static ChainDeleteEdgesResumeProcessor* instance(StorageEnv* env, const std::string& val) { + return new ChainDeleteEdgesResumeProcessor(env, val); + } + + folly::SemiFuture prepareLocal() override; + + folly::SemiFuture processRemote(nebula::cpp2::ErrorCode code) override; + + folly::SemiFuture processLocal(nebula::cpp2::ErrorCode code) override; + + virtual ~ChainDeleteEdgesResumeProcessor() = default; + + protected: + ChainDeleteEdgesResumeProcessor(StorageEnv* env, const std::string& val); +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainDeleteEdgesResumeRemoteProcessor.cpp b/src/storage/transaction/ChainDeleteEdgesResumeRemoteProcessor.cpp new file mode 100644 index 00000000000..a0e0cdbc84f --- /dev/null +++ b/src/storage/transaction/ChainDeleteEdgesResumeRemoteProcessor.cpp @@ -0,0 +1,65 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#include "storage/transaction/ChainDeleteEdgesResumeRemoteProcessor.h" + +#include "storage/StorageFlags.h" + +namespace nebula { +namespace storage { + +ChainDeleteEdgesResumeRemoteProcessor::ChainDeleteEdgesResumeRemoteProcessor(StorageEnv* env, + const std::string& val) + : ChainDeleteEdgesLocalProcessor(env) { + req_ = DeleteEdgesRequestHelper::parseDeleteEdgesRequest(val); +} + +folly::SemiFuture ChainDeleteEdgesResumeRemoteProcessor::prepareLocal() { + code_ = checkRequest(req_); + return code_; +} + +folly::SemiFuture ChainDeleteEdgesResumeRemoteProcessor::processRemote(Code code) { + VLOG(1) << txnId_ << " prepareLocal() " << apache::thrift::util::enumNameSafe(code); + + return ChainDeleteEdgesLocalProcessor::processRemote(code); +} + +folly::SemiFuture ChainDeleteEdgesResumeRemoteProcessor::processLocal(Code code) { + VLOG(1) << txnId_ << " processRemote() " << apache::thrift::util::enumNameSafe(code); + + setErrorCode(code); + + if (code == Code::E_RPC_FAILURE) { + return code_; + } + + if (code == Code::SUCCEEDED) { + // if there are something wrong other than rpc failure + // we need to keep the resume retry(by not remove double prime key) + std::vector doublePrimeKeys; + for (auto& partOfKeys : req_.get_parts()) { + std::string key; + for (auto& edgeKey : partOfKeys.second) { + doublePrimeKeys.emplace_back(); + doublePrimeKeys.back() = ConsistUtil::doublePrimeTable().append( + ConsistUtil::edgeKey(spaceVidLen_, localPartId_, edgeKey)); + } + } + + folly::Baton baton; + env_->kvstore_->asyncMultiRemove( + spaceId_, localPartId_, std::move(doublePrimeKeys), [this, &baton](auto&& rc) { + this->code_ = rc; + baton.post(); + }); + baton.wait(); + } + + return code_; +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainDeleteEdgesResumeRemoteProcessor.h b/src/storage/transaction/ChainDeleteEdgesResumeRemoteProcessor.h new file mode 100644 index 00000000000..31c091f5962 --- /dev/null +++ b/src/storage/transaction/ChainDeleteEdgesResumeRemoteProcessor.h @@ -0,0 +1,32 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#pragma once + +#include "storage/transaction/ChainDeleteEdgesLocalProcessor.h" + +namespace nebula { +namespace storage { + +class ChainDeleteEdgesResumeRemoteProcessor : public ChainDeleteEdgesLocalProcessor { + public: + static ChainDeleteEdgesResumeRemoteProcessor* instance(StorageEnv* env, const std::string& val) { + return new ChainDeleteEdgesResumeRemoteProcessor(env, val); + } + + folly::SemiFuture prepareLocal() override; + + folly::SemiFuture processRemote(nebula::cpp2::ErrorCode code) override; + + folly::SemiFuture processLocal(nebula::cpp2::ErrorCode code) override; + + virtual ~ChainDeleteEdgesResumeRemoteProcessor() = default; + + protected: + ChainDeleteEdgesResumeRemoteProcessor(StorageEnv* env, const std::string& val); +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainProcessorFactory.cpp b/src/storage/transaction/ChainProcessorFactory.cpp index 16d1dbf656e..b8a60e60e4d 100644 --- a/src/storage/transaction/ChainProcessorFactory.cpp +++ b/src/storage/transaction/ChainProcessorFactory.cpp @@ -5,6 +5,8 @@ #include "storage/transaction/ChainProcessorFactory.h" +#include "storage/transaction/ChainDeleteEdgesResumeProcessor.h" +#include "storage/transaction/ChainDeleteEdgesResumeRemoteProcessor.h" #include "storage/transaction/ConsistUtil.h" #include "storage/transaction/ResumeAddEdgeProcessor.h" #include "storage/transaction/ResumeAddEdgeRemoteProcessor.h" @@ -51,6 +53,22 @@ ChainBaseProcessor* ChainProcessorFactory::makeProcessor(StorageEnv* env, } break; } + case RequestType::DELETE: { + switch (options.resumeType) { + case ResumeType::RESUME_CHAIN: { + ret = ChainDeleteEdgesResumeProcessor::instance(env, options.primeValue); + break; + } + case ResumeType::RESUME_REMOTE: { + ret = ChainDeleteEdgesResumeRemoteProcessor::instance(env, options.primeValue); + break; + } + case ResumeType::UNKNOWN: { + LOG(FATAL) << "ResumeType::UNKNOWN: not supposed run here"; + } + } + break; + } case RequestType::UNKNOWN: { LOG(FATAL) << "RequestType::UNKNOWN: not supposed run here"; } diff --git a/src/storage/transaction/ChainResumeProcessor.cpp b/src/storage/transaction/ChainResumeProcessor.cpp index 8a0bc3dfd11..4fad8f13749 100644 --- a/src/storage/transaction/ChainResumeProcessor.cpp +++ b/src/storage/transaction/ChainResumeProcessor.cpp @@ -5,9 +5,9 @@ #include "storage/transaction/ChainResumeProcessor.h" -#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ChainAddEdgesLocalProcessor.h" #include "storage/transaction/ChainProcessorFactory.h" -#include "storage/transaction/ChainUpdateEdgeProcessorLocal.h" +#include "storage/transaction/ChainUpdateEdgeLocalProcessor.h" #include "storage/transaction/ConsistUtil.h" #include "storage/transaction/TransactionManager.h" @@ -15,7 +15,7 @@ namespace nebula { namespace storage { void ChainResumeProcessor::process() { - auto* table = env_->txnMan_->getReserveTable(); + auto* table = env_->txnMan_->getDangleEdges(); std::unique_ptr iter; for (auto it = table->begin(); it != table->end(); ++it) { auto spaceId = *reinterpret_cast(const_cast(it->first.c_str())); @@ -40,10 +40,14 @@ void ChainResumeProcessor::process() { env_->txnMan_->delPrime(spaceId, edgeKey); } continue; + } else if (rc == nebula::cpp2::ErrorCode::E_KEY_NOT_FOUND) { + // raft may rollback want we scanned. + env_->txnMan_->delPrime(spaceId, edgeKey); } else { LOG(WARNING) << "kvstore->get() failed, " << apache::thrift::util::enumNameSafe(rc); continue; } + ResumeOptions opt(it->second, val); auto* proc = ChainProcessorFactory::makeProcessor(env_, opt); auto fut = proc->getFinished(); @@ -52,6 +56,8 @@ void ChainResumeProcessor::process() { .thenValue([=](auto&& code) { if (code == Code::SUCCEEDED) { env_->txnMan_->delPrime(spaceId, edgeKey); + } else { + VLOG(1) << "recover failed: " << apache::thrift::util::enumNameSafe(rc); } }) .get(); diff --git a/src/storage/transaction/ChainResumeProcessor.h b/src/storage/transaction/ChainResumeProcessor.h index 1e5d90aa336..ac3572e319f 100644 --- a/src/storage/transaction/ChainResumeProcessor.h +++ b/src/storage/transaction/ChainResumeProcessor.h @@ -7,9 +7,9 @@ #include "clients/storage/InternalStorageClient.h" #include "common/utils/NebulaKeyUtils.h" -#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ChainAddEdgesLocalProcessor.h" #include "storage/transaction/ChainBaseProcessor.h" -#include "storage/transaction/ChainUpdateEdgeProcessorLocal.h" +#include "storage/transaction/ChainUpdateEdgeLocalProcessor.h" #include "storage/transaction/TransactionManager.h" namespace nebula { diff --git a/src/storage/transaction/ChainUpdateEdgeProcessorLocal.cpp b/src/storage/transaction/ChainUpdateEdgeLocalProcessor.cpp similarity index 67% rename from src/storage/transaction/ChainUpdateEdgeProcessorLocal.cpp rename to src/storage/transaction/ChainUpdateEdgeLocalProcessor.cpp index 443da4093fa..d2246ecb002 100644 --- a/src/storage/transaction/ChainUpdateEdgeProcessorLocal.cpp +++ b/src/storage/transaction/ChainUpdateEdgeLocalProcessor.cpp @@ -3,7 +3,7 @@ * This source code is licensed under Apache 2.0 License. */ -#include "storage/transaction/ChainUpdateEdgeProcessorLocal.h" +#include "storage/transaction/ChainUpdateEdgeLocalProcessor.h" #include @@ -15,7 +15,7 @@ namespace nebula { namespace storage { -void ChainUpdateEdgeProcessorLocal::process(const cpp2::UpdateEdgeRequest& req) { +void ChainUpdateEdgeLocalProcessor::process(const cpp2::UpdateEdgeRequest& req) { if (!prepareRequest(req)) { onFinished(); } @@ -23,22 +23,19 @@ void ChainUpdateEdgeProcessorLocal::process(const cpp2::UpdateEdgeRequest& req) env_->txnMan_->addChainTask(this); } -bool ChainUpdateEdgeProcessorLocal::prepareRequest(const cpp2::UpdateEdgeRequest& req) { +bool ChainUpdateEdgeLocalProcessor::prepareRequest(const cpp2::UpdateEdgeRequest& req) { req_ = req; spaceId_ = req.get_space_id(); - partId_ = req_.get_part_id(); + localPartId_ = req_.get_part_id(); auto rc = getSpaceVidLen(spaceId_); if (rc != nebula::cpp2::ErrorCode::SUCCEEDED) { - pushResultCode(rc, partId_); + pushResultCode(rc, localPartId_); return false; } - auto __term = env_->txnMan_->getTerm(req_.get_space_id(), partId_); - if (__term.ok()) { - termOfPrepare_ = __term.value(); - } else { - pushResultCode(Code::E_PART_NOT_FOUND, partId_); + std::tie(term_, code_) = env_->txnMan_->getTerm(spaceId_, localPartId_); + if (code_ != Code::SUCCEEDED) { return false; } return true; @@ -48,13 +45,13 @@ bool ChainUpdateEdgeProcessorLocal::prepareRequest(const cpp2::UpdateEdgeRequest * 1. set mem lock * 2. set edge prime * */ -folly::SemiFuture ChainUpdateEdgeProcessorLocal::prepareLocal() { +folly::SemiFuture ChainUpdateEdgeLocalProcessor::prepareLocal() { if (!setLock()) { LOG(INFO) << "set lock failed, return E_WRITE_WRITE_CONFLICT"; return Code::E_WRITE_WRITE_CONFLICT; } - auto key = ConsistUtil::primeKey(spaceVidLen_, partId_, req_.get_edge_key()); + auto key = ConsistUtil::primeKey(spaceVidLen_, localPartId_, req_.get_edge_key()); std::string val; apache::thrift::CompactSerializer::serialize(req_, &val); @@ -63,7 +60,7 @@ folly::SemiFuture ChainUpdateEdgeProcessorLocal::prepareLocal() { std::vector data{{key, val}}; auto c = folly::makePromiseContract(); env_->kvstore_->asyncMultiPut( - spaceId_, partId_, std::move(data), [p = std::move(c.first), this](auto rc) mutable { + spaceId_, localPartId_, std::move(data), [p = std::move(c.first), this](auto rc) mutable { if (rc == nebula::cpp2::ErrorCode::SUCCEEDED) { primeInserted_ = true; } else { @@ -74,7 +71,7 @@ folly::SemiFuture ChainUpdateEdgeProcessorLocal::prepareLocal() { return std::move(c.second); } -folly::SemiFuture ChainUpdateEdgeProcessorLocal::processRemote(Code code) { +folly::SemiFuture ChainUpdateEdgeLocalProcessor::processRemote(Code code) { LOG(INFO) << "prepareLocal()=" << apache::thrift::util::enumNameSafe(code); if (code != Code::SUCCEEDED) { return code; @@ -84,15 +81,16 @@ folly::SemiFuture ChainUpdateEdgeProcessorLocal::processRemote(Code code) return std::move(fut); } -folly::SemiFuture ChainUpdateEdgeProcessorLocal::processLocal(Code code) { +folly::SemiFuture ChainUpdateEdgeLocalProcessor::processLocal(Code code) { LOG(INFO) << "processRemote(), code = " << apache::thrift::util::enumNameSafe(code); if (code != Code::SUCCEEDED && code_ == Code::SUCCEEDED) { code_ = code; } - if (!checkTerm()) { - LOG(WARNING) << "checkTerm() failed"; - return Code::E_OUTDATED_TERM; + auto currTerm = env_->txnMan_->getTerm(spaceId_, localPartId_); + if (currTerm.first != term_) { + LOG(WARNING) << "E_LEADER_CHANGED during prepare and commit local"; + code_ = Code::E_LEADER_CHANGED; } if (code == Code::E_RPC_FAILURE) { @@ -112,7 +110,7 @@ folly::SemiFuture ChainUpdateEdgeProcessorLocal::processLocal(Code code) { return code_; } -void ChainUpdateEdgeProcessorLocal::doRpc(folly::Promise&& promise, int retry) noexcept { +void ChainUpdateEdgeLocalProcessor::doRpc(folly::Promise&& promise, int retry) noexcept { try { if (retry > retryLimit_) { promise.setValue(Code::E_LEADER_CHANGED); @@ -123,7 +121,7 @@ void ChainUpdateEdgeProcessorLocal::doRpc(folly::Promise&& promise, int re auto reversedReq = reverseRequest(req_); auto f = p.getFuture(); - iClient->chainUpdateEdge(reversedReq, termOfPrepare_, ver_, std::move(p)); + iClient->chainUpdateEdge(reversedReq, term_, ver_, std::move(p)); std::move(f) .thenTry([=, p = std::move(promise)](auto&& t) mutable { auto code = t.hasValue() ? t.value() : Code::E_RPC_FAILURE; @@ -144,20 +142,20 @@ void ChainUpdateEdgeProcessorLocal::doRpc(folly::Promise&& promise, int re } } -void ChainUpdateEdgeProcessorLocal::erasePrime() { - auto key = ConsistUtil::primeKey(spaceVidLen_, partId_, req_.get_edge_key()); +void ChainUpdateEdgeLocalProcessor::erasePrime() { + auto key = ConsistUtil::primeKey(spaceVidLen_, localPartId_, req_.get_edge_key()); kvErased_.emplace_back(std::move(key)); } -void ChainUpdateEdgeProcessorLocal::appendDoublePrime() { - auto key = ConsistUtil::doublePrime(spaceVidLen_, partId_, req_.get_edge_key()); +void ChainUpdateEdgeLocalProcessor::appendDoublePrime() { + auto key = ConsistUtil::doublePrime(spaceVidLen_, localPartId_, req_.get_edge_key()); std::string val; apache::thrift::CompactSerializer::serialize(req_, &val); val += ConsistUtil::updateIdentifier(); kvAppend_.emplace_back(std::make_pair(std::move(key), std::move(val))); } -void ChainUpdateEdgeProcessorLocal::forwardToDelegateProcessor() { +void ChainUpdateEdgeLocalProcessor::forwardToDelegateProcessor() { kUpdateEdgeCounters.init("update_edge"); UpdateEdgeProcessor::ContextAdjuster fn = [=](EdgeContext& ctx) { ctx.kvAppend = std::move(kvAppend_); @@ -176,37 +174,22 @@ void ChainUpdateEdgeProcessorLocal::forwardToDelegateProcessor() { std::swap(resp_, resp); } -Code ChainUpdateEdgeProcessorLocal::checkAndBuildContexts(const cpp2::UpdateEdgeRequest&) { +Code ChainUpdateEdgeLocalProcessor::checkAndBuildContexts(const cpp2::UpdateEdgeRequest&) { return Code::SUCCEEDED; } -std::string ChainUpdateEdgeProcessorLocal::sEdgeKey(const cpp2::UpdateEdgeRequest& req) { +std::string ChainUpdateEdgeLocalProcessor::sEdgeKey(const cpp2::UpdateEdgeRequest& req) { return ConsistUtil::edgeKey(spaceVidLen_, req.get_part_id(), req.get_edge_key()); } -void ChainUpdateEdgeProcessorLocal::finish() { - LOG(INFO) << "ChainUpdateEdgeProcessorLocal::finish()"; +void ChainUpdateEdgeLocalProcessor::finish() { + LOG(INFO) << "ChainUpdateEdgeLocalProcessor::finish()"; pushResultCode(code_, req_.get_part_id()); onFinished(); } -bool ChainUpdateEdgeProcessorLocal::checkTerm() { - return env_->txnMan_->checkTerm(req_.get_space_id(), req_.get_part_id(), termOfPrepare_); -} - -bool ChainUpdateEdgeProcessorLocal::checkVersion() { - if (!ver_) { - return true; - } - auto [ver, rc] = ConsistUtil::versionOfUpdateReq(env_, req_); - if (rc != Code::SUCCEEDED) { - return false; - } - return *ver_ == ver; -} - -void ChainUpdateEdgeProcessorLocal::abort() { - auto key = ConsistUtil::primeKey(spaceVidLen_, partId_, req_.get_edge_key()); +void ChainUpdateEdgeLocalProcessor::abort() { + auto key = ConsistUtil::primeKey(spaceVidLen_, localPartId_, req_.get_edge_key()); kvErased_.emplace_back(std::move(key)); folly::Baton baton; @@ -221,7 +204,7 @@ void ChainUpdateEdgeProcessorLocal::abort() { baton.wait(); } -cpp2::UpdateEdgeRequest ChainUpdateEdgeProcessorLocal::reverseRequest( +cpp2::UpdateEdgeRequest ChainUpdateEdgeLocalProcessor::reverseRequest( const cpp2::UpdateEdgeRequest& req) { cpp2::UpdateEdgeRequest reversedRequest(req); auto reversedEdgeKey = ConsistUtil::reverseEdgeKey(req.get_edge_key()); @@ -236,7 +219,7 @@ cpp2::UpdateEdgeRequest ChainUpdateEdgeProcessorLocal::reverseRequest( return reversedRequest; } -bool ChainUpdateEdgeProcessorLocal::setLock() { +bool ChainUpdateEdgeLocalProcessor::setLock() { auto spaceId = req_.get_space_id(); auto* lockCore = env_->txnMan_->getLockCore(spaceId, req_.get_part_id()); if (lockCore == nullptr) { @@ -247,20 +230,7 @@ bool ChainUpdateEdgeProcessorLocal::setLock() { return lk_->isLocked(); } -int64_t ChainUpdateEdgeProcessorLocal::getVersion(const cpp2::UpdateEdgeRequest& req) { - int64_t invalidVer = -1; - auto spaceId = req.get_space_id(); - auto vIdLen = env_->metaClient_->getSpaceVidLen(spaceId); - if (!vIdLen.ok()) { - LOG(WARNING) << vIdLen.status().toString(); - return invalidVer; - } - auto partId = req.get_part_id(); - auto key = ConsistUtil::edgeKey(vIdLen.value(), partId, req.get_edge_key()); - return ConsistUtil::getSingleEdgeVer(env_->kvstore_, spaceId, partId, key); -} - -nebula::cpp2::ErrorCode ChainUpdateEdgeProcessorLocal::getErrorCode( +nebula::cpp2::ErrorCode ChainUpdateEdgeLocalProcessor::getErrorCode( const cpp2::UpdateResponse& resp) { auto& respCommon = resp.get_result(); auto& parts = respCommon.get_failed_parts(); @@ -270,7 +240,7 @@ nebula::cpp2::ErrorCode ChainUpdateEdgeProcessorLocal::getErrorCode( return parts.front().get_code(); } -void ChainUpdateEdgeProcessorLocal::addUnfinishedEdge(ResumeType type) { +void ChainUpdateEdgeLocalProcessor::addUnfinishedEdge(ResumeType type) { LOG(INFO) << "addUnfinishedEdge()"; if (lk_ != nullptr) { lk_->forceUnlock(); diff --git a/src/storage/transaction/ChainUpdateEdgeProcessorLocal.h b/src/storage/transaction/ChainUpdateEdgeLocalProcessor.h similarity index 86% rename from src/storage/transaction/ChainUpdateEdgeProcessorLocal.h rename to src/storage/transaction/ChainUpdateEdgeLocalProcessor.h index ecf61e8dc6d..2f84f343a83 100644 --- a/src/storage/transaction/ChainUpdateEdgeProcessorLocal.h +++ b/src/storage/transaction/ChainUpdateEdgeLocalProcessor.h @@ -16,15 +16,15 @@ namespace nebula { namespace storage { -class ChainUpdateEdgeProcessorLocal +class ChainUpdateEdgeLocalProcessor : public QueryBaseProcessor, public ChainBaseProcessor { friend struct ChainUpdateEdgeTestHelper; public: using Code = ::nebula::cpp2::ErrorCode; - static ChainUpdateEdgeProcessorLocal* instance(StorageEnv* env) { - return new ChainUpdateEdgeProcessorLocal(env); + static ChainUpdateEdgeLocalProcessor* instance(StorageEnv* env) { + return new ChainUpdateEdgeLocalProcessor(env); } void process(const cpp2::UpdateEdgeRequest& req) override; @@ -39,20 +39,16 @@ class ChainUpdateEdgeProcessorLocal void finish() override; - virtual ~ChainUpdateEdgeProcessorLocal() = default; + virtual ~ChainUpdateEdgeLocalProcessor() = default; protected: - explicit ChainUpdateEdgeProcessorLocal(StorageEnv* env) + explicit ChainUpdateEdgeLocalProcessor(StorageEnv* env) : QueryBaseProcessor(env, nullptr) {} std::string edgeKey(const cpp2::UpdateEdgeRequest& req); void doRpc(folly::Promise&& promise, int retry = 0) noexcept; - bool checkTerm(); - - bool checkVersion(); - folly::SemiFuture processNormalLocal(Code code); void abort(); @@ -82,9 +78,9 @@ class ChainUpdateEdgeProcessorLocal protected: cpp2::UpdateEdgeRequest req_; std::unique_ptr lk_; - PartitionID partId_; + PartitionID localPartId_; int retryLimit_{10}; - TermID termOfPrepare_{-1}; + TermID term_{-1}; // set to true when prime insert succeed // in processLocal(), we check this to determine if need to do abort() diff --git a/src/storage/transaction/ChainUpdateEdgeProcessorRemote.cpp b/src/storage/transaction/ChainUpdateEdgeRemoteProcessor.cpp similarity index 50% rename from src/storage/transaction/ChainUpdateEdgeProcessorRemote.cpp rename to src/storage/transaction/ChainUpdateEdgeRemoteProcessor.cpp index d52cbf8087d..1dbfb1daeaf 100644 --- a/src/storage/transaction/ChainUpdateEdgeProcessorRemote.cpp +++ b/src/storage/transaction/ChainUpdateEdgeRemoteProcessor.cpp @@ -3,7 +3,7 @@ * This source code is licensed under Apache 2.0 License. */ -#include "storage/transaction/ChainUpdateEdgeProcessorRemote.h" +#include "storage/transaction/ChainUpdateEdgeRemoteProcessor.h" #include "storage/mutate/UpdateEdgeProcessor.h" #include "storage/transaction/ConsistUtil.h" @@ -14,18 +14,16 @@ namespace storage { using Code = ::nebula::cpp2::ErrorCode; -void ChainUpdateEdgeProcessorRemote::process(const cpp2::ChainUpdateEdgeRequest& req) { +void ChainUpdateEdgeRemoteProcessor::process(const cpp2::ChainUpdateEdgeRequest& req) { auto rc = Code::SUCCEEDED; - if (!checkTerm(req)) { + auto spaceId = req.get_space_id(); + auto localPartId = getLocalPart(req); + auto localTerm = req.get_term(); + if (!env_->txnMan_->checkTermFromCache(spaceId, localPartId, localTerm)) { LOG(WARNING) << "invalid term"; rc = Code::E_OUTDATED_TERM; } - if (!checkVersion(req)) { - LOG(WARNING) << "invalid term"; - rc = Code::E_OUTDATED_EDGE; - } - auto& updateRequest = req.get_update_edge_request(); if (rc != Code::SUCCEEDED) { pushResultCode(rc, updateRequest.get_part_id()); @@ -35,26 +33,13 @@ void ChainUpdateEdgeProcessorRemote::process(const cpp2::ChainUpdateEdgeRequest& onFinished(); } -bool ChainUpdateEdgeProcessorRemote::checkTerm(const cpp2::ChainUpdateEdgeRequest& req) { - auto partId = req.get_update_edge_request().get_part_id(); - return env_->txnMan_->checkTerm(req.get_space_id(), partId, req.get_term()); -} - -bool ChainUpdateEdgeProcessorRemote::checkVersion(const cpp2::ChainUpdateEdgeRequest& req) { - if (!req.edge_version_ref()) { - return true; - } - auto verExpected = *req.edge_version_ref(); - auto& updateRequest = req.get_update_edge_request(); - auto [verActually, rc] = ConsistUtil::versionOfUpdateReq(env_, updateRequest); - if (rc != Code::SUCCEEDED) { - return false; - } - return verExpected >= verActually; +PartitionID ChainUpdateEdgeRemoteProcessor::getLocalPart(const cpp2::ChainUpdateEdgeRequest& req) { + auto& edgeKey = req.get_update_edge_request().get_edge_key(); + return NebulaKeyUtils::getPart(edgeKey.dst()->getStr()); } // forward to UpdateEdgeProcessor -void ChainUpdateEdgeProcessorRemote::updateEdge(const cpp2::ChainUpdateEdgeRequest& req) { +void ChainUpdateEdgeRemoteProcessor::updateEdge(const cpp2::ChainUpdateEdgeRequest& req) { auto* proc = UpdateEdgeProcessor::instance(env_, counters_); auto f = proc->getFuture(); proc->process(req.get_update_edge_request()); diff --git a/src/storage/transaction/ChainUpdateEdgeProcessorRemote.h b/src/storage/transaction/ChainUpdateEdgeRemoteProcessor.h similarity index 64% rename from src/storage/transaction/ChainUpdateEdgeProcessorRemote.h rename to src/storage/transaction/ChainUpdateEdgeRemoteProcessor.h index 0170c54d964..b3034fc9d3d 100644 --- a/src/storage/transaction/ChainUpdateEdgeProcessorRemote.h +++ b/src/storage/transaction/ChainUpdateEdgeRemoteProcessor.h @@ -12,24 +12,22 @@ namespace nebula { namespace storage { -class ChainUpdateEdgeProcessorRemote : public BaseProcessor { +class ChainUpdateEdgeRemoteProcessor : public BaseProcessor { public: - static ChainUpdateEdgeProcessorRemote* instance(StorageEnv* env) { - return new ChainUpdateEdgeProcessorRemote(env); + static ChainUpdateEdgeRemoteProcessor* instance(StorageEnv* env) { + return new ChainUpdateEdgeRemoteProcessor(env); } void process(const cpp2::ChainUpdateEdgeRequest& req); private: - explicit ChainUpdateEdgeProcessorRemote(StorageEnv* env) + explicit ChainUpdateEdgeRemoteProcessor(StorageEnv* env) : BaseProcessor(env) {} - bool checkTerm(const cpp2::ChainUpdateEdgeRequest& req); - - bool checkVersion(const cpp2::ChainUpdateEdgeRequest& req); - void updateEdge(const cpp2::ChainUpdateEdgeRequest& req); + PartitionID getLocalPart(const cpp2::ChainUpdateEdgeRequest& req); + private: std::unique_ptr lk_; }; diff --git a/src/storage/transaction/ConsistTypes.h b/src/storage/transaction/ConsistTypes.h new file mode 100644 index 00000000000..eb0cb2c89db --- /dev/null +++ b/src/storage/transaction/ConsistTypes.h @@ -0,0 +1,41 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License. + */ + +#pragma once + +#include + +#include "kvstore/LogEncoder.h" + +namespace nebula { +namespace storage { + +enum class RequestType { + UNKNOWN = 0, + INSERT, + UPDATE, + DELETE, +}; + +enum class ResumeType { + UNKNOWN = 0, + RESUME_CHAIN, + RESUME_REMOTE, +}; + +struct ResumeOptions { + ResumeOptions(ResumeType tp, std::string val) : resumeType(tp), primeValue(std::move(val)) {} + ResumeType resumeType; + std::string primeValue; +}; + +struct HookFuncPara { + std::optional*> keys; + std::optional<::nebula::kvstore::BatchHolder*> batch; + std::optional result; +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ConsistUtil.cpp b/src/storage/transaction/ConsistUtil.cpp index e720012bd3b..d80d288b2f7 100644 --- a/src/storage/transaction/ConsistUtil.cpp +++ b/src/storage/transaction/ConsistUtil.cpp @@ -65,6 +65,8 @@ RequestType ConsistUtil::parseType(folly::StringPiece val) { return RequestType::UPDATE; case 'a': return RequestType::INSERT; + case 'd': + return RequestType::DELETE; default: LOG(FATAL) << "should not happen, identifier is " << identifier; } @@ -98,43 +100,6 @@ std::string ConsistUtil::ConsistUtil::edgeKey(size_t vIdLen, (*key.dst_ref()).getStr()); } -std::vector ConsistUtil::getMultiEdgeVers(kvstore::KVStore* store, - GraphSpaceID spaceId, - PartitionID partId, - const std::vector& keys) { - std::vector ret(keys.size()); - std::vector _keys(keys); - auto rc = nebula::cpp2::ErrorCode::SUCCEEDED; - std::vector status; - std::vector vals; - std::tie(rc, status) = store->multiGet(spaceId, partId, std::move(_keys), &vals); - if (rc != nebula::cpp2::ErrorCode::SUCCEEDED && rc != nebula::cpp2::ErrorCode::E_PARTIAL_RESULT) { - return ret; - } - for (auto i = 0U; i != ret.size(); ++i) { - ret[i] = getTimestamp(vals[i]); - } - return ret; -} - -// return -1 if edge version not exist -int64_t ConsistUtil::getSingleEdgeVer(kvstore::KVStore* store, - GraphSpaceID spaceId, - PartitionID partId, - const std::string& key) { - static int64_t invalidEdgeVer = -1; - std::string val; - auto rc = store->get(spaceId, partId, key, &val); - if (rc != nebula::cpp2::ErrorCode::SUCCEEDED) { - return invalidEdgeVer; - } - return getTimestamp(val); -} - -int64_t ConsistUtil::getTimestamp(const std::string& val) noexcept { - return *reinterpret_cast(val.data() + (val.size() - sizeof(int64_t))); -} - cpp2::AddEdgesRequest ConsistUtil::toAddEdgesRequest(const cpp2::ChainAddEdgesRequest& req) { cpp2::AddEdgesRequest ret; ret.space_id_ref() = req.get_space_id(); @@ -157,62 +122,71 @@ void ConsistUtil::reverseEdgeKeyInplace(cpp2::EdgeKey& edgeKey) { *edgeKey.edge_type_ref() = 0 - edgeKey.get_edge_type(); } -std::pair ConsistUtil::versionOfUpdateReq( - StorageEnv* env, const cpp2::UpdateEdgeRequest& req) { - int64_t ver = -1; - auto rc = nebula::cpp2::ErrorCode::SUCCEEDED; +int64_t ConsistUtil::toInt(const ::nebula::Value& val) { + // return ConsistUtil::toInt2(val.toString()); + auto str = val.toString(); + if (str.size() < 3) { + return 0; + } + return *reinterpret_cast(const_cast(str.data() + 1)); +} - do { - auto spaceId = req.get_space_id(); - auto stVidLen = env->metaClient_->getSpaceVidLen(spaceId); - if (!stVidLen.ok()) { - rc = nebula::cpp2::ErrorCode::E_SPACE_NOT_FOUND; - break; - } - auto vIdLen = stVidLen.value(); - auto partId = req.get_part_id(); - auto key = ConsistUtil::edgeKey(vIdLen, partId, req.get_edge_key()); - ver = ConsistUtil::getSingleEdgeVer(env->kvstore_, spaceId, partId, key); - } while (0); +int64_t ConsistUtil::toInt2(const std::string& str) { + if (str.size() < 8) { + return 0; + } + return *reinterpret_cast(const_cast(str.data())); +} - return std::make_pair(ver, rc); +std::string ConsistUtil::readableKey(size_t vidLen, const std::string& rawKey) { + auto src = NebulaKeyUtils::getSrcId(vidLen, rawKey); + auto dst = NebulaKeyUtils::getDstId(vidLen, rawKey); + auto rank = NebulaKeyUtils::getRank(vidLen, rawKey); + std::stringstream ss; + ss << ConsistUtil::toInt2(src.str()) << "->" << ConsistUtil::toInt2(dst.str()) << "@" << rank; + return ss.str(); } -std::string ConsistUtil::dumpAddEdgeReq(const cpp2::AddEdgesRequest& req) { - std::stringstream oss; - oss << "prop_names.size() = " << req.get_prop_names().size() << " "; - for (auto& name : req.get_prop_names()) { - oss << name << " "; - } - oss << " "; - for (auto& part : req.get_parts()) { - // oss << dumpParts(part.second); - for (auto& edge : part.second) { - oss << " edge: " << folly::hexlify(edge.get_key().get_src().toString()) << "->" - << folly::hexlify(edge.get_key().get_dst().toString()) - << ", type=" << edge.get_key().get_edge_type() - << ", rank=" << edge.get_key().get_ranking() << ", vals: "; - for (auto& val : edge.get_props()) { - oss << val.toString() << ", "; - } - oss << "\n"; +std::vector ConsistUtil::toStrKeys(const cpp2::DeleteEdgesRequest& req, int vIdLen) { + std::vector ret; + for (auto& edgesOfPart : req.get_parts()) { + auto partId = edgesOfPart.first; + for (auto& key : edgesOfPart.second) { + ret.emplace_back(ConsistUtil::edgeKey(vIdLen, partId, key)); } } - return oss.str(); + return ret; +} + +::nebula::cpp2::ErrorCode ConsistUtil::getErrorCode(const cpp2::ExecResponse& resp) { + auto ret = ::nebula::cpp2::ErrorCode::SUCCEEDED; + auto& respComn = resp.get_result(); + for (auto& part : respComn.get_failed_parts()) { + ret = part.code; + } + return ret; +} + +cpp2::DeleteEdgesRequest DeleteEdgesRequestHelper::toDeleteEdgesRequest( + const cpp2::ChainDeleteEdgesRequest& req) { + cpp2::DeleteEdgesRequest ret; + ret.space_id_ref() = req.get_space_id(); + ret.parts_ref() = req.get_parts(); + return ret; +} + +cpp2::DeleteEdgesRequest DeleteEdgesRequestHelper::parseDeleteEdgesRequest(const std::string& val) { + cpp2::DeleteEdgesRequest req; + apache::thrift::CompactSerializer::deserialize(val, req); + return req; } -std::string ConsistUtil::dumpParts(const Parts& parts) { +std::string DeleteEdgesRequestHelper::explain(const cpp2::DeleteEdgesRequest& req) { std::stringstream oss; - for (auto& part : parts) { - for (auto& edge : part.second) { - oss << " edge: " << folly::hexlify(edge.get_key().get_src().toString()) << "->" - << folly::hexlify(edge.get_key().get_dst().toString()) - << ", type=" << edge.get_key().get_edge_type() - << ", rank=" << edge.get_key().get_ranking() << ", vals: "; - for (auto& val : edge.get_props()) { - oss << val.toString() << ", "; - } - oss << "\n"; + for (auto& partOfKeys : req.get_parts()) { + for (auto& key : partOfKeys.second) { + oss << ConsistUtil::toInt(key.get_src()) << "->" << ConsistUtil::toInt(key.get_dst()) << "@" + << key.get_ranking() << ", "; } } return oss.str(); diff --git a/src/storage/transaction/ConsistUtil.h b/src/storage/transaction/ConsistUtil.h index a507802bdf0..0ca2fc918d5 100644 --- a/src/storage/transaction/ConsistUtil.h +++ b/src/storage/transaction/ConsistUtil.h @@ -11,34 +11,20 @@ #include "interface/gen-cpp2/storage_types.h" #include "kvstore/KVStore.h" #include "storage/CommonUtils.h" +#include "storage/transaction/ConsistTypes.h" namespace nebula { namespace storage { - -enum class RequestType { - UNKNOWN, - INSERT, - UPDATE, -}; - -enum class ResumeType { - UNKNOWN = 0, - RESUME_CHAIN, - RESUME_REMOTE, -}; - -struct ResumeOptions { - ResumeOptions(ResumeType tp, std::string val) : resumeType(tp), primeValue(std::move(val)) {} - ResumeType resumeType; - std::string primeValue; -}; - class ConsistUtil final { public: static std::string primeTable(); static std::string doublePrimeTable(); + static std::string deletePrimeTable(); + + static std::string deleteDoublePrimeTable(); + static std::string edgeKey(size_t vIdLen, PartitionID partId, const cpp2::EdgeKey& key); static std::string primeKey(size_t vIdLen, PartitionID partId, const cpp2::EdgeKey& edgeKey); @@ -75,21 +61,6 @@ class ConsistUtil final { static std::string strUUID(); - static std::string tempRequestTable(); - - static std::vector getMultiEdgeVers(kvstore::KVStore* store, - GraphSpaceID spaceId, - PartitionID partId, - const std::vector& keys); - - // return -1 if edge version not exist - static int64_t getSingleEdgeVer(kvstore::KVStore* store, - GraphSpaceID spaceId, - PartitionID partId, - const std::string& key); - - static int64_t getTimestamp(const std::string& val) noexcept; - static cpp2::AddEdgesRequest toAddEdgesRequest(const cpp2::ChainAddEdgesRequest& req); static cpp2::EdgeKey reverseEdgeKey(const cpp2::EdgeKey& edgeKey); @@ -104,13 +75,36 @@ class ConsistUtil final { return "u"; } - static std::pair versionOfUpdateReq( - StorageEnv* env, const cpp2::UpdateEdgeRequest& req); + static std::string deleteIdentifier() noexcept { + return "d"; + } + + /** + * @brief if the vid of space is created as "Fixed string" + * when trying to print this vid, it will show a hex string + * This function trying to transform it to human readable format. + * @return -1 if failed + */ + static int64_t toInt(const ::nebula::Value& val); + + static int64_t toInt2(const std::string& val); + + static std::string readableKey(size_t vidLen, const std::string& rawKey); + + static std::vector toStrKeys(const cpp2::DeleteEdgesRequest& req, int vidLen); + + static ::nebula::cpp2::ErrorCode getErrorCode(const cpp2::ExecResponse& resp); +}; + +struct DeleteEdgesRequestHelper final { + static cpp2::DeleteEdgesRequest toDeleteEdgesRequest(const cpp2::ChainDeleteEdgesRequest& req); + + static cpp2::ChainDeleteEdgesRequest toChainDeleteEdgesRequest( + const cpp2::DeleteEdgesRequest& req); - static std::string dumpAddEdgeReq(const cpp2::AddEdgesRequest& req); + static cpp2::DeleteEdgesRequest parseDeleteEdgesRequest(const std::string& val); - using Parts = std::unordered_map>; - static std::string dumpParts(const Parts& parts); + static std::string explain(const cpp2::DeleteEdgesRequest& req); }; } // namespace storage diff --git a/src/storage/transaction/ResumeAddEdgeProcessor.cpp b/src/storage/transaction/ResumeAddEdgeProcessor.cpp index a23b34aad29..3ca1bfb18c5 100644 --- a/src/storage/transaction/ResumeAddEdgeProcessor.cpp +++ b/src/storage/transaction/ResumeAddEdgeProcessor.cpp @@ -9,13 +9,13 @@ namespace nebula { namespace storage { ResumeAddEdgeProcessor::ResumeAddEdgeProcessor(StorageEnv* env, const std::string& val) - : ChainAddEdgesProcessorLocal(env) { + : ChainAddEdgesLocalProcessor(env) { req_ = ConsistUtil::parseAddRequest(val); uuid_ = ConsistUtil::strUUID(); readableEdgeDesc_ = makeReadableEdge(req_); VLOG(1) << uuid_ << " resume prime " << readableEdgeDesc_; - ChainAddEdgesProcessorLocal::prepareRequest(req_); + ChainAddEdgesLocalProcessor::prepareRequest(req_); } folly::SemiFuture ResumeAddEdgeProcessor::prepareLocal() { @@ -28,32 +28,31 @@ folly::SemiFuture ResumeAddEdgeProcessor::prepareLocal( return Code::E_SPACE_NOT_FOUND; } auto& parts = req_.get_parts(); + auto& srcId = parts.begin()->second.back().get_key().get_src().getStr(); auto& dstId = parts.begin()->second.back().get_key().get_dst().getStr(); + localPartId_ = env_->metaClient_->partId(numOfPart.value(), srcId); remotePartId_ = env_->metaClient_->partId(numOfPart.value(), dstId); - std::vector keys = sEdgeKey(req_); - auto vers = ConsistUtil::getMultiEdgeVers(env_->kvstore_, spaceId, localPartId_, keys); - edgeVer_ = vers.front(); - - return Code::SUCCEEDED; + return code_; } folly::SemiFuture ResumeAddEdgeProcessor::processRemote(Code code) { VLOG(1) << uuid_ << " prepareLocal() " << apache::thrift::util::enumNameSafe(code); - return ChainAddEdgesProcessorLocal::processRemote(code); + return ChainAddEdgesLocalProcessor::processRemote(code); } folly::SemiFuture ResumeAddEdgeProcessor::processLocal(Code code) { VLOG(1) << uuid_ << " processRemote() " << apache::thrift::util::enumNameSafe(code); setErrorCode(code); - if (!checkTerm(req_)) { - LOG(WARNING) << this << "E_OUTDATED_TERM"; - return Code::E_OUTDATED_TERM; + auto currTerm = env_->txnMan_->getTerm(spaceId_, localPartId_); + if (currTerm.first != term_) { + LOG(WARNING) << "E_LEADER_CHANGED during prepare and commit local"; + code_ = Code::E_LEADER_CHANGED; } if (code == Code::E_RPC_FAILURE) { - kvAppend_ = ChainAddEdgesProcessorLocal::makeDoublePrime(); + kvAppend_ = ChainAddEdgesLocalProcessor::makeDoublePrime(); } if (code == Code::E_RPC_FAILURE || code == Code::SUCCEEDED) { diff --git a/src/storage/transaction/ResumeAddEdgeProcessor.h b/src/storage/transaction/ResumeAddEdgeProcessor.h index 5608cd60360..797bf7979aa 100644 --- a/src/storage/transaction/ResumeAddEdgeProcessor.h +++ b/src/storage/transaction/ResumeAddEdgeProcessor.h @@ -5,12 +5,12 @@ #pragma once -#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ChainAddEdgesLocalProcessor.h" namespace nebula { namespace storage { -class ResumeAddEdgeProcessor : public ChainAddEdgesProcessorLocal { +class ResumeAddEdgeProcessor : public ChainAddEdgesLocalProcessor { public: static ResumeAddEdgeProcessor* instance(StorageEnv* env, const std::string& val) { return new ResumeAddEdgeProcessor(env, val); diff --git a/src/storage/transaction/ResumeAddEdgeRemoteProcessor.cpp b/src/storage/transaction/ResumeAddEdgeRemoteProcessor.cpp index bdd1be3b664..21259f74afa 100644 --- a/src/storage/transaction/ResumeAddEdgeRemoteProcessor.cpp +++ b/src/storage/transaction/ResumeAddEdgeRemoteProcessor.cpp @@ -9,21 +9,15 @@ namespace nebula { namespace storage { ResumeAddEdgeRemoteProcessor::ResumeAddEdgeRemoteProcessor(StorageEnv* env, const std::string& val) - : ChainAddEdgesProcessorLocal(env) { + : ChainAddEdgesLocalProcessor(env) { req_ = ConsistUtil::parseAddRequest(val); - LOG(WARNING) << ConsistUtil::dumpAddEdgeReq(req_); - ChainAddEdgesProcessorLocal::prepareRequest(req_); + ChainAddEdgesLocalProcessor::prepareRequest(req_); } folly::SemiFuture ResumeAddEdgeRemoteProcessor::prepareLocal() { - if (!checkTerm(req_)) { - LOG(WARNING) << this << "E_OUTDATED_TERM"; - return Code::E_OUTDATED_TERM; - } - - if (!checkVersion(req_)) { - LOG(WARNING) << this << "E_OUTDATED_EDGE"; - return Code::E_OUTDATED_EDGE; + std::tie(term_, code_) = env_->txnMan_->getTerm(spaceId_, localPartId_); + if (code_ != Code::SUCCEEDED) { + return code_; } auto spaceId = req_.get_space_id(); @@ -34,26 +28,19 @@ folly::SemiFuture ResumeAddEdgeRemoteProcessor::prepare auto& parts = req_.get_parts(); auto& dstId = parts.begin()->second.back().get_key().get_dst().getStr(); remotePartId_ = env_->metaClient_->partId(numOfPart.value(), dstId); - std::vector keys = sEdgeKey(req_); - auto vers = ConsistUtil::getMultiEdgeVers(env_->kvstore_, spaceId, localPartId_, keys); - edgeVer_ = vers.front(); return Code::SUCCEEDED; } folly::SemiFuture ResumeAddEdgeRemoteProcessor::processRemote(Code code) { - return ChainAddEdgesProcessorLocal::processRemote(code); + return ChainAddEdgesLocalProcessor::processRemote(code); } folly::SemiFuture ResumeAddEdgeRemoteProcessor::processLocal(Code code) { - if (!checkTerm(req_)) { - LOG(WARNING) << this << "E_OUTDATED_TERM"; - return Code::E_OUTDATED_TERM; - } - - if (!checkVersion(req_)) { - LOG(WARNING) << this << "E_OUTDATED_EDGE"; - return Code::E_OUTDATED_EDGE; + auto currTerm = env_->txnMan_->getTerm(spaceId_, localPartId_); + if (currTerm.first != term_) { + LOG(WARNING) << "E_LEADER_CHANGED during prepare and commit local"; + code_ = Code::E_LEADER_CHANGED; } if (code == Code::E_OUTDATED_TERM) { @@ -70,7 +57,7 @@ folly::SemiFuture ResumeAddEdgeRemoteProcessor::processLocal(Code code) { if (code == Code::SUCCEEDED) { // if there are something wrong other than rpc failure // we need to keep the resume retry(by not remove those prime key) - ChainAddEdgesProcessorLocal::eraseDoublePrime(); + ChainAddEdgesLocalProcessor::eraseDoublePrime(); code_ = forwardToDelegateProcessor().get(); return code_; } diff --git a/src/storage/transaction/ResumeAddEdgeRemoteProcessor.h b/src/storage/transaction/ResumeAddEdgeRemoteProcessor.h index 9e5aed28351..a9046814064 100644 --- a/src/storage/transaction/ResumeAddEdgeRemoteProcessor.h +++ b/src/storage/transaction/ResumeAddEdgeRemoteProcessor.h @@ -5,12 +5,12 @@ #pragma once -#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ChainAddEdgesLocalProcessor.h" namespace nebula { namespace storage { -class ResumeAddEdgeRemoteProcessor : public ChainAddEdgesProcessorLocal { +class ResumeAddEdgeRemoteProcessor : public ChainAddEdgesLocalProcessor { public: static ResumeAddEdgeRemoteProcessor* instance(StorageEnv* env, const std::string& val) { return new ResumeAddEdgeRemoteProcessor(env, val); diff --git a/src/storage/transaction/ResumeUpdateProcessor.cpp b/src/storage/transaction/ResumeUpdateProcessor.cpp index ee883e86d01..075d0c10a2d 100644 --- a/src/storage/transaction/ResumeUpdateProcessor.cpp +++ b/src/storage/transaction/ResumeUpdateProcessor.cpp @@ -13,34 +13,29 @@ namespace nebula { namespace storage { ResumeUpdateProcessor::ResumeUpdateProcessor(StorageEnv* env, const std::string& val) - : ChainUpdateEdgeProcessorLocal(env) { + : ChainUpdateEdgeLocalProcessor(env) { req_ = ConsistUtil::parseUpdateRequest(val); - ChainUpdateEdgeProcessorLocal::prepareRequest(req_); + ChainUpdateEdgeLocalProcessor::prepareRequest(req_); } folly::SemiFuture ResumeUpdateProcessor::prepareLocal() { - ver_ = getVersion(req_); - - return Code::SUCCEEDED; + std::tie(term_, code_) = env_->txnMan_->getTerm(spaceId_, localPartId_); + return code_; } folly::SemiFuture ResumeUpdateProcessor::processRemote(Code code) { - LOG_IF(INFO, FLAGS_trace_toss) << "prepareLocal()=" << apache::thrift::util::enumNameSafe(code); - return ChainUpdateEdgeProcessorLocal::processRemote(code); + VLOG(1) << "prepareLocal()=" << apache::thrift::util::enumNameSafe(code); + return ChainUpdateEdgeLocalProcessor::processRemote(code); } folly::SemiFuture ResumeUpdateProcessor::processLocal(Code code) { - LOG_IF(INFO, FLAGS_trace_toss) << "processRemote()=" << apache::thrift::util::enumNameSafe(code); + VLOG(1) << "processRemote()=" << apache::thrift::util::enumNameSafe(code); setErrorCode(code); - if (!checkTerm()) { - LOG(WARNING) << "E_OUTDATED_TERM"; - return Code::E_OUTDATED_TERM; - } - - if (!checkVersion()) { - LOG(WARNING) << "E_OUTDATED_EDGE"; - return Code::E_OUTDATED_EDGE; + auto currTerm = env_->txnMan_->getTerm(spaceId_, localPartId_); + if (currTerm.first != term_) { + LOG(WARNING) << "E_LEADER_CHANGED during prepare and commit local"; + code_ = Code::E_LEADER_CHANGED; } if (code == Code::E_RPC_FAILURE) { @@ -50,7 +45,7 @@ folly::SemiFuture ResumeUpdateProcessor::processLocal(Code code) { if (code == Code::E_RPC_FAILURE || code == Code::SUCCEEDED) { // if there are something wrong other than rpc failure // we need to keep the resume retry(by not remove those prime key) - auto key = ConsistUtil::primeKey(spaceVidLen_, partId_, req_.get_edge_key()); + auto key = ConsistUtil::primeKey(spaceVidLen_, localPartId_, req_.get_edge_key()); kvErased_.emplace_back(std::move(key)); forwardToDelegateProcessor(); return code_; @@ -60,7 +55,7 @@ folly::SemiFuture ResumeUpdateProcessor::processLocal(Code code) { } void ResumeUpdateProcessor::finish() { - LOG_IF(INFO, FLAGS_trace_toss) << "commitLocal()=" << apache::thrift::util::enumNameSafe(code_); + VLOG(1) << "commitLocal()=" << apache::thrift::util::enumNameSafe(code_); finished_.setValue(code_); onFinished(); } diff --git a/src/storage/transaction/ResumeUpdateProcessor.h b/src/storage/transaction/ResumeUpdateProcessor.h index ea6272e43ef..557e351b4ed 100644 --- a/src/storage/transaction/ResumeUpdateProcessor.h +++ b/src/storage/transaction/ResumeUpdateProcessor.h @@ -5,7 +5,7 @@ #pragma once -#include "storage/transaction/ChainUpdateEdgeProcessorLocal.h" +#include "storage/transaction/ChainUpdateEdgeLocalProcessor.h" namespace nebula { namespace storage { @@ -15,7 +15,7 @@ namespace storage { * if the TxnManager background resume thread found a prime key * it will create this processor to resume the complete update process */ -class ResumeUpdateProcessor : public ChainUpdateEdgeProcessorLocal { +class ResumeUpdateProcessor : public ChainUpdateEdgeLocalProcessor { public: static ResumeUpdateProcessor* instance(StorageEnv* env, const std::string& val) { return new ResumeUpdateProcessor(env, val); diff --git a/src/storage/transaction/ResumeUpdateRemoteProcessor.cpp b/src/storage/transaction/ResumeUpdateRemoteProcessor.cpp index aca10ccad0c..5bfa6ed2a65 100644 --- a/src/storage/transaction/ResumeUpdateRemoteProcessor.cpp +++ b/src/storage/transaction/ResumeUpdateRemoteProcessor.cpp @@ -11,36 +11,33 @@ namespace nebula { namespace storage { ResumeUpdateRemoteProcessor::ResumeUpdateRemoteProcessor(StorageEnv* env, const std::string& val) - : ChainUpdateEdgeProcessorLocal(env) { + : ChainUpdateEdgeLocalProcessor(env) { req_ = ConsistUtil::parseUpdateRequest(val); - ChainUpdateEdgeProcessorLocal::prepareRequest(req_); + ChainUpdateEdgeLocalProcessor::prepareRequest(req_); } folly::SemiFuture ResumeUpdateRemoteProcessor::prepareLocal() { - return Code::SUCCEEDED; + std::tie(term_, code_) = env_->txnMan_->getTerm(spaceId_, localPartId_); + return code_; } folly::SemiFuture ResumeUpdateRemoteProcessor::processRemote(Code code) { - return ChainUpdateEdgeProcessorLocal::processRemote(code); + return ChainUpdateEdgeLocalProcessor::processRemote(code); } folly::SemiFuture ResumeUpdateRemoteProcessor::processLocal(Code code) { setErrorCode(code); - if (!checkTerm()) { - LOG(WARNING) << "E_OUTDATED_TERM"; - return Code::E_OUTDATED_TERM; - } - - if (!checkVersion()) { - LOG(WARNING) << "E_OUTDATED_EDGE"; - return Code::E_OUTDATED_EDGE; + auto currTerm = env_->txnMan_->getTerm(spaceId_, localPartId_); + if (currTerm.first != term_) { + LOG(WARNING) << "E_LEADER_CHANGED during prepare and commit local"; + code_ = Code::E_LEADER_CHANGED; } if (code == Code::SUCCEEDED) { // if there are something wrong other than rpc failure // we need to keep the resume retry(by not remove those prime key) - auto key = ConsistUtil::doublePrime(spaceVidLen_, partId_, req_.get_edge_key()); + auto key = ConsistUtil::doublePrime(spaceVidLen_, localPartId_, req_.get_edge_key()); kvErased_.emplace_back(std::move(key)); forwardToDelegateProcessor(); return code; diff --git a/src/storage/transaction/ResumeUpdateRemoteProcessor.h b/src/storage/transaction/ResumeUpdateRemoteProcessor.h index d1ce5d93438..bb3171d061b 100644 --- a/src/storage/transaction/ResumeUpdateRemoteProcessor.h +++ b/src/storage/transaction/ResumeUpdateRemoteProcessor.h @@ -5,7 +5,7 @@ #pragma once -#include "storage/transaction/ChainUpdateEdgeProcessorLocal.h" +#include "storage/transaction/ChainUpdateEdgeLocalProcessor.h" namespace nebula { namespace storage { @@ -15,7 +15,7 @@ namespace storage { * if the TxnManager background resume thread found a prime key * it will create this processor to resume the complete update process */ -class ResumeUpdateRemoteProcessor : public ChainUpdateEdgeProcessorLocal { +class ResumeUpdateRemoteProcessor : public ChainUpdateEdgeLocalProcessor { public: static ResumeUpdateRemoteProcessor* instance(StorageEnv* env, const std::string& val) { return new ResumeUpdateRemoteProcessor(env, val); diff --git a/src/storage/transaction/TransactionManager.cpp b/src/storage/transaction/TransactionManager.cpp index 3c42694adc1..2c91b20b8de 100644 --- a/src/storage/transaction/TransactionManager.cpp +++ b/src/storage/transaction/TransactionManager.cpp @@ -39,7 +39,7 @@ TransactionManager::LockCore* TransactionManager::getLockCore(GraphSpaceID space GraphSpaceID partId, bool checkWhiteList) { if (checkWhiteList) { - if (whiteListParts_.find(std::make_pair(spaceId, partId)) == whiteListParts_.end()) { + if (scannedParts_.find(std::make_pair(spaceId, partId)) == scannedParts_.end()) { return nullptr; } } @@ -52,16 +52,27 @@ TransactionManager::LockCore* TransactionManager::getLockCore(GraphSpaceID space return item.first->second.get(); } -StatusOr TransactionManager::getTerm(GraphSpaceID spaceId, PartitionID partId) { - return env_->metaClient_->getTermFromCache(spaceId, partId); +std::pair TransactionManager::getTerm(GraphSpaceID spaceId, PartitionID partId) { + TermID termId = -1; + auto rc = Code::SUCCEEDED; + auto part = env_->kvstore_->part(spaceId, partId); + if (nebula::ok(part)) { + termId = nebula::value(part)->termId(); + } else { + rc = nebula::error(part); + } + return std::make_pair(termId, rc); } -bool TransactionManager::checkTerm(GraphSpaceID spaceId, PartitionID partId, TermID term) { +bool TransactionManager::checkTermFromCache(GraphSpaceID spaceId, + PartitionID partId, + TermID termId) { auto termOfMeta = env_->metaClient_->getTermFromCache(spaceId, partId); if (termOfMeta.ok()) { - if (term < termOfMeta.value()) { + if (termId < termOfMeta.value()) { LOG(WARNING) << "checkTerm() failed: " - << "spaceId=" << spaceId << ", partId=" << partId << ", in-coming term=" << term + << "spaceId=" << spaceId << ", partId=" << partId + << ", in-coming term=" << termId << ", term in meta cache=" << termOfMeta.value(); return false; } @@ -69,12 +80,12 @@ bool TransactionManager::checkTerm(GraphSpaceID spaceId, PartitionID partId, Ter auto partUUID = std::make_pair(spaceId, partId); auto it = cachedTerms_.find(partUUID); if (it != cachedTerms_.cend()) { - if (term < it->second) { + if (termId < it->second) { LOG(WARNING) << "term < it->second"; return false; } } - cachedTerms_.assign(partUUID, term); + cachedTerms_.assign(partUUID, termId); return true; } @@ -115,23 +126,21 @@ std::string TransactionManager::getEdgeKey(const std::string& lockKey) { } void TransactionManager::addPrime(GraphSpaceID spaceId, const std::string& edge, ResumeType type) { - LOG_IF(INFO, FLAGS_trace_toss) << "addPrime() space=" << spaceId - << ", hex=" << folly::hexlify(edge) - << ", ResumeType=" << static_cast(type); + VLOG(1) << "addPrime() space=" << spaceId << ", hex=" << folly::hexlify(edge) + << ", ResumeType=" << static_cast(type); auto key = makeLockKey(spaceId, edge); - reserveTable_.insert(std::make_pair(key, type)); + dangleEdges_.insert(std::make_pair(key, type)); } void TransactionManager::delPrime(GraphSpaceID spaceId, const std::string& edge) { - LOG_IF(INFO, FLAGS_trace_toss) << "delPrime() space=" << spaceId - << ", hex=" << folly::hexlify(edge); + VLOG(1) << "delPrime() space=" << spaceId << ", hex=" << folly::hexlify(edge) << ", readable " + << ConsistUtil::readableKey(8, edge); auto key = makeLockKey(spaceId, edge); - reserveTable_.erase(key); + dangleEdges_.erase(key); auto partId = NebulaKeyUtils::getPart(edge); auto* lk = getLockCore(spaceId, partId, false); - auto lockKey = makeLockKey(spaceId, edge); - lk->unlock(lockKey); + lk->unlock(edge); } void TransactionManager::scanAll() { @@ -148,6 +157,7 @@ void TransactionManager::scanAll() { scanPrimes(spaceId, partId); } } + LOG(INFO) << "finish scanAll()"; } void TransactionManager::onNewPartAdded(std::shared_ptr& part) { @@ -165,7 +175,8 @@ void TransactionManager::onLeaderLostWrapper(const ::nebula::kvstore::Part::Call opt.spaceId, opt.partId, opt.term); - whiteListParts_.erase(std::make_pair(opt.spaceId, opt.partId)); + scannedParts_.erase(std::make_pair(opt.spaceId, opt.partId)); + dangleEdges_.clear(); } void TransactionManager::onLeaderElectedWrapper( @@ -183,9 +194,10 @@ void TransactionManager::scanPrimes(GraphSpaceID spaceId, PartitionID partId) { if (rc == nebula::cpp2::ErrorCode::SUCCEEDED) { for (; iter->valid(); iter->next()) { auto edgeKey = ConsistUtil::edgeKeyFromPrime(iter->key()); - VLOG(1) << "scanned edgekey: " << folly::hexlify(edgeKey); + VLOG(1) << "scanned edgekey: " << folly::hexlify(edgeKey) + << ", readable: " << ConsistUtil::readableKey(8, edgeKey.str()); auto lockKey = makeLockKey(spaceId, edgeKey.str()); - auto insSucceed = reserveTable_.insert(std::make_pair(lockKey, ResumeType::RESUME_CHAIN)); + auto insSucceed = dangleEdges_.insert(std::make_pair(lockKey, ResumeType::RESUME_CHAIN)); if (!insSucceed.second) { LOG(ERROR) << "not supposed to insert fail: " << folly::hexlify(edgeKey); } @@ -201,13 +213,14 @@ void TransactionManager::scanPrimes(GraphSpaceID spaceId, PartitionID partId) { return; } } + prefix = ConsistUtil::doublePrimePrefix(partId); rc = env_->kvstore_->prefix(spaceId, partId, prefix, &iter); if (rc == nebula::cpp2::ErrorCode::SUCCEEDED) { for (; iter->valid(); iter->next()) { auto edgeKey = ConsistUtil::edgeKeyFromDoublePrime(iter->key()); auto lockKey = makeLockKey(spaceId, edgeKey.str()); - auto insSucceed = reserveTable_.insert(std::make_pair(lockKey, ResumeType::RESUME_REMOTE)); + auto insSucceed = dangleEdges_.insert(std::make_pair(lockKey, ResumeType::RESUME_REMOTE)); if (!insSucceed.second) { LOG(ERROR) << "not supposed to insert fail: " << folly::hexlify(edgeKey); } @@ -224,13 +237,13 @@ void TransactionManager::scanPrimes(GraphSpaceID spaceId, PartitionID partId) { } } auto partOfSpace = std::make_pair(spaceId, partId); - auto insRet = whiteListParts_.insert(std::make_pair(partOfSpace, 0)); + auto insRet = scannedParts_.insert(std::make_pair(partOfSpace, 0)); LOG(INFO) << "insert space=" << spaceId << ", part=" << partId - << ", into white list suc=" << insRet.second; + << ", into white list suc=" << std::boolalpha << insRet.second; } -folly::ConcurrentHashMap* TransactionManager::getReserveTable() { - return &reserveTable_; +folly::ConcurrentHashMap* TransactionManager::getDangleEdges() { + return &dangleEdges_; } } // namespace storage diff --git a/src/storage/transaction/TransactionManager.h b/src/storage/transaction/TransactionManager.h index 83441b958ea..acfc2517506 100644 --- a/src/storage/transaction/TransactionManager.h +++ b/src/storage/transaction/TransactionManager.h @@ -33,7 +33,9 @@ class TransactionManager { public: explicit TransactionManager(storage::StorageEnv* env); - ~TransactionManager() = default; + ~TransactionManager() { + stop(); + } void addChainTask(ChainBaseProcessor* proc) { folly::async([=] { @@ -49,19 +51,25 @@ class TransactionManager { return exec_.get(); } + bool start(); + + void stop(); + LockCore* getLockCore(GraphSpaceID spaceId, PartitionID partId, bool checkWhiteList = true); InternalStorageClient* getInternalClient() { return iClient_; } - StatusOr getTerm(GraphSpaceID spaceId, PartitionID partId); + // get term of part from kvstore, may fail if this part is not exist + std::pair getTerm(GraphSpaceID spaceId, PartitionID partId); - bool checkTerm(GraphSpaceID spaceId, PartitionID partId, TermID term); + // check get term from local term cache + // this is used by Chain...RemoteProcessor, + // to avoid an old leader request overrider a newer leader's + bool checkTermFromCache(GraphSpaceID spaceId, PartitionID partId, TermID termId); - bool start(); - - void stop(); + void reportFailed(); // leave a record for (double)prime edge, to let resume processor there is one dangling edge void addPrime(GraphSpaceID spaceId, const std::string& edgeKey, ResumeType type); @@ -70,11 +78,7 @@ class TransactionManager { bool checkUnfinishedEdge(GraphSpaceID spaceId, const folly::StringPiece& key); - // return false if there is no "edge" in reserveTable_ - // true if there is, and also erase the edge from reserveTable_. - bool takeDanglingEdge(GraphSpaceID spaceId, const std::string& edge); - - folly::ConcurrentHashMap* getReserveTable(); + folly::ConcurrentHashMap* getDangleEdges(); void scanPrimes(GraphSpaceID spaceId, PartitionID partId); @@ -106,18 +110,16 @@ class TransactionManager { std::unique_ptr resumeThread_; /** - * an update request may re-entered to an existing (double)prime key - * and wants to have its own (double)prime. - * also MVCC doesn't work. - * because (double)prime can't judge if remote side succeeded. - * to prevent insert/update re + * edges need to recover will put into this, + * resume processor will get edge from this then do resume. * */ - folly::ConcurrentHashMap reserveTable_; + folly::ConcurrentHashMap dangleEdges_; /** - * @brief only part in this white list allowed to get lock + * @brief every raft part need to do a scan, + * only scanned part allowed to insert edges */ - folly::ConcurrentHashMap, int> whiteListParts_; + folly::ConcurrentHashMap, int> scannedParts_; }; } // namespace storage diff --git a/tests/cert/test.ca.password b/tests/cert/test.ca.password index 143be9ab9c5..60b7570cd13 100644 --- a/tests/cert/test.ca.password +++ b/tests/cert/test.ca.password @@ -1 +1 @@ -vesoft +vesoft \ No newline at end of file