From 8c2e50fc561ec6e4e85d2b8f941df5840a4675f6 Mon Sep 17 00:00:00 2001 From: "lionel.liu@vesoft.com" <52276794+liuyu85cn@users.noreply.github.com> Date: Wed, 29 Dec 2021 18:02:39 +0800 Subject: [PATCH 1/7] To support delete in toss (#3374) * To support delte in TOSS dummy copy from chain add edges processor reguraly update some update may compile add some simple UT adjusting UT add explain DeleteEdgesRequest add some UT clear reverse table early expose storage interface update license & fix gcc compile error fix compile error update license fix chain interface bug fix mem-leak rename some processor continue rename Processor add some log for debug print readable edge debug keys when delete print conflict key and scanned key print rank delPrime delete the wrong key from memory lock fix raft may rollback want we scanned * rebase master * fix format * fix compile error * fix lint * address comments * fix lint * looks like some test may fail due to not stop kvstore Co-authored-by: Doodle <13706157+critical27@users.noreply.github.com> Co-authored-by: Sophie <84560950+Sophie-Xie@users.noreply.github.com> --- src/clients/storage/InternalStorageClient.cpp | 45 +- src/clients/storage/InternalStorageClient.h | 6 + src/clients/storage/StorageClient.cpp | 6 +- src/common/utils/Types.h | 2 + src/graph/executor/mutate/DeleteExecutor.cpp | 2 + src/interface/storage.thrift | 23 +- src/kvstore/Part.cpp | 4 +- src/kvstore/raftex/RaftPart.cpp | 1 - src/mock/MockData.cpp | 12 +- src/mock/MockData.h | 6 +- src/storage/CMakeLists.txt | 13 +- src/storage/GraphStorageServiceHandler.cpp | 11 +- src/storage/GraphStorageServiceHandler.h | 3 + src/storage/InternalStorageServiceHandler.cpp | 15 +- src/storage/InternalStorageServiceHandler.h | 3 + src/storage/mutate/AddEdgesProcessor.h | 2 +- src/storage/mutate/DeleteEdgesProcessor.cpp | 23 +- src/storage/mutate/DeleteEdgesProcessor.h | 11 + src/storage/test/CMakeLists.txt | 15 + src/storage/test/ChainAddEdgesTest.cpp | 28 +- src/storage/test/ChainDeleteEdgesTest.cpp | 521 ++++++++++++++++++ src/storage/test/ChainResumeEdgeTest.cpp | 115 ++-- src/storage/test/ChainTestUtils.h | 101 +++- src/storage/test/ChainUpdateEdgeTest.cpp | 12 +- .../ChainAddEdgesGroupProcessor.cpp | 4 +- ...al.cpp => ChainAddEdgesLocalProcessor.cpp} | 124 ++--- ...rLocal.h => ChainAddEdgesLocalProcessor.h} | 22 +- ...e.cpp => ChainAddEdgesRemoteProcessor.cpp} | 51 +- ...emote.h => ChainAddEdgesRemoteProcessor.h} | 14 +- .../ChainDeleteEdgesGroupProcessor.cpp | 75 +++ .../ChainDeleteEdgesGroupProcessor.h | 34 ++ .../ChainDeleteEdgesLocalProcessor.cpp | 360 ++++++++++++ .../ChainDeleteEdgesLocalProcessor.h | 118 ++++ .../ChainDeleteEdgesRemoteProcessor.cpp | 72 +++ .../ChainDeleteEdgesRemoteProcessor.h | 35 ++ .../ChainDeleteEdgesResumeProcessor.cpp | 59 ++ .../ChainDeleteEdgesResumeProcessor.h | 32 ++ .../ChainDeleteEdgesResumeRemoteProcessor.cpp | 65 +++ .../ChainDeleteEdgesResumeRemoteProcessor.h | 32 ++ .../transaction/ChainProcessorFactory.cpp | 18 + .../transaction/ChainResumeProcessor.cpp | 12 +- .../transaction/ChainResumeProcessor.h | 4 +- ....cpp => ChainUpdateEdgeLocalProcessor.cpp} | 96 ++-- ...ocal.h => ChainUpdateEdgeLocalProcessor.h} | 18 +- ...cpp => ChainUpdateEdgeRemoteProcessor.cpp} | 35 +- ...ote.h => ChainUpdateEdgeRemoteProcessor.h} | 14 +- src/storage/transaction/ConsistTypes.h | 41 ++ src/storage/transaction/ConsistUtil.cpp | 142 ++--- src/storage/transaction/ConsistUtil.h | 72 ++- .../transaction/ResumeAddEdgeProcessor.cpp | 23 +- .../transaction/ResumeAddEdgeProcessor.h | 4 +- .../ResumeAddEdgeRemoteProcessor.cpp | 35 +- .../ResumeAddEdgeRemoteProcessor.h | 4 +- .../transaction/ResumeUpdateProcessor.cpp | 31 +- .../transaction/ResumeUpdateProcessor.h | 4 +- .../ResumeUpdateRemoteProcessor.cpp | 23 +- .../transaction/ResumeUpdateRemoteProcessor.h | 4 +- .../transaction/TransactionManager.cpp | 63 ++- src/storage/transaction/TransactionManager.h | 40 +- 59 files changed, 2150 insertions(+), 610 deletions(-) create mode 100644 src/storage/test/ChainDeleteEdgesTest.cpp rename src/storage/transaction/{ChainAddEdgesProcessorLocal.cpp => ChainAddEdgesLocalProcessor.cpp} (82%) rename src/storage/transaction/{ChainAddEdgesProcessorLocal.h => ChainAddEdgesLocalProcessor.h} (89%) rename src/storage/transaction/{ChainAddEdgesProcessorRemote.cpp => ChainAddEdgesRemoteProcessor.cpp} (59%) rename src/storage/transaction/{ChainAddEdgesProcessorRemote.h => ChainAddEdgesRemoteProcessor.h} (58%) create mode 100644 src/storage/transaction/ChainDeleteEdgesGroupProcessor.cpp create mode 100644 src/storage/transaction/ChainDeleteEdgesGroupProcessor.h create mode 100644 src/storage/transaction/ChainDeleteEdgesLocalProcessor.cpp create mode 100644 src/storage/transaction/ChainDeleteEdgesLocalProcessor.h create mode 100644 src/storage/transaction/ChainDeleteEdgesRemoteProcessor.cpp create mode 100644 src/storage/transaction/ChainDeleteEdgesRemoteProcessor.h create mode 100644 src/storage/transaction/ChainDeleteEdgesResumeProcessor.cpp create mode 100644 src/storage/transaction/ChainDeleteEdgesResumeProcessor.h create mode 100644 src/storage/transaction/ChainDeleteEdgesResumeRemoteProcessor.cpp create mode 100644 src/storage/transaction/ChainDeleteEdgesResumeRemoteProcessor.h rename src/storage/transaction/{ChainUpdateEdgeProcessorLocal.cpp => ChainUpdateEdgeLocalProcessor.cpp} (67%) rename src/storage/transaction/{ChainUpdateEdgeProcessorLocal.h => ChainUpdateEdgeLocalProcessor.h} (86%) rename src/storage/transaction/{ChainUpdateEdgeProcessorRemote.cpp => ChainUpdateEdgeRemoteProcessor.cpp} (50%) rename src/storage/transaction/{ChainUpdateEdgeProcessorRemote.h => ChainUpdateEdgeRemoteProcessor.h} (64%) create mode 100644 src/storage/transaction/ConsistTypes.h 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/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/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/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/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/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/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/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/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/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/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 From f749d775b50b377758316634e575c6e4e5b496b3 Mon Sep 17 00:00:00 2001 From: panda-sheep <59197347+panda-sheep@users.noreply.github.com> Date: Wed, 29 Dec 2021 19:15:22 +0800 Subject: [PATCH 2/7] Synchronize the enterprise version to reduce conflicts (#3588) Co-authored-by: Sophie <84560950+Sophie-Xie@users.noreply.github.com> --- src/clients/meta/MetaClient.cpp | 54 +++++++----- src/clients/meta/MetaClient.h | 27 +++--- src/common/meta/SchemaManager.h | 3 +- src/common/meta/ServerBasedSchemaManager.cpp | 7 +- src/common/meta/ServerBasedSchemaManager.h | 3 +- src/common/utils/MetaKeyUtils.cpp | 38 +++++---- src/common/utils/MetaKeyUtils.h | 11 ++- src/graph/executor/CMakeLists.txt | 6 +- src/graph/executor/Executor.cpp | 18 ++-- .../admin/ShowServiceClientsExecutor.cpp | 48 +++++++++++ .../admin/ShowServiceClientsExecutor.h | 28 ++++++ .../executor/admin/ShowTSClientsExecutor.cpp | 41 --------- .../executor/admin/ShowTSClientsExecutor.h | 28 ------ ...Executor.cpp => SignInServiceExecutor.cpp} | 15 ++-- .../executor/admin/SignInServiceExecutor.h | 28 ++++++ .../executor/admin/SignInTSServiceExecutor.h | 28 ------ ...xecutor.cpp => SignOutServiceExecutor.cpp} | 15 ++-- ...iceExecutor.h => SignOutServiceExecutor.h} | 8 +- src/graph/planner/plan/Admin.h | 68 ++++++++++----- src/graph/planner/plan/PlanNode.cpp | 14 +-- src/graph/planner/plan/PlanNode.h | 8 +- src/graph/service/PermissionCheck.cpp | 8 +- src/graph/util/FTIndexUtils.cpp | 2 +- src/graph/validator/AdminValidator.cpp | 27 +++--- src/graph/validator/AdminValidator.h | 17 ++-- src/graph/validator/Validator.cpp | 12 +-- .../validator/test/MockSchemaManager.cpp | 3 +- src/graph/validator/test/MockSchemaManager.h | 3 +- src/interface/common.thrift | 1 + src/interface/meta.thrift | 27 +++--- .../plugins/elasticsearch/ESListener.cpp | 2 +- src/meta/CMakeLists.txt | 2 +- src/meta/MetaServiceHandler.cpp | 20 ++--- src/meta/MetaServiceHandler.h | 10 +-- src/meta/processors/Common.h | 2 +- .../processors/index/FTServiceProcessor.cpp | 80 ----------------- .../processors/index/FTServiceProcessor.h | 54 ------------ .../processors/service/ServiceProcessor.cpp | 85 +++++++++++++++++++ .../processors/service/ServiceProcessor.h | 56 ++++++++++++ src/meta/test/MetaClientTest.cpp | 16 ++-- src/mock/AdHocSchemaManager.cpp | 10 ++- src/mock/AdHocSchemaManager.h | 7 +- src/mock/MockCluster.cpp | 5 +- src/mock/MockCluster.h | 2 +- src/parser/AdminSentences.cpp | 29 +++++-- src/parser/AdminSentences.h | 54 ++++++++---- src/parser/Sentence.h | 6 +- src/parser/parser.yy | 50 +++++------ .../test/ElasticSearchBulkInsertTest.cpp | 2 +- 49 files changed, 607 insertions(+), 481 deletions(-) create mode 100644 src/graph/executor/admin/ShowServiceClientsExecutor.cpp create mode 100644 src/graph/executor/admin/ShowServiceClientsExecutor.h delete mode 100644 src/graph/executor/admin/ShowTSClientsExecutor.cpp delete mode 100644 src/graph/executor/admin/ShowTSClientsExecutor.h rename src/graph/executor/admin/{SignInTSServiceExecutor.cpp => SignInServiceExecutor.cpp} (57%) create mode 100644 src/graph/executor/admin/SignInServiceExecutor.h delete mode 100644 src/graph/executor/admin/SignInTSServiceExecutor.h rename src/graph/executor/admin/{SignOutTSServiceExecutor.cpp => SignOutServiceExecutor.cpp} (51%) rename src/graph/executor/admin/{SignOutTSServiceExecutor.h => SignOutServiceExecutor.h} (68%) delete mode 100644 src/meta/processors/index/FTServiceProcessor.cpp delete mode 100644 src/meta/processors/index/FTServiceProcessor.h create mode 100644 src/meta/processors/service/ServiceProcessor.cpp create mode 100644 src/meta/processors/service/ServiceProcessor.h diff --git a/src/clients/meta/MetaClient.cpp b/src/clients/meta/MetaClient.cpp index 5d03a17f95c..961d00f8162 100644 --- a/src/clients/meta/MetaClient.cpp +++ b/src/clients/meta/MetaClient.cpp @@ -184,8 +184,8 @@ bool MetaClient::loadData() { return false; } - if (!loadFulltextClients()) { - LOG(ERROR) << "Load fulltext services Failed"; + if (!loadGlobalServiceClients()) { + LOG(ERROR) << "Load global services Failed"; return false; } @@ -518,15 +518,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; }, @@ -3294,13 +3294,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; }, @@ -3309,25 +3310,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/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 603cf9e228b..862144e3669 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/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..b482cf1ab1c 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" @@ -489,17 +489,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)); 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/planner/plan/Admin.h b/src/graph/planner/plan/Admin.h index 0e6536587e9..fd1acf3682d 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/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/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 22b1b8fcb17..e9895ebb26a 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 c7291171200..1ee7bbacb68 100644 --- a/src/interface/meta.thrift +++ b/src/interface/meta.thrift @@ -1005,32 +1005,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 { @@ -1259,9 +1262,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/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/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/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/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/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/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 9b5f730b8b2..a6b0dd93ff2 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 @@ -1916,26 +1916,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; @@ -1944,7 +1944,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; @@ -1954,7 +1954,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; @@ -1965,29 +1965,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); } ; @@ -3418,7 +3418,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(); @@ -3900,8 +3900,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/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; From 6dd7446db4b6d3ed8c38759c806ff4251de6cf37 Mon Sep 17 00:00:00 2001 From: Doodle <13706157+critical27@users.noreply.github.com> Date: Wed, 29 Dec 2021 06:27:46 -0600 Subject: [PATCH 3/7] Revert "Support auto regist (#3548)" (#3592) This reverts commit 68493839c619872725ba96388cca5c3e7792e8d4. Co-authored-by: Yee <2520865+yixinglu@users.noreply.github.com> --- src/meta/processors/admin/HBProcessor.cpp | 4 +--- src/meta/processors/admin/HBProcessor.h | 2 -- src/meta/test/HBProcessorTest.cpp | 1 - src/meta/test/ProcessorTest.cpp | 4 ---- 4 files changed, 1 insertion(+), 10 deletions(-) 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/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/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()); { From 6a1c204a25d9932821ea27e6b189bad857d0b899 Mon Sep 17 00:00:00 2001 From: "jie.wang" <38901892+jievince@users.noreply.github.com> Date: Wed, 29 Dec 2021 21:01:32 +0800 Subject: [PATCH 4/7] add some space level metrics (#3590) --- src/graph/executor/Executor.cpp | 20 +++++++++++++++++++- src/graph/optimizer/rule/IndexScanRule.cpp | 1 + src/graph/service/GraphService.cpp | 10 ++++++++-- src/graph/service/QueryInstance.cpp | 19 ++++++++++++++++++- src/graph/session/ClientSession.cpp | 9 +++++++++ src/graph/stats/GraphStats.cpp | 5 +++-- src/graph/stats/GraphStats.h | 2 +- 7 files changed, 59 insertions(+), 7 deletions(-) diff --git a/src/graph/executor/Executor.cpp b/src/graph/executor/Executor.cpp index b482cf1ab1c..4e1dfee089b 100644 --- a/src/graph/executor/Executor.cpp +++ b/src/graph/executor/Executor.cpp @@ -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: { @@ -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/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/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/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; From 18d2fc9f1cf3c31c1122289412086742304d4c32 Mon Sep 17 00:00:00 2001 From: "lionel.liu@vesoft.com" <52276794+liuyu85cn@users.noreply.github.com> Date: Wed, 29 Dec 2021 21:38:28 +0800 Subject: [PATCH 5/7] Adjust get neighbor plan (#3458) * add GroupTagNode * address some comments * fix some comments * address comments from panda Co-authored-by: Sophie <84560950+Sophie-Xie@users.noreply.github.com> --- src/storage/CommonUtils.h | 4 + src/storage/exec/FilterNode.h | 50 +++- src/storage/exec/GetNeighborsNode.h | 10 +- src/storage/exec/MultiTagNode.h | 118 ++++++++++ src/storage/query/GetNeighborsProcessor.cpp | 88 ++++--- src/storage/test/GetNeighborsTest.cpp | 242 ++++++++++++++++---- 6 files changed, 427 insertions(+), 85 deletions(-) create mode 100644 src/storage/exec/MultiTagNode.h 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/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/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/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"}; From fa6d83853fd0bcd901f76f2d4e222780e6125afd Mon Sep 17 00:00:00 2001 From: shylock <33566796+Shylock-Hg@users.noreply.github.com> Date: Wed, 29 Dec 2021 22:11:09 +0800 Subject: [PATCH 6/7] Fix mistake modification. (#3596) Co-authored-by: Sophie <84560950+Sophie-Xie@users.noreply.github.com> --- tests/cert/test.ca.password | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 From 321f54916b5e446ab81309c365f062530daee97c Mon Sep 17 00:00:00 2001 From: "endy.li" <25311962+heroicNeZha@users.noreply.github.com> Date: Thu, 30 Dec 2021 10:46:20 +0800 Subject: [PATCH 7/7] simplify graph signal handler (#3542) * simplify graph signal handler * style * style * fix some comment Co-authored-by: Yee <2520865+yixinglu@users.noreply.github.com> Co-authored-by: Sophie <84560950+Sophie-Xie@users.noreply.github.com> --- src/daemons/GraphDaemon.cpp | 56 +++------------- src/graph/service/CMakeLists.txt | 1 + src/graph/service/GraphServer.cpp | 105 ++++++++++++++++++++++++++++++ src/graph/service/GraphServer.h | 46 +++++++++++++ 4 files changed, 162 insertions(+), 46 deletions(-) create mode 100644 src/graph/service/GraphServer.cpp create mode 100644 src/graph/service/GraphServer.h 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/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