diff --git a/src/clients/meta/MetaClient.cpp b/src/clients/meta/MetaClient.cpp index 7ce35d8acf7..ed4b380c549 100644 --- a/src/clients/meta/MetaClient.cpp +++ b/src/clients/meta/MetaClient.cpp @@ -2247,17 +2247,16 @@ bool MetaClient::checkShadowAccountFromCache(const std::string& account) const { return false; } -TermID MetaClient::getTermFromCache(GraphSpaceID spaceId, PartitionID partId) const { - static TermID notFound = -1; +StatusOr MetaClient::getTermFromCache(GraphSpaceID spaceId, PartitionID partId) const { folly::RWSpinLock::ReadHolder holder(localCacheLock_); auto spaceInfo = localCache_.find(spaceId); if (spaceInfo == localCache_.end()) { - return notFound; + return Status::Error("Term not found!"); } auto termInfo = spaceInfo->second->termOfPartition_.find(partId); if (termInfo == spaceInfo->second->termOfPartition_.end()) { - return notFound; + return Status::Error("Term not found!"); } return termInfo->second; diff --git a/src/clients/meta/MetaClient.h b/src/clients/meta/MetaClient.h index 546640b497d..84f49629755 100644 --- a/src/clients/meta/MetaClient.h +++ b/src/clients/meta/MetaClient.h @@ -31,6 +31,12 @@ DECLARE_int32(meta_client_retry_times); DECLARE_int32(heartbeat_interval_secs); +namespace nebula { +namespace storage { +class MetaClientTestUpdater; +} // namespace storage +} // namespace nebula + namespace nebula { namespace meta { @@ -182,6 +188,8 @@ class MetaClient { FRIEND_TEST(MetaClientTest, RetryUntilLimitTest); FRIEND_TEST(MetaClientTest, RocksdbOptionsTest); friend class KillQueryMetaWrapper; + FRIEND_TEST(ChainAddEdgesTest, AddEdgesLocalTest); + friend class storage::MetaClientTestUpdater; public: MetaClient(std::shared_ptr ioThreadPool, @@ -552,9 +560,9 @@ class MetaClient { bool authCheckFromCache(const std::string& account, const std::string& password) const; - bool checkShadowAccountFromCache(const std::string& account) const; + StatusOr getTermFromCache(GraphSpaceID spaceId, PartitionID) const; - TermID getTermFromCache(GraphSpaceID spaceId, PartitionID) const; + bool checkShadowAccountFromCache(const std::string& account) const; StatusOr> getStorageHosts() const; diff --git a/src/clients/storage/GraphStorageClient.cpp b/src/clients/storage/GraphStorageClient.cpp index 2c0ead60f3f..3c4961d25ad 100644 --- a/src/clients/storage/GraphStorageClient.cpp +++ b/src/clients/storage/GraphStorageClient.cpp @@ -165,7 +165,7 @@ folly::SemiFuture> GraphStorageClient::ad evb, std::move(requests), [=](cpp2::GraphStorageServiceAsyncClient* client, const cpp2::AddEdgesRequest& r) { - return useToss ? client->future_addEdgesAtomic(r) : client->future_addEdges(r); + return useToss ? client->future_chainAddEdges(r) : client->future_addEdges(r); }); } diff --git a/src/clients/storage/InternalStorageClient.cpp b/src/clients/storage/InternalStorageClient.cpp index 56a5bb122c7..1d20ccbc0df 100644 --- a/src/clients/storage/InternalStorageClient.cpp +++ b/src/clients/storage/InternalStorageClient.cpp @@ -12,7 +12,7 @@ namespace nebula { namespace storage { template -nebula::cpp2::ErrorCode extractErrorCode(T& tryResp) { +::nebula::cpp2::ErrorCode getErrorCode(T& tryResp) { if (!tryResp.hasValue()) { LOG(ERROR) << tryResp.exception().what(); return nebula::cpp2::ErrorCode::E_RPC_FAILURE; @@ -37,112 +37,98 @@ nebula::cpp2::ErrorCode extractErrorCode(T& tryResp) { return nebula::cpp2::ErrorCode::SUCCEEDED; } -StatusOr InternalStorageClient::getFuzzyLeader(GraphSpaceID spaceId, - PartitionID partId) const { - return getLeader(spaceId, partId); -} - -folly::SemiFuture InternalStorageClient::forwardTransaction( - int64_t txnId, - GraphSpaceID spaceId, - PartitionID partId, - std::string&& data, - folly::EventBase* evb) { - auto c = folly::makePromiseContract(); - forwardTransactionImpl(txnId, spaceId, partId, std::move(data), std::move(c.first), evb); - return std::move(c.second); -} - -void InternalStorageClient::forwardTransactionImpl(int64_t txnId, - GraphSpaceID spaceId, - PartitionID partId, - std::string&& data, - folly::Promise p, - folly::EventBase* evb) { - VLOG(1) << "forwardTransactionImpl txnId=" << txnId; - auto statusOrLeader = getFuzzyLeader(spaceId, partId); - if (!statusOrLeader.ok()) { - p.setValue(nebula::cpp2::ErrorCode::E_SPACE_NOT_FOUND); +void InternalStorageClient::chainUpdateEdge(cpp2::UpdateEdgeRequest& reversedRequest, + TermID termOfSrc, + folly::Optional optVersion, + folly::Promise<::nebula::cpp2::ErrorCode>&& p, + folly::EventBase* evb) { + auto spaceId = reversedRequest.get_space_id(); + auto partId = reversedRequest.get_part_id(); + auto optLeader = getLeader(spaceId, 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& dest = statusOrLeader.value(); - dest.port += kInternalPortOffset; - - cpp2::InternalTxnRequest interReq; - interReq.set_txn_id(txnId); - interReq.set_space_id(spaceId); - interReq.set_part_id(partId); - interReq.set_position(1); - (*interReq.data_ref()).resize(2); - (*interReq.data_ref()).back().emplace_back(data); - getResponse( + HostAddr& leader = optLeader.value(); + leader.port += kInternalPortOffset; + + cpp2::ChainUpdateEdgeRequest chainReq; + chainReq.set_update_edge_request(reversedRequest); + chainReq.set_term(termOfSrc); + if (optVersion) { + chainReq.set_edge_version(optVersion.value()); + } + auto resp = getResponse( evb, - std::make_pair(dest, interReq), - [](cpp2::InternalStorageServiceAsyncClient* client, const cpp2::InternalTxnRequest& r) { - return client->future_forwardTransaction(r); - }) - .thenTry([=, p = std::move(p)](auto&& t) mutable { - auto code = extractErrorCode(t); - if (code == nebula::cpp2::ErrorCode::E_LEADER_CHANGED) { - std::this_thread::sleep_for(std::chrono::milliseconds(500)); - return forwardTransactionImpl(txnId, spaceId, partId, std::move(data), std::move(p), evb); - } else { - p.setValue(code); - } + std::make_pair(leader, chainReq), + [](cpp2::InternalStorageServiceAsyncClient* client, const cpp2::ChainUpdateEdgeRequest& r) { + return client->future_chainUpdateEdge(r); }); -} - -folly::SemiFuture InternalStorageClient::getValue(size_t vIdLen, - GraphSpaceID spaceId, - folly::StringPiece key, - folly::EventBase* evb) { - auto srcVid = key.subpiece(sizeof(PartitionID), vIdLen); - auto partId = metaClient_->partId(spaceId, srcVid.str()); - auto c = folly::makePromiseContract(); - getValueImpl(spaceId, partId, key.str(), std::move(c.first), evb); - return std::move(c.second); -} - -void InternalStorageClient::getValueImpl(GraphSpaceID spaceId, - PartitionID partId, - std::string&& key, - folly::Promise p, - folly::EventBase* evb) { - std::pair req; - auto stLeaderHost = getFuzzyLeader(spaceId, partId); - if (!stLeaderHost.ok()) { - if (stLeaderHost.status().toString().find("partid")) { - p.setValue(nebula::cpp2::ErrorCode::E_PART_NOT_FOUND); + 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)); + chainUpdateEdge(reversedRequest, termOfSrc, optVersion, std::move(p)); } else { - p.setValue(nebula::cpp2::ErrorCode::E_SPACE_NOT_FOUND); + p.setValue(code); } return; - } - req.first = stLeaderHost.value(); - req.first.port += kInternalPortOffset; + }); +} - req.second.set_space_id(spaceId); - req.second.set_part_id(partId); - req.second.set_key(key); +void InternalStorageClient::chainAddEdges(cpp2::AddEdgesRequest& directReq, + TermID termId, + folly::Optional optVersion, + folly::Promise&& p, + folly::EventBase* evb) { + auto spaceId = directReq.get_space_id(); + 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); + p.setValue(::nebula::cpp2::ErrorCode::E_SPACE_NOT_FOUND); + return; + } + HostAddr& leader = optLeader.value(); + leader.port += kInternalPortOffset; + LOG(INFO) << "leader host: " << leader; - auto remote = [](cpp2::InternalStorageServiceAsyncClient* client, - const cpp2::GetValueRequest& r) { return client->future_getValue(r); }; + cpp2::ChainAddEdgesRequest chainReq = makeChainAddReq(directReq, termId, optVersion); + auto resp = getResponse( + evb, + std::make_pair(leader, chainReq), + [](cpp2::InternalStorageServiceAsyncClient* client, const cpp2::ChainAddEdgesRequest& r) { + return client->future_chainAddEdges(r); + }); - auto cb = [=, p = std::move(p)](auto&& t) mutable { - auto code = extractErrorCode(t); - if (code == nebula::cpp2::ErrorCode::SUCCEEDED) { - p.setValue(t.value().value().get_value()); - } else if (code == nebula::cpp2::ErrorCode::E_LEADER_CHANGED) { - // retry directly may easily get same error + 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)); - return getValueImpl(spaceId, partId, std::move(key), std::move(p), evb); + chainAddEdges(directReq, termId, optVersion, std::move(p)); } else { p.setValue(code); } - }; + return; + }); +} - getResponse(evb, std::move(req), remote).thenTry(std::move(cb)); +cpp2::ChainAddEdgesRequest InternalStorageClient::makeChainAddReq(const cpp2::AddEdgesRequest& req, + TermID termId, + folly::Optional ver) { + cpp2::ChainAddEdgesRequest ret; + ret.set_space_id(req.get_space_id()); + ret.set_parts(req.get_parts()); + ret.set_prop_names(req.get_prop_names()); + ret.set_if_not_exists(req.get_if_not_exists()); + ret.set_term(termId); + if (ver) { + ret.set_edge_version(ver.value()); + } + return ret; } } // namespace storage diff --git a/src/clients/storage/InternalStorageClient.h b/src/clients/storage/InternalStorageClient.h index d3e7ae0ee6e..6ef772cb4f5 100644 --- a/src/clients/storage/InternalStorageClient.h +++ b/src/clients/storage/InternalStorageClient.h @@ -17,8 +17,6 @@ namespace nebula { namespace storage { -typedef ErrorOr ErrOrVal; - /** * A wrapper class for InternalStorageServiceAsyncClient thrift API * @@ -33,32 +31,22 @@ class InternalStorageClient : public StorageClientBase forwardTransaction(int64_t txnId, - GraphSpaceID spaceId, - PartitionID partId, - std::string&& data, - folly::EventBase* evb = nullptr); - - folly::SemiFuture getValue(size_t vIdLen, - GraphSpaceID spaceId, - folly::StringPiece key, - folly::EventBase* evb = nullptr); - - protected: - StatusOr getFuzzyLeader(GraphSpaceID spaceId, PartitionID partId) const; - - void forwardTransactionImpl(int64_t txnId, - GraphSpaceID spaceId, - PartitionID partId, - std::string&& data, - folly::Promise p, - folly::EventBase* evb); - - void getValueImpl(GraphSpaceID spaceId, - PartitionID partId, - std::string&& key, - folly::Promise p, - folly::EventBase* evb = nullptr); + virtual void chainUpdateEdge(cpp2::UpdateEdgeRequest& reversedRequest, + TermID termOfSrc, + folly::Optional optVersion, + folly::Promise<::nebula::cpp2::ErrorCode>&& p, + folly::EventBase* evb = nullptr); + + virtual void chainAddEdges(cpp2::AddEdgesRequest& req, + TermID termId, + folly::Optional optVersion, + folly::Promise<::nebula::cpp2::ErrorCode>&& p, + folly::EventBase* evb = nullptr); + + private: + cpp2::ChainAddEdgesRequest makeChainAddReq(const cpp2::AddEdgesRequest& req, + TermID termId, + folly::Optional optVersion); }; } // namespace storage diff --git a/src/clients/storage/StorageClientBase.h b/src/clients/storage/StorageClientBase.h index bb15f105f58..165a1e7526d 100644 --- a/src/clients/storage/StorageClientBase.h +++ b/src/clients/storage/StorageClientBase.h @@ -209,13 +209,13 @@ class StorageClientBase { return {req.get_part_id()}; } - std::vector getReqPartsId(const cpp2::InternalTxnRequest& req) const { - return {req.get_part_id()}; - } + // std::vector getReqPartsId(const cpp2::InternalTxnRequest& req) const { + // return {req.get_part_id()}; + // } - std::vector getReqPartsId(const cpp2::GetValueRequest& req) const { - return {req.get_part_id()}; - } + // std::vector getReqPartsId(const cpp2::GetValueRequest& req) const { + // return {req.get_part_id()}; + // } std::vector getReqPartsId(const cpp2::ScanEdgeRequest& req) const { return {req.get_part_id()}; diff --git a/src/common/utils/MemoryLockWrapper.h b/src/common/utils/MemoryLockWrapper.h index 77d157f167a..ca8f2c90179 100644 --- a/src/common/utils/MemoryLockWrapper.h +++ b/src/common/utils/MemoryLockWrapper.h @@ -68,6 +68,13 @@ class MemoryLockGuard { return *iter_; } + // this will manual set the lock to unlocked state + // which mean will not release all locks automaticly + // please make sure you really know the side effect + void forceLock() { locked_ = true; } + + void forceUnlock() { locked_ = false; } + protected: MemoryLockCore* lock_; std::vector keys_; diff --git a/src/graph/executor/mutate/InsertExecutor.cpp b/src/graph/executor/mutate/InsertExecutor.cpp index 55eb86ca087..a625ca8581a 100644 --- a/src/graph/executor/mutate/InsertExecutor.cpp +++ b/src/graph/executor/mutate/InsertExecutor.cpp @@ -8,6 +8,7 @@ #include "graph/context/QueryContext.h" #include "graph/planner/plan/Mutate.h" +#include "graph/service/GraphFlags.h" #include "graph/util/ScopedTimer.h" namespace nebula { @@ -55,7 +56,7 @@ folly::Future InsertEdgesExecutor::insertEdges() { ieNode->getPropNames(), ieNode->getIfNotExists(), nullptr, - ieNode->useChainInsert()) + FLAGS_enable_experimental_feature) .via(runner()) .ensure( [addEdgeTime]() { VLOG(1) << "Add edge time: " << addEdgeTime.elapsedInUSec() << "us"; }) diff --git a/src/graph/service/GraphFlags.cpp b/src/graph/service/GraphFlags.cpp index 08e3164ae75..e1bc9abebb9 100644 --- a/src/graph/service/GraphFlags.cpp +++ b/src/graph/service/GraphFlags.cpp @@ -62,3 +62,5 @@ DEFINE_bool(disable_octal_escape_char, false, "Octal escape character will be disabled" " in next version to ensure compatibility with cypher."); + +DEFINE_bool(enable_experimental_feature, false, "Whether to enable experimental feature"); diff --git a/src/graph/service/GraphFlags.h b/src/graph/service/GraphFlags.h index f9af5d756a9..40b9fd5bd71 100644 --- a/src/graph/service/GraphFlags.h +++ b/src/graph/service/GraphFlags.h @@ -46,4 +46,6 @@ DECLARE_int64(max_allowed_connections); DECLARE_string(local_ip); +DECLARE_bool(enable_experimental_feature); + #endif // GRAPH_GRAPHFLAGS_H_ diff --git a/src/graph/validator/AdminValidator.cpp b/src/graph/validator/AdminValidator.cpp index f797e4e08ca..6436bb6c82a 100644 --- a/src/graph/validator/AdminValidator.cpp +++ b/src/graph/validator/AdminValidator.cpp @@ -93,8 +93,6 @@ Status CreateSpaceValidator::validateImpl() { case SpaceOptItem::ATOMIC_EDGE: { if (item->getAtomicEdge()) { spaceDesc_.set_isolation_level(meta::cpp2::IsolationLevel::TOSS); - // for 2.0 GA, no matter how this option set, don't use toss. - return ::nebula::Status::NotSupported("not support enable toss in 2.0 GA"); } else { spaceDesc_.set_isolation_level(meta::cpp2::IsolationLevel::DEFAULT); } diff --git a/src/graph/validator/MutateValidator.cpp b/src/graph/validator/MutateValidator.cpp index a99edceadb5..d820e5c3616 100644 --- a/src/graph/validator/MutateValidator.cpp +++ b/src/graph/validator/MutateValidator.cpp @@ -191,10 +191,11 @@ Status InsertEdgesValidator::check() { } Status InsertEdgesValidator::prepareEdges() { - using IsoLevel = meta::cpp2::IsolationLevel; - auto isoLevel = space_.spaceDesc.isolation_level_ref().value_or(IsoLevel::DEFAULT); - auto useToss = isoLevel == IsoLevel::TOSS; - auto size = useToss ? rows_.size() : rows_.size() * 2; + // using IsoLevel = meta::cpp2::IsolationLevel; + // auto isoLevel = space_.spaceDesc.isolation_level_ref().value_or(IsoLevel::DEFAULT); + // auto useToss = isoLevel == IsoLevel::TOSS; + // auto size = useToss ? rows_.size() : rows_.size() * 2; + auto size = FLAGS_enable_experimental_feature ? rows_.size() : rows_.size() * 2; edges_.reserve(size); for (auto i = 0u; i < rows_.size(); i++) { auto *row = rows_[i]; @@ -243,7 +244,8 @@ Status InsertEdgesValidator::prepareEdges() { edge.set_key(key); edge.set_props(std::move(props)); edges_.emplace_back(edge); - if (!useToss) { + // if (!useToss) { + if (!FLAGS_enable_experimental_feature) { // inbound key.set_src(dstId); key.set_dst(srcId); @@ -759,10 +761,11 @@ Status UpdateEdgeValidator::toPlan() { {}, condition_, {}); - using IsoLevel = meta::cpp2::IsolationLevel; - auto isoLevel = space_.spaceDesc.isolation_level_ref().value_or(IsoLevel::DEFAULT); - auto useToss = isoLevel == IsoLevel::TOSS; - if (useToss) { + // using IsoLevel = meta::cpp2::IsolationLevel; + // auto isoLevel = space_.spaceDesc.isolation_level_ref().value_or(IsoLevel::DEFAULT); + // auto useToss = isoLevel == IsoLevel::TOSS; + // if (useToss) { + if (FLAGS_enable_experimental_feature) { root_ = outNode; tail_ = root_; } else { diff --git a/src/interface/common.thrift b/src/interface/common.thrift index 6f05fb32fbf..9763691f246 100644 --- a/src/interface/common.thrift +++ b/src/interface/common.thrift @@ -397,6 +397,11 @@ enum ErrorCode { E_TASK_EXECUTION_FAILED = -3053, E_PLAN_IS_KILLED = -3060, + // toss + E_NO_TERM = -3070 + E_OUTDATED_TERM = -3071, + E_OUTDATED_EDGE = -3072, + E_WRITE_WRITE_CONFLICT = -3073, E_UNKNOWN = -8000, } (cpp.enum_strict) diff --git a/src/interface/storage.thrift b/src/interface/storage.thrift index d3afad1ee22..302f0682176 100644 --- a/src/interface/storage.thrift +++ b/src/interface/storage.thrift @@ -662,7 +662,9 @@ service GraphStorageService { LookupIndexResp lookupIndex(1: LookupIndexRequest req); GetNeighborsResponse lookupAndTraverse(1: LookupAndTraverseRequest req); - ExecResponse addEdgesAtomic(1: AddEdgesRequest req); + + UpdateResponse chainUpdateEdge(1: UpdateEdgeRequest req); + ExecResponse chainAddEdges(1: AddEdgesRequest req); } @@ -863,27 +865,49 @@ service GeneralStorageService { // transaction request struct InternalTxnRequest { - 1: i64 txn_id, - 2: i32 space_id, - // need this(part_id) to satisfy getResponse - 3: i32 part_id, - // position of chain - 4: i32 position, - 5: list> data + 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 GetValueRequest { - 1: common.GraphSpaceID space_id, - 2: common.PartitionID part_id, - 3: binary key +// +// Response for data modification requests +// +struct ChainResponse { + 1: required ResponseCommon result, } -struct GetValueResponse { - 1: required ResponseCommon result - 2: binary value + +struct ChainAddEdgesRequest { + 1: common.GraphSpaceID space_id, + // partId => edges + 2: map>( + cpp.template = "std::unordered_map") parts, + // A list of property names. The order of the property names should match + // the data order specified in the NewEdge.props + 3: list prop_names, + // if ture, 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>( + // cpp.template = "std::unordered_map") edge_ver, +} + + +struct ChainUpdateEdgeRequest { + 1: UpdateEdgeRequest update_edge_request, + 2: i64 term, + 3: optional i64 edge_version + 4: common.GraphSpaceID space_id, + 5: required list parts, } service InternalStorageService { - GetValueResponse getValue(1: GetValueRequest req); - ExecResponse forwardTransaction(1: InternalTxnRequest req); + ExecResponse chainAddEdges(1: ChainAddEdgesRequest req); + UpdateResponse chainUpdateEdge(1: ChainUpdateEdgeRequest req); } diff --git a/src/meta/ActiveHostsMan.cpp b/src/meta/ActiveHostsMan.cpp index 1a2a46c24c2..19c2e2e9bb1 100644 --- a/src/meta/ActiveHostsMan.cpp +++ b/src/meta/ActiveHostsMan.cpp @@ -37,17 +37,16 @@ nebula::cpp2::ErrorCode ActiveHostsMan::updateHostInfo(kvstore::KVStore* kv, } auto keys = leaderKeys; std::vector vals; - // let see if this c++17 syntax can pass - auto [rc, statuses] = kv->multiGet(kDefaultSpaceId, kDefaultPartId, std::move(keys), &vals); + auto [rc, statusVec] = kv->multiGet(kDefaultSpaceId, kDefaultPartId, std::move(keys), &vals); if (rc != nebula::cpp2::ErrorCode::SUCCEEDED && rc != nebula::cpp2::ErrorCode::E_PARTIAL_RESULT) { LOG(INFO) << "error rc = " << apache::thrift::util::enumNameSafe(rc); return rc; } - TermID term; + TermID term = -1; nebula::cpp2::ErrorCode code; - for (auto i = 0U; i != statuses.size(); ++i) { - if (statuses[i].ok()) { + for (auto i = 0U; i != leaderKeys.size(); ++i) { + if (statusVec[i].ok()) { std::tie(std::ignore, term, code) = MetaServiceUtils::parseLeaderValV3(vals[i]); if (code != nebula::cpp2::ErrorCode::SUCCEEDED) { LOG(WARNING) << apache::thrift::util::enumNameSafe(code); @@ -61,6 +60,8 @@ nebula::cpp2::ErrorCode ActiveHostsMan::updateHostInfo(kvstore::KVStore* kv, auto val = MetaServiceUtils::leaderValV3(hostAddr, terms[i]); data.emplace_back(std::make_pair(leaderKeys[i], std::move(val))); } + } else { + // do nothing } // indicate whether any leader info is updated bool hasUpdate = false; diff --git a/src/meta/processors/parts/GetPartsAllocProcessor.cpp b/src/meta/processors/parts/GetPartsAllocProcessor.cpp index ead2ce8a964..16a7cdb3ae7 100644 --- a/src/meta/processors/parts/GetPartsAllocProcessor.cpp +++ b/src/meta/processors/parts/GetPartsAllocProcessor.cpp @@ -34,8 +34,60 @@ void GetPartsAllocProcessor::process(const cpp2::GetPartsAllocReq& req) { } handleErrorCode(nebula::cpp2::ErrorCode::SUCCEEDED); resp_.set_parts(std::move(parts)); + auto terms = getTerm(spaceId); + if (!terms.empty()) { + resp_.set_terms(std::move(terms)); + } onFinished(); } +std::unordered_map GetPartsAllocProcessor::getTerm(GraphSpaceID spaceId) { + std::unordered_map ret; + + auto spaceKey = MetaServiceUtils::spaceKey(spaceId); + auto spaceVal = doGet(spaceKey); + if (!nebula::ok(spaceVal)) { + auto rc = nebula::error(spaceVal); + LOG(ERROR) << "Get Space SpaceId: " << spaceId + << " error: " << apache::thrift::util::enumNameSafe(rc); + handleErrorCode(rc); + return ret; + } + + auto properties = MetaServiceUtils::parseSpace(nebula::value(spaceVal)); + auto partNum = properties.get_partition_num(); + + std::vector partIdVec; + std::vector leaderKeys; + for (auto partId = 1; partId <= partNum; ++partId) { + partIdVec.emplace_back(partId); + leaderKeys.emplace_back(MetaServiceUtils::leaderKey(spaceId, partId)); + } + + std::vector vals; + auto [code, statusVec] = kvstore_->multiGet(0, 0, std::move(leaderKeys), &vals); + if (code != nebula::cpp2::ErrorCode::SUCCEEDED && + code != nebula::cpp2::ErrorCode::E_PARTIAL_RESULT) { + LOG(INFO) << "error rc = " << apache::thrift::util::enumNameSafe(code); + return ret; + } + + TermID term; + for (auto i = 0U; i != vals.size(); ++i) { + if (statusVec[i].ok()) { + std::tie(std::ignore, term, code) = MetaServiceUtils::parseLeaderValV3(vals[i]); + if (code != nebula::cpp2::ErrorCode::SUCCEEDED) { + LOG(WARNING) << apache::thrift::util::enumNameSafe(code); + LOG(INFO) << folly::sformat("term of part {} is invalid", partIdVec[i]); + continue; + } + LOG(INFO) << folly::sformat("term of part {} is {}", partIdVec[i], term); + ret[partIdVec[i]] = term; + } + } + + return ret; +} + } // namespace meta } // namespace nebula diff --git a/src/meta/processors/parts/GetPartsAllocProcessor.h b/src/meta/processors/parts/GetPartsAllocProcessor.h index b5c0be2d71f..1fb318c45bc 100644 --- a/src/meta/processors/parts/GetPartsAllocProcessor.h +++ b/src/meta/processors/parts/GetPartsAllocProcessor.h @@ -23,6 +23,8 @@ class GetPartsAllocProcessor : public BaseProcessor { private: explicit GetPartsAllocProcessor(kvstore::KVStore* kvstore) : BaseProcessor(kvstore) {} + + std::unordered_map getTerm(GraphSpaceID space); }; } // namespace meta diff --git a/src/mock/MockCluster.cpp b/src/mock/MockCluster.cpp index 821da555839..3e5c3de80d1 100644 --- a/src/mock/MockCluster.cpp +++ b/src/mock/MockCluster.cpp @@ -16,6 +16,7 @@ #include "storage/GeneralStorageServiceHandler.h" #include "storage/GraphStorageServiceHandler.h" #include "storage/StorageAdminServiceHandler.h" +#include "storage/transaction/TransactionManager.h" DECLARE_int32(heartbeat_interval_secs); @@ -210,6 +211,9 @@ void MockCluster::initStorageKV(const char* dataPath, storageEnv_->rebuildIndexGuard_ = std::make_unique(); storageEnv_->verticesML_ = std::make_unique(); storageEnv_->edgesML_ = std::make_unique(); + + txnMan_ = std::make_unique(storageEnv_.get()); + storageEnv_->txnMan_ = txnMan_.get(); } void MockCluster::startStorage(HostAddr addr, diff --git a/src/mock/MockCluster.h b/src/mock/MockCluster.h index da9e24d5afd..076821cb916 100644 --- a/src/mock/MockCluster.h +++ b/src/mock/MockCluster.h @@ -24,6 +24,7 @@ #include "storage/BaseProcessor.h" #include "storage/GraphStorageServiceHandler.h" #include "storage/StorageAdminServiceHandler.h" +#include "storage/transaction/TransactionManager.h" namespace nebula { namespace mock { @@ -135,6 +136,7 @@ class MockCluster { std::unique_ptr esListener_{nullptr}; std::unique_ptr lSchemaMan_; std::unique_ptr lMetaClient_{nullptr}; + std::unique_ptr txnMan_{nullptr}; ObjectPool pool_; }; diff --git a/src/storage/CMakeLists.txt b/src/storage/CMakeLists.txt index 8589e5e6a83..4080b350d9c 100644 --- a/src/storage/CMakeLists.txt +++ b/src/storage/CMakeLists.txt @@ -32,7 +32,6 @@ nebula_add_library( mutate/DeleteVerticesProcessor.cpp mutate/DeleteTagsProcessor.cpp mutate/AddEdgesProcessor.cpp - mutate/AddEdgesAtomicProcessor.cpp mutate/DeleteEdgesProcessor.cpp mutate/UpdateVertexProcessor.cpp mutate/UpdateEdgeProcessor.cpp @@ -67,9 +66,18 @@ nebula_add_library( nebula_add_library( storage_transaction_executor OBJECT transaction/TransactionManager.cpp - transaction/TransactionProcessor.cpp - transaction/GetValueProcessor.cpp - transaction/TransactionUtils.cpp + transaction/ConsistUtil.cpp + transaction/ChainUpdateEdgeProcessorLocal.cpp + transaction/ChainUpdateEdgeProcessorRemote.cpp + transaction/ChainResumeProcessor.cpp + transaction/ChainAddEdgesGroupProcessor.cpp + transaction/ChainAddEdgesProcessorLocal.cpp + transaction/ChainAddEdgesProcessorRemote.cpp + transaction/ResumeAddEdgeProcessor.cpp + transaction/ResumeAddEdgeRemoteProcessor.cpp + transaction/ResumeUpdateProcessor.cpp + transaction/ResumeUpdateRemoteProcessor.cpp + transaction/ChainProcessorFactory.cpp ) nebula_add_library( diff --git a/src/storage/CommonUtils.h b/src/storage/CommonUtils.h index 52f57b0ea47..b2e5f1df373 100644 --- a/src/storage/CommonUtils.h +++ b/src/storage/CommonUtils.h @@ -61,6 +61,7 @@ using VerticesMemLock = MemoryLockCore; using EdgesMemLock = MemoryLockCore; class TransactionManager; +class InternalStorageClient; // unify TagID, EdgeType using SchemaID = TagID; @@ -74,6 +75,7 @@ class StorageEnv { std::atomic onFlyingRequest_{0}; std::unique_ptr rebuildIndexGuard_{nullptr}; meta::MetaClient* metaClient_{nullptr}; + InternalStorageClient* interClient_{nullptr}; TransactionManager* txnMan_{nullptr}; std::unique_ptr verticesML_{nullptr}; std::unique_ptr edgesML_{nullptr}; diff --git a/src/storage/GraphStorageServiceHandler.cpp b/src/storage/GraphStorageServiceHandler.cpp index 6a95903a3a7..ec77f713943 100644 --- a/src/storage/GraphStorageServiceHandler.cpp +++ b/src/storage/GraphStorageServiceHandler.cpp @@ -7,7 +7,6 @@ #include "storage/GraphStorageServiceHandler.h" #include "storage/index/LookupProcessor.h" -#include "storage/mutate/AddEdgesAtomicProcessor.h" #include "storage/mutate/AddEdgesProcessor.h" #include "storage/mutate/AddVerticesProcessor.h" #include "storage/mutate/DeleteEdgesProcessor.h" @@ -19,7 +18,8 @@ #include "storage/query/GetPropProcessor.h" #include "storage/query/ScanEdgeProcessor.h" #include "storage/query/ScanVertexProcessor.h" -#include "storage/transaction/TransactionProcessor.h" +#include "storage/transaction/ChainAddEdgesGroupProcessor.h" +#include "storage/transaction/ChainUpdateEdgeProcessorLocal.h" #define RETURN_FUTURE(processor) \ auto f = processor->getFuture(); \ @@ -48,7 +48,6 @@ GraphStorageServiceHandler::GraphStorageServiceHandler(StorageEnv* env) : env_(e // Initialize all counters kAddVerticesCounters.init("add_vertices"); kAddEdgesCounters.init("add_edges"); - kAddEdgesAtomicCounters.init("add_edges_atomic"); kDelVerticesCounters.init("delete_vertices"); kDelTagsCounters.init("delete_tags"); kDelEdgesCounters.init("delete_edges"); @@ -106,6 +105,12 @@ folly::Future GraphStorageServiceHandler::future_updateEdg RETURN_FUTURE(processor); } +folly::Future GraphStorageServiceHandler::future_chainUpdateEdge( + const cpp2::UpdateEdgeRequest& req) { + auto* proc = ChainUpdateEdgeProcessorLocal::instance(env_); + RETURN_FUTURE(proc); +} + folly::Future GraphStorageServiceHandler::future_getNeighbors( const cpp2::GetNeighborsRequest& req) { auto* processor = @@ -140,11 +145,13 @@ folly::Future GraphStorageServiceHandler::future_scanEdg folly::Future GraphStorageServiceHandler::future_getUUID( const cpp2::GetUUIDReq&) { LOG(FATAL) << "Unsupported in version 2.0"; + cpp2::GetUUIDResp ret; + return ret; } -folly::Future GraphStorageServiceHandler::future_addEdgesAtomic( +folly::Future GraphStorageServiceHandler::future_chainAddEdges( const cpp2::AddEdgesRequest& req) { - auto* processor = AddEdgesAtomicProcessor::instance(env_, &kAddEdgesAtomicCounters); + auto* processor = ChainAddEdgesGroupProcessor::instance(env_); RETURN_FUTURE(processor); } diff --git a/src/storage/GraphStorageServiceHandler.h b/src/storage/GraphStorageServiceHandler.h index ef0a0efd915..a46c43fee8e 100644 --- a/src/storage/GraphStorageServiceHandler.h +++ b/src/storage/GraphStorageServiceHandler.h @@ -22,7 +22,6 @@ class StorageEnv; class GraphStorageServiceHandler final : public cpp2::GraphStorageServiceSvIf { public: explicit GraphStorageServiceHandler(StorageEnv* env); - // Vertice section folly::Future future_addVertices( const cpp2::AddVerticesRequest& req) override; @@ -52,8 +51,10 @@ class GraphStorageServiceHandler final : public cpp2::GraphStorageServiceSvIf { folly::Future future_lookupIndex( const cpp2::LookupIndexRequest& req) override; - folly::Future future_addEdgesAtomic( - const cpp2::AddEdgesRequest& req) override; + folly::Future future_chainUpdateEdge( + const cpp2::UpdateEdgeRequest& req) override; + + folly::Future future_chainAddEdges(const cpp2::AddEdgesRequest& req) override; folly::Future future_scanVertex( const cpp2::ScanVertexRequest& req) override; diff --git a/src/storage/InternalStorageServiceHandler.cpp b/src/storage/InternalStorageServiceHandler.cpp index 09b05046850..5fb972629f7 100644 --- a/src/storage/InternalStorageServiceHandler.cpp +++ b/src/storage/InternalStorageServiceHandler.cpp @@ -6,8 +6,8 @@ #include "storage/InternalStorageServiceHandler.h" -#include "storage/transaction/GetValueProcessor.h" -#include "storage/transaction/TransactionProcessor.h" +#include "storage/transaction/ChainAddEdgesProcessorRemote.h" +#include "storage/transaction/ChainUpdateEdgeProcessorRemote.h" #define RETURN_FUTURE(processor) \ auto f = processor->getFuture(); \ @@ -17,20 +17,17 @@ namespace nebula { namespace storage { -InternalStorageServiceHandler::InternalStorageServiceHandler(StorageEnv* env) : env_(env) { - kForwardTranxCounters.init("forward_tranx"); - kGetValueCounters.init("get_value"); -} +InternalStorageServiceHandler::InternalStorageServiceHandler(StorageEnv* env) : env_(env) {} -folly::Future InternalStorageServiceHandler::future_forwardTransaction( - const cpp2::InternalTxnRequest& req) { - auto* processor = InterTxnProcessor::instance(env_); +folly::Future InternalStorageServiceHandler::future_chainAddEdges( + const cpp2::ChainAddEdgesRequest& req) { + auto* processor = ChainAddEdgesProcessorRemote::instance(env_); RETURN_FUTURE(processor); } -folly::Future InternalStorageServiceHandler::future_getValue( - const cpp2::GetValueRequest& req) { - auto* processor = GetValueProcessor::instance(env_); +folly::Future InternalStorageServiceHandler::future_chainUpdateEdge( + const cpp2::ChainUpdateEdgeRequest& req) { + auto* processor = ChainUpdateEdgeProcessorRemote::instance(env_); RETURN_FUTURE(processor); } diff --git a/src/storage/InternalStorageServiceHandler.h b/src/storage/InternalStorageServiceHandler.h index d173ca9c5a5..60ebe668156 100644 --- a/src/storage/InternalStorageServiceHandler.h +++ b/src/storage/InternalStorageServiceHandler.h @@ -23,10 +23,10 @@ class InternalStorageServiceHandler final : public cpp2::InternalStorageServiceS public: explicit InternalStorageServiceHandler(StorageEnv* env); - folly::Future future_forwardTransaction( - const cpp2::InternalTxnRequest& req) override; + folly::Future future_chainAddEdges(const cpp2::ChainAddEdgesRequest& p_req); - folly::Future future_getValue(const cpp2::GetValueRequest& req) override; + folly::Future future_chainUpdateEdge( + const cpp2::ChainUpdateEdgeRequest& p_req); private: StorageEnv* env_{nullptr}; diff --git a/src/storage/StorageServer.cpp b/src/storage/StorageServer.cpp index 687448f3077..a13d435065e 100644 --- a/src/storage/StorageServer.cpp +++ b/src/storage/StorageServer.cpp @@ -159,14 +159,21 @@ bool StorageServer::start() { return false; } + interClient_ = std::make_unique(ioThreadPool_, metaClient_.get()); + env_ = std::make_unique(); env_->kvstore_ = kvstore_.get(); env_->indexMan_ = indexMan_.get(); env_->schemaMan_ = schemaMan_.get(); env_->rebuildIndexGuard_ = std::make_unique(); env_->metaClient_ = metaClient_.get(); + env_->interClient_ = interClient_.get(); txnMan_ = std::make_unique(env_.get()); + if (!txnMan_->start()) { + LOG(ERROR) << "Start transaction manager failed!"; + return false; + } env_->txnMan_ = txnMan_.get(); env_->verticesML_ = std::make_unique(); @@ -300,6 +307,10 @@ void StorageServer::stop() { if (adminServer_) { adminServer_->stop(); } + if (txnMan_) { + txnMan_->stop(); + txnMan_.reset(); + } if (internalStorageServer_) { internalStorageServer_->stop(); } diff --git a/src/storage/StorageServer.h b/src/storage/StorageServer.h index 21d7027da08..3fffcb98ba8 100644 --- a/src/storage/StorageServer.h +++ b/src/storage/StorageServer.h @@ -17,6 +17,7 @@ #include "kvstore/NebulaStore.h" #include "storage/CommonUtils.h" #include "storage/admin/AdminTaskManager.h" +#include "storage/transaction/TransactionManager.h" namespace nebula { @@ -81,7 +82,9 @@ class StorageServer final { std::string listenerPath_; AdminTaskManager* taskMgr_{nullptr}; - std::unique_ptr txnMan_; + std::unique_ptr txnMan_{nullptr}; + // used for communicate between one storaged to another + std::unique_ptr interClient_; }; } // namespace storage diff --git a/src/storage/exec/EdgeNode.h b/src/storage/exec/EdgeNode.h index a63285b9820..64b5d145b2b 100644 --- a/src/storage/exec/EdgeNode.h +++ b/src/storage/exec/EdgeNode.h @@ -10,8 +10,6 @@ #include "common/base/Base.h" #include "storage/exec/RelNode.h" #include "storage/exec/StorageIterator.h" -#include "storage/transaction/TossEdgeIterator.h" -#include "storage/transaction/TransactionManager.h" namespace nebula { namespace storage { diff --git a/src/storage/exec/UpdateNode.h b/src/storage/exec/UpdateNode.h index 05e56ab5cdc..8d40a993ac6 100644 --- a/src/storage/exec/UpdateNode.h +++ b/src/storage/exec/UpdateNode.h @@ -510,33 +510,20 @@ class UpdateEdgeNode : public UpdateNode { } }; - if (context_->env()->txnMan_ && context_->env()->txnMan_->enableToss(context_->spaceId())) { - LOG(INFO) << "before update edge atomic" << TransactionUtils::dumpKey(edgeKey); - auto f = context_->env()->txnMan_->updateEdgeAtomic( - context_->vIdLen(), context_->spaceId(), partId, edgeKey, std::move(op)); - f.wait(); - - if (f.valid()) { - ret = f.value(); - } else { - ret = nebula::cpp2::ErrorCode::E_UNKNOWN; - } - } else { - auto batch = op(); - if (batch == folly::none) { - return this->exeResult_; - } + auto batch = op(); + if (batch == folly::none) { + return this->exeResult_; + } - folly::Baton baton; - auto callback = [&ret, &baton](nebula::cpp2::ErrorCode code) { - ret = code; - baton.post(); - }; + folly::Baton baton; + auto callback = [&ret, &baton](nebula::cpp2::ErrorCode code) { + ret = code; + baton.post(); + }; - context_->env()->kvstore_->asyncAppendBatch( - context_->spaceId(), partId, std::move(batch).value(), callback); - baton.wait(); - } + context_->planContext_->env_->kvstore_->asyncAppendBatch( + context_->planContext_->spaceId_, partId, std::move(batch).value(), callback); + baton.wait(); return ret; } @@ -730,6 +717,14 @@ class UpdateEdgeNode : public UpdateNode { } // step 3, insert new edge data batchHolder->put(std::move(key_), std::move(nVal)); + + // extra phase: if there are some extra requirement. + for (auto& [k, v] : edgeContext_->kvAppend) { + batchHolder->put(std::move(k), std::move(v)); + } + for (auto& k : edgeContext_->kvErased) { + batchHolder->remove(std::move(k)); + } return encodeBatchValue(batchHolder->getBatch()); } diff --git a/src/storage/index/LookupBaseProcessor-inl.h b/src/storage/index/LookupBaseProcessor-inl.h index 9980ddb30bc..54333452337 100644 --- a/src/storage/index/LookupBaseProcessor-inl.h +++ b/src/storage/index/LookupBaseProcessor-inl.h @@ -7,6 +7,7 @@ #pragma once #include "LookupBaseProcessor.h" +#include "folly/container/Enumerate.h" namespace nebula { namespace storage { diff --git a/src/storage/mutate/AddEdgesAtomicProcessor.cpp b/src/storage/mutate/AddEdgesAtomicProcessor.cpp deleted file mode 100644 index 966ee62cb0f..00000000000 --- a/src/storage/mutate/AddEdgesAtomicProcessor.cpp +++ /dev/null @@ -1,148 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License, - * attached with Common Clause Condition 1.0, found in the LICENSES directory. - */ - -#include "storage/mutate/AddEdgesAtomicProcessor.h" - -#include -#include -#include - -#include - -#include "codec/RowWriterV2.h" -#include "common/utils/IndexKeyUtils.h" -#include "common/utils/NebulaKeyUtils.h" -#include "storage/transaction/TransactionManager.h" -#include "storage/transaction/TransactionUtils.h" - -namespace nebula { -namespace storage { - -ProcessorCounters kAddEdgesAtomicCounters; - -// use localPart vs remotePart to identify different channel. -using ChainId = std::pair; - -void AddEdgesAtomicProcessor::process(const cpp2::AddEdgesRequest& req) { - propNames_ = req.get_prop_names(); - spaceId_ = req.get_space_id(); - - auto stVidLen = env_->schemaMan_->getSpaceVidLen(spaceId_); - if (!stVidLen.ok()) { - LOG(ERROR) << stVidLen.status(); - for (auto& part : req.get_parts()) { - pushResultCode(nebula::cpp2::ErrorCode::E_INVALID_SPACEVIDLEN, part.first); - } - onFinished(); - return; - } - vIdLen_ = stVidLen.value(); - processByChain(req); -} - -void AddEdgesAtomicProcessor::processByChain(const cpp2::AddEdgesRequest& req) { - std::unordered_map> edgesByChain; - std::unordered_map failedPart; - // split req into chains - for (auto& part : *req.parts_ref()) { - auto localPart = part.first; - for (auto& edge : part.second) { - auto remotePart = - env_->metaClient_->partId(spaceId_, (*(*edge.key_ref()).dst_ref()).getStr()); - ChainId cid{localPart, remotePart}; - if (FLAGS_trace_toss) { - auto& ekey = *edge.key_ref(); - LOG(INFO) << "ekey.src.hex=" << folly::hexlify((*ekey.src_ref()).toString()) - << ", ekey.dst.hex=" << folly::hexlify((*ekey.dst_ref()).toString()); - } - auto key = TransactionUtils::edgeKey(vIdLen_, localPart, edge.get_key()); - std::string val; - auto code = encodeSingleEdgeProps(edge, val); - if (code != nebula::cpp2::ErrorCode::SUCCEEDED) { - failedPart[localPart] = code; - break; - } - edgesByChain[cid].emplace_back(std::make_pair(std::move(key), std::move(val))); - } - } - - if (!failedPart.empty()) { - for (auto& part : failedPart) { - pushResultCode(part.second, part.first); - } - onFinished(); - return; - } - - CHECK_NOTNULL(env_->indexMan_); - auto stIndex = env_->indexMan_->getEdgeIndexes(spaceId_); - if (!stIndex.ok()) { - for (auto& part : *req.parts_ref()) { - pushResultCode(nebula::cpp2::ErrorCode::E_SPACE_NOT_FOUND, part.first); - } - onFinished(); - return; - } - if (!stIndex.value().empty()) { - processor_.reset(AddEdgesProcessor::instance(env_)); - processor_->indexes_ = stIndex.value(); - } - - std::list> futures; - for (auto& chain : edgesByChain) { - auto localPart = chain.first.first; - auto remotePart = chain.first.second; - auto& localData = chain.second; - - futures.emplace_back( - env_->txnMan_ - ->addSamePartEdges( - vIdLen_, spaceId_, localPart, remotePart, localData, processor_.get()) - .thenTry([=](auto&& t) { - auto code = nebula::cpp2::ErrorCode::SUCCEEDED; - if (!t.hasValue()) { - code = nebula::cpp2::ErrorCode::E_UNKNOWN; - } else if (t.value() != nebula::cpp2::ErrorCode::SUCCEEDED) { - code = t.value(); - } - LOG_IF(INFO, FLAGS_trace_toss) << folly::sformat( - "addSamePartEdges: (space,localPart,remotePart)=({},{},{}), " - "code={}", - spaceId_, - localPart, - remotePart, - apache::thrift::util::enumNameSafe(code)); - if (code != nebula::cpp2::ErrorCode::SUCCEEDED) { - pushResultCode(code, localPart); - } - })); - } - folly::collectAll(futures).via(env_->txnMan_->getExecutor()).thenValue([=](auto&&) { - onFinished(); - }); -} - -nebula::cpp2::ErrorCode AddEdgesAtomicProcessor::encodeSingleEdgeProps(const cpp2::NewEdge& e, - std::string& encodedVal) { - auto edgeType = e.get_key().get_edge_type(); - auto schema = env_->schemaMan_->getEdgeSchema(spaceId_, std::abs(edgeType)); - if (!schema) { - LOG(ERROR) << "Space " << spaceId_ << ", Edge " << edgeType << " invalid"; - return nebula::cpp2::ErrorCode::E_SPACE_NOT_FOUND; - } - WriteResult wRet; - auto& edgeProps = e.get_props(); - auto stEncodedVal = encodeRowVal(schema.get(), propNames_, edgeProps, wRet); - if (!stEncodedVal.ok()) { - LOG(ERROR) << stEncodedVal.status(); - return nebula::cpp2::ErrorCode::E_DATA_TYPE_MISMATCH; - } - encodedVal = stEncodedVal.value(); - return nebula::cpp2::ErrorCode::SUCCEEDED; -} - -} // namespace storage -} // namespace nebula diff --git a/src/storage/mutate/AddEdgesAtomicProcessor.h b/src/storage/mutate/AddEdgesAtomicProcessor.h deleted file mode 100644 index f6fd6ce92a4..00000000000 --- a/src/storage/mutate/AddEdgesAtomicProcessor.h +++ /dev/null @@ -1,48 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License, - * attached with Common Clause Condition 1.0, found in the LICENSES directory. - */ - -#ifndef STORAGE_MUTATE_ADDEDGESATOMICPROCESSOR_H_ -#define STORAGE_MUTATE_ADDEDGESATOMICPROCESSOR_H_ - -#include "common/base/Base.h" -#include "kvstore/LogEncoder.h" -#include "storage/BaseProcessor.h" -#include "storage/StorageFlags.h" -#include "storage/mutate/AddEdgesProcessor.h" -#include "storage/transaction/TransactionManager.h" - -namespace nebula { -namespace storage { - -extern ProcessorCounters kAddEdgesAtomicCounters; - -class AddEdgesAtomicProcessor : public BaseProcessor { - public: - static AddEdgesAtomicProcessor* instance( - StorageEnv* env, const ProcessorCounters* counters = &kAddEdgesAtomicCounters) { - return new AddEdgesAtomicProcessor(env, counters); - } - - void process(const cpp2::AddEdgesRequest& req); - - void processByChain(const cpp2::AddEdgesRequest& req); - - private: - AddEdgesAtomicProcessor(StorageEnv* env, const ProcessorCounters* counters) - : BaseProcessor(env, counters) {} - - nebula::cpp2::ErrorCode encodeSingleEdgeProps(const cpp2::NewEdge& e, std::string& encodedVal); - - GraphSpaceID spaceId_; - int64_t vIdLen_; - std::vector propNames_; - std::unique_ptr processor_; - std::vector> indexes_; -}; - -} // namespace storage -} // namespace nebula -#endif // STORAGE_MUTATE_ADDEDGESATOMICPROCESSOR_H_ diff --git a/src/storage/mutate/AddEdgesProcessor.cpp b/src/storage/mutate/AddEdgesProcessor.cpp index add739a54b5..2d0b29f5916 100644 --- a/src/storage/mutate/AddEdgesProcessor.cpp +++ b/src/storage/mutate/AddEdgesProcessor.cpp @@ -130,7 +130,18 @@ void AddEdgesProcessor::doProcess(const cpp2::AddEdgesRequest& req) { if (code != nebula::cpp2::ErrorCode::SUCCEEDED) { handleAsync(spaceId_, partId, code); } else { - doPut(spaceId_, partId, std::move(data)); + if (consistOp_) { + auto batchHolder = std::make_unique(); + (*consistOp_)(*batchHolder, &data); + auto batch = encodeBatchValue(std::move(batchHolder)->getBatch()); + + env_->kvstore_->asyncAppendBatch( + spaceId_, partId, std::move(batch), [partId, this](auto rc) { + handleAsync(spaceId_, partId, rc); + }); + } else { + doPut(spaceId_, partId, std::move(data)); + } } } } @@ -236,8 +247,7 @@ void AddEdgesProcessor::doProcessWithIndex(const cpp2::AddEdgesRequest& req) { if (oReader != nullptr) { auto oi = indexKey(partId, oReader.get(), key, index); if (!oi.empty()) { - // Check the index is building for the specified partition or - // not. + // Check the index is building for the specified partition or not. auto indexState = env_->getIndexState(spaceId_, partId); if (env_->checkRebuilding(indexState)) { auto delOpKey = OperationKeyUtils::deleteOperationKey(partId); @@ -259,8 +269,7 @@ void AddEdgesProcessor::doProcessWithIndex(const cpp2::AddEdgesRequest& req) { if (!nik.empty()) { auto v = CommonUtils::ttlValue(schema.get(), nReader.get()); auto niv = v.ok() ? IndexKeyUtils::indexVal(std::move(v).value()) : ""; - // Check the index is building for the specified partition or - // not. + // Check the index is building for the specified partition or not. auto indexState = env_->getIndexState(spaceId_, partId); if (env_->checkRebuilding(indexState)) { auto opKey = OperationKeyUtils::modifyOperationKey(partId, std::move(nik)); @@ -287,6 +296,9 @@ void AddEdgesProcessor::doProcessWithIndex(const cpp2::AddEdgesRequest& req) { handleAsync(spaceId_, partId, code); continue; } + if (consistOp_) { + (*consistOp_)(*batchHolder, nullptr); + } auto batch = encodeBatchValue(batchHolder->getBatch()); DCHECK(!batch.empty()); nebula::MemoryLockGuard lg(env_->edgesML_.get(), std::move(dummyLock), false, false); @@ -318,8 +330,7 @@ ErrorOr AddEdgesProcessor::addEdges( * kv(part1_src1_edgeType1_rank1_dst1 , v3) * kv(part1_src1_edgeType1_rank1_dst1 , v4) * - * Ultimately, kv(part1_src1_edgeType1_rank1_dst1 , v4) . It's just what I - * need. + * Ultimately, kv(part1_src1_edgeType1_rank1_dst1 , v4) . It's just what I need. */ std::unordered_map newEdges; std::for_each( diff --git a/src/storage/mutate/AddEdgesProcessor.h b/src/storage/mutate/AddEdgesProcessor.h index 9834d1ee9df..086fb52f9ca 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 AddEdgesAtomicProcessor; + friend class ChainAddEdgesProcessorLocal; public: static AddEdgesProcessor* instance(StorageEnv* env, @@ -52,6 +52,9 @@ class AddEdgesProcessor : public BaseProcessor { GraphSpaceID spaceId_; std::vector> indexes_; bool ifNotExists_{false}; + + using ConsistOper = std::function*)>; + folly::Optional consistOp_; }; } // namespace storage diff --git a/src/storage/mutate/UpdateEdgeProcessor.cpp b/src/storage/mutate/UpdateEdgeProcessor.cpp index af79246902a..0b8f387d4fa 100644 --- a/src/storage/mutate/UpdateEdgeProcessor.cpp +++ b/src/storage/mutate/UpdateEdgeProcessor.cpp @@ -54,9 +54,6 @@ void UpdateEdgeProcessor::doProcess(const cpp2::UpdateEdgeRequest& req) { this->planContext_ = std::make_unique( this->env_, spaceId_, this->spaceVidLen_, this->isIntId_, req.common_ref()); context_ = std::make_unique(planContext_.get()); - if (env_->txnMan_ && env_->txnMan_->enableToss(spaceId_)) { - planContext_->defaultEdgeVer_ = 1L; - } retCode = checkAndBuildContexts(req); if (retCode != nebula::cpp2::ErrorCode::SUCCEEDED) { LOG(ERROR) << "Failure build contexts: " << apache::thrift::util::enumNameSafe(retCode); @@ -94,6 +91,10 @@ void UpdateEdgeProcessor::doProcess(const cpp2::UpdateEdgeRequest& req) { return; } +void UpdateEdgeProcessor::adjustContext(UpdateEdgeProcessor::ContextAdjuster fn) { + ctxAdjuster_.emplace_back(std::move(fn)); +} + nebula::cpp2::ErrorCode UpdateEdgeProcessor::checkAndBuildContexts( const cpp2::UpdateEdgeRequest& req) { // Build edgeContext_.schemas_ @@ -110,12 +111,15 @@ nebula::cpp2::ErrorCode UpdateEdgeProcessor::checkAndBuildContexts( // Build edgeContext_.ttlInfo_ buildEdgeTTLInfo(); + + for (auto& adjuster : ctxAdjuster_) { + adjuster(edgeContext_); + } return nebula::cpp2::ErrorCode::SUCCEEDED; } /* The storage plan of update(upsert) edge looks like this: - +--------+----------+ | UpdateEdgeResNode | +--------+----------+ | diff --git a/src/storage/mutate/UpdateEdgeProcessor.h b/src/storage/mutate/UpdateEdgeProcessor.h index 6976e0209d9..1c4b9bf5b8c 100644 --- a/src/storage/mutate/UpdateEdgeProcessor.h +++ b/src/storage/mutate/UpdateEdgeProcessor.h @@ -31,6 +31,9 @@ class UpdateEdgeProcessor void doProcess(const cpp2::UpdateEdgeRequest& req); + using ContextAdjuster = folly::Function; + void adjustContext(ContextAdjuster fn); + private: UpdateEdgeProcessor(StorageEnv* env, const ProcessorCounters* counters, folly::Executor* executor) : QueryBaseProcessor(env, counters, executor) { @@ -53,8 +56,8 @@ class UpdateEdgeProcessor // std::vector result; // result.resize(returnPropsExp_.size()); // auto get = [] (auto &ptr) {return ptr.get(); }; - // std::transform(returnPropsExp_.begin(), returnPropsExp_.end(), - // result.begin(), get); return result; + // std::transform(returnPropsExp_.begin(), returnPropsExp_.end(), result.begin(), get); + // return result; return returnPropsExp_; } @@ -71,6 +74,9 @@ class UpdateEdgeProcessor // update std::vector updatedProps_; + folly::Optional> returnProps_; + folly::Optional condition_; + // return props expression std::vector returnPropsExp_; @@ -79,6 +85,8 @@ class UpdateEdgeProcessor // updatedProps_ dependent props in value expression std::vector>> depPropMap_; + + std::list ctxAdjuster_; }; } // namespace storage diff --git a/src/storage/query/QueryBaseProcessor.h b/src/storage/query/QueryBaseProcessor.h index 2c3fab7348d..55b1ec009ed 100644 --- a/src/storage/query/QueryBaseProcessor.h +++ b/src/storage/query/QueryBaseProcessor.h @@ -125,6 +125,10 @@ struct EdgeContext { // offset is the start index of first edge type in a response row size_t offset_; size_t statCount_ = 0; + + // additional operator for eventually-consistent edges + std::vector> kvAppend; + std::vector kvErased; }; template diff --git a/src/storage/test/CMakeLists.txt b/src/storage/test/CMakeLists.txt index 437185d4692..eb103810438 100644 --- a/src/storage/test/CMakeLists.txt +++ b/src/storage/test/CMakeLists.txt @@ -230,39 +230,7 @@ nebula_add_executable( boost_regex ) -nebula_add_executable( - NAME - toss_test - SOURCES - TossTest.cpp - OBJECTS - ${storage_test_deps} - LIBRARIES - ${ROCKSDB_LIBRARIES} - ${THRIFT_LIBRARIES} - ${PROXYGEN_LIBRARIES} - wangle - gtest - boost_regex -) -nebula_add_executable( - NAME - toss_bm - SOURCES - TossBenchmark.cpp - OBJECTS - ${storage_test_deps} - $ - $ - LIBRARIES - ${ROCKSDB_LIBRARIES} - ${THRIFT_LIBRARIES} - ${PROXYGEN_LIBRARIES} - wangle - follybenchmark - boost_regex -) nebula_add_test( NAME @@ -685,6 +653,51 @@ nebula_add_test( gtest ) +nebula_add_test( + NAME + chain_add_edge_test + SOURCES + ChainAddEdgesTest.cpp + OBJECTS + ${storage_test_deps} + LIBRARIES + ${ROCKSDB_LIBRARIES} + ${THRIFT_LIBRARIES} + ${PROXYGEN_LIBRARIES} + wangle + gtest +) + +nebula_add_executable( + NAME + chain_update_edge_test + SOURCES + ChainUpdateEdgeTest.cpp + OBJECTS + ${storage_test_deps} + LIBRARIES + ${ROCKSDB_LIBRARIES} + ${THRIFT_LIBRARIES} + ${PROXYGEN_LIBRARIES} + wangle + gtest +) + +nebula_add_executable( + NAME + chain_resume_edge_test + SOURCES + ChainResumeEdgeTest.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 new file mode 100644 index 00000000000..30b56f087bb --- /dev/null +++ b/src/storage/test/ChainAddEdgesTest.cpp @@ -0,0 +1,206 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#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/test/ChainTestUtils.h" +#include "storage/test/TestUtils.h" +#include "storage/transaction/ChainAddEdgesGroupProcessor.h" +#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ConsistUtil.h" + +namespace nebula { +namespace storage { + +constexpr int32_t mockSpaceId = 1; +constexpr int32_t mockPartNum = 1; +constexpr int32_t fackTerm = 1; + +// make sure test class works well +TEST(ChainAddEdgesTest, TestUtilsTest) { + fs::TempDir rootPath("/tmp/AddEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + env->metaClient_ = mClient.get(); + MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, fackTerm); + + auto* processor = new FakeChainAddEdgesProcessorLocal(env); + + processor->rcPrepareLocal = nebula::cpp2::ErrorCode::SUCCEEDED; + processor->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; + processor->rcProcessLocal = nebula::cpp2::ErrorCode::SUCCEEDED; + + LOG(INFO) << "Build AddEdgesRequest..."; + cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, 1); + + LOG(INFO) << "Test AddEdgesProcessor..."; + auto fut = processor->getFuture(); + processor->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + + LOG(INFO) << "Check data in kv store..."; + // The number of data in serve is 334 + checkAddEdgesData(req, env, 0, 0); +} + +TEST(ChainAddEdgesTest, prepareLocalSucceedTest) { + fs::TempDir rootPath("/tmp/AddEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + env->metaClient_ = mClient.get(); + MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, fackTerm); + auto* proc = new FakeChainAddEdgesProcessorLocal(env); + + proc->rcProcessRemote = nebula::cpp2::ErrorCode::E_RPC_FAILURE; + + LOG(INFO) << "Build AddEdgesRequest..."; + cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, 1); + + LOG(INFO) << "Test AddEdgesProcessor..."; + auto fut = proc->getFuture(); + proc->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + + ChainTestUtils util; + // none of really edge key should be inserted + EXPECT_EQ(334, numOfKey(req, util.genKey, env)); + EXPECT_EQ(0, numOfKey(req, util.genPrime, env)); + EXPECT_EQ(334, numOfKey(req, util.genDoublePrime, env)); +} + +TEST(ChainAddEdgesTest, processRemoteSucceededTest) { + fs::TempDir rootPath("/tmp/AddEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + + env->metaClient_ = mClient.get(); + auto* proc = new FakeChainAddEdgesProcessorLocal(env); + MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, fackTerm); + + proc->rcProcessRemote = nebula::cpp2::ErrorCode::SUCCEEDED; + + LOG(INFO) << "Build AddEdgesRequest..."; + cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, 1); + + LOG(INFO) << "Test AddEdgesProcessor..."; + auto fut = proc->getFuture(); + proc->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + + ChainTestUtils util; + // none of really edge key should be inserted + EXPECT_EQ(334, numOfKey(req, util.genKey, env)); + // prime key should be deleted + EXPECT_EQ(0, numOfKey(req, util.genPrime, env)); + EXPECT_EQ(0, numOfKey(req, util.genDoublePrime, env)); +} + +// check prepareLocal() will set prime key properly +TEST(ChainAddEdgesTest, processRemoteFailedTest) { + fs::TempDir rootPath("/tmp/AddEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + env->metaClient_ = mClient.get(); + MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, fackTerm); + + auto* proc = new FakeChainAddEdgesProcessorLocal(env); + proc->rcProcessRemote = nebula::cpp2::ErrorCode::E_OUTDATED_TERM; + + LOG(INFO) << "Build AddEdgesRequest..."; + cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, 1); + + LOG(INFO) << "Test AddEdgesProcessor..."; + auto fut = proc->getFuture(); + proc->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + + ChainTestUtils util; + // none of really edge key should be inserted + EXPECT_EQ(0, numOfKey(req, util.genKey, env)); + // prime key should be deleted + EXPECT_EQ(0, numOfKey(req, util.genPrime, env)); + EXPECT_EQ(0, numOfKey(req, util.genDoublePrime, env)); +} + +TEST(ChainAddEdgesTest, processRemoteUnknownTest) { + fs::TempDir rootPath("/tmp/AddEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + env->metaClient_ = mClient.get(); + MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, fackTerm); + + auto* proc = new FakeChainAddEdgesProcessorLocal(env); + + proc->rcProcessRemote = nebula::cpp2::ErrorCode::E_RPC_FAILURE; + + LOG(INFO) << "Build AddEdgesRequest..."; + cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, 1); + + LOG(INFO) << "Test AddEdgesProcessor..."; + auto fut = proc->getFuture(); + proc->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + + ChainTestUtils util; + // none of really edge key should be inserted + EXPECT_EQ(334, numOfKey(req, util.genKey, env)); + // prime key should be deleted + EXPECT_EQ(0, numOfKey(req, util.genPrime, env)); + EXPECT_EQ(334, numOfKey(req, util.genDoublePrime, env)); +} + +// make a reversed request, make sure it can be added successfully +TEST(ChainAddEdgesTest, processRemoteTest) { + fs::TempDir rootPath("/tmp/AddEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + + env->metaClient_ = mClient.get(); + MetaClientTestUpdater::addPartTerm(env->metaClient_, mockSpaceId, mockPartNum, fackTerm); + + auto* proc = new FakeChainAddEdgesProcessorLocal(env); + LOG(INFO) << "Build AddEdgesRequest..."; + cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, 1); + + auto reversedRequest = proc->reverseRequestForward(req); +} + +} // namespace storage +} // namespace nebula + +int main(int argc, char** argv) { + testing::InitGoogleTest(&argc, argv); + folly::init(&argc, &argv, false); + google::SetStderrLogging(google::INFO); + return RUN_ALL_TESTS(); +} diff --git a/src/storage/test/ChainResumeEdgeTest.cpp b/src/storage/test/ChainResumeEdgeTest.cpp new file mode 100644 index 00000000000..e67b67067ce --- /dev/null +++ b/src/storage/test/ChainResumeEdgeTest.cpp @@ -0,0 +1,596 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#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/test/ChainTestUtils.h" +#include "storage/test/QueryTestUtils.h" +#include "storage/test/TestUtils.h" +#include "storage/transaction/ChainAddEdgesGroupProcessor.h" +#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ChainResumeProcessor.h" +#include "storage/transaction/ConsistUtil.h" + +namespace nebula { +namespace storage { + +constexpr int32_t mockSpaceId = 1; +constexpr int32_t mockPartNum = 6; +constexpr int32_t mockSpaceVidLen = 32; + +ChainTestUtils gTestUtil; +ChainUpdateEdgeTestHelper helper; + +/** + * @brief resumeTest1 (resume insert prime) + * insert resume + * prepareLocal succeed succeed + * processRemote skip succeed + * processLocal skip succeed + * expect: edge/prime/double : 334/0/0 + * keyOfRequest: false + */ +TEST(ChainResumeEdgesTest, resumeTest1) { + fs::TempDir rootPath("/tmp/AddEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + + env->metaClient_ = mClient.get(); + auto* proc = new FakeChainAddEdgesProcessorLocal(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); + + auto fut = proc->getFuture(); + proc->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + EXPECT_EQ(0, numOfKey(req, gTestUtil.genKey, env)); + EXPECT_EQ(334, numOfKey(req, gTestUtil.genPrime, env)); + EXPECT_EQ(0, numOfKey(req, gTestUtil.genDoublePrime, env)); + + auto* iClient = FakeInternalStorageClient::instance(env); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + ChainResumeProcessor resumeProc(env); + resumeProc.process(); + + EXPECT_EQ(334, numOfKey(req, gTestUtil.genKey, env)); + EXPECT_EQ(0, numOfKey(req, gTestUtil.genPrime, env)); + EXPECT_EQ(0, numOfKey(req, gTestUtil.genDoublePrime, env)); +} + +/** + * @brief resumeTest2 (resume insert prime, remote failed) + * previous resume + * prepareLocal succeed succeed + * processRemote skip failed + * processLocal skip failed + * expect: edge/prime/double : 0/0/0 + * keyOfRequest: false + */ +TEST(ChainResumeEdgesTest, resumeTest2) { + fs::TempDir rootPath("/tmp/AddEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + + env->metaClient_ = mClient.get(); + auto* proc = new FakeChainAddEdgesProcessorLocal(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); + + LOG(INFO) << "Test AddEdgesProcessor..."; + auto fut = proc->getFuture(); + proc->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + + ChainTestUtils util; + EXPECT_EQ(0, numOfKey(req, util.genKey, env)); + EXPECT_EQ(334, numOfKey(req, util.genPrime, env)); + EXPECT_EQ(0, numOfKey(req, util.genDoublePrime, env)); + + auto* iClient = FakeInternalStorageClient::instance(env, nebula::cpp2::ErrorCode::E_UNKNOWN); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + ChainResumeProcessor resumeProc(env); + resumeProc.process(); + + EXPECT_EQ(0, numOfKey(req, util.genKey, env)); + EXPECT_EQ(334, numOfKey(req, util.genPrime, env)); + EXPECT_EQ(0, numOfKey(req, util.genDoublePrime, env)); +} + +/** + * @brief resumePrimeTest3 (resume insert prime outdated) + */ +TEST(ChainResumeEdgesTest, resumePrimeTest3) { + fs::TempDir rootPath("/tmp/AddEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + + env->metaClient_ = mClient.get(); + auto* proc = new FakeChainAddEdgesProcessorLocal(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); + + LOG(INFO) << "Test AddEdgesProcessor..."; + auto fut = proc->getFuture(); + proc->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + + ChainTestUtils util; + EXPECT_EQ(0, numOfKey(req, util.genKey, env)); + EXPECT_EQ(334, numOfKey(req, util.genPrime, env)); + EXPECT_EQ(0, numOfKey(req, util.genDoublePrime, env)); + + auto error = nebula::cpp2::ErrorCode::E_RPC_FAILURE; + auto* iClient = FakeInternalStorageClient::instance(env, error); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + ChainResumeProcessor resumeProc(env); + resumeProc.process(); + + // none of really edge key should be inserted + EXPECT_EQ(334, numOfKey(req, util.genKey, env)); + EXPECT_EQ(0, numOfKey(req, util.genPrime, env)); + EXPECT_EQ(334, numOfKey(req, util.genDoublePrime, env)); +} + +/** + * @brief resumeTest4 (resume double prime, resume failed) + * insert resume + * prepareLocal succeed succeed + * processRemote outdate failed + * processLocal succeed succeed + * expect: edge/prime/double : 334/0/334 + * keyOfRequest: true + */ +TEST(ChainResumeEdgesTest, resumeTest4) { + fs::TempDir rootPath("/tmp/AddEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + + env->metaClient_ = mClient.get(); + auto* proc = new FakeChainAddEdgesProcessorLocal(env); + + proc->rcProcessRemote = nebula::cpp2::ErrorCode::E_RPC_FAILURE; + + LOG(INFO) << "Build AddEdgesRequest..."; + int partNum = 1; + cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, partNum); + + LOG(INFO) << "Test AddEdgesProcessor..."; + auto fut = proc->getFuture(); + proc->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + + // ChainTestUtils util; + EXPECT_EQ(334, numOfKey(req, gTestUtil.genKey, env)); + EXPECT_EQ(0, numOfKey(req, gTestUtil.genPrime, env)); + EXPECT_EQ(334, numOfKey(req, gTestUtil.genDoublePrime, env)); + + auto error = nebula::cpp2::ErrorCode::E_UNKNOWN; + auto* iClient = FakeInternalStorageClient::instance(env, error); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + ChainResumeProcessor resumeProc(env); + resumeProc.process(); + + EXPECT_EQ(334, numOfKey(req, gTestUtil.genKey, env)); + EXPECT_EQ(0, numOfKey(req, gTestUtil.genPrime, env)); + EXPECT_EQ(334, numOfKey(req, gTestUtil.genDoublePrime, env)); +} + +/** + * @brief resumeTest5 (resume double prime, but outdated) + */ +TEST(ChainResumeEdgesTest, resumeTest5) { + fs::TempDir rootPath("/tmp/AddEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + + env->metaClient_ = mClient.get(); + auto* proc = new FakeChainAddEdgesProcessorLocal(env); + + proc->rcProcessRemote = nebula::cpp2::ErrorCode::E_RPC_FAILURE; + + LOG(INFO) << "Build AddEdgesRequest..."; + cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, 1); + + LOG(INFO) << "Test AddEdgesProcessor..."; + auto fut = proc->getFuture(); + proc->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + + ChainTestUtils util; + EXPECT_EQ(334, numOfKey(req, util.genKey, env)); + EXPECT_EQ(0, numOfKey(req, util.genPrime, env)); + EXPECT_EQ(334, numOfKey(req, util.genDoublePrime, env)); + + auto error = nebula::cpp2::ErrorCode::E_RPC_FAILURE; + auto* iClient = FakeInternalStorageClient::instance(env, error); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + ChainResumeProcessor resumeProc(env); + resumeProc.process(); + + EXPECT_EQ(334, numOfKey(req, util.genKey, env)); + EXPECT_EQ(0, numOfKey(req, util.genPrime, env)); + EXPECT_EQ(334, numOfKey(req, util.genDoublePrime, env)); +} + +/** + * @brief resumeTest6 (resume add edge double prime, succeeded) + */ +TEST(ChainResumeEdgesTest, resumeTest6) { + fs::TempDir rootPath("/tmp/AddEdgesTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + + env->metaClient_ = mClient.get(); + auto* proc = new FakeChainAddEdgesProcessorLocal(env); + + proc->rcProcessRemote = nebula::cpp2::ErrorCode::E_RPC_FAILURE; + + LOG(INFO) << "Build AddEdgesRequest..."; + cpp2::AddEdgesRequest req = mock::MockData::mockAddEdgesReq(false, 1); + + LOG(INFO) << "Test AddEdgesProcessor..."; + auto fut = proc->getFuture(); + proc->process(req); + auto resp = std::move(fut).get(); + EXPECT_EQ(0, resp.result.failed_parts.size()); + + ChainTestUtils util; + EXPECT_EQ(334, numOfKey(req, util.genKey, env)); + EXPECT_EQ(0, numOfKey(req, util.genPrime, env)); + EXPECT_EQ(334, numOfKey(req, util.genDoublePrime, env)); + + auto* iClient = FakeInternalStorageClient::instance(env); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + 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)); +} + +// resume an update left prime, check resume succeeded +TEST(ChainUpdateEdgeTest, resumeTest7) { + fs::TempDir rootPath("/tmp/UpdateEdgeTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + env->metaClient_ = mClient.get(); + + auto parts = cluster.getTotalParts(); + EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); + + LOG(INFO) << "Test UpdateEdgeRequest..."; + auto req = helper.makeDefaultRequest(); + + LOG(INFO) << "Fake Prime..."; + auto* proc = new FakeChainUpdateProcessor(env); + auto f = proc->getFuture(); + proc->rcProcessRemote = Code::SUCCEEDED; + proc->rcProcessLocal = Code::SUCCEEDED; + proc->process(req); + auto resp = std::move(f).get(); + + // EXPECT_TRUE(helper.checkResp(req, resp)); + EXPECT_FALSE(helper.checkRequestUpdated(env, req)); + EXPECT_TRUE(helper.edgeExist(env, req)); + EXPECT_TRUE(helper.primeExist(env, req)); + EXPECT_FALSE(helper.doublePrimeExist(env, req)); + + auto* iClient = FakeInternalStorageClient::instance(env); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + ChainResumeProcessor resumeProc(env); + resumeProc.process(); + + EXPECT_TRUE(helper.edgeExist(env, req)); + EXPECT_FALSE(helper.primeExist(env, req)); + EXPECT_FALSE(helper.doublePrimeExist(env, req)); +} + +// resume an update left prime, resume failed +TEST(ChainUpdateEdgeTest, resumeTest8) { + fs::TempDir rootPath("/tmp/UpdateEdgeTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + env->metaClient_ = mClient.get(); + + auto parts = cluster.getTotalParts(); + EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); + + LOG(INFO) << "Test UpdateEdgeRequest..."; + auto req = helper.makeDefaultRequest(); + + LOG(INFO) << "Fake Prime..."; + auto* proc = new FakeChainUpdateProcessor(env); + auto f = proc->getFuture(); + proc->rcProcessRemote = Code::SUCCEEDED; + proc->rcProcessLocal = Code::SUCCEEDED; + proc->process(req); + auto resp = std::move(f).get(); + + // EXPECT_TRUE(helper.checkResp(req, resp)); + EXPECT_FALSE(helper.checkRequestUpdated(env, req)); + EXPECT_TRUE(helper.edgeExist(env, req)); + EXPECT_TRUE(helper.primeExist(env, req)); + EXPECT_FALSE(helper.doublePrimeExist(env, req)); + + auto* iClient = FakeInternalStorageClient::instance(env); + iClient->setErrorCode(Code::E_UNKNOWN); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + ChainResumeProcessor resumeProc(env); + resumeProc.process(); + + EXPECT_TRUE(helper.edgeExist(env, req)); + EXPECT_TRUE(helper.primeExist(env, req)); + EXPECT_FALSE(helper.doublePrimeExist(env, req)); +} + +// resume an update left prime, resume outdated +TEST(ChainUpdateEdgeTest, resumeTest9) { + fs::TempDir rootPath("/tmp/UpdateEdgeTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + env->metaClient_ = mClient.get(); + + auto parts = cluster.getTotalParts(); + EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); + + LOG(INFO) << "Test UpdateEdgeRequest..."; + auto req = helper.makeDefaultRequest(); + + LOG(INFO) << "Fake Prime..."; + auto* proc = new FakeChainUpdateProcessor(env); + auto f = proc->getFuture(); + proc->rcProcessRemote = Code::SUCCEEDED; + proc->rcProcessLocal = Code::SUCCEEDED; + proc->process(req); + auto resp = std::move(f).get(); + + // EXPECT_TRUE(helper.checkResp(req, resp)); + EXPECT_FALSE(helper.checkRequestUpdated(env, req)); + EXPECT_TRUE(helper.edgeExist(env, req)); + EXPECT_TRUE(helper.primeExist(env, req)); + EXPECT_FALSE(helper.doublePrimeExist(env, req)); + + auto* iClient = FakeInternalStorageClient::instance(env); + iClient->setErrorCode(Code::E_RPC_FAILURE); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + ChainResumeProcessor resumeProc(env); + resumeProc.process(); + + EXPECT_TRUE(helper.edgeExist(env, req)); + EXPECT_FALSE(helper.primeExist(env, req)); + EXPECT_TRUE(helper.doublePrimeExist(env, req)); +} + +// resume an update left prime, check resume succeeded +TEST(ChainUpdateEdgeTest, resumeTest10) { + fs::TempDir rootPath("/tmp/UpdateEdgeTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + env->metaClient_ = mClient.get(); + + auto parts = cluster.getTotalParts(); + EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); + + LOG(INFO) << "Test UpdateEdgeRequest..."; + auto req = helper.makeDefaultRequest(); + + LOG(INFO) << "Fake Prime..."; + auto* proc = new FakeChainUpdateProcessor(env); + auto f = proc->getFuture(); + proc->rcProcessRemote = Code::E_RPC_FAILURE; + // proc->rcProcessLocal = Code::SUCCEEDED; + proc->process(req); + auto resp = std::move(f).get(); + + EXPECT_TRUE(helper.checkRequestUpdated(env, req)); + EXPECT_TRUE(helper.edgeExist(env, req)); + EXPECT_FALSE(helper.primeExist(env, req)); + EXPECT_TRUE(helper.doublePrimeExist(env, req)); + + auto* iClient = FakeInternalStorageClient::instance(env); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + ChainResumeProcessor resumeProc(env); + resumeProc.process(); + + EXPECT_TRUE(helper.edgeExist(env, req)); + EXPECT_FALSE(helper.primeExist(env, req)); + EXPECT_FALSE(helper.doublePrimeExist(env, req)); +} + +// resume an update left prime, resume failed +TEST(ChainUpdateEdgeTest, resumeTest11) { + fs::TempDir rootPath("/tmp/UpdateEdgeTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + env->metaClient_ = mClient.get(); + + auto parts = cluster.getTotalParts(); + EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); + + LOG(INFO) << "Test UpdateEdgeRequest..."; + auto req = helper.makeDefaultRequest(); + + LOG(INFO) << "Fake Prime..."; + auto* proc = new FakeChainUpdateProcessor(env); + auto f = proc->getFuture(); + proc->rcProcessRemote = Code::E_RPC_FAILURE; + // proc->rcProcessLocal = Code::SUCCEEDED; + proc->process(req); + auto resp = std::move(f).get(); + + EXPECT_TRUE(helper.checkRequestUpdated(env, req)); + EXPECT_TRUE(helper.edgeExist(env, req)); + EXPECT_FALSE(helper.primeExist(env, req)); + EXPECT_TRUE(helper.doublePrimeExist(env, req)); + + auto* iClient = FakeInternalStorageClient::instance(env); + iClient->setErrorCode(Code::E_UNKNOWN); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + ChainResumeProcessor resumeProc(env); + resumeProc.process(); + + EXPECT_TRUE(helper.edgeExist(env, req)); + EXPECT_FALSE(helper.primeExist(env, req)); + EXPECT_TRUE(helper.doublePrimeExist(env, req)); +} + +// resume an update left prime, resume outdated +TEST(ChainUpdateEdgeTest, resumeTest12) { + fs::TempDir rootPath("/tmp/UpdateEdgeTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + env->metaClient_ = mClient.get(); + + auto parts = cluster.getTotalParts(); + EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); + + LOG(INFO) << "Test UpdateEdgeRequest..."; + auto req = helper.makeDefaultRequest(); + + LOG(INFO) << "Fake Prime..."; + auto* proc = new FakeChainUpdateProcessor(env); + auto f = proc->getFuture(); + proc->rcProcessRemote = Code::E_RPC_FAILURE; + // proc->rcProcessLocal = Code::SUCCEEDED; + proc->process(req); + auto resp = std::move(f).get(); + + EXPECT_TRUE(helper.checkRequestUpdated(env, req)); + EXPECT_TRUE(helper.edgeExist(env, req)); + EXPECT_FALSE(helper.primeExist(env, req)); + EXPECT_TRUE(helper.doublePrimeExist(env, req)); + + auto* iClient = FakeInternalStorageClient::instance(env); + iClient->setErrorCode(Code::E_RPC_FAILURE); + FakeInternalStorageClient::hookInternalStorageClient(env, iClient); + ChainResumeProcessor resumeProc(env); + resumeProc.process(); + + EXPECT_TRUE(helper.edgeExist(env, req)); + EXPECT_FALSE(helper.primeExist(env, req)); + EXPECT_TRUE(helper.doublePrimeExist(env, req)); +} +} // namespace storage +} // namespace nebula + +int main(int argc, char** argv) { + testing::InitGoogleTest(&argc, argv); + folly::init(&argc, &argv, false); + google::SetStderrLogging(google::INFO); + return RUN_ALL_TESTS(); +} + +/** + * @brief resumeTest1 (prime add edge can be resumed) + * insert resume + * prepareLocal succeed succeed + * processRemote skip succeed + * processLocal skip succeed + * expect: edge/prime/double : 334/0/0 + * keyOfRequest: false + */ + +/** + * @brief resumeTest2 (double prime add edge can be resumed) + * insert resume + * prepareLocal succeed succeed + * processRemote skip failed + * processLocal skip failed + * expect: edge/prime/double : 0/0/0 + * keyOfRequest: false + */ + +/** + * @brief resumePrimeTest3 + * insert resume + * prepareLocal succeed succeed + * processRemote skip outdate + * processLocal skip succeed + * expect: edge/prime/double : 334/0/334 + * keyOfRequest: true + */ + +/** + * @brief resumeTest4 + * insert resume + * prepareLocal succeed succeed + * processRemote outdate failed + * processLocal succeed succeed + * expect: edge/prime/double : 334/0/334 + * keyOfRequest: true + */ + +/** + * @brief resumeTest5 + * insert resume + * prepareLocal succeed succeed + * processRemote outdate outdate + * processLocal succeed succeed + * expect: edge/prime/double : 334/0/334 + * keyOfRequest: true + */ + +/** + * @brief resumeTest6 + * insert resume + * prepareLocal succeed succeed + * processRemote outdate succeed + * processLocal succeed succeed + * expect: edge/prime/double : 334/0/0 + * keyOfRequest: false + */ diff --git a/src/storage/test/ChainTestUtils.h b/src/storage/test/ChainTestUtils.h new file mode 100644 index 00000000000..0c262f6d746 --- /dev/null +++ b/src/storage/test/ChainTestUtils.h @@ -0,0 +1,494 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#pragma once + +#include "storage/CommonUtils.h" +#include "storage/transaction/ChainResumeProcessor.h" +#include "storage/transaction/ChainUpdateEdgeProcessorLocal.h" +#include "storage/transaction/ChainUpdateEdgeProcessorRemote.h" + +namespace nebula { +namespace storage { + +extern const int32_t mockSpaceId; +extern const int32_t mockPartNum; +extern const int32_t mockSpaceVidLen; + +using KeyGenerator = std::function; + +class ChainTestUtils { + public: + ChainTestUtils() { + genKey = [&](PartitionID partId, const cpp2::NewEdge& edge) { + auto key = ConsistUtil::edgeKey(spaceVidLen_, partId, edge.get_key()); + return key; + }; + genPrime = [&](PartitionID partId, const cpp2::NewEdge& edge) { + auto key = ConsistUtil::primeKey(spaceVidLen_, partId, edge.get_key()); + return key; + }; + genDoublePrime = [&](PartitionID partId, const cpp2::NewEdge& edge) { + auto key = ConsistUtil::doublePrime(spaceVidLen_, partId, edge.get_key()); + return key; + }; + } + + public: + int32_t spaceVidLen_{32}; + KeyGenerator genKey; + KeyGenerator genPrime; + KeyGenerator genDoublePrime; +}; + +// , StorageEnv* env +int numOfKey(const cpp2::AddEdgesRequest& req, KeyGenerator gen, StorageEnv* env) { + int numOfEdges = 0; + int totalEdge = 0; + auto spaceId = req.get_space_id(); + for (auto& edgesOfPart : *req.parts_ref()) { + auto partId = edgesOfPart.first; + auto& edgeVec = edgesOfPart.second; + for (auto& edge : edgeVec) { + ++totalEdge; + auto key = gen(partId, edge); + std::unique_ptr iter; + EXPECT_EQ(Code::SUCCEEDED, env->kvstore_->prefix(spaceId, partId, key, &iter)); + if (iter && iter->valid()) { + ++numOfEdges; + } else { + // LOG(INFO) << "key: " << key << " not exist"; + } + } + } + LOG(INFO) << "numOfEdges = " << numOfEdges; + LOG(INFO) << "totalEdge = " << totalEdge; + return numOfEdges; +} + +std::pair extractSpaceAndPart(const cpp2::AddEdgesRequest& req) { + auto spaceId = req.get_space_id(); + CHECK_EQ(req.get_parts().size(), 1); + auto partId = req.get_parts().begin()->first; + return std::make_pair(spaceId, partId); +} + +bool keyExist(StorageEnv* env, GraphSpaceID spaceId, PartitionID partId, std::string key) { + // std::unique_ptr iter; + std::string ignoreVal; + auto rc = env->kvstore_->get(spaceId, partId, key, &ignoreVal); + return rc == Code::SUCCEEDED; +} + +class FakeChainAddEdgesProcessorLocal : public ChainAddEdgesProcessorLocal { + FRIEND_TEST(ChainAddEdgesTest, prepareLocalSucceededTest); + FRIEND_TEST(ChainAddEdgesTest, processRemoteSucceededTest); + FRIEND_TEST(ChainAddEdgesTest, processRemoteFailedTest); + FRIEND_TEST(ChainAddEdgesTest, processRemoteOutdatedTest); + // 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) { + spaceVidLen_ = 32; + } + + folly::SemiFuture prepareLocal() override { + LOG(INFO) << "FakeChainAddEdgesProcessorLocal::" << __func__ << "()"; + if (rcPrepareLocal) { + LOG(INFO) << "Fake return " << apache::thrift::util::enumNameSafe(*rcPrepareLocal); + return *rcPrepareLocal; + } + LOG(INFO) << "forward to ChainAddEdgesProcessorLocal::prepareLocal()"; + return ChainAddEdgesProcessorLocal::prepareLocal(); + } + + folly::SemiFuture processRemote(Code code) override { + LOG(INFO) << "FakeChainAddEdgesProcessorLocal::" << __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); + } + + folly::SemiFuture processLocal(Code code) override { + LOG(INFO) << "FakeChainAddEdgesProcessorLocal::" << __func__ << "()"; + if (rcProcessLocal) { + LOG(INFO) << "Fake return " << apache::thrift::util::enumNameSafe(*rcProcessLocal); + return *rcProcessLocal; + } + LOG(INFO) << "forward to ChainAddEdgesProcessorLocal::processLocal()"; + return ChainAddEdgesProcessorLocal::processLocal(code); + } + + cpp2::AddEdgesRequest reverseRequestForward(const cpp2::AddEdgesRequest& req) { + return ChainAddEdgesProcessorLocal::reverseRequest(req); + } + + folly::Optional rcPrepareLocal; + + folly::Optional rcProcessRemote; + + folly::Optional rcProcessLocal; +}; + +class FakeChainUpdateProcessor : public ChainUpdateEdgeProcessorLocal { + public: + explicit FakeChainUpdateProcessor(StorageEnv* env) : ChainUpdateEdgeProcessorLocal(env) { + spaceVidLen_ = 32; + } + + folly::SemiFuture prepareLocal() override { + LOG(INFO) << "FakeChainUpdateEdgeProcessorA::" << __func__ << "()"; + if (rcPrepareLocal) { + LOG(INFO) << "Fake return " << apache::thrift::util::enumNameSafe(*rcPrepareLocal); + return *rcPrepareLocal; + } + LOG(INFO) << "forward to ChainUpdateEdgeProcessorLocal::prepareLocal()"; + return ChainUpdateEdgeProcessorLocal::prepareLocal(); + } + + folly::SemiFuture processRemote(Code code) override { + LOG(INFO) << "FakeChainUpdateEdgeProcessorA::" << __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 ChainUpdateEdgeProcessorLocal::processRemote()"; + return ChainUpdateEdgeProcessorLocal::processRemote(code); + } + + folly::SemiFuture processLocal(Code code) override { + LOG(INFO) << "FakeChainUpdateEdgeProcessorA::" << __func__ << "()"; + if (rcProcessLocal) { + LOG(INFO) << "processLocal() fake return " + << apache::thrift::util::enumNameSafe(*rcProcessLocal); + return *rcProcessLocal; + } + LOG(INFO) << "forward to ChainUpdateEdgeProcessorLocal::processLocal()"; + return ChainUpdateEdgeProcessorLocal::processLocal(code); + } + + public: + folly::Optional rcPrepareLocal; + folly::Optional rcProcessRemote; + folly::Optional rcProcessLocal; +}; + +class MetaClientTestUpdater { + public: + MetaClientTestUpdater() = default; + + static void addLocalCache(meta::MetaClient& mClient, + GraphSpaceID spaceId, + std::shared_ptr spInfoCache) { + mClient.localCache_[spaceId] = spInfoCache; + } + + static meta::SpaceInfoCache* getLocalCache(meta::MetaClient* mClient, GraphSpaceID spaceId) { + if (mClient->localCache_.count(spaceId) == 0) { + return nullptr; + } + return mClient->localCache_[spaceId].get(); + } + + static void addPartTerm(meta::MetaClient* mClient, + GraphSpaceID spaceId, + PartitionID partId, + TermID termId) { + auto* pCache = getLocalCache(mClient, spaceId); + if (pCache == nullptr) { + auto spCache = std::make_shared(); + addLocalCache(*mClient, spaceId, spCache); + pCache = getLocalCache(mClient, spaceId); + } + pCache->termOfPartition_[partId] = termId; + } + + static std::unique_ptr makeDefaultMetaClient() { + auto exec = std::make_shared(3); + std::vector addrs(1); + meta::MetaClientOptions options; + + auto mClient = std::make_unique(exec, addrs, options); + mClient->localCache_[mockSpaceId] = std::make_shared(); + for (int i = 0; i != mockPartNum; ++i) { + mClient->localCache_[mockSpaceId]->termOfPartition_[i] = i; + auto ignoreItem = mClient->localCache_[mockSpaceId]->partsAlloc_[i]; + UNUSED(ignoreItem); + } + meta::cpp2::ColumnTypeDef type; + type.set_type(meta::cpp2::PropertyType::FIXED_STRING); + type.set_type_length(32); + + mClient->localCache_[mockSpaceId]->spaceDesc_.set_vid_type(std::move(type)); + mClient->ready_ = true; + return mClient; + } +}; + +class FakeInternalStorageClient : public InternalStorageClient { + public: + explicit FakeInternalStorageClient(StorageEnv* env, + std::shared_ptr pool, + Code code) + : InternalStorageClient(pool, env->metaClient_), env_(env), code_(code) {} + + void chainUpdateEdge(cpp2::UpdateEdgeRequest& req, + TermID termOfSrc, + folly::Optional optVersion, + folly::Promise&& p, + folly::EventBase* evb = nullptr) override { + cpp2::ChainUpdateEdgeRequest chainReq; + chainReq.set_update_edge_request(req); + chainReq.set_term(termOfSrc); + + auto* proc = ChainUpdateEdgeProcessorRemote::instance(env_); + auto f = proc->getFuture(); + proc->process(chainReq); + auto resp = std::move(f).get(); + + p.setValue(code_); + UNUSED(optVersion); + UNUSED(evb); + } + + void setErrorCode(Code code) { code_ = code; } + + void chainAddEdges(cpp2::AddEdgesRequest& req, + TermID termId, + folly::Optional optVersion, + folly::Promise<::nebula::cpp2::ErrorCode>&& p, + folly::EventBase* evb = nullptr) override { + UNUSED(req); + UNUSED(termId); + UNUSED(optVersion); + 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 void hookInternalStorageClient(StorageEnv* env, InternalStorageClient* client) { + env->txnMan_->iClient_ = client; + } + + private: + StorageEnv* env_{nullptr}; + Code code_{Code::SUCCEEDED}; +}; + +struct ChainUpdateEdgeTestHelper { + ChainUpdateEdgeTestHelper() { sEdgeType = std::to_string(std::abs(edgeType_)); } + + cpp2::EdgeKey defaultEdgeKey() { + cpp2::EdgeKey ret; + ret.set_src(srcId_); + ret.set_edge_type(edgeType_); + ret.set_ranking(rank_); + ret.set_dst(dstId_); + return ret; + } + + std::vector defaultUpdateProps() { + ObjectPool objPool; + LOG(INFO) << "Build updated props..."; + std::vector props; + // int: 101.teamCareer = 20 + cpp2::UpdatedProp uProp1; + uProp1.set_name("teamCareer"); + // ConstantExpression val1(20); + const auto& val1 = *ConstantExpression::make(&objPool, 20); + uProp1.set_value(Expression::encode(val1)); + props.emplace_back(uProp1); + + // bool: 101.type = trade + cpp2::UpdatedProp uProp2; + uProp2.set_name("type"); + std::string colnew("trade"); + // ConstantExpression val2(colnew); + const auto& val2 = *ConstantExpression::make(&objPool, colnew); + uProp2.set_value(Expression::encode(val2)); + props.emplace_back(uProp2); + return props; + } + + std::vector defaultRetProps() { + ObjectPool objPool; + std::vector props; + std::vector cols{ + "playerName", "teamName", "teamCareer", "type", kSrc, kType, kRank, kDst}; + for (auto& colName : cols) { + const auto& exp = *EdgePropertyExpression::make(&objPool, sEdgeType, colName); + props.emplace_back(Expression::encode(exp)); + } + return props; + } + + cpp2::UpdateEdgeRequest makeDefaultRequest() { + auto edgeKey = defaultEdgeKey(); + auto updateProps = defaultUpdateProps(); + auto retProps = defaultRetProps(); + return makeRequest(edgeKey, updateProps, retProps); + } + + cpp2::UpdateEdgeRequest reverseRequest(StorageEnv* env, const cpp2::UpdateEdgeRequest& req) { + ChainUpdateEdgeProcessorLocal proc(env); + return proc.reverseRequest(req); + } + + cpp2::UpdateEdgeRequest makeInvalidRequest() { + VertexID srcInvalid{"Spurssssssss"}; + auto edgeKey = defaultEdgeKey(); + edgeKey.set_src(srcInvalid); + auto updateProps = defaultUpdateProps(); + auto retProps = defaultRetProps(); + return makeRequest(edgeKey, updateProps, retProps); + } + + cpp2::UpdateEdgeRequest makeRequest(const cpp2::EdgeKey& edgeKey, + const std::vector& updateProps, + const std::vector& retCols) { + cpp2::UpdateEdgeRequest req; + auto partId = std::hash()(edgeKey.get_src().getStr()) % mockPartNum + 1; + req.set_space_id(mockSpaceId); + req.set_part_id(partId); + req.set_edge_key(edgeKey); + req.set_updated_props(updateProps); + req.set_return_props(retCols); + req.set_insertable(false); + return req; + } + + bool checkResp2(cpp2::UpdateResponse& resp) { + LOG(INFO) << "checkResp2(cpp2::UpdateResponse& resp)"; + if (!resp.props_ref()) { + LOG(INFO) << "!resp.props_ref()"; + return false; + } else { + LOG(INFO) << "resp.props_ref()"; + EXPECT_EQ(9, (*resp.props_ref()).colNames.size()); + EXPECT_EQ("_inserted", (*resp.props_ref()).colNames[0]); + EXPECT_EQ("101.playerName", (*resp.props_ref()).colNames[1]); + EXPECT_EQ("101.teamName", (*resp.props_ref()).colNames[2]); + EXPECT_EQ("101.teamCareer", (*resp.props_ref()).colNames[3]); + EXPECT_EQ("101.type", (*resp.props_ref()).colNames[4]); + EXPECT_EQ(std::string("101.").append(kSrc), (*resp.props_ref()).colNames[5]); + EXPECT_EQ(std::string("101.").append(kType), (*resp.props_ref()).colNames[6]); + EXPECT_EQ(std::string("101.").append(kRank), (*resp.props_ref()).colNames[7]); + EXPECT_EQ(std::string("101.").append(kDst), (*resp.props_ref()).colNames[8]); + EXPECT_EQ(1, (*resp.props_ref()).rows.size()); + EXPECT_EQ(9, (*resp.props_ref()).rows[0].values.size()); + EXPECT_EQ(false, (*resp.props_ref()).rows[0].values[0].getBool()); + EXPECT_EQ("Tim Duncan", (*resp.props_ref()).rows[0].values[1].getStr()); + EXPECT_EQ("Spurs", (*resp.props_ref()).rows[0].values[2].getStr()); + EXPECT_EQ(20, (*resp.props_ref()).rows[0].values[3].getInt()); + EXPECT_EQ("trade", (*resp.props_ref()).rows[0].values[4].getStr()); + EXPECT_EQ("Spurs", (*resp.props_ref()).rows[0].values[5].getStr()); + EXPECT_EQ(-101, (*resp.props_ref()).rows[0].values[6].getInt()); + EXPECT_EQ(1997, (*resp.props_ref()).rows[0].values[7].getInt()); + EXPECT_EQ("Tim Duncan", (*resp.props_ref()).rows[0].values[8].getStr()); + } + return true; + } + + bool edgeExist(StorageEnv* env, cpp2::UpdateEdgeRequest& req) { + auto partId = req.get_part_id(); + auto key = ConsistUtil::edgeKey(mockSpaceVidLen, partId, req.get_edge_key()); + return keyExist(env, mockSpaceId, partId, key); + } + + bool primeExist(StorageEnv* env, cpp2::UpdateEdgeRequest& req) { + auto partId = req.get_part_id(); + auto key = ConsistUtil::primeKey(mockSpaceVidLen, partId, req.get_edge_key()); + return keyExist(env, mockSpaceId, partId, key); + } + + bool doublePrimeExist(StorageEnv* env, cpp2::UpdateEdgeRequest& req) { + auto partId = req.get_part_id(); + auto key = ConsistUtil::doublePrime(mockSpaceVidLen, partId, req.get_edge_key()); + return keyExist(env, mockSpaceId, partId, key); + } + + bool keyExist(StorageEnv* env, GraphSpaceID spaceId, PartitionID partId, const std::string& key) { + std::string val; + auto rc = env->kvstore_->get(spaceId, partId, key, &val); + return rc == Code::SUCCEEDED; + } + + bool checkRequestUpdated(StorageEnv* env, cpp2::UpdateEdgeRequest& req) { + // get serve from kvstore directly + bool ret = true; + auto& key = req.get_edge_key(); + auto partId = req.get_part_id(); + auto prefix = ConsistUtil::edgeKey(mockSpaceVidLen, partId, req.get_edge_key()); + std::unique_ptr iter; + auto rc = env->kvstore_->prefix(mockSpaceId, partId, prefix, &iter); + EXPECT_EQ(Code::SUCCEEDED, rc); + EXPECT_TRUE(iter && iter->valid()); + + auto edgeType = key.get_edge_type(); + auto edgeReader = RowReaderWrapper::getEdgePropReader( + env->schemaMan_, mockSpaceId, std::abs(edgeType), iter->val()); + + LOG(INFO) << "req.get_updated_props().size() = " << req.get_updated_props().size(); + ObjectPool objPool; + for (auto& prop : req.get_updated_props()) { + LOG(INFO) << "prop name = " << prop.get_name(); + auto enVal = prop.get_value(); + auto expression = Expression::decode(&objPool, enVal); + ConstantExpression* cexpr = static_cast(expression); + auto val1 = cexpr->value(); + auto val2 = edgeReader->getValueByName(prop.get_name()); + + // EXPECT_EQ(val1, val2); + if (val1 != val2) { + ret = false; + } + } + + return ret; + } + + public: + VertexID srcId_{"Spurs"}; + VertexID dstId_{"Tim Duncan"}; + EdgeRanking rank_{1997}; + EdgeType edgeType_{-101}; + storage::cpp2::EdgeKey updateKey_; + std::string sEdgeType; +}; + +// class ChainResumeProcessorTestHelper { +// public: +// explicit ChainResumeProcessorTestHelper(ChainResumeProcessor* proc) : proc_(proc) {} + +// void setAddEdgeProc(ChainAddEdgesProcessorLocal* proc) { +// proc_->addProc = proc; +// } + +// // setUpdProc +// void setUpdProc(ChainUpdateEdgeProcessorLocal* proc) { +// proc_->updProc = proc; +// } + +// std::string getTxnId() { +// return proc_->addProc->txnId_; +// } +// public: +// ChainResumeProcessor* proc_{nullptr}; +// }; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/test/ChainUpdateEdgeTest.cpp b/src/storage/test/ChainUpdateEdgeTest.cpp new file mode 100644 index 00000000000..0b5c96f4e3f --- /dev/null +++ b/src/storage/test/ChainUpdateEdgeTest.cpp @@ -0,0 +1,268 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#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/mutate/UpdateEdgeProcessor.h" +#include "storage/test/ChainTestUtils.h" +#include "storage/test/QueryTestUtils.h" +#include "storage/test/TestUtils.h" +#include "storage/transaction/ChainAddEdgesGroupProcessor.h" +#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ChainResumeProcessor.h" +#include "storage/transaction/ChainUpdateEdgeProcessorRemote.h" +#include "storage/transaction/ConsistUtil.h" + +namespace nebula { +namespace storage { + +// using Code = ::nebula::cpp2::ErrorCode; + +constexpr int32_t mockSpaceId = 1; +constexpr int32_t mockPartNum = 6; +constexpr int32_t mockSpaceVidLen = 32; + +ChainTestUtils gTestUtil; + +ChainUpdateEdgeTestHelper helper; +TEST(ChainUpdateEdgeTest, updateTest1) { + fs::TempDir rootPath("/tmp/UpdateEdgeTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + env->metaClient_ = mClient.get(); + + auto parts = cluster.getTotalParts(); + EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); + + LOG(INFO) << "Test updateTest1..."; + auto req = helper.makeDefaultRequest(); + + env->txnMan_->iClient_ = FakeInternalStorageClient::instance(env); + auto reversedRequest = helper.reverseRequest(env, req); + + auto* proc = new FakeChainUpdateProcessor(env); + LOG(INFO) << "proc: " << proc; + auto f = proc->getFuture(); + proc->process(req); + auto resp = std::move(f).get(); + + EXPECT_TRUE(helper.checkResp2(resp)); + EXPECT_TRUE(helper.checkRequestUpdated(env, req)); + EXPECT_TRUE(helper.checkRequestUpdated(env, reversedRequest)); + EXPECT_TRUE(helper.edgeExist(env, req)); + EXPECT_FALSE(helper.primeExist(env, req)); + EXPECT_FALSE(helper.doublePrimeExist(env, req)); +} + +TEST(ChainUpdateEdgeTest, updateTest2) { + fs::TempDir rootPath("/tmp/UpdateEdgeTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + env->metaClient_ = mClient.get(); + + auto parts = cluster.getTotalParts(); + EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); + + LOG(INFO) << "Test UpdateEdgeRequest..."; + auto goodRequest = helper.makeDefaultRequest(); + EXPECT_TRUE(helper.edgeExist(env, goodRequest)); + EXPECT_FALSE(helper.primeExist(env, goodRequest)); + EXPECT_FALSE(helper.doublePrimeExist(env, goodRequest)); + + auto badRequest = helper.makeInvalidRequest(); + + auto* proc = new FakeChainUpdateProcessor(env); + auto f = proc->getFuture(); + proc->rcProcessRemote = Code::E_KEY_NOT_FOUND; + proc->process(badRequest); + auto resp = std::move(f).get(); + + EXPECT_EQ(1, (*resp.result_ref()).failed_parts.size()); + EXPECT_FALSE(helper.checkResp2(resp)); + EXPECT_FALSE(helper.edgeExist(env, badRequest)); + EXPECT_FALSE(helper.primeExist(env, badRequest)); + EXPECT_FALSE(helper.doublePrimeExist(env, badRequest)); +} + +TEST(ChainUpdateEdgeTest, updateTest3) { + fs::TempDir rootPath("/tmp/UpdateEdgeTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + env->metaClient_ = mClient.get(); + + auto parts = cluster.getTotalParts(); + EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); + + LOG(INFO) << "Test UpdateEdgeRequest..."; + auto goodRequest = helper.makeDefaultRequest(); + EXPECT_TRUE(helper.edgeExist(env, goodRequest)); + EXPECT_FALSE(helper.primeExist(env, goodRequest)); + EXPECT_FALSE(helper.doublePrimeExist(env, goodRequest)); + + auto* proc = new FakeChainUpdateProcessor(env); + auto f = proc->getFuture(); + proc->rcProcessRemote = Code::SUCCEEDED; + proc->rcProcessLocal = Code::SUCCEEDED; + proc->process(goodRequest); + auto resp = std::move(f).get(); + + EXPECT_TRUE(helper.edgeExist(env, goodRequest)); + EXPECT_TRUE(helper.primeExist(env, goodRequest)); + EXPECT_FALSE(helper.doublePrimeExist(env, goodRequest)); +} + +TEST(ChainUpdateEdgeTest, updateTest4) { + fs::TempDir rootPath("/tmp/UpdateEdgeTest.XXXXXX"); + mock::MockCluster cluster; + cluster.initStorageKV(rootPath.path()); + auto* env = cluster.storageEnv_.get(); + auto mClient = MetaClientTestUpdater::makeDefaultMetaClient(); + env->metaClient_ = mClient.get(); + + auto parts = cluster.getTotalParts(); + EXPECT_TRUE(QueryTestUtils::mockEdgeData(env, parts, mockSpaceVidLen)); + + LOG(INFO) << "Test UpdateEdgeRequest..."; + auto goodRequest = helper.makeDefaultRequest(); + EXPECT_TRUE(helper.edgeExist(env, goodRequest)); + EXPECT_FALSE(helper.primeExist(env, goodRequest)); + EXPECT_FALSE(helper.doublePrimeExist(env, goodRequest)); + + auto* proc = new FakeChainUpdateProcessor(env); + auto f = proc->getFuture(); + proc->rcProcessRemote = Code::E_RPC_FAILURE; + proc->process(goodRequest); + auto resp = std::move(f).get(); + + EXPECT_TRUE(helper.edgeExist(env, goodRequest)); + EXPECT_FALSE(helper.primeExist(env, goodRequest)); + EXPECT_TRUE(helper.doublePrimeExist(env, goodRequest)); +} + +} // namespace storage +} // namespace nebula + +int main(int argc, char** argv) { + testing::InitGoogleTest(&argc, argv); + folly::init(&argc, &argv, false); + google::SetStderrLogging(google::INFO); + return RUN_ALL_TESTS(); +} + +// ***** Test Plan ***** +/** + * @brief updateTest1 (update a normal edge will succeed) + * previous update + * prepareLocal succeed succeed + * processRemote succeed succeed + * processLocal succeed succeed + * expect: edge true + * edge prime false + * double prime false + * prop changed true + */ + +/** + * @brief updateTest2 (update non-exist edge will fail) + * previous update + * prepareLocal failed succeed + * processRemote skip succeed + * processLocal failed succeed + * expect: edge false + * edge prime false + * double prime false + * prop changed true + */ + +/** + * @brief updateTest3 (remote update failed will not change anything) + * previous update + * prepareLocal succeed succeed + * processRemote skip failed + * processLocal skip failed + * expect: edge true + * edge prime true + * double prime false + * prop changed false + */ + +/** + * @brief updateTest4 (remote update outdate will add double prime) + * previous update + * prepareLocal succeed succeed + * processRemote skip outdate + * processLocal skip succeed + * expect: edge true + * edge prime false + * double prime true + * prop changed false + */ + +// /** +// * @brief updateTest5 (update1 + resume) +// * previous update +// * prepareLocal succeed succeed +// * processRemote skip succeed +// * processLocal succeed succeed +// * expect: edge true +// * edge prime false +// * double prime false +// * prop changed true +// */ + +// /** +// * @brief updateTest6 (update2 + resume) +// * previous update +// * prepareLocal failed succeed +// * processRemote skip succeed +// * processLocal failed succeed +// * expect: edge false +// * edge prime false +// * double prime false +// * prop changed true +// */ + +// /** +// * @brief updateTest7 (updateTest3 + resume) +// * previous resume +// * prepareLocal succeed succeed +// * processRemote skip failed +// * processLocal skip failed +// * expect: edge true +// * edge prime true +// * double prime false +// * prop changed false +// */ + +// /** +// * @brief updateTest8 +// * previous resume +// * prepareLocal succeed succeed +// * processRemote skip outdate +// * processLocal skip succeed +// * expect: edge true +// * edge prime false +// * double prime true +// * prop changed false +// */ + +// ***** End Test Plan ***** diff --git a/src/storage/test/ConsistUtilTest.cpp b/src/storage/test/ConsistUtilTest.cpp new file mode 100644 index 00000000000..ea9544e9ada --- /dev/null +++ b/src/storage/test/ConsistUtilTest.cpp @@ -0,0 +1,31 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#include +#include +#include +#include +#include + +#include "folly/String.h" +#include "storage/transaction/ConsistUtil.h" + +#define LOG_FMT(...) LOG(INFO) << folly::sformat(__VA_ARGS__) + +namespace nebula { +namespace storage { + +TEST(ConsistUtilTest, primeTable) {} + +} // namespace storage +} // namespace nebula + +int main(int argc, char** argv) { + testing::InitGoogleTest(&argc, argv); + folly::init(&argc, &argv, false); + google::SetStderrLogging(google::INFO); + return RUN_ALL_TESTS(); +} diff --git a/src/storage/test/TossBenchmark.cpp b/src/storage/test/TossBenchmark.cpp deleted file mode 100644 index 7be77099ffc..00000000000 --- a/src/storage/test/TossBenchmark.cpp +++ /dev/null @@ -1,246 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License, - * attached with Common Clause Condition 1.0, found in the LICENSES directory. - */ - -#include -#include -#include -#include - -#include - -#include "TossEnvironment.h" - -#define oneReqOneEdge 1 -#define oneReqTenEdges 1 -#define oneReqHundredEdges 1 -#define oneReqThousandEdges 1 - -namespace nebula { -namespace storage { - -using StorageClient = storage::GraphStorageClient; - -bool useToss = true; -bool notToss = false; - -int32_t gRank = 0; -std::map gAddedEdges; - -std::vector gTypes; - -std::vector genColDefs(const std::vector& types) { - auto N = types.size(); - auto colNames = TossEnvironment::makeColNames(N); - std::vector ret(N); - for (auto i = 0U; i != N; ++i) { - ret[i].set_name(colNames[i]); - meta::cpp2::ColumnTypeDef tpDef; - tpDef.set_type(types[i]); - ret[i].set_type(tpDef); - ret[i].set_nullable(true); - } - return ret; -} - -void setupEnvironment() { - auto env = TossEnvironment::getInstance(kMetaName, kMetaPort); - std::vector types; - gTypes.emplace_back(meta::cpp2::PropertyType::INT64); - gTypes.emplace_back(meta::cpp2::PropertyType::STRING); - - auto colDefs = genColDefs(gTypes); - env->init(kSpaceName, kPart, kReplica, colDefs); -} - -#if oneReqOneEdge -BENCHMARK(bmOneReqOneEdgeTossNo) { - int32_t srcId = 10000; - ++gAddedEdges[srcId]; - auto dstId = gAddedEdges[srcId]; - std::vector vals(gTypes.size()); - vals[0].setInt(srcId); - vals[1].setStr(folly::sformat("bmoneReqOneEdgeTossNo, src={}, dst={}", srcId, dstId)); - - auto env = TossEnvironment::getInstance(kMetaName, kMetaPort); - auto edge = env->generateEdge(srcId, gRank, vals, dstId); - - env->syncAddEdge(edge, notToss); -} - -BENCHMARK_RELATIVE(bmOneReqOneEdgeTossYes) { - int32_t srcId = 20000; - ++gAddedEdges[srcId]; - auto dstId = gAddedEdges[srcId]; - - std::vector vals(gTypes.size()); - vals[0].setInt(srcId); - vals[1].setStr(folly::sformat("bmoneReqOneEdgeTossYes, src={}, dst={}", srcId, dstId)); - - auto env = TossEnvironment::getInstance(kMetaName, kMetaPort); - auto edge = env->generateEdge(srcId, gRank, vals, dstId); - - env->syncAddEdge(edge, useToss); -} - -BENCHMARK_DRAW_LINE(); -#endif - -#if oneReqTenEdges -BENCHMARK(bmOneReqTenEdges) { - size_t cnt = 10; - int32_t srcId = 30000; - std::vector vals(gTypes.size()); - vals[0].setInt(srcId); - vals[1].setStr(folly::sformat("{}", __func__)); - - auto env = TossEnvironment::getInstance(kMetaName, kMetaPort); - std::vector edges; - edges.reserve(cnt); - for (auto i = 0U; i < cnt; ++i) { - ++gAddedEdges[srcId]; - auto dstId = gAddedEdges[srcId]; - edges.emplace_back(env->generateEdge(srcId, gRank, vals, dstId)); - } - env->addEdgesAsync(edges, notToss).wait(); -} - -BENCHMARK_RELATIVE(bmOneReqTenEdgesToss) { - size_t cnt = 10; - int32_t srcId = 40000; - std::vector vals(gTypes.size()); - vals[0].setInt(srcId); - vals[1].setStr(folly::sformat("{}", __func__)); - - auto env = TossEnvironment::getInstance(kMetaName, kMetaPort); - std::vector edges; - edges.reserve(cnt); - for (auto i = 0U; i < cnt; ++i) { - ++gAddedEdges[srcId]; - auto dstId = gAddedEdges[srcId]; - edges.emplace_back(env->generateEdge(srcId, gRank, vals, dstId)); - } - env->addEdgesAsync(edges, useToss).wait(); -} -#endif - -#if oneReqHundredEdges -BENCHMARK_DRAW_LINE(); -BENCHMARK(bmOneReqHundredEdges) { - size_t cnt = 100; - int32_t srcId = 50000; - std::vector vals(gTypes.size()); - vals[0].setInt(srcId); - vals[1].setStr(folly::sformat("{}", __func__)); - - auto env = TossEnvironment::getInstance(kMetaName, kMetaPort); - std::vector edges; - edges.reserve(cnt); - for (auto i = 0U; i < cnt; ++i) { - ++gAddedEdges[srcId]; - auto dstId = gAddedEdges[srcId]; - edges.emplace_back(env->generateEdge(srcId, gRank, vals, dstId)); - } - env->addEdgesAsync(edges, notToss).wait(); -} - -BENCHMARK_RELATIVE(bmOneReqHundredEdgesToss) { - size_t cnt = 100; - int32_t srcId = 60000; - std::vector vals(gTypes.size()); - vals[0].setInt(srcId); - vals[1].setStr(folly::sformat("{}", __func__)); - - auto env = TossEnvironment::getInstance(kMetaName, kMetaPort); - std::vector edges; - edges.reserve(cnt); - for (auto i = 0U; i < cnt; ++i) { - ++gAddedEdges[srcId]; - auto dstId = gAddedEdges[srcId]; - edges.emplace_back(env->generateEdge(srcId, gRank, vals, dstId)); - } - env->addEdgesAsync(edges, useToss).wait(); -} -#endif - -#if oneReqThousandEdges -BENCHMARK_DRAW_LINE(); -BENCHMARK(bmOneReqThousandEdges) { - size_t cnt = 1000; - int32_t srcId = 70000; - std::vector vals(gTypes.size()); - vals[0].setInt(srcId); - vals[1].setStr(folly::sformat("{}", __func__)); - - auto env = TossEnvironment::getInstance(kMetaName, kMetaPort); - std::vector edges; - edges.reserve(cnt); - for (auto i = 0U; i < cnt; ++i) { - ++gAddedEdges[srcId]; - auto dstId = gAddedEdges[srcId]; - edges.emplace_back(env->generateEdge(srcId, gRank, vals, dstId)); - } - env->addEdgesAsync(edges, notToss).wait(); -} - -BENCHMARK_RELATIVE(bmOneReqThousandEdgesToss) { - size_t cnt = 1000; - int32_t srcId = 80000; - std::vector vals(gTypes.size()); - vals[0].setInt(srcId); - vals[1].setStr(folly::sformat("{}", __func__)); - - auto env = TossEnvironment::getInstance(kMetaName, kMetaPort); - std::vector edges; - edges.reserve(cnt); - for (auto i = 0U; i < cnt; ++i) { - ++gAddedEdges[srcId]; - auto dstId = gAddedEdges[srcId]; - edges.emplace_back(env->generateEdge(srcId, gRank, vals, dstId)); - } - env->addEdgesAsync(edges, useToss).wait(); -} -#endif - -} // namespace storage -} // namespace nebula - -using namespace nebula::storage; // NOLINT - -int main(int argc, char** argv) { - FLAGS_heartbeat_interval_secs = 1; - FLAGS_logtostderr = 1; - - folly::init(&argc, &argv, false); - setupEnvironment(); - folly::runBenchmarks(); - LOG(INFO) << "exit main"; - - auto env = TossEnvironment::getInstance(kMetaName, kMetaPort); - for (auto& item : gAddedEdges) { - std::vector edges; - std::vector vals(gTypes.size()); - edges.emplace_back(env->generateEdge(item.first, gRank, vals)); - auto strNei = env->getNeiProps(edges); - auto cnt = env->countSquareBrackets(strNei); - LOG(INFO) << folly::sformat("testId={}, testCalled={}, cnt={}", item.first, item.second, cnt); - } -} - -// ============================================================================ -// TossBenchmark.cpprelative (fcb6a91) time/iter iters/s -// ============================================================================ -// bmOneReqOneEdgeTossNo 1.82ms 549.76 -// bmOneReqOneEdgeTossYes 42.83% 4.25ms 235.46 -// ---------------------------------------------------------------------------- -// bmOneReqTenEdges 1.96ms 511.07 -// bmOneReqTenEdgesToss 20.05% 9.76ms 102.49 -// ---------------------------------------------------------------------------- -// bmOneReqHundredEdges 6.93ms 144.31 -// bmOneReqHundredEdgesToss 49.95% 13.87ms 72.08 -// ---------------------------------------------------------------------------- -// bmOneReqThousandEdges 33.76ms 29.62 -// bmOneReqThousandEdgesToss 69.75% 48.40ms 20.66 -// ============================================================================ diff --git a/src/storage/test/TossEnvironment.h b/src/storage/test/TossEnvironment.h deleted file mode 100644 index 3cb82fbc2b9..00000000000 --- a/src/storage/test/TossEnvironment.h +++ /dev/null @@ -1,686 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License, - * attached with Common Clause Condition 1.0, found in the LICENSES directory. - */ - -#pragma once - -#include - -#include "TossTestUtils.h" -#include "common/meta/ServerBasedSchemaManager.h" - -#define FLOG_FMT(...) LOG(INFO) << folly::sformat(__VA_ARGS__) - -DECLARE_int32(heartbeat_interval_secs); -DEFINE_string(meta_server, "127.0.0.1:44500", "Meta servers' address."); - -namespace nebula { -namespace storage { - -const std::string kMetaName = "hp-server"; // NOLINT -constexpr int32_t kMetaPort = 6500; -const std::string kSpaceName = "test"; // NOLINT -constexpr int32_t kPart = 5; -constexpr int32_t kReplica = 3; -constexpr int32_t kSum = 10000 * 10000; - -using StorageClient = storage::GraphStorageClient; -struct TossEnvironment { - static TossEnvironment* getInstance(const std::string& metaName, int32_t metaPort) { - static TossEnvironment inst(metaName, metaPort); - return &inst; - } - - TossEnvironment(const std::string& metaName, int32_t metaPort) { - executor_ = std::make_shared(20); - mClient_ = setupMetaClient(metaName, metaPort); - sClient_ = std::make_unique(executor_, mClient_.get()); - interClient_ = std::make_unique(executor_, mClient_.get()); - schemaMan_ = meta::ServerBasedSchemaManager::create(mClient_.get()); - } - - std::unique_ptr setupMetaClient(const std::string& metaName, - uint32_t metaPort) { - std::vector metas; - metas.emplace_back(HostAddr(metaName, metaPort)); - meta::MetaClientOptions options; - auto client = std::make_unique(executor_, metas, options); - if (!client->waitForMetadReady()) { - LOG(FATAL) << "!client->waitForMetadReady()"; - } - return client; - } - - /* - * setup space and edge type, then describe the cluster - * */ - void init(const std::string& spaceName, - size_t part, - int replica, - std::vector& colDefs) { - if (spaceId_ != 0) { - return; - } - spaceId_ = setupSpace(spaceName, part, replica); - edgeType_ = setupEdgeSchema("test_edge", colDefs); - - int sleepSecs = FLAGS_heartbeat_interval_secs + 2; - while (sleepSecs) { - LOG(INFO) << "sleep for " << sleepSecs-- << " sec"; - std::this_thread::sleep_for(std::chrono::seconds(1)); - } - - auto stVIdLen = mClient_->getSpaceVidLen(spaceId_); - LOG_IF(FATAL, !stVIdLen.ok()); - vIdLen_ = stVIdLen.value(); - - bool leaderLoaded = false; - while (!leaderLoaded) { - auto statusOrLeaderMap = mClient_->getLeaderInfo(); - if (!statusOrLeaderMap.ok()) { - LOG(FATAL) << "mClient_->loadLeader() failed!!!!!!"; - } - - auto leaderMap = statusOrLeaderMap.value().leaderMap_; - for (auto& leader : leaderMap) { - LOG(INFO) << "spaceId=" << leader.first.first << ", part=" << leader.first.second - << ", host=" << leader.second; - if (leader.first.first == spaceId_) { - leaderLoaded = true; - } - } - } - } - - int32_t getSpaceId() { return spaceId_; } - - int setupSpace(const std::string& spaceName, int nPart, int nReplica) { - auto fDropSpace = mClient_->dropSpace(spaceName, true); - fDropSpace.wait(); - LOG(INFO) << "drop space " << spaceName; - - meta::cpp2::SpaceDesc spaceDesc; - spaceDesc.set_space_name(spaceName); - spaceDesc.set_partition_num(nPart); - spaceDesc.set_replica_factor(nReplica); - meta::cpp2::ColumnTypeDef colType; - colType.set_type(meta::cpp2::PropertyType::INT64); - spaceDesc.set_vid_type(colType); - spaceDesc.set_isolation_level(meta::cpp2::IsolationLevel::TOSS); - - auto fCreateSpace = mClient_->createSpace(spaceDesc, true); - fCreateSpace.wait(); - if (!fCreateSpace.valid()) { - LOG(FATAL) << "!fCreateSpace.valid()"; - } - if (!fCreateSpace.value().ok()) { - LOG(FATAL) << "!fCreateSpace.value().ok(): " << fCreateSpace.value().status().toString(); - } - auto spaceId = fCreateSpace.value().value(); - LOG(INFO) << folly::sformat("spaceId = {}", spaceId); - return spaceId; - } - - EdgeType setupEdgeSchema(const std::string& edgeName, - std::vector columns) { - meta::cpp2::Schema schema; - schema.set_columns(std::move(columns)); - - auto fCreateEdgeSchema = mClient_->createEdgeSchema(spaceId_, edgeName, schema, true); - fCreateEdgeSchema.wait(); - - if (!fCreateEdgeSchema.valid() || !fCreateEdgeSchema.value().ok()) { - LOG(FATAL) << "createEdgeSchema failed"; - } - return fCreateEdgeSchema.value().value(); - } - - cpp2::EdgeKey generateEdgeKey(int64_t srcId, int rank, int dstId = 0) { - cpp2::EdgeKey edgeKey; - edgeKey.set_src(srcId); - - edgeKey.set_edge_type(edgeType_); - edgeKey.set_ranking(rank); - if (dstId == 0) { - dstId = kSum - srcId; - } - edgeKey.set_dst(dstId); - return edgeKey; - } - - cpp2::NewEdge generateEdge(int srcId, - int rank, - std::vector values, - int dstId = 0) { - cpp2::NewEdge newEdge; - - cpp2::EdgeKey edgeKey = generateEdgeKey(srcId, rank, dstId); - newEdge.set_key(std::move(edgeKey)); - newEdge.set_props(std::move(values)); - - return newEdge; - } - - cpp2::NewEdge reverseEdge(const cpp2::NewEdge& e0) { - cpp2::NewEdge e(e0); - std::swap(*(*e.key_ref()).src_ref(), *(*e.key_ref()).dst_ref()); - (*e.key_ref()).set_edge_type(-e.get_key().get_edge_type()); - return e; - } - - nebula::cpp2::ErrorCode syncAddMultiEdges(std::vector& edges, bool useToss) { - bool retLeaderChange = false; - int32_t retry = 0; - int32_t retryMax = 10; - do { - if (retry > 0 && retry != retryMax) { - LOG(INFO) << "\n leader changed, retry=" << retry; - std::this_thread::sleep_for(std::chrono::milliseconds(500)); - retLeaderChange = false; - } - auto f = addEdgesAsync(edges, useToss); - f.wait(); - if (!f.valid()) { - auto retcode = nebula::cpp2::ErrorCode::E_UNKNOWN; - LOG(INFO) << apache::thrift::util::enumNameSafe(retcode); - return retcode; - } - if (!f.value().succeeded()) { - LOG(INFO) << "addEdgeAsync() !f.value().succeeded()"; - LOG(INFO) << "f.value().failedParts().size()=" << f.value().failedParts().size(); - for (auto& part : f.value().failedParts()) { - LOG(INFO) << "partId=" << part.first - << ", ec=" << apache::thrift::util::enumNameSafe(part.second); - if (part.second == nebula::cpp2::ErrorCode::E_LEADER_CHANGED) { - retLeaderChange = true; - } - } - } - - std::vector& execResps = f.value().responses(); - for (auto& execResp : execResps) { - // ResponseCommon - auto& respComn = execResp.get_result(); - auto& failedParts = respComn.get_failed_parts(); - for (auto& part : failedParts) { - if (part.code == nebula::cpp2::ErrorCode::E_LEADER_CHANGED) { - retLeaderChange = true; - LOG(INFO) << "addEdgeAsync() !f.value().succeeded(), retry"; - } - } - } - - if (++retry == retryMax) { - break; - } - } while (retLeaderChange); - LOG(INFO) << "addEdgeAsync() succeeded"; - return nebula::cpp2::ErrorCode::SUCCEEDED; - } - - nebula::cpp2::ErrorCode syncAddEdge(const cpp2::NewEdge& edge, bool useToss = true) { - std::vector edges{edge}; - return syncAddMultiEdges(edges, useToss); - } - - folly::SemiFuture> addEdgesAsync( - const std::vector& edges, bool useToss = true) { - auto propNames = makeColNames(edges.back().get_props().size()); - return sClient_->addEdges(spaceId_, 0, 0, edges, propNames, true, nullptr, useToss); - } - - static std::vector makeColNames(size_t n) { - std::vector colNames; - for (auto i = 0U; i < n; ++i) { - colNames.emplace_back(folly::sformat("c{}", i + 1)); - } - return colNames; - } - - std::vector getProps(cpp2::NewEdge edge) { - // nebula::DataSet ds; ===> will crash if not set - std::vector ret; - nebula::Row row; - row.values.emplace_back(edge.get_key().get_src()); - row.values.emplace_back(edge.get_key().get_edge_type()); - row.values.emplace_back(edge.get_key().get_ranking()); - // auto sDst = std::string(reinterpret_cast(&edge.key.dst.getInt()), 8); - row.values.emplace_back(edge.get_key().get_dst()); - - nebula::DataSet ds; - ds.rows.emplace_back(std::move(row)); - - std::vector props; - cpp2::EdgeProp oneProp; - oneProp.set_type(edge.get_key().get_edge_type()); - props.emplace_back(oneProp); - - auto needRetry = false; - int retries = 0; - int retryLimit = 5; - // folly::Future> frpc; - do { - auto frpc = sClient_ - ->getProps(spaceId_, - 0, - 0, - ds, /*DataSet*/ - nullptr, /*vector*/ - &props, /*vector*/ - nullptr /*expressions*/) - .via(executor_.get()); - frpc.wait(); - if (!frpc.valid()) { - LOG(INFO) << "getProps rpc invalid()"; - needRetry = true; - continue; - } - - // StorageRpcResponse - auto& rpcResp = frpc.value(); - LOG(INFO) << "rpcResp.succeeded()=" << rpcResp.succeeded() - << ", responses().size()=" << rpcResp.responses().size(); - if (!rpcResp.failedParts().empty()) { - LOG(INFO) << "rpcResp.failedParts().size()=" << rpcResp.failedParts().size(); - for (auto& p : rpcResp.failedParts()) { - LOG(INFO) << "failedPart: " << p.first - << ", err=" << apache::thrift::util::enumNameSafe(p.second); - if (p.second == nebula::cpp2::ErrorCode::E_LEADER_CHANGED) { - needRetry = true; - continue; - } - } - if (needRetry) { - continue; - } - } - auto resps = frpc.value().responses(); - if (resps.empty()) { - LOG(FATAL) << "getProps() resps.empty())"; - } - cpp2::GetPropResponse& propResp = resps.front(); - cpp2::ResponseCommon result = propResp.get_result(); - std::vector& fparts = result.failed_parts; - if (!fparts.empty()) { - for (cpp2::PartitionResult& res : fparts) { - LOG(INFO) << "part_id: " << res.part_id << ", part leader " << res.get_leader() - << ", code " << apache::thrift::util::enumNameSafe(res.code); - } - LOG(FATAL) << "getProps() !failed_parts.empty())"; - } - auto& dataSet = *propResp.props_ref(); - auto& rows = dataSet.rows; - if (rows.empty()) { - LOG(FATAL) << "getProps() dataSet.rows.empty())"; - } - ret = rows[0].values; - if (ret.empty()) { - LOG(FATAL) << "getProps() ret.empty())"; - } - } while (needRetry && ++retries < retryLimit); - return ret; - } - - folly::SemiFuture> getNeighborsWrapper( - const std::vector& edges, - int64_t limit = std::numeric_limits::max()) { - // para3 - std::vector vertices; - std::set vids; - for (auto& e : edges) { - vids.insert(e.get_key().get_src()); - } - for (auto& vid : vids) { - Row row; - row.emplace_back(vid); - vertices.emplace_back(row); - } - // para 4 - std::vector edgeTypes; - // para 5 - cpp2::EdgeDirection edgeDirection = cpp2::EdgeDirection::BOTH; - // para 6 - std::vector* statProps = nullptr; - // para 7 - std::vector* vertexProps = nullptr; - // para 8 - const std::vector edgeProps; - // para 9 - const std::vector* expressions = nullptr; - // para 10 - bool dedup = false; - // para 11 - bool random = false; - // para 12 - const std::vector orderBy = std::vector(); - - auto colNames = makeColNames(edges.back().get_props().size()); - - return sClient_->getNeighbors(spaceId_, - 0, - 0, - colNames, - vertices, - edgeTypes, - edgeDirection, - statProps, - vertexProps, - &edgeProps, - expressions, - dedup, - random, - orderBy, - limit); - } - - /** - * @brief Get the Nei Props object, - * will unique same src & dst input edges. - */ - std::vector getNeiProps(const std::vector& _edges, - int64_t limit = std::numeric_limits::max()) { - bool retLeaderChange = false; - auto edges(_edges); - std::sort(edges.begin(), edges.end(), [](const auto& a, const auto& b) { - if (a.get_key().get_src() == b.get_key().get_src()) { - return a.get_key().get_dst() < b.get_key().get_dst(); - } - return a.get_key().get_src() < b.get_key().get_src(); - }); - auto last = std::unique(edges.begin(), edges.end(), [](const auto& a, const auto& b) { - return a.get_key().get_src() == b.get_key().get_src() && - a.get_key().get_dst() == b.get_key().get_dst(); - }); - edges.erase(last, edges.end()); - LOG(INFO) << "_edges.size()=" << _edges.size() << ", edges.size()=" << edges.size(); - do { - auto f = getNeighborsWrapper(edges, limit); - f.wait(); - if (!f.valid()) { - LOG(ERROR) << "!f.valid()"; - break; - } - if (f.value().succeeded()) { - return extractMultiRpcResp(f.value()); - } else { - LOG(ERROR) << "!f.value().succeeded()"; - } - auto parts = f.value().failedParts(); - for (auto& part : parts) { - if (part.second == nebula::cpp2::ErrorCode::E_LEADER_CHANGED) { - retLeaderChange = true; - break; - } - } - } while (retLeaderChange); - LOG(ERROR) << "getOutNeighborsProps failed"; - std::vector ret; - return ret; - } - - using RpcResp = StorageRpcResponse; - - static std::vector extractMultiRpcResp(RpcResp& rpc) { - std::vector ret; - LOG(INFO) << "rpc.responses().size()=" << rpc.responses().size(); - for (auto& resp : rpc.responses()) { - auto sub = extractNeiProps(resp); - ret.insert(ret.end(), sub.begin(), sub.end()); - } - return ret; - } - - static std::vector extractNeiProps(cpp2::GetNeighborsResponse& resp) { - std::vector ret; - auto& ds = *resp.vertices_ref(); - LOG(INFO) << "ds.rows.size()=" << ds.rows.size(); - for (auto& row : ds.rows) { - if (row.values.size() < 4) { - continue; - } - LOG(INFO) << "row.values.size()=" << row.values.size(); - ret.emplace_back(row.values[3].toString()); - } - return ret; - } - - int32_t getCountOfNeighbors(const std::vector& edges) { - int32_t ret = 0; - auto neiPropsVec = getNeiProps(edges); - for (auto& prop : neiPropsVec) { - ret += countSquareBrackets(prop); - } - return ret; - } - - static int32_t countSquareBrackets(const std::vector& str) { - int32_t ret = 0; - for (auto& s : str) { - ret += countSquareBrackets(s); - } - return ret; - } - - static int32_t countSquareBrackets(const std::string& str) { - auto ret = TossTestUtils::splitNeiResult(str); - return ret.size(); - } - - static std::vector extractProps(const std::string& props) { - std::vector ret; - if (props.find('[') == std::string::npos) { - return ret; - } - // props string shoule be [[...]], trim the [] - auto props1 = props.substr(2, props.size() - 4); - folly::split("],[", props1, ret, true); - std::sort(ret.begin(), ret.end(), [](const std::string& a, const std::string& b) { - std::vector svec1; - std::vector svec2; - folly::split(",", a, svec1); - folly::split(",", b, svec2); - auto i1 = std::atoi(svec1[4].data()); - auto i2 = std::atoi(svec2[4].data()); - return i1 < i2; - }); - return ret; - } - - std::set extractStrVals(const std::vector& svec) { - auto len = 36; - std::set strVals; - for (auto& e : svec) { - strVals.insert(e.substr(e.size() - len)); - } - // std::sort(strVals.begin(), strVals.end()); - return strVals; - } - - std::vector diffProps(std::vector actual, - std::vector expect) { - std::sort(actual.begin(), actual.end()); - std::sort(expect.begin(), expect.end()); - std::vector diff; - - std::set_difference(actual.begin(), - actual.end(), - expect.begin(), - expect.end(), - std::inserter(diff, diff.begin())); - return diff; - } - - cpp2::NewEdge dupEdge(const cpp2::NewEdge& e) { - cpp2::NewEdge dupEdge{e}; - int n = e.get_props()[0].getInt() / 1024 + 1; - dupEdge.set_props(TossTestUtils::genSingleVal(n)); - return dupEdge; - } - - /** - * @brief gen num edges base from src - * dst shoule begin from [src + 1, src + num + 1) - * @param extraSameKey - * if set, gen another edge, has the same src & dst will always equal - * to src + 1 - */ - std::vector generateMultiEdges(int num, int64_t src, bool extraSameKey = false) { - LOG_IF(FATAL, num <= 0) << "num must > 0"; - num += static_cast(extraSameKey); - auto vals = TossTestUtils::genValues(num); - std::vector edges; - for (int i = 0; i < num; ++i) { - auto dst = extraSameKey ? 1 : src + i + 1; - edges.emplace_back(generateEdge(src, 0, vals[i], dst)); - auto keyPair = makeRawKey(edges.back().get_key()); - LOG(INFO) << "gen key=" << folly::hexlify(keyPair.first) - << ", val=" << edges.back().get_props().back().toString(); - } - return edges; - } - - void insertMutliLocks(const std::vector& edges) { - UNUSED(edges); - // auto lockKey = ; - } - - int32_t getPartId(const std::string& src) { - // auto stPart = mClient_->partId(spaceId_, edgeKey.src.getStr()); - return mClient_->partId(spaceId_, src); - } - - /** - * @brief gen rawkey and partId from EdgeKey - * @return std::pair rawkey vs partId - */ - std::pair makeRawKey(const cpp2::EdgeKey& e) { - auto edgeKey = TossTestUtils::toVidKey(e); - auto partId = getPartId(edgeKey.get_src().getStr()); - - auto rawKey = TransactionUtils::edgeKey(vIdLen_, partId, edgeKey); - return std::make_pair(rawKey, partId); - } - - std::string encodeProps(const cpp2::NewEdge& e) { - auto edgeType = e.get_key().get_edge_type(); - auto pSchema = schemaMan_->getEdgeSchema(spaceId_, std::abs(edgeType)).get(); - LOG_IF(FATAL, !pSchema) << "Space " << spaceId_ << ", Edge " << edgeType << " invalid"; - auto propNames = makeColNames(e.get_props().size()); - return encodeRowVal(pSchema, propNames, e.get_props()); - } - - std::string insertEdge(const cpp2::NewEdge& e) { - std::string rawKey; - int32_t partId = 0; - std::tie(rawKey, partId) = makeRawKey(e.get_key()); - auto encodedProps = encodeProps(e); - putValue(rawKey, encodedProps, partId); - return rawKey; - } - - cpp2::EdgeKey reverseEdgeKey(const cpp2::EdgeKey& input) { - cpp2::EdgeKey ret(input); - std::swap(*ret.src_ref(), *ret.dst_ref()); - ret.set_edge_type(-ret.get_edge_type()); - return ret; - } - - std::string insertReverseEdge(const cpp2::NewEdge& _e) { - cpp2::NewEdge e(_e); - e.set_key(reverseEdgeKey(_e.get_key())); - return insertEdge(e); - } - - /** - * @brief insert a lock according to the given edge e. - * also insert reverse edge - * @return lockKey - */ - std::string insertLock(const cpp2::NewEdge& e, bool insertInEdge) { - if (insertInEdge) { - insertReverseEdge(e); - } - - std::string rawKey; - int32_t lockPartId; - std::tie(rawKey, lockPartId) = makeRawKey(e.get_key()); - - auto lockKey = NebulaKeyUtils::toLockKey(rawKey); - auto lockVal = encodeProps(e); - - putValue(lockKey, lockVal, lockPartId); - - return lockKey; - } - - void putValue(std::string key, std::string val, int32_t partId) { - LOG(INFO) << "put value, partId=" << partId << ", key=" << folly::hexlify(key); - kvstore::BatchHolder bat; - bat.put(std::move(key), std::move(val)); - auto batch = encodeBatchValue(bat.getBatch()); - - auto txnId = 0; - auto sf = interClient_->forwardTransaction(txnId, spaceId_, partId, std::move(batch)); - sf.wait(); - - if (sf.value() != nebula::cpp2::ErrorCode::SUCCEEDED) { - LOG(FATAL) << "forward txn return=" << apache::thrift::util::enumNameSafe(sf.value()); - } - } - - bool keyExist(folly::StringPiece key) { - auto sf = interClient_->getValue(vIdLen_, spaceId_, key); - sf.wait(); - if (!sf.hasValue()) { - LOG(FATAL) << "interClient_->getValue has no value"; - return false; - } - return nebula::ok(sf.value()); - } - - // simple copy of Storage::BaseProcessor::encodeRowVal - std::string encodeRowVal(const meta::NebulaSchemaProvider* schema, - const std::vector& propNames, - const std::vector& props) { - RowWriterV2 rowWrite(schema); - WriteResult wRet; - if (!propNames.empty()) { - for (size_t i = 0; i < propNames.size(); i++) { - wRet = rowWrite.setValue(propNames[i], props[i]); - if (wRet != WriteResult::SUCCEEDED) { - LOG(FATAL) << "Add field faild"; - } - } - } else { - for (size_t i = 0; i < props.size(); i++) { - wRet = rowWrite.setValue(i, props[i]); - if (wRet != WriteResult::SUCCEEDED) { - LOG(FATAL) << "Add field faild"; - } - } - } - wRet = rowWrite.finish(); - if (wRet != WriteResult::SUCCEEDED) { - LOG(FATAL) << "Add field faild"; - } - - return std::move(rowWrite).moveEncodedStr(); - } - - public: - std::shared_ptr executor_; - std::unique_ptr mClient_; - std::unique_ptr sClient_; - std::unique_ptr interClient_; - std::unique_ptr schemaMan_; - - int32_t spaceId_{0}; - int32_t edgeType_{0}; - int32_t vIdLen_{0}; -}; - -} // namespace storage -} // namespace nebula diff --git a/src/storage/test/TossTest.cpp b/src/storage/test/TossTest.cpp deleted file mode 100644 index 40c804ee26e..00000000000 --- a/src/storage/test/TossTest.cpp +++ /dev/null @@ -1,1363 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License, - * attached with Common Clause Condition 1.0, found in the LICENSES directory. - */ - -#include - -#include "TossEnvironment.h" -#include "folly/String.h" -#define LOG_FMT(...) LOG(INFO) << folly::sformat(__VA_ARGS__) - -namespace nebula { -namespace storage { - -using StorageClient = storage::GraphStorageClient; - -constexpr bool kUseToss = true; -constexpr bool kNotUseToss = false; - -static int32_t b_ = 1; -static int32_t gap = 1000; - -enum class TossTestEnum { - NO_TOSS = 1, - // add one edge - ADD_ONES_EDGE = 2, - // add two edges(same local part, same remote part) - TWO_EDGES_CASE_1, - // add two edges(same local part, diff remote part) - TWO_EDGES_CASE_2, - // add two edges(diff local part, same remote part) - TWO_EDGES_CASE_3, - // add two edges(diff local part, diff remote part) - TWO_EDGES_CASE_4, - // add 10 edges(same local part, same remote part) - TEN_EDGES_CASE_1, - // add 10 edges(same local part, diff remote part) - TEN_EDGES_CASE_2, - // add 10 edges(diff local part, same remote part) - TEN_EDGES_CASE_3, - // add 10 edges(diff local part, diff remote part) - TEN_EDGES_CASE_4, -}; - -class TossTest : public ::testing::Test { - public: - TossTest() { - types_.emplace_back(meta::cpp2::PropertyType::INT64); - types_.emplace_back(meta::cpp2::PropertyType::STRING); - colDefs_ = genColDefs(types_); - - boost::uuids::uuid u; - values_.emplace_back(); - values_.back().setInt(1024); - values_.emplace_back(); - values_.back().setStr("defaul:" + boost::uuids::to_string(u)); - - env_ = TossEnvironment::getInstance(kMetaName, kMetaPort); - env_->init(kSpaceName, kPart, kReplica, colDefs_); - } - - void SetUp() override { - if (b_ % gap) { - b_ = (b_ / gap + 1) * gap; - } else { - b_ += gap; - } - } - - void TearDown() override {} - - protected: - TossEnvironment* env_{nullptr}; - - std::vector types_; - std::vector colDefs_; - std::vector values_; - boost::uuids::random_generator gen_; - - // generate ColumnDefs from data types, set default value to NULL. - std::vector genColDefs( - const std::vector& types) { - auto N = types.size(); - auto colNames = TossEnvironment::makeColNames(N); - std::vector ret(N); - for (auto i = 0U; i != N; ++i) { - ret[i].set_name(colNames[i]); - meta::cpp2::ColumnTypeDef tpDef; - tpDef.set_type(types[i]); - ret[i].set_type(tpDef); - ret[i].set_nullable(true); - } - return ret; - } -}; - -// make sure environment ok -TEST_F(TossTest, NO_TOSS) { - int32_t num = 1; - auto values = TossTestUtils::genValues(num); - - int32_t __e = static_cast(TossTestEnum::NO_TOSS) * 100; - int64_t src = env_->getSpaceId() * __e * kPart; - - std::vector edges; - edges.emplace_back(env_->generateEdge(src, 0, values.back(), src + kPart)); - - std::vector startWith{edges[0]}; - auto props = env_->getNeiProps(startWith); - EXPECT_EQ(env_->countSquareBrackets(props), 0); - - LOG(INFO) << "going to add edge:" << edges.back().get_props().back(); - auto code = env_->syncAddMultiEdges(edges, kNotUseToss); - ASSERT_EQ(code, nebula::cpp2::ErrorCode::SUCCEEDED); - - props = env_->getNeiProps(startWith); - - LOG(INFO) << "props.size()=" << props.size(); - for (auto& prop : props) { - LOG(INFO) << "prop: " << prop; - } - EXPECT_EQ(env_->countSquareBrackets(props), 1); -} - -TEST_F(TossTest, ONE_EDGE) { - auto num = 1U; - auto values = TossTestUtils::genValues(num); - - std::vector edges; - edges.emplace_back(env_->generateEdge(b_, 0, values.back(), b_ + kPart)); - - std::vector startWith{edges[0]}; - auto props = env_->getNeiProps(startWith); - EXPECT_EQ(env_->countSquareBrackets(props), 0); - - LOG(INFO) << "going to add edge:" << TossTestUtils::hexEdgeId(edges.back().get_key()); - auto code = env_->syncAddMultiEdges(edges, kUseToss); - ASSERT_EQ(code, nebula::cpp2::ErrorCode::SUCCEEDED); - - props = env_->getNeiProps(startWith); - - LOG(INFO) << "props.size()=" << props.size(); - for (auto& prop : props) { - LOG(INFO) << "prop: " << prop; - } - auto results = TossTestUtils::splitNeiResults(props); - auto cnt = results.size(); - EXPECT_EQ(cnt, num); - for (auto& res : results) { - LOG(INFO) << "res: " << res; - } -} - -TEST_F(TossTest, TWO_EDGES_CASE_1) { - int32_t num = 2; - auto values = TossTestUtils::genValues(num); - - std::vector edges; - edges.emplace_back(env_->generateEdge(b_, 0, values[0], b_ + kPart)); - edges.emplace_back(env_->generateEdge(b_, 0, values[1], b_ + kPart * 2)); - - for (auto& e : edges) { - LOG(INFO) << "going to add edge: " << TossTestUtils::hexEdgeId(e.get_key()); - } - - std::vector startWith{edges[0]}; - auto props = env_->getNeiProps(startWith); - EXPECT_EQ(env_->countSquareBrackets(props), 0); - - auto code = env_->syncAddMultiEdges(edges, kUseToss); - ASSERT_EQ(code, nebula::cpp2::ErrorCode::SUCCEEDED); - - props = env_->getNeiProps(startWith); - EXPECT_EQ(env_->countSquareBrackets(props), 2); - if (env_->countSquareBrackets(props) != 2) { - TossTestUtils::print_svec(props); - } -} - -TEST_F(TossTest, TWO_EDGES_CASE_2) { - int32_t num = 2; - auto values = TossTestUtils::genValues(num); - - int32_t __e = static_cast(TossTestEnum::TWO_EDGES_CASE_2) * 100; - int64_t src = env_->getSpaceId() * __e * kPart; - - std::vector edges; - edges.emplace_back(env_->generateEdge(src, 0, values[0], src + kPart)); - edges.emplace_back(env_->generateEdge(src, 0, values[1], src + kPart + 1)); - - LOG(INFO) << "src=" << src << ", hex=" << TossTestUtils::hexVid(src); - - std::vector first{edges[0]}; - auto props = env_->getNeiProps(first); - EXPECT_EQ(env_->countSquareBrackets(props), 0); - - auto code = env_->syncAddMultiEdges(edges, kUseToss); - LOG_IF(FATAL, code != nebula::cpp2::ErrorCode::SUCCEEDED) - << "fatal code=" << apache::thrift::util::enumNameSafe(code); - - props = env_->getNeiProps(first); - EXPECT_EQ(env_->countSquareBrackets(props), 2); - if (env_->countSquareBrackets(props) != 2) { - TossTestUtils::print_svec(props); - } -} - -TEST_F(TossTest, TWO_EDGES_CASE_3) { - int32_t num = 2; - auto values = TossTestUtils::genValues(num); - - int32_t __e = static_cast(TossTestEnum::TWO_EDGES_CASE_3) * 100; - int64_t src = env_->getSpaceId() * __e * kPart; - int64_t src1st = src; - int64_t src2nd = src1st + 1; - - std::vector edges; - edges.emplace_back(env_->generateEdge(src, 0, values[0], src + kPart)); - edges.emplace_back(env_->generateEdge(src2nd, 0, values[1], src + kPart)); - - LOG(INFO) << "src1st=" << src1st << ", hex=" << TossTestUtils::hexVid(src1st); - LOG(INFO) << "src2nd=" << src2nd << ", hex=" << TossTestUtils::hexVid(src2nd); - - std::vector first{edges[0]}; - std::vector second{edges[1]}; - auto props = env_->getNeiProps(first); - EXPECT_EQ(env_->countSquareBrackets(props), 0); - props = env_->getNeiProps(second); - EXPECT_EQ(env_->countSquareBrackets(props), 0); - - auto code = env_->syncAddMultiEdges(edges, kUseToss); - ASSERT_EQ(code, nebula::cpp2::ErrorCode::SUCCEEDED); - - props = env_->getNeiProps(first); - LOG(INFO) << "props.size()=" << props.size(); - for (auto& prop : props) { - LOG(INFO) << prop; - } - EXPECT_EQ(env_->countSquareBrackets(props), 1); - - props = env_->getNeiProps(second); - LOG(INFO) << "props.size()=" << props.size(); - for (auto& prop : props) { - LOG(INFO) << prop; - } - EXPECT_EQ(env_->countSquareBrackets(props), 1); - - props = env_->getNeiProps(edges); - LOG(INFO) << "props.size()=" << props.size(); - for (auto& prop : props) { - LOG(INFO) << prop; - } - EXPECT_EQ(env_->countSquareBrackets(props), 2); -} - -TEST_F(TossTest, TWO_EDGES_CASE_4) { - int32_t num = 2; - auto values = TossTestUtils::genValues(num); - - int32_t __e = static_cast(TossTestEnum::TWO_EDGES_CASE_4) * 100; - int64_t src = env_->getSpaceId() * __e * kPart; - - std::vector edges; - edges.emplace_back(env_->generateEdge(src, 0, values[0], src + kPart + 1)); - edges.emplace_back(env_->generateEdge(src + 1, 0, values[1], src + kPart)); - - std::vector startWith{edges[0]}; - auto props = env_->getNeiProps(startWith); - EXPECT_EQ(env_->countSquareBrackets(props), 0); - - auto code = env_->syncAddMultiEdges(edges, kUseToss); - LOG_IF(FATAL, code != nebula::cpp2::ErrorCode::SUCCEEDED) - << "fatal code=" << apache::thrift::util::enumNameSafe(code); - - props = env_->getNeiProps(startWith); - for (auto& prop : props) { - LOG(INFO) << prop; - } - EXPECT_EQ(env_->countSquareBrackets(props), 1); -} - -TEST_F(TossTest, TEN_EDGES_CASE_1) { - int32_t num = 10; - auto values = TossTestUtils::genValues(num); - - int32_t __e = static_cast(TossTestEnum::TEN_EDGES_CASE_1) * 100; - int64_t src = env_->getSpaceId() * __e * kPart; - - std::vector edges; - for (int32_t i = 0; i != num; ++i) { - edges.emplace_back(env_->generateEdge(src, 0, values[i], src + kPart + i + 1)); - } - - std::vector first{edges[0]}; - // auto props = env_->getNeiProps(startWith); - // EXPECT_EQ(env_->countSquareBrackets(props), 0); - - auto code = env_->syncAddMultiEdges(edges, kUseToss); - LOG_IF(FATAL, code != nebula::cpp2::ErrorCode::SUCCEEDED) - << "fatal code=" << apache::thrift::util::enumNameSafe(code); - - auto props = env_->getNeiProps(first); - // LOG(INFO) << "props: " << props; - EXPECT_EQ(env_->countSquareBrackets(props), num); -} - -TEST_F(TossTest, TEN_EDGES_CASE_2) { - auto num = 100U; - auto values = TossTestUtils::genValues(num); - - int32_t __e = static_cast(TossTestEnum::TEN_EDGES_CASE_2) * 100; - int64_t src = env_->getSpaceId() * __e * kPart; - - std::vector edges; - for (auto i = 0U; i != num; ++i) { - edges.emplace_back(env_->generateEdge(src, 0, values[i], src + kPart + i + 1)); - } - - std::vector first{edges[0]}; - - auto code = env_->syncAddMultiEdges(edges, kUseToss); - LOG_IF(FATAL, code != nebula::cpp2::ErrorCode::SUCCEEDED) - << "fatal code=" << apache::thrift::util::enumNameSafe(code); - - auto props = env_->getNeiProps(first); - // auto actual = env_->countSquareBrackets(props); - auto svec = TossTestUtils::splitNeiResults(props); - auto actual = svec.size(); - EXPECT_EQ(actual, num); - if (actual != num) { - for (auto& s : svec) { - LOG(INFO) << "s" << s; - } - } -} - -TEST_F(TossTest, lock_test_0) { - auto num = 1; - std::vector edges = env_->generateMultiEdges(num, b_); - - auto edge = TossTestUtils::toVertexIdEdge(edges[0]); - - auto vIdLen = 8; - auto partId = 5; // just a random number - auto rawKey = TransactionUtils::edgeKey(vIdLen, partId, edge.get_key()); - auto lockKey = NebulaKeyUtils::toLockKey(rawKey); - - ASSERT_TRUE(NebulaKeyUtils::isLock(vIdLen, lockKey)); -} - -/** - * @brief good lock - */ -TEST_F(TossTest, lock_test_1) { - auto num = 1; - LOG(INFO) << "b_=" << b_; - std::vector edges = env_->generateMultiEdges(num, b_); - - auto lockKey = env_->insertLock(edges[0], true); - LOG(INFO) << "lock_test_1 lock hexlify = " << folly::hexlify(lockKey); - - auto props = env_->getNeiProps(edges); - auto svec = TossTestUtils::splitNeiResults(props); - - ASSERT_FALSE(env_->keyExist(lockKey)); - - auto rawKey = NebulaKeyUtils::toEdgeKey(lockKey); - ASSERT_TRUE(env_->keyExist(rawKey)); - - auto reversedEdgeKey = env_->reverseEdgeKey(edges[0].get_key()); - auto reversedRawKey = env_->makeRawKey(reversedEdgeKey); - ASSERT_TRUE(env_->keyExist(reversedRawKey.first)); - - ASSERT_EQ(svec.size(), num); -} - -/** - * @brief good lock + edge - */ -TEST_F(TossTest, lock_test_2) { - LOG(INFO) << "b_=" << b_; - auto num = 1; - std::vector edges = env_->generateMultiEdges(num, b_, true); - - auto lockKey = env_->insertLock(edges[0], true); - - LOG(INFO) << "lock_test_1 lock hexlify = " << folly::hexlify(lockKey); - auto rawKey = env_->insertEdge(edges[1]); - - auto props = env_->getNeiProps(edges); - auto svec = TossTestUtils::splitNeiResults(props); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey)); - - // step 2nd: edge key exist - // auto rawKey = NebulaKeyUtils::toEdgeKey(lockKey); - ASSERT_TRUE(env_->keyExist(rawKey)); - - // step 3rd: reverse edge key exist - auto reversedEdgeKey = env_->reverseEdgeKey(edges[0].get_key()); - auto reversedRawKey = env_->makeRawKey(reversedEdgeKey); - ASSERT_TRUE(env_->keyExist(reversedRawKey.first)); - - // step 4th: the get neighbors result is from lock - auto lockStrVal = edges[0].get_props()[1].toString(); - ASSERT_TRUE(svec.back().size() > lockStrVal.size()); - auto neighborStrVal = svec.back().substr(svec.back().size() - lockStrVal.size()); - ASSERT_EQ(lockStrVal, neighborStrVal); - - ASSERT_EQ(svec.size(), num); -} - -/** - * @brief bad lock - */ -TEST_F(TossTest, lock_test_3) { - LOG(INFO) << "b_=" << b_; - auto num = 1; - std::vector edges = env_->generateMultiEdges(num, b_, true); - - auto lockKey = env_->insertLock(edges[0], false); - ASSERT_TRUE(env_->keyExist(lockKey)); - LOG(INFO) << "lock_test_1 lock hexlify = " << folly::hexlify(lockKey); - - auto props = env_->getNeiProps(edges); - auto svec = TossTestUtils::splitNeiResults(props); - EXPECT_EQ(svec.size(), 0); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey)); -} - -/** - * @brief bad lock + edge - */ -TEST_F(TossTest, lock_test_4) { - LOG(INFO) << "b_=" << b_; - auto num = 1; - std::vector edges = env_->generateMultiEdges(num, b_, true); - ASSERT_EQ(edges.size(), 2); - - auto lockKey = env_->insertLock(edges[0], false); - ASSERT_TRUE(env_->keyExist(lockKey)); - LOG(INFO) << "lock_test_1 lock hexlify = " << folly::hexlify(lockKey); - - auto rawKey = env_->insertEdge(edges[1]); - - auto props = env_->getNeiProps(edges); - auto svec = TossTestUtils::splitNeiResults(props); - ASSERT_EQ(svec.size(), num); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey)); - - // step 2nd: edge key exist - // auto rawKey = NebulaKeyUtils::toEdgeKey(lockKey); - ASSERT_TRUE(env_->keyExist(rawKey)); - - // step 3rd: reverse edge key exist - auto reversedEdgeKey = env_->reverseEdgeKey(edges[0].get_key()); - auto reversedRawKey = env_->makeRawKey(reversedEdgeKey); - ASSERT_FALSE(env_->keyExist(reversedRawKey.first)); - - // step 4th: the get neighbors result is from lock - auto edgeStrVal = edges[1].get_props()[1].toString(); - ASSERT_TRUE(svec.back().size() > edgeStrVal.size()); - auto neighborStrVal = svec.back().substr(svec.back().size() - edgeStrVal.size()); - ASSERT_EQ(edgeStrVal, neighborStrVal); -} - -/** - * @brief neighbor edge + edge - * check normal data(without lock) can be read without err - */ -TEST_F(TossTest, neighbors_test_1) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto rawKey1 = env_->insertEdge(edges[0]); - auto rawKey2 = env_->insertEdge(edges[1]); - - auto props = env_->getNeiProps(edges); - auto svec = TossTestUtils::splitNeiResults(props); - ASSERT_TRUE(TossTestUtils::compareSize(svec, num)); -} - -/** - * @brief neighbor edge + good lock - */ -TEST_F(TossTest, neighbors_test_2) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto rawKey1 = env_->insertEdge(edges[0]); - - auto lockKey = env_->insertLock(edges[1], true); - LOG(INFO) << "lock_test_1 lock hexlify = " << folly::hexlify(lockKey); - - auto props = env_->getNeiProps(edges); - auto svec = TossTestUtils::splitNeiResults(props); - ASSERT_TRUE(TossTestUtils::compareSize(svec, num)); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey)); - - // step 2nd: edge key exist - auto rawKey = NebulaKeyUtils::toEdgeKey(lockKey); - ASSERT_TRUE(env_->keyExist(rawKey)); - - // step 3rd: reverse edge key exist - auto reversedEdgeKey = env_->reverseEdgeKey(edges[1].get_key()); - auto reversedRawKey = env_->makeRawKey(reversedEdgeKey); - ASSERT_TRUE(env_->keyExist(reversedRawKey.first)); -} - -/** - * @brief neighbor edge + good lock + edge - */ -TEST_F(TossTest, neighbors_test_3) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - auto lockEdge = env_->dupEdge(edges[1]); - - auto rawKey1 = env_->insertEdge(edges[0]); - auto rawKey2 = env_->insertEdge(edges[1]); - auto lockKey = env_->insertLock(lockEdge, true); - LOG(INFO) << "lock_test_1 lock hexlify = " << folly::hexlify(lockKey); - - auto props = env_->getNeiProps(edges); - auto svec = TossTestUtils::splitNeiResults(props); - ASSERT_TRUE(TossTestUtils::compareSize(svec, num)); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey)); - - // step 2nd: edge key exist - ASSERT_TRUE(env_->keyExist(rawKey1)); - ASSERT_TRUE(env_->keyExist(rawKey2)); - - // step 3rd: reverse edge key exist - auto reversedEdgeKey = env_->reverseEdgeKey(edges[1].get_key()); - auto reversedRawKey = env_->makeRawKey(reversedEdgeKey); - ASSERT_TRUE(env_->keyExist(reversedRawKey.first)); - - // TossTestUtils::print_svec(svec); - LOG(INFO) << "edges[0]=" << edges[0].get_props()[1].toString(); - LOG(INFO) << "edges[1]=" << edges[1].get_props()[1].toString(); - LOG(INFO) << "lockEdge=" << lockEdge.get_props()[1].toString(); - - LOG(INFO) << "lockEdge.size()=" << lockEdge.get_props()[1].toString().size(); - - // step 4th: the get neighbors result is from lock - auto strProps = env_->extractStrVals(svec); - decltype(strProps) expect{edges[0].get_props()[1].toString(), lockEdge.get_props()[1].toString()}; - ASSERT_EQ(strProps, expect); -} - -/** - * @brief neighbor edge + bad lock - */ -TEST_F(TossTest, neighbors_test_4) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto rawKey1 = env_->insertEdge(edges[0]); - auto lockEdge = env_->dupEdge(edges[1]); - - auto lockKey = env_->insertLock(lockEdge, false); - LOG(INFO) << "lock_test_1 lock hexlify = " << folly::hexlify(lockKey); - - auto props = env_->getNeiProps(edges); - auto svec = TossTestUtils::splitNeiResults(props); - ASSERT_TRUE(TossTestUtils::compareSize(svec, num - 1)); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey)); - - // step 2nd: edge key exist - ASSERT_TRUE(env_->keyExist(rawKey1)); - - // step 3rd: reverse edge key exist - auto reversedEdgeKey = env_->reverseEdgeKey(lockEdge.get_key()); - auto reversedRawKey = env_->makeRawKey(reversedEdgeKey); - ASSERT_FALSE(env_->keyExist(reversedRawKey.first)); - - // TossTestUtils::print_svec(svec); - LOG(INFO) << "edges[0]=" << edges[0].get_props()[1].toString(); - - // step 4th: the get neighbors result is from lock - auto strProps = env_->extractStrVals(svec); - decltype(strProps) expect{edges[0].get_props()[1].toString()}; - ASSERT_EQ(strProps, expect); -} - -/** - * @brief neighbor edge + bad lock + edge - */ -TEST_F(TossTest, neighbors_test_5) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto lock1 = env_->dupEdge(edges[1]); - - auto lockKey1 = env_->insertLock(lock1, false); - LOG(INFO) << "lock_test_1 lock hexlify = " << folly::hexlify(lockKey1); - - auto rawKey0 = env_->insertEdge(edges[0]); - auto rawKey1 = env_->insertEdge(edges[1]); - - auto props = env_->getNeiProps(edges); - auto svec = TossTestUtils::splitNeiResults(props); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey1)); - - // step 2nd: edge key exist - ASSERT_TRUE(env_->keyExist(rawKey1)); - - // step 3rd: reverse edge key exist - auto reversedEdgeKey = env_->reverseEdgeKey(lock1.get_key()); - auto reversedRawKey = env_->makeRawKey(reversedEdgeKey); - ASSERT_FALSE(env_->keyExist(reversedRawKey.first)); - - LOG(INFO) << "edges[0]=" << edges[0].get_props()[1].toString(); - - // step 4th: the get neighbors result is from lock - auto actualProps = env_->extractStrVals(svec); - decltype(actualProps) expect{edges[0].get_props()[1].toString(), - edges[1].get_props()[1].toString()}; - ASSERT_EQ(actualProps, expect); - - ASSERT_TRUE(TossTestUtils::compareSize(svec, num)); -} - -/** - * @brief neighbor good lock + edge - */ -TEST_F(TossTest, neighbors_test_6) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto lockEdge = env_->dupEdge(edges[0]); - - auto lockKey = env_->insertLock(lockEdge, true); - LOG(INFO) << "lock_test_1 lock hexlify = " << folly::hexlify(lockKey); - - // auto rawKey0 = env_->insertEdge(edges[0]); - auto rawKey1 = env_->insertEdge(edges[1]); - - auto props = env_->getNeiProps(edges); - auto svec = TossTestUtils::splitNeiResults(props); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey)); - - // step 2nd: edge key exist - ASSERT_TRUE(env_->keyExist(rawKey1)); - - // step 3rd: reverse edge key exist - auto reversedEdgeKey = env_->reverseEdgeKey(lockEdge.get_key()); - auto reversedRawKey = env_->makeRawKey(reversedEdgeKey); - ASSERT_TRUE(env_->keyExist(reversedRawKey.first)); - - // step 4th: the get neighbors result is from lock - auto actualProps = env_->extractStrVals(svec); - decltype(actualProps) expect{lockEdge.get_props()[1].toString(), - edges[1].get_props()[1].toString()}; - ASSERT_EQ(actualProps, expect); - - ASSERT_TRUE(TossTestUtils::compareSize(svec, num)); -} - -/** - * @brief neighbor good lock + good lock - */ -TEST_F(TossTest, neighbors_test_7) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto lock0 = env_->dupEdge(edges[0]); - auto lock1 = env_->dupEdge(edges[1]); - - auto lockKey0 = env_->insertLock(lock0, true); - auto lockKey1 = env_->insertLock(lock1, true); - LOG(INFO) << "lockKey0 hexlify = " << folly::hexlify(lockKey0); - LOG(INFO) << "lockKey1 hexlify = " << folly::hexlify(lockKey1); - - // auto rawKey0 = env_->insertEdge(edges[0]); - // auto rawKey1 = env_->insertEdge(edges[1]); - auto rawKey0 = env_->makeRawKey(edges[0].get_key()); - auto rawKey1 = env_->makeRawKey(edges[1].get_key()); - - auto props = env_->getNeiProps(edges); - auto svec = TossTestUtils::splitNeiResults(props); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey0)); - ASSERT_FALSE(env_->keyExist(lockKey1)); - - // step 2nd: edge key exist - ASSERT_TRUE(env_->keyExist(rawKey0.first)); - ASSERT_TRUE(env_->keyExist(rawKey1.first)); - - // step 3rd: reverse edge key exist - auto rRawKey0 = env_->makeRawKey(env_->reverseEdgeKey(lock0.get_key())); - auto rRawKey1 = env_->makeRawKey(env_->reverseEdgeKey(lock1.get_key())); - - ASSERT_TRUE(env_->keyExist(rRawKey0.first)); - ASSERT_TRUE(env_->keyExist(rRawKey1.first)); - - // step 4th: the get neighbors result is from lock - auto actualProps = env_->extractStrVals(svec); - decltype(actualProps) expect{lock0.get_props()[1].toString(), lock1.get_props()[1].toString()}; - ASSERT_EQ(actualProps, expect); - - ASSERT_TRUE(TossTestUtils::compareSize(svec, num)); -} - -/** - * @brief neighbor good lock + good lock + edge - */ -TEST_F(TossTest, neighbors_test_8) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto lock0 = env_->dupEdge(edges[0]); - auto lock1 = env_->dupEdge(edges[1]); - - auto lockKey0 = env_->insertLock(lock0, true); - auto lockKey1 = env_->insertLock(lock1, true); - LOG(INFO) << "lockKey0 hexlify = " << folly::hexlify(lockKey0); - LOG(INFO) << "lockKey1 hexlify = " << folly::hexlify(lockKey1); - - auto rawKey0 = env_->makeRawKey(edges[0].get_key()).first; - // auto rawKey1 = env_->makeRawKey(edges[1].key).first; - - // auto rawKey0 = env_->insertEdge(edges[0]); - auto rawKey1 = env_->insertEdge(edges[1]); - - auto props = env_->getNeiProps(edges); - auto svec = TossTestUtils::splitNeiResults(props); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey0)); - ASSERT_FALSE(env_->keyExist(lockKey1)); - - // step 2nd: edge key exist - ASSERT_TRUE(env_->keyExist(rawKey0)); - ASSERT_TRUE(env_->keyExist(rawKey1)); - - // step 3rd: reverse edge key exist - auto rRawKey0 = env_->makeRawKey(env_->reverseEdgeKey(lock0.get_key())); - auto rRawKey1 = env_->makeRawKey(env_->reverseEdgeKey(lock1.get_key())); - - ASSERT_TRUE(env_->keyExist(rRawKey0.first)); - ASSERT_TRUE(env_->keyExist(rRawKey1.first)); - - // step 4th: the get neighbors result is from lock - auto actualProps = env_->extractStrVals(svec); - decltype(actualProps) expect{lock0.get_props()[1].toString(), lock1.get_props()[1].toString()}; - ASSERT_EQ(actualProps, expect); - - ASSERT_TRUE(TossTestUtils::compareSize(svec, num)); -} - -/** - * @brief neighbor good lock + bad lock - */ -TEST_F(TossTest, neighbors_test_9) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto lock0 = env_->dupEdge(edges[0]); - auto lock1 = env_->dupEdge(edges[1]); - - auto lockKey0 = env_->insertLock(lock0, true); - auto lockKey1 = env_->insertLock(lock1, false); - LOG(INFO) << "lockKey0 hexlify = " << folly::hexlify(lockKey0); - LOG(INFO) << "lockKey1 hexlify = " << folly::hexlify(lockKey1); - - auto rawKey0 = env_->makeRawKey(edges[0].get_key()).first; - auto rawKey1 = env_->makeRawKey(edges[1].get_key()).first; - - // auto rawKey0 = env_->insertEdge(edges[0]); - // auto rawKey1 = env_->insertEdge(edges[1]); - - auto props = env_->getNeiProps(edges); - auto svec = TossTestUtils::splitNeiResults(props); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey0)); - ASSERT_FALSE(env_->keyExist(lockKey1)); - - // step 2nd: edge key exist - ASSERT_TRUE(env_->keyExist(rawKey0)); - ASSERT_FALSE(env_->keyExist(rawKey1)); - - // step 3rd: reverse edge key exist - auto rRawKey0 = env_->makeRawKey(env_->reverseEdgeKey(lock0.get_key())); - auto rRawKey1 = env_->makeRawKey(env_->reverseEdgeKey(lock1.get_key())); - - ASSERT_TRUE(env_->keyExist(rRawKey0.first)); - ASSERT_FALSE(env_->keyExist(rRawKey1.first)); - - // step 4th: the get neighbors result is from lock - auto actualProps = env_->extractStrVals(svec); - decltype(actualProps) expect{lock0.get_props()[1].toString()}; - ASSERT_EQ(actualProps, expect); - - // ASSERT_TRUE(TossTestUtils::compareSize(svec, num)); -} - -/** - * @brief neighbor good lock + bad lock + edge - */ -TEST_F(TossTest, neighbors_test_10) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto lock0 = env_->dupEdge(edges[0]); - auto lock1 = env_->dupEdge(edges[1]); - - auto lockKey0 = env_->insertLock(lock0, true); - auto lockKey1 = env_->insertLock(lock1, false); - LOG(INFO) << "lockKey0 hexlify = " << folly::hexlify(lockKey0); - LOG(INFO) << "lockKey1 hexlify = " << folly::hexlify(lockKey1); - - auto rawKey0 = env_->makeRawKey(edges[0].get_key()).first; - auto rawKey1 = env_->makeRawKey(edges[1].get_key()).first; - - // auto rawKey0 = env_->insertEdge(edges[0]); - env_->syncAddEdge(edges[1]); - - auto props = env_->getNeiProps(edges); - auto svec = TossTestUtils::splitNeiResults(props); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey0)); - ASSERT_FALSE(env_->keyExist(lockKey1)); - - // step 2nd: edge key exist - ASSERT_TRUE(env_->keyExist(rawKey0)); - ASSERT_TRUE(env_->keyExist(rawKey1)); - - // step 3rd: reverse edge key exist - auto rRawKey0 = env_->makeRawKey(env_->reverseEdgeKey(lock0.get_key())); - auto rRawKey1 = env_->makeRawKey(env_->reverseEdgeKey(lock1.get_key())); - - ASSERT_TRUE(env_->keyExist(rRawKey0.first)); - ASSERT_TRUE(env_->keyExist(rRawKey1.first)); - - // step 4th: the get neighbors result is from lock - auto actualProps = env_->extractStrVals(svec); - decltype(actualProps) expect{lock0.get_props()[1].toString(), edges[1].get_props()[1].toString()}; - ASSERT_EQ(actualProps, expect); - - ASSERT_TRUE(TossTestUtils::compareSize(svec, num)); -} - -/** - * @brief neighbor bad lock + edge - */ -TEST_F(TossTest, neighbors_test_11) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto lock0 = env_->dupEdge(edges[0]); - auto lock1 = env_->dupEdge(edges[1]); - - auto lockKey0 = env_->insertLock(lock0, false); - // auto lockKey1 = env_->insertLock(lock1, false); - LOG(INFO) << "lockKey0 hexlify = " << folly::hexlify(lockKey0); - // LOG(INFO) << "lockKey1 hexlify = " << folly::hexlify(lockKey1); - - auto rawKey0 = env_->makeRawKey(edges[0].get_key()).first; - auto rawKey1 = env_->makeRawKey(edges[1].get_key()).first; - - env_->syncAddEdge(edges[1]); - - auto props = env_->getNeiProps(edges); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey0)); - // ASSERT_FALSE(env_->keyExist(lockKey1)); - - // step 2nd: edge key exist - ASSERT_FALSE(env_->keyExist(rawKey0)); - ASSERT_TRUE(env_->keyExist(rawKey1)); - - // step 3rd: reverse edge key exist - auto rRawKey0 = env_->makeRawKey(env_->reverseEdgeKey(lock0.get_key())); - auto rRawKey1 = env_->makeRawKey(env_->reverseEdgeKey(lock1.get_key())); - - ASSERT_FALSE(env_->keyExist(rRawKey0.first)); - ASSERT_TRUE(env_->keyExist(rRawKey1.first)); - - // step 4th: the get neighbors result is from lock - auto svec = TossTestUtils::splitNeiResults(props); - auto actualProps = env_->extractStrVals(svec); - decltype(actualProps) expect{edges[1].get_props()[1].toString()}; - ASSERT_EQ(actualProps, expect); - - ASSERT_TRUE(TossTestUtils::compareSize(svec, num - 1)); -} - -/** - * @brief neighbor bad lock + good lock - */ -TEST_F(TossTest, neighbors_test_12) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto lock0 = env_->dupEdge(edges[0]); - auto lock1 = env_->dupEdge(edges[1]); - - auto rawKey0 = env_->makeRawKey(edges[0].get_key()).first; - auto rawKey1 = env_->makeRawKey(edges[1].get_key()).first; - - // env_->syncAddEdge(edges[1]); - auto lockKey0 = env_->insertLock(lock0, false); - auto lockKey1 = env_->insertLock(lock1, true); - LOG(INFO) << "lockKey0 hexlify = " << folly::hexlify(lockKey0); - LOG(INFO) << "lockKey1 hexlify = " << folly::hexlify(lockKey1); - - auto props = env_->getNeiProps(edges); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey0)); - ASSERT_FALSE(env_->keyExist(lockKey1)); - - // step 2nd: edge key exist - ASSERT_FALSE(env_->keyExist(rawKey0)); - ASSERT_TRUE(env_->keyExist(rawKey1)); - - // step 3rd: reverse edge key exist - auto rRawKey0 = env_->makeRawKey(env_->reverseEdgeKey(lock0.get_key())); - auto rRawKey1 = env_->makeRawKey(env_->reverseEdgeKey(lock1.get_key())); - - ASSERT_FALSE(env_->keyExist(rRawKey0.first)); - ASSERT_TRUE(env_->keyExist(rRawKey1.first)); - - // step 4th: the get neighbors result is from lock - auto svec = TossTestUtils::splitNeiResults(props); - auto actualProps = env_->extractStrVals(svec); - decltype(actualProps) expect{lock1.get_props()[1].toString()}; - ASSERT_EQ(actualProps, expect); - - ASSERT_TRUE(TossTestUtils::compareSize(svec, num - 1)); -} - -/** - * @brief neighbor bad lock + good lock + edge - */ -TEST_F(TossTest, neighbors_test_13) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto lock0 = env_->dupEdge(edges[0]); - auto lock1 = env_->dupEdge(edges[1]); - - auto rawKey0 = env_->makeRawKey(edges[0].get_key()).first; - auto rawKey1 = env_->makeRawKey(edges[1].get_key()).first; - - env_->syncAddEdge(edges[1]); - auto lockKey0 = env_->insertLock(lock0, false); - auto lockKey1 = env_->insertLock(lock1, true); - LOG(INFO) << "lockKey0 hexlify = " << folly::hexlify(lockKey0); - LOG(INFO) << "lockKey1 hexlify = " << folly::hexlify(lockKey1); - - auto props = env_->getNeiProps(edges); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey0)); - ASSERT_FALSE(env_->keyExist(lockKey1)); - - // step 2nd: edge key exist - ASSERT_FALSE(env_->keyExist(rawKey0)); - ASSERT_TRUE(env_->keyExist(rawKey1)); - - // step 3rd: reverse edge key exist - auto rRawKey0 = env_->makeRawKey(env_->reverseEdgeKey(lock0.get_key())); - auto rRawKey1 = env_->makeRawKey(env_->reverseEdgeKey(lock1.get_key())); - - ASSERT_FALSE(env_->keyExist(rRawKey0.first)); - ASSERT_TRUE(env_->keyExist(rRawKey1.first)); - - // step 4th: the get neighbors result is from lock - auto svec = TossTestUtils::splitNeiResults(props); - auto actualProps = env_->extractStrVals(svec); - decltype(actualProps) expect{lock1.get_props()[1].toString()}; - ASSERT_EQ(actualProps, expect); - - ASSERT_TRUE(TossTestUtils::compareSize(svec, num - 1)); -} - -/** - * @brief neighbor bad lock + bad lock - */ -TEST_F(TossTest, neighbors_test_14) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto lock0 = env_->dupEdge(edges[0]); - auto lock1 = env_->dupEdge(edges[1]); - - auto rawKey0 = env_->makeRawKey(edges[0].get_key()).first; - auto rawKey1 = env_->makeRawKey(edges[1].get_key()).first; - - // env_->syncAddEdge(edges[1]); - - auto lockKey0 = env_->insertLock(lock0, false); - auto lockKey1 = env_->insertLock(lock1, false); - LOG(INFO) << "lockKey0 hexlify = " << folly::hexlify(lockKey0); - LOG(INFO) << "lockKey1 hexlify = " << folly::hexlify(lockKey1); - - auto props = env_->getNeiProps(edges); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey0)); - ASSERT_FALSE(env_->keyExist(lockKey1)); - - // step 2nd: edge key exist - ASSERT_FALSE(env_->keyExist(rawKey0)); - ASSERT_FALSE(env_->keyExist(rawKey1)); - - // step 3rd: reverse edge key exist - auto rRawKey0 = env_->makeRawKey(env_->reverseEdgeKey(lock0.get_key())); - auto rRawKey1 = env_->makeRawKey(env_->reverseEdgeKey(lock1.get_key())); - - ASSERT_FALSE(env_->keyExist(rRawKey0.first)); - ASSERT_FALSE(env_->keyExist(rRawKey1.first)); - - // step 4th: the get neighbors result is from lock - auto svec = TossTestUtils::splitNeiResults(props); - auto actualProps = env_->extractStrVals(svec); - decltype(actualProps) expect{}; - ASSERT_EQ(actualProps, expect); - - ASSERT_TRUE(TossTestUtils::compareSize(svec, 0)); -} - -/** - * @brief neighbor bad lock + bad lock + edge - */ -TEST_F(TossTest, neighbors_test_15) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto lock0 = env_->dupEdge(edges[0]); - auto lock1 = env_->dupEdge(edges[1]); - - auto rawKey0 = env_->makeRawKey(edges[0].get_key()).first; - auto rawKey1 = env_->makeRawKey(edges[1].get_key()).first; - - env_->syncAddEdge(edges[1]); - - auto lockKey0 = env_->insertLock(lock0, false); - auto lockKey1 = env_->insertLock(lock1, false); - LOG(INFO) << "lockKey0 hexlify = " << folly::hexlify(lockKey0); - LOG(INFO) << "lockKey1 hexlify = " << folly::hexlify(lockKey1); - - auto props = env_->getNeiProps(edges); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey0)); - ASSERT_FALSE(env_->keyExist(lockKey1)); - - // step 2nd: edge key exist - ASSERT_FALSE(env_->keyExist(rawKey0)); - ASSERT_TRUE(env_->keyExist(rawKey1)); - - // step 3rd: reverse edge key exist - auto rRawKey0 = env_->makeRawKey(env_->reverseEdgeKey(lock0.get_key())); - auto rRawKey1 = env_->makeRawKey(env_->reverseEdgeKey(lock1.get_key())); - - ASSERT_FALSE(env_->keyExist(rRawKey0.first)); - ASSERT_TRUE(env_->keyExist(rRawKey1.first)); - - // step 4th: the get neighbors result is from lock - auto svec = TossTestUtils::splitNeiResults(props); - auto actualProps = env_->extractStrVals(svec); - decltype(actualProps) expect{edges[1].get_props()[1].toString()}; - ASSERT_EQ(actualProps, expect); - - ASSERT_TRUE(TossTestUtils::compareSize(svec, 1)); -} - -/** - * @brief neighbor good lock + neighbor edge + good lock + edge - */ -TEST_F(TossTest, neighbors_test_16) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto lock0 = env_->dupEdge(edges[0]); - auto lock1 = env_->dupEdge(edges[1]); - - auto rawKey0 = env_->makeRawKey(edges[0].get_key()).first; - auto rawKey1 = env_->makeRawKey(edges[1].get_key()).first; - - env_->syncAddEdge(edges[1]); - - auto lockKey0 = env_->insertLock(lock0, true); - auto lockKey1 = env_->insertLock(lock1, true); - LOG(INFO) << "lockKey0 hexlify = " << folly::hexlify(lockKey0); - LOG(INFO) << "lockKey1 hexlify = " << folly::hexlify(lockKey1); - - auto props = env_->getNeiProps(edges); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey0)); - ASSERT_FALSE(env_->keyExist(lockKey1)); - - // step 2nd: edge key exist - ASSERT_TRUE(env_->keyExist(rawKey0)); - ASSERT_TRUE(env_->keyExist(rawKey1)); - - // step 3rd: reverse edge key exist - auto rRawKey0 = env_->makeRawKey(env_->reverseEdgeKey(lock0.get_key())); - auto rRawKey1 = env_->makeRawKey(env_->reverseEdgeKey(lock1.get_key())); - - ASSERT_TRUE(env_->keyExist(rRawKey0.first)); - ASSERT_TRUE(env_->keyExist(rRawKey1.first)); - - // step 4th: the get neighbors result is from lock - auto svec = TossTestUtils::splitNeiResults(props); - auto actualProps = env_->extractStrVals(svec); - decltype(actualProps) expect{lock0.get_props()[1].toString(), lock1.get_props()[1].toString()}; - ASSERT_EQ(actualProps, expect); - - ASSERT_TRUE(TossTestUtils::compareSize(svec, 2)); -} - -/** - * @brief neighbor good lock + neighbor edge + bad lock + edge - */ -TEST_F(TossTest, neighbors_test_17) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto lock0 = env_->dupEdge(edges[0]); - auto lock1 = env_->dupEdge(edges[1]); - - auto rawKey0 = env_->makeRawKey(edges[0].get_key()).first; - auto rawKey1 = env_->makeRawKey(edges[1].get_key()).first; - - env_->syncAddEdge(edges[1]); - - auto lockKey0 = env_->insertLock(lock0, true); - auto lockKey1 = env_->insertLock(lock1, false); - LOG(INFO) << "lockKey0 hexlify = " << folly::hexlify(lockKey0); - LOG(INFO) << "lockKey1 hexlify = " << folly::hexlify(lockKey1); - - auto props = env_->getNeiProps(edges); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey0)); - ASSERT_FALSE(env_->keyExist(lockKey1)); - - // step 2nd: edge key exist - ASSERT_TRUE(env_->keyExist(rawKey0)); - ASSERT_TRUE(env_->keyExist(rawKey1)); - - // step 3rd: reverse edge key exist - auto rRawKey0 = env_->makeRawKey(env_->reverseEdgeKey(lock0.get_key())); - auto rRawKey1 = env_->makeRawKey(env_->reverseEdgeKey(lock1.get_key())); - - ASSERT_TRUE(env_->keyExist(rRawKey0.first)); - ASSERT_TRUE(env_->keyExist(rRawKey1.first)); - - // step 4th: the get neighbors result is from lock - auto svec = TossTestUtils::splitNeiResults(props); - auto actualProps = env_->extractStrVals(svec); - decltype(actualProps) expect{lock0.get_props()[1].toString(), edges[1].get_props()[1].toString()}; - ASSERT_EQ(actualProps, expect); - - ASSERT_TRUE(TossTestUtils::compareSize(svec, 2)); -} - -/** - * @brief neighbor bad lock + neighbor edge + good lock + edge - */ -TEST_F(TossTest, neighbors_test_18) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto lock0 = env_->dupEdge(edges[0]); - auto lock1 = env_->dupEdge(edges[1]); - - auto rawKey0 = env_->makeRawKey(edges[0].get_key()).first; - auto rawKey1 = env_->makeRawKey(edges[1].get_key()).first; - - env_->syncAddEdge(edges[0]); - env_->syncAddEdge(edges[1]); - - auto lockKey0 = env_->insertLock(lock0, false); - auto lockKey1 = env_->insertLock(lock1, true); - LOG(INFO) << "lockKey0 hexlify = " << folly::hexlify(lockKey0); - LOG(INFO) << "lockKey1 hexlify = " << folly::hexlify(lockKey1); - - auto props = env_->getNeiProps(edges); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey0)); - ASSERT_FALSE(env_->keyExist(lockKey1)); - - // step 2nd: edge key exist - ASSERT_TRUE(env_->keyExist(rawKey0)); - ASSERT_TRUE(env_->keyExist(rawKey1)); - - // step 3rd: reverse edge key exist - auto rRawKey0 = env_->makeRawKey(env_->reverseEdgeKey(lock0.get_key())); - auto rRawKey1 = env_->makeRawKey(env_->reverseEdgeKey(lock1.get_key())); - - ASSERT_TRUE(env_->keyExist(rRawKey0.first)); - ASSERT_TRUE(env_->keyExist(rRawKey1.first)); - - // step 4th: the get neighbors result is from lock - auto svec = TossTestUtils::splitNeiResults(props); - auto actualProps = env_->extractStrVals(svec); - decltype(actualProps) expect{edges[0].get_props()[1].toString(), lock1.get_props()[1].toString()}; - ASSERT_EQ(actualProps, expect); - - ASSERT_TRUE(TossTestUtils::compareSize(svec, 2)); -} - -/** - * @brief neighbor bad lock + neighbor edge + bad lock + edge - */ -TEST_F(TossTest, neighbors_test_19) { - LOG(INFO) << "b_=" << b_; - auto num = 2; - std::vector edges = env_->generateMultiEdges(num, b_); - ASSERT_EQ(edges.size(), 2); - - auto lock0 = env_->dupEdge(edges[0]); - auto lock1 = env_->dupEdge(edges[1]); - - auto rawKey0 = env_->makeRawKey(edges[0].get_key()).first; - auto rawKey1 = env_->makeRawKey(edges[1].get_key()).first; - - env_->syncAddEdge(edges[0]); - env_->syncAddEdge(edges[1]); - - auto lockKey0 = env_->insertLock(lock0, false); - auto lockKey1 = env_->insertLock(lock1, false); - LOG(INFO) << "lockKey0 hexlify = " << folly::hexlify(lockKey0); - LOG(INFO) << "lockKey1 hexlify = " << folly::hexlify(lockKey1); - - auto props = env_->getNeiProps(edges); - - // step 1st: lock key not exist - ASSERT_FALSE(env_->keyExist(lockKey0)); - ASSERT_FALSE(env_->keyExist(lockKey1)); - - // step 2nd: edge key exist - ASSERT_TRUE(env_->keyExist(rawKey0)); - ASSERT_TRUE(env_->keyExist(rawKey1)); - - // step 3rd: reverse edge key exist - auto rRawKey0 = env_->makeRawKey(env_->reverseEdgeKey(lock0.get_key())); - auto rRawKey1 = env_->makeRawKey(env_->reverseEdgeKey(lock1.get_key())); - - ASSERT_TRUE(env_->keyExist(rRawKey0.first)); - ASSERT_TRUE(env_->keyExist(rRawKey1.first)); - - // step 4th: the get neighbors result is from lock - auto svec = TossTestUtils::splitNeiResults(props); - auto actualProps = env_->extractStrVals(svec); - decltype(actualProps) expect{edges[0].get_props()[1].toString(), - edges[1].get_props()[1].toString()}; - ASSERT_EQ(actualProps, expect); - - ASSERT_TRUE(TossTestUtils::compareSize(svec, 2)); -} - -/** - * @brief - */ -TEST_F(TossTest, get_props_test_0) { - LOG(INFO) << "getProps_test_0 b_=" << b_; - auto num = 1; - std::vector edges = env_->generateMultiEdges(num, b_); - - LOG(INFO) << "going to add edge:" << edges.back().get_props().back(); - auto code = env_->syncAddMultiEdges(edges, kNotUseToss); - ASSERT_EQ(code, nebula::cpp2::ErrorCode::SUCCEEDED); - - auto vvec = env_->getProps(edges[0]); - LOG(INFO) << "vvec.size()=" << vvec.size(); - for (auto& val : vvec) { - LOG(INFO) << "val.toString()=" << val.toString(); - } -} - -/** - * @brief - */ -TEST_F(TossTest, get_props_test_1) { - LOG(INFO) << __func__ << " b_=" << b_; - auto num = 1; - std::vector edges = env_->generateMultiEdges(num, b_); - - LOG(INFO) << "going to add edge:" << edges.back().get_props().back(); - auto code = env_->syncAddMultiEdges(edges, kUseToss); - ASSERT_EQ(code, nebula::cpp2::ErrorCode::SUCCEEDED); - - auto vvec = env_->getProps(edges[0]); - LOG(INFO) << "vvec.size()=" << vvec.size(); - for (auto& val : vvec) { - LOG(INFO) << "val.toString()=" << val.toString(); - } -} - -} // namespace storage -} // namespace nebula - -int main(int argc, char** argv) { - FLAGS_heartbeat_interval_secs = 1; - - testing::InitGoogleTest(&argc, argv); - folly::init(&argc, &argv, false); - google::SetStderrLogging(google::INFO); - return RUN_ALL_TESTS(); -} diff --git a/src/storage/test/TossTestExecutor.h b/src/storage/test/TossTestExecutor.h new file mode 100644 index 00000000000..3bea706c0d7 --- /dev/null +++ b/src/storage/test/TossTestExecutor.h @@ -0,0 +1,380 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#pragma once + +#include "TossEnvironment.h" + +namespace nebula { +namespace storage { + +using StorageClient = storage::GraphStorageClient; + +template +class StorageResponseReader { + public: + explicit StorageResponseReader(StorageRpcResponse& resp) : resp_(&resp) {} + + bool isLeaderChange() { + auto& c = resp_->failedParts(); + return std::any_of( + c.begin(), c.end(), [](auto it) { return it.second == cpp2::ErrorCode::E_LEADER_CHANGED; }); + } + + std::vector& data() { return resp_->responses(); } + + cpp2::ErrorCode firstErrCode() { + for (auto& p : resp_->failedParts()) { + if (p.second != cpp2::ErrorCode::SUCCEEDED) { + return p.second; + } + } + return cpp2::ErrorCode::SUCCEEDED; + } + + private: + StorageRpcResponse* resp_; +}; + +class GetNeighborsExecutor { + TestSpace* s_; + std::vector data_; + + public: + GetNeighborsExecutor(TestSpace* s, cpp2::NewEdge edge) + : GetNeighborsExecutor(s, std::vector{edge}) {} + + GetNeighborsExecutor(TestSpace* s, + std::vector edges, + int64_t limit = std::numeric_limits::max()) + : s_(s), edges_(std::move(edges)), limit_(limit) { + env_ = TossEnvironment::getInstance(); + auto uniEdges = uniqueEdges(edges_); + data_ = run(uniEdges); + } + + std::vector data() { return data_; } + + std::vector uniqueEdges(const std::vector& __edges) { + std::vector edges(__edges); + std::sort(edges.begin(), edges.end(), [](const auto& a, const auto& b) { + if (a.key.src == b.key.src) { + return a.key.dst < b.key.dst; + } + return a.key.src < b.key.src; + }); + auto last = std::unique(edges.begin(), edges.end(), [](const auto& a, const auto& b) { + return a.key.src == b.key.src && a.key.dst == b.key.dst; + }); + edges.erase(last, edges.end()); + return edges; + } + + /** + * @brief Get the Nei Props object, + * will unique same src & dst input edges. + */ + std::vector run(const std::vector& edges) { + bool retLeaderChange = false; + LOG(INFO) << "edges.size()=" << edges.size() << ", edges.size()=" << edges.size(); + do { + auto f = rpc(edges); + f.wait(); + if (!f.valid()) { + LOG(ERROR) << "!f.valid()"; + break; + } + if (f.value().succeeded()) { + return readStorageResp(f.value()); + } else { + LOG(ERROR) << "!f.value().succeeded()"; + } + auto parts = f.value().failedParts(); + for (auto& part : parts) { + if (part.second == cpp2::ErrorCode::E_LEADER_CHANGED) { + retLeaderChange = true; + break; + } + } + } while (retLeaderChange); + + LOG(ERROR) << "getOutNeighborsProps failed"; + std::vector ret; + return ret; + } + + folly::SemiFuture> rpc( + const std::vector& edges) { + // para3 + std::vector vertices; + std::set vids; + for (auto& e : edges) { + // vids.insert(e.key.src); + vids.insert(e.key.dst); + } + for (auto& vid : vids) { + Row row; + row.emplace_back(vid); + vertices.emplace_back(row); + } + // para 4 + std::vector edgeTypes; + // para 5 + cpp2::EdgeDirection edgeDirection = cpp2::EdgeDirection::BOTH; + // para 6 + std::vector* statProps = nullptr; + // para 7 + std::vector* vertexProps = nullptr; + // para 8 + // const std::vector edgeProps(1); + std::vector edgeProps(1); + edgeProps.back().type = 0 - edges[0].key.edge_type; + // para 9 + const std::vector* expressions = nullptr; + // para 10 + bool dedup = false; + // para 11 + bool random = false; + // para 12 + const std::vector orderBy = std::vector(); + + auto colNames = TossTestUtils::makeColNames(edges.back().props.size()); + + return s_->sClient_->getNeighbors(s_->getSpaceId(), + colNames, + vertices, + edgeTypes, + edgeDirection, + statProps, + vertexProps, + &edgeProps, + expressions, + dedup, + random, + orderBy, + limit_); + } + + std::vector readStorageResp(StorageRpcResponse& rpc) { + std::vector ret; + LOG(INFO) << "rpc.responses().size()=" << rpc.responses().size(); + for (auto& resp : rpc.responses()) { + auto sub = readGetNeighborsResp(resp); + ret.insert(ret.end(), sub.begin(), sub.end()); + } + return ret; + } + + std::vector readGetNeighborsResp(cpp2::GetNeighborsResponse& resp) { + std::vector ret; + auto& ds = resp.vertices; + LOG(INFO) << "ds.rows.size()=" << ds.rows.size(); + for (auto& row : ds.rows) { + LOG(INFO) << "row.values.size()=" << row.values.size(); + for (auto& val : row.values) { + LOG(INFO) << "row.val = " << val.toString(); + } + ret.emplace_back(row.values[3].toString()); + } + return ret; + } + + private: + std::vector edges_; + int64_t limit_{0}; + TossEnvironment* env_; +}; + +struct AddEdgeExecutor { + TestSpace* s_; + std::vector edges_; + cpp2::ErrorCode code_; + + public: + AddEdgeExecutor(TestSpace* s, cpp2::NewEdge edge) : s_(s) { + appendEdge(std::move(edge)); + rpc(); + } + + AddEdgeExecutor(TestSpace* s, std::vector edges) : s_(s) { + for (auto& eg : edges) { + appendEdge(std::move(eg)); + } + rpc(); + } + + void appendEdge(cpp2::NewEdge&& edge) { + CHECK_GT(edge.key.edge_type, 0); + edges_.emplace_back(std::move(edge)); + edges_.back().key.edge_type = 0 - edges_.back().key.edge_type; + std::swap(edges_.back().key.src, edges_.back().key.dst); + } + + void rpc() { + // LOG(INFO) << "AddEdgeExecutor::rpc(), spaceId=" << s_->getSpaceId(); + auto propNames = TossTestUtils::makeColNames(edges_.back().props.size()); + bool overwritable = true; + folly::EventBase* evb = nullptr; + bool useToss = s_->useToss_; + int retry = 10; + do { + // LOG(INFO) << "AddEdgeExecutor::rpc(), do retry = " << retry; + auto f = + s_->sClient_->addEdges(s_->getSpaceId(), edges_, propNames, overwritable, evb, useToss); + f.wait(); + CHECK(f.hasValue()); + StorageResponseReader reader(f.value()); + if (reader.isLeaderChange()) { + continue; + } + code_ = reader.firstErrCode(); + break; + } while (retry-- > 0); + } + + bool ok() { return code_ == cpp2::ErrorCode::SUCCEEDED; } + + cpp2::ErrorCode code() { return code_; } +}; + +struct GetPropsExecutor { + TossEnvironment* env_; + TestSpace* s_; + cpp2::NewEdge edge_; + std::vector result_; + + public: + GetPropsExecutor(TestSpace* s, cpp2::NewEdge edge) : s_(s), edge_(std::move(edge)) { + env_ = TossEnvironment::getInstance(); + CHECK_GT(edge_.key.edge_type, 0); + std::swap(edge_.key.src, edge_.key.dst); + edge_.key.edge_type = 0 - edge_.key.edge_type; + rpc(); + } + + GetPropsExecutor(TestSpace* s, std::vector edges) + : GetPropsExecutor(s, edges[0]) {} + + std::vector data() { return result_; } + + void rpc() { + nebula::Row row; + row.values.emplace_back(edge_.key.src); + row.values.emplace_back(edge_.key.edge_type); + row.values.emplace_back(edge_.key.ranking); + row.values.emplace_back(edge_.key.dst); + + nebula::DataSet ds; + ds.rows.emplace_back(std::move(row)); + + std::vector props(1); + props.back().type = edge_.key.edge_type; + + int retry = 10; + + do { + LOG(INFO) << "enter do{} while"; + auto frpc = s_->sClient_ + ->getProps(s_->getSpaceId(), + ds, /*DataSet*/ + nullptr, /*vector*/ + &props, /*vector*/ + nullptr /*expressions*/) + .via(env_->executor_.get()); + frpc.wait(); + CHECK(frpc.hasValue()); + StorageResponseReader reader(frpc.value()); + if (reader.isLeaderChange()) { + continue; + } + + auto resps = reader.data(); + CHECK_EQ(resps.size(), 1); + cpp2::GetPropResponse& propResp = resps.front(); + cpp2::ResponseCommon result = propResp.get_result(); + nebula::DataSet& dataSet = propResp.props; + std::vector& rows = dataSet.rows; + if (rows.empty()) { + LOG(FATAL) << "getProps() dataSet.rows.empty())"; + } + LOG(INFO) << "rows.size() = " << rows.size(); + for (auto& r : rows) { + LOG(INFO) << "values.size() = " << r.values.size(); + for (auto& val : r.values) { + LOG(INFO) << "val: " << val.toString(); + } + } + result_ = rows[0].values; + if (result_.empty()) { + LOG(FATAL) << "getProps() ret.empty())"; + } + break; + } while (retry-- > 0); + } +}; + +struct UpdateExecutor { + TossEnvironment* env_; + TestSpace* s_; + cpp2::NewEdge edge_; + std::vector updatedProps_; + bool insertable_{false}; + std::vector returnProps_; + std::string condition_; + + public: + UpdateExecutor(TestSpace* s, cpp2::NewEdge edge) : s_(s), edge_(std::move(edge)) { + env_ = TossEnvironment::getInstance(); + CHECK_GT(edge_.key.edge_type, 0); + std::swap(edge_.key.src, edge_.key.dst); + edge_.key.edge_type = 0 - edge_.key.edge_type; + prepareParameters(); + rpc(); + } + + void prepareParameters() { + cpp2::UpdatedProp uProp1; + uProp1.set_name("c1"); + ConstantExpression val1(edge_.props[0].getInt()); + uProp1.set_value(Expression::encode(val1)); + updatedProps_.emplace_back(uProp1); + + cpp2::UpdatedProp uProp2; + uProp2.set_name("c2"); + ConstantExpression val2(edge_.props[1].getStr()); + uProp2.set_value(Expression::encode(val2)); + + updatedProps_.emplace_back(uProp2); + } + + void rpc() { + for (;;) { + auto f = s_->sClient_->updateEdge( + s_->getSpaceId(), edge_.key, updatedProps_, insertable_, returnProps_, condition_); + f.wait(); + CHECK(f.hasValue()); + if (!f.value().ok()) { + LOG(FATAL) << f.value().status().toString(); + } + storage::cpp2::UpdateResponse resp = f.value().value(); + if (resp.result.get_failed_parts().empty()) { + break; + } + + CHECK_EQ(resp.result.get_failed_parts().size(), 1U); + cpp2::PartitionResult result = resp.result.get_failed_parts().back(); + if (result.code == cpp2::ErrorCode::E_LEADER_CHANGED) { + LOG(INFO) << "update edge leader changed, retry"; + continue; + } + LOG(ERROR) << "update edge err: " << cpp2::_ErrorCode_VALUES_TO_NAMES.at(result.code); + break; + } + } +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/test/TossTestUtils.h b/src/storage/test/TossTestUtils.h index d269676470b..59ca454f5f5 100644 --- a/src/storage/test/TossTestUtils.h +++ b/src/storage/test/TossTestUtils.h @@ -154,12 +154,92 @@ struct TossTestUtils { return ret; } - static cpp2::EdgeKey toVidKey(const cpp2::EdgeKey& input) { + static cpp2::EdgeKey makeEdgeKeyS(const cpp2::EdgeKey& input) { + if (input.src.type() == Value::Type::STRING) { + return input; + } cpp2::EdgeKey ret(input); ret.set_src(std::string(reinterpret_cast(&input.get_src().getInt()), 8)); ret.set_dst(std::string(reinterpret_cast(&input.get_dst().getInt()), 8)); return ret; } + + static std::vector makeISValue(int64_t iVal) { + boost::uuids::random_generator gen; + std::vector vals(2); + vals[0].setInt(iVal); + vals[1].setStr(boost::uuids::to_string(gen())); + return vals; + } + + // generate a vector of values, 1st is ant i64, 2nd is a random string. + static std::vector> genISValues(size_t num) { + std::vector> ret; + for (auto i = 0U; i != num; ++i) { + int32_t n = 1024 * (1 + i); + ret.emplace_back(makeISValue(n)); + } + return ret; + } + + // generate num different edges with same dst + // the first src is dst + 1, and increase 1 for each + static std::vector makeNeighborEdges(int64_t dst, int edgeType, size_t num) { + auto values = genISValues(num); + std::vector edges; + auto rank = 0; + for (auto i = 0U; i < num; ++i) { + auto src = dst + i + 1; + auto ekey = makeEdgeKeyI(src, edgeType, rank, dst); + edges.emplace_back(); + edges.back().set_key(std::move(ekey)); + edges.back().set_props(std::move(values[i])); + } + return edges; + } + + static cpp2::NewEdge makeEdge(int64_t src, int edgeType) { + cpp2::NewEdge edge; + edge.set_key(makeEdgeKeyI(src, edgeType, 0, src + 1)); + edge.set_props(makeISValue(1024)); + return edge; + } + + static cpp2::NewEdge makeEdgeS(int64_t src, int edgeType) { + cpp2::NewEdge edge = makeEdge(src, edgeType); + edge.key = makeEdgeKeyS(edge.key); + return edge; + } + + static cpp2::NewEdge makeTwinEdge(const cpp2::NewEdge& oldEdge) { + cpp2::NewEdge newEdge(oldEdge); + auto newVal = makeISValue(newEdge.props[0].getInt() + 1024); + newEdge.set_props(newVal); + return newEdge; + } + + static std::vector makeColNames(size_t n) { + std::vector colNames; + for (auto i = 0U; i < n; ++i) { + colNames.emplace_back(folly::sformat("c{}", i + 1)); + } + return colNames; + } + + static std::vector makeColDefs( + const std::vector& types) { + auto N = types.size(); + auto colNames = makeColNames(N); + std::vector columnDefs(N); + for (auto i = 0U; i != N; ++i) { + columnDefs[i].set_name(colNames[i]); + meta::cpp2::ColumnTypeDef colTypeDef; + colTypeDef.set_type(types[i]); + columnDefs[i].set_type(colTypeDef); + columnDefs[i].set_nullable(true); + } + return columnDefs; + } }; // end TossTestUtils } // namespace storage diff --git a/src/storage/transaction/ChainAddEdgesGroupProcessor.cpp b/src/storage/transaction/ChainAddEdgesGroupProcessor.cpp new file mode 100644 index 00000000000..e389867f565 --- /dev/null +++ b/src/storage/transaction/ChainAddEdgesGroupProcessor.cpp @@ -0,0 +1,78 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#include "storage/transaction/ChainAddEdgesGroupProcessor.h" + +#include "storage/StorageFlags.h" +#include "storage/mutate/AddEdgesProcessor.h" +#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ConsistUtil.h" +#include "storage/transaction/TransactionManager.h" + +namespace nebula { +namespace storage { + +void ChainAddEdgesGroupProcessor::process(const cpp2::AddEdgesRequest& req) { + auto space = req.get_space_id(); + ShuffledReq shuffledReq; + shuffleRequest(req, shuffledReq); + + callingNum_ = shuffledReq.size(); + + auto delegateProcess = [&](auto& item) { + auto localPartId = item.first.first; + auto* proc = ChainAddEdgesProcessorLocal::instance(env_); + proc->setRemotePartId(item.first.second); + 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(space, localPartId, code); + }); + proc->process(item.second); + }; + + std::for_each(shuffledReq.begin(), shuffledReq.end(), delegateProcess); +} + +void ChainAddEdgesGroupProcessor::shuffleRequest(const cpp2::AddEdgesRequest& req, + ShuffledReq& shuffledReq) { + auto numOfPart = env_->metaClient_->partsNum(req.get_space_id()); + if (!numOfPart.ok()) { + return; + } + auto getPart = [&](auto& vid) { return env_->metaClient_->partId(numOfPart.value(), vid); }; + + auto genNewReq = [&](auto& reqIn) { + cpp2::AddEdgesRequest ret; + ret.set_space_id(reqIn.get_space_id()); + ret.set_prop_names(reqIn.get_prop_names()); + ret.set_if_not_exists(reqIn.get_if_not_exists()); + return ret; + }; + + auto shufflePart = [&](auto&& part) { + auto& localPart = part.first; + + auto shuffleEdge = [&](auto&& edge) { + auto remotePart = getPart(edge.get_key().get_dst().getStr()); + auto key = std::make_pair(localPart, remotePart); + auto it = shuffledReq.find(key); + if (it == shuffledReq.end()) { + shuffledReq[key] = genNewReq(req); + } + (*shuffledReq[key].parts_ref())[localPart].emplace_back(edge); + }; + + std::for_each(part.second.begin(), part.second.end(), shuffleEdge); + }; + + auto& parts = req.get_parts(); + std::for_each(parts.begin(), parts.end(), shufflePart); +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainAddEdgesGroupProcessor.h b/src/storage/transaction/ChainAddEdgesGroupProcessor.h new file mode 100644 index 00000000000..3f4fbdb8a57 --- /dev/null +++ b/src/storage/transaction/ChainAddEdgesGroupProcessor.h @@ -0,0 +1,34 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#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 ChainAddEdgesGroupProcessor : public BaseProcessor { + public: + static ChainAddEdgesGroupProcessor* instance(StorageEnv* env) { + return new ChainAddEdgesGroupProcessor(env); + } + + void process(const cpp2::AddEdgesRequest& req); + + protected: + explicit ChainAddEdgesGroupProcessor(StorageEnv* env) : BaseProcessor(env) {} + + using Chain = std::pair; + using ShuffledReq = std::unordered_map; + void shuffleRequest(const cpp2::AddEdgesRequest& src, ShuffledReq& dst); +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainAddEdgesProcessorLocal.cpp b/src/storage/transaction/ChainAddEdgesProcessorLocal.cpp new file mode 100644 index 00000000000..6fc5d6f709a --- /dev/null +++ b/src/storage/transaction/ChainAddEdgesProcessorLocal.cpp @@ -0,0 +1,436 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#include "storage/transaction/ChainAddEdgesProcessorLocal.h" + +#include + +#include "common/utils/DefaultValueContext.h" +#include "kvstore/Part.h" +#include "storage/StorageFlags.h" +#include "storage/mutate/AddEdgesProcessor.h" +#include "storage/transaction/ConsistUtil.h" +#include "storage/transaction/TransactionManager.h" + +namespace nebula { +namespace storage { + +void ChainAddEdgesProcessorLocal::process(const cpp2::AddEdgesRequest& req) { + if (!prepareRequest(req)) { + finish(); + return; + } + env_->txnMan_->addChainTask(this); +} + +int64_t toInt(const std::string& str) { + if (str.size() < 3) { + return 0; + } + auto str2 = str.substr(1, str.size() - 2); + return atoll(str2.c_str()); +} + +std::string makeReadableEdge(const cpp2::AddEdgesRequest& req) { + if (req.get_parts().size() != 1) { + LOG(INFO) << req.get_parts().size(); + return ""; + } + if (req.get_parts().begin()->second.size() != 1) { + LOG(INFO) << req.get_parts().begin()->second.size(); + return ""; + } + auto& edge = req.get_parts().begin()->second.back(); + + auto isrc = toInt(edge.get_key().get_src().toString()); + auto idst = toInt(edge.get_key().get_dst().toString()); + std::stringstream oss; + oss << isrc << "->" << idst << ", val: "; + for (auto& val : edge.get_props()) { + oss << val.toString() << " "; + } + return oss.str(); +} + +/** + * @brief + * 1. check term + * 2. set mem lock + * 3. write edge prime(key = edge prime, value = transaction id) + */ +folly::SemiFuture ChainAddEdgesProcessorLocal::prepareLocal() { + uuid_ = ConsistUtil::strUUID(); + readableEdgeDesc_ = makeReadableEdge(req_); + if (!readableEdgeDesc_.empty()) { + uuid_.append(" ").append(readableEdgeDesc_); + } + + if (!lockEdges(req_)) { + return Code::E_WRITE_WRITE_CONFLICT; + } + + auto [pro, fut] = folly::makePromiseContract(); + auto primes = makePrime(); + for (auto& kv : primes) { + LOG(INFO) << uuid_ << " put prime " << folly::hexlify(kv.first); + } + + erasePrime(); + env_->kvstore_->asyncMultiPut( + spaceId_, localPartId_, std::move(primes), [p = std::move(pro)](auto rc) mutable { + LOG_IF(WARNING, rc != nebula::cpp2::ErrorCode::SUCCEEDED) + << "kvstore err: " << static_cast(rc); + p.setValue(rc); + }); + return std::move(fut); +} + +folly::SemiFuture ChainAddEdgesProcessorLocal::processRemote(Code code) { + LOG(INFO) << uuid_ << " prepareLocal(), code = " << apache::thrift::util::enumNameSafe(code); + if (code != Code::SUCCEEDED) { + return code; + } + CHECK_EQ(req_.get_parts().size(), 1); + auto reversedRequest = reverseRequest(req_); + CHECK_EQ(reversedRequest.get_parts().size(), 1); + auto [pro, fut] = folly::makePromiseContract(); + doRpc(std::move(pro), std::move(reversedRequest)); + return std::move(fut); +} + +folly::SemiFuture ChainAddEdgesProcessorLocal::processLocal(Code code) { + LOG(INFO) << uuid_ << " processRemote(), code = " << apache::thrift::util::enumNameSafe(code); + if (!checkTerm(req_)) { + LOG(WARNING) << "E_OUTDATED_TERM"; + return Code::E_OUTDATED_TERM; + } + + if (code == Code::E_RPC_FAILURE) { + kvAppend_ = makeDoublePrime(); + markDanglingEdge(); + } + + if (code == Code::SUCCEEDED || code == Code::E_RPC_FAILURE) { + return forwardToDelegateProcessor(); + } else { + return abort(); + } + + if (code == Code::E_RPC_FAILURE) { + return Code::SUCCEEDED; + } + return code; +} + +void ChainAddEdgesProcessorLocal::markDanglingEdge() { + lk_->forceUnlock(); + auto keys = sEdgeKey(req_); + for (auto& key : keys) { + env_->txnMan_->markDanglingEdge(spaceId_, key); + } +} + +bool ChainAddEdgesProcessorLocal::prepareRequest(const cpp2::AddEdgesRequest& req) { + CHECK_EQ(req.get_parts().size(), 1); + req_ = req; + spaceId_ = req_.get_space_id(); + localPartId_ = req.get_parts().begin()->first; + // replaceNullWithDefaultValue(req_); + auto part = env_->kvstore_->part(spaceId_, localPartId_); + if (!nebula::ok(part)) { + pushResultCode(nebula::error(part), localPartId_); + return false; + } + localTerm_ = (nebula::value(part))->termId(); + + auto vidLen = env_->schemaMan_->getSpaceVidLen(spaceId_); + if (!vidLen.ok()) { + LOG(ERROR) << "getSpaceVidLen failed, spaceId_: " << spaceId_ + << ", status: " << vidLen.status(); + setErrorCode(Code::E_INVALID_SPACEVIDLEN); + return false; + } + spaceVidLen_ = vidLen.value(); + return true; +} + +folly::SemiFuture ChainAddEdgesProcessorLocal::forwardToDelegateProcessor() { + auto* proc = AddEdgesProcessor::instance(env_, nullptr); + proc->consistOp_ = [&](kvstore::BatchHolder& a, std::vector* b) { + callbackOfChainOp(a, b); + }; + auto futProc = proc->getFuture(); + auto [pro, fut] = folly::makePromiseContract(); + std::move(futProc).thenValue([&, p = std::move(pro)](auto&& resp) mutable { + auto rc = extractRpcError(resp); + if (rc != Code::SUCCEEDED) { + LOG(INFO) << uuid_ << " forwardToDelegateProcessor(), code = " + << apache::thrift::util::enumNameSafe(rc); + markDanglingEdge(); + } + p.setValue(rc); + }); + proc->process(req_); + return std::move(fut); +} + +Code ChainAddEdgesProcessorLocal::extractRpcError(const cpp2::ExecResponse& resp) { + Code ret = Code::SUCCEEDED; + auto& respComn = resp.get_result(); + for (auto& part : respComn.get_failed_parts()) { + ret = part.code; + } + return ret; +} + +void ChainAddEdgesProcessorLocal::doRpc(folly::Promise&& promise, + cpp2::AddEdgesRequest&& req, + int retry) noexcept { + if (retry > retryLimit_) { + promise.setValue(Code::E_LEADER_CHANGED); + return; + } + // CHECK_NE(req.get_parts().size(), 1); + auto* iClient = env_->txnMan_->getInternalClient(); + folly::Promise p; + auto f = p.getFuture(); + iClient->chainAddEdges(req, localTerm_, 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; + switch (code) { + case Code::E_LEADER_CHANGED: + doRpc(std::move(p), std::move(req), ++retry); + break; + default: + p.setValue(code); + break; + } + return code; + }); +} + +void ChainAddEdgesProcessorLocal::callbackOfChainOp(kvstore::BatchHolder& batch, + std::vector* pData) { + if (pData != nullptr) { + for (auto& kv : *pData) { + batch.put(std::string(kv.first), std::string(kv.second)); + } + } + for (auto& key : kvErased_) { + batch.remove(std::string(key)); + } + for (auto& kv : kvAppend_) { + batch.put(std::string(kv.first), std::string(kv.second)); + } + // LOG(WARNING) << "will do perf optimize later"; +} + +folly::SemiFuture ChainAddEdgesProcessorLocal::abort() { + if (kvErased_.empty()) { + return Code::SUCCEEDED; + } + auto [pro, fut] = folly::makePromiseContract(); + env_->kvstore_->asyncMultiRemove(req_.get_space_id(), + localPartId_, + std::move(kvErased_), + [p = std::move(pro), this](auto rc) mutable { + if (rc != Code::SUCCEEDED) { + LOG(INFO) << this << " forwardToDelegateProcessor(), code = " + << apache::thrift::util::enumNameSafe(rc); + markDanglingEdge(); + } + p.setValue(rc); + }); + return std::move(fut); +} + +std::vector ChainAddEdgesProcessorLocal::makePrime() { + std::vector ret; + for (auto& edge : req_.get_parts().begin()->second) { + auto key = ConsistUtil::primeKey(spaceVidLen_, localPartId_, edge.get_key()); + + auto req = makeSingleEdgeRequest(localPartId_, edge); + std::string val; + apache::thrift::CompactSerializer::serialize(req, &val); + val.append(ConsistUtil::insertIdentifier()); + + ret.emplace_back(std::make_pair(std::move(key), std::move(val))); + } + return ret; +} + +std::vector ChainAddEdgesProcessorLocal::makeDoublePrime() { + std::vector ret; + for (auto& edge : req_.get_parts().begin()->second) { + auto key = ConsistUtil::doublePrime(spaceVidLen_, localPartId_, edge.get_key()); + + auto req = makeSingleEdgeRequest(localPartId_, edge); + std::string val; + apache::thrift::CompactSerializer::serialize(req, &val); + val.append(ConsistUtil::insertIdentifier()); + + ret.emplace_back(std::make_pair(std::move(key), std::move(val))); + } + return ret; +} + +void ChainAddEdgesProcessorLocal::erasePrime() { + auto fn = [&](const cpp2::NewEdge& edge) { + auto key = ConsistUtil::primeKey(spaceVidLen_, localPartId_, edge.get_key()); + return key; + }; + for (auto& edge : req_.get_parts().begin()->second) { + kvErased_.push_back(fn(edge)); + } +} + +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) { + std::vector keys; + auto partId = req.get_parts().begin()->first; + for (auto& edge : req.get_parts().begin()->second) { + keys.emplace_back(ConsistUtil::edgeKey(spaceVidLen_, partId, edge.get_key())); + } + auto* lockCore = env_->txnMan_->getLockCore(req.get_space_id()); + lk_ = std::make_unique(lockCore, keys); + 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 ret = env_->txnMan_->checkTerm(space, partId, localTerm_); + LOG_IF(WARNING, !ret) << "check term failed, localTerm_ = " << localTerm_; + return ret; +} + +// 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 ret; + for (auto& edgesOfPart : req.get_parts()) { + auto partId = edgesOfPart.first; + for (auto& edge : edgesOfPart.second) { + ret.emplace_back(ConsistUtil::edgeKey(spaceVidLen_, partId, edge.get_key())); + } + } + return ret; +} + +cpp2::AddEdgesRequest ChainAddEdgesProcessorLocal::reverseRequest( + const cpp2::AddEdgesRequest& req) { + cpp2::AddEdgesRequest reversedRequest; + for (auto& edgesOfPart : *req.parts_ref()) { + for (auto& newEdge : edgesOfPart.second) { + (*reversedRequest.parts_ref())[remotePartId_].emplace_back(newEdge); + auto& newEdgeRef = (*reversedRequest.parts_ref())[remotePartId_].back(); + ConsistUtil::reverseEdgeKeyInplace(*newEdgeRef.key_ref()); + } + } + reversedRequest.set_space_id(req.get_space_id()); + reversedRequest.set_prop_names(req.get_prop_names()); + reversedRequest.set_if_not_exists(req.get_if_not_exists()); + return reversedRequest; +} + +void ChainAddEdgesProcessorLocal::finish() { + LOG(INFO) << uuid_ << " commitLocal(), code_ = " << apache::thrift::util::enumNameSafe(code_); + pushResultCode(code_, localPartId_); + onFinished(); +} + +cpp2::AddEdgesRequest ChainAddEdgesProcessorLocal::makeSingleEdgeRequest( + PartitionID partId, const cpp2::NewEdge& edge) { + cpp2::AddEdgesRequest req; + req.set_space_id(req_.get_space_id()); + req.set_prop_names(req_.get_prop_names()); + req.set_if_not_exists(req_.get_if_not_exists()); + + std::unordered_map> newParts; + newParts[partId].emplace_back(edge); + + req.set_parts(newParts); + return req; +} + +/*** consider the following case: + * + * create edge known(kdate datetime default datetime(), degree int); + * insert edge known(degree) VALUES "100" -> "101":(95); + * + * storage will insert datetime() as default value on both + * in/out edge, but they will calculate independent + * which lead to inconsistance + * + * that why we need to replace the inconsistance prone value + * at the monment the request comes + * */ +void ChainAddEdgesProcessorLocal::replaceNullWithDefaultValue(cpp2::AddEdgesRequest& req) { + DefaultValueContext expCtx; + auto& propNames = *req.prop_names_ref(); + for (auto& part : *req.parts_ref()) { + for (auto& edge : part.second) { + auto edgeKey = edge.get_key(); + auto edgeType = std::abs(*edgeKey.edge_type_ref()); + auto schema = env_->schemaMan_->getEdgeSchema(spaceId_, edgeType); + auto& vals = *edge.props_ref(); + for (auto i = 0U; i < schema->getNumFields(); ++i) { + std::string fieldName(schema->getFieldName(i)); + auto it = std::find(propNames.begin(), propNames.end(), fieldName); + if (it == propNames.end()) { + auto field = schema->field(i); + if (field->hasDefault()) { + auto expr = field->defaultValue()->clone(); + propNames.emplace_back(fieldName); + auto defVal = Expression::eval(expr, expCtx); + switch (defVal.type()) { + case Value::Type::DATE: + vals.emplace_back(defVal.getDate()); + break; + case Value::Type::TIME: + vals.emplace_back(defVal.getTime()); + break; + case Value::Type::DATETIME: + vals.emplace_back(defVal.getDateTime()); + break; + default: + // for other type, local and remote should behavior same. + break; + } + } else { + // it's ok if this field doesn't have a default value + } + } + } + } + } +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainAddEdgesProcessorLocal.h b/src/storage/transaction/ChainAddEdgesProcessorLocal.h new file mode 100644 index 00000000000..0daa8fb9b23 --- /dev/null +++ b/src/storage/transaction/ChainAddEdgesProcessorLocal.h @@ -0,0 +1,142 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#pragma once + +#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 ChainAddEdgesProcessorLocal : public BaseProcessor, + public ChainBaseProcessor { + friend class ChainResumeProcessorTestHelper; // for test friendly + public: + static ChainAddEdgesProcessorLocal* instance(StorageEnv* env) { + return new ChainAddEdgesProcessorLocal(env); + } + + virtual ~ChainAddEdgesProcessorLocal() = default; + + virtual void process(const cpp2::AddEdgesRequest& req); + + folly::SemiFuture prepareLocal() override; + + folly::SemiFuture processRemote(Code code) override; + + folly::SemiFuture processLocal(Code code) override; + + void setRemotePartId(PartitionID remotePartId) { remotePartId_ = remotePartId; } + + void finish() override; + + protected: + explicit ChainAddEdgesProcessorLocal(StorageEnv* env) : BaseProcessor(env) {} + + bool prepareRequest(const cpp2::AddEdgesRequest& req); + + /** + * @brief resume and set req_ & txnId_ from the val of (double)prime + * @return true if resume succeeded + */ + bool deserializeRequest(GraphSpaceID spaceId, PartitionID partId, folly::StringPiece val); + + void doRpc(folly::Promise&& pro, cpp2::AddEdgesRequest&& req, int retry = 0) noexcept; + + 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 + * @param batch if target edge has index + * @param pData if target edge has no index. + */ + void callbackOfChainOp(kvstore::BatchHolder& batch, std::vector* pData); + + /** + * @brief helper function to generate string form of keys of request + */ + std::vector sEdgeKey(const cpp2::AddEdgesRequest& req); + + /** + * @brief normally, the prime/double prime keys will be deleted at AddEdgeProcessor + * (in a batch of finial edges), but sometimes ChainProcessor will stop early + * and we need this do the clean thing. + */ + folly::SemiFuture abort(); + + /** + * @brief helper function to get the reversed request of the normal incoming req. + * (to use that for reversed edge) + */ + cpp2::AddEdgesRequest reverseRequest(const cpp2::AddEdgesRequest& req); + + Code extractRpcError(const cpp2::ExecResponse& resp); + + /** + * @brief a normal AddEdgeRequest may contain multi edges + * even though they will fail or succeed as a batch in this time + * some of them may by overwrite by othere request + * so when resume each edge + */ + cpp2::AddEdgesRequest makeSingleEdgeRequest(PartitionID partId, const cpp2::NewEdge& edge); + + std::vector makePrime(); + + std::vector makeDoublePrime(); + + void erasePrime(); + + void eraseDoublePrime(); + + folly::SemiFuture forwardToDelegateProcessor(); + + void markDanglingEdge(); + + /*** consider the following case: + * + * create edge known(kdate datetime default datetime(), degree int); + * insert edge known(degree) VALUES "100" -> "101":(95); + * + * storage will insert datetime() as default value on both + * in/out edge, but they will calculate independent + * which lead to inconsistance + * + * that why we need to replace the inconsistance prone value + * at the monment the request comes + * */ + void replaceNullWithDefaultValue(cpp2::AddEdgesRequest& req); + + protected: + GraphSpaceID spaceId_; + PartitionID localPartId_; + PartitionID remotePartId_; + cpp2::AddEdgesRequest req_; + std::unique_ptr lk_; + int retryLimit_{10}; + TermID localTerm_{-1}; + + std::vector kvErased_; + std::vector kvAppend_; + folly::Optional edgeVer_{folly::none}; + int64_t resumedEdgeVer_{-1}; + + std::string uuid_; + // 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/ChainAddEdgesProcessorRemote.cpp b/src/storage/transaction/ChainAddEdgesProcessorRemote.cpp new file mode 100644 index 00000000000..b2680c6fed4 --- /dev/null +++ b/src/storage/transaction/ChainAddEdgesProcessorRemote.cpp @@ -0,0 +1,96 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#include "storage/transaction/ChainAddEdgesProcessorRemote.h" + +#include "storage/mutate/AddEdgesProcessor.h" +#include "storage/transaction/ConsistUtil.h" +#include "storage/transaction/TransactionManager.h" + +namespace nebula { +namespace storage { + +void ChainAddEdgesProcessorRemote::process(const cpp2::ChainAddEdgesRequest& req) { + LOG(INFO) << this << ConsistUtil::dumpParts(req.get_parts()); + auto partId = req.get_parts().begin()->first; + auto code = nebula::cpp2::ErrorCode::SUCCEEDED; + do { + if (!checkTerm(req)) { + LOG(WARNING) << "invalid term, incoming part " << partId << ", 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; + break; + } else { + spaceVidLen_ = vIdLen.value(); + } + } while (0); + + if (code == nebula::cpp2::ErrorCode::SUCCEEDED) { + forwardRequest(req); + } else { + pushResultCode(code, partId); + 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) { + auto spaceId = req.get_space_id(); + auto* proc = AddEdgesProcessor::instance(env_); + proc->getFuture().thenValue([&](auto&& resp) { + Code rc = Code::SUCCEEDED; + for (auto& part : resp.get_result().get_failed_parts()) { + handleErrorCode(part.code, spaceId, part.get_part_id()); + } + LOG(INFO) << this << " " << apache::thrift::util::enumNameSafe(rc); + this->result_ = resp.get_result(); + this->onFinished(); + }); + proc->process(ConsistUtil::makeDirectAddReq(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( + const cpp2::ChainAddEdgesRequest& req) { + std::vector ret; + for (auto& edgesOfPart : req.get_parts()) { + auto partId = edgesOfPart.first; + for (auto& edge : edgesOfPart.second) { + ret.emplace_back(ConsistUtil::edgeKey(spaceVidLen_, partId, edge.get_key())); + } + } + return ret; +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainAddEdgesProcessorRemote.h b/src/storage/transaction/ChainAddEdgesProcessorRemote.h new file mode 100644 index 00000000000..19b795b71d4 --- /dev/null +++ b/src/storage/transaction/ChainAddEdgesProcessorRemote.h @@ -0,0 +1,36 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#pragma once + +#include "storage/BaseProcessor.h" +#include "storage/transaction/ChainBaseProcessor.h" + +namespace nebula { +namespace storage { + +class ChainAddEdgesProcessorRemote : public BaseProcessor { + public: + static ChainAddEdgesProcessorRemote* instance(StorageEnv* env) { + return new ChainAddEdgesProcessorRemote(env); + } + + void process(const cpp2::ChainAddEdgesRequest& req); + + private: + explicit ChainAddEdgesProcessorRemote(StorageEnv* env) : BaseProcessor(env) {} + + bool checkTerm(const cpp2::ChainAddEdgesRequest& req); + + bool checkVersion(const cpp2::ChainAddEdgesRequest& req); + + void forwardRequest(const cpp2::ChainAddEdgesRequest& req); + + std::vector getStrEdgeKeys(const cpp2::ChainAddEdgesRequest& req); +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainBaseProcessor.h b/src/storage/transaction/ChainBaseProcessor.h new file mode 100644 index 00000000000..93fd3a9bd3b --- /dev/null +++ b/src/storage/transaction/ChainBaseProcessor.h @@ -0,0 +1,50 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#pragma once + +#include "common/utils/MemoryLockWrapper.h" +#include "storage/CommonUtils.h" + +namespace nebula { +namespace storage { + +constexpr int32_t chainRetryLimit = 10; + +using Code = ::nebula::cpp2::ErrorCode; + +/** + * @brief interface for all chain processor + * + */ +class ChainBaseProcessor { + public: + virtual ~ChainBaseProcessor() = default; + + virtual folly::SemiFuture prepareLocal() { return Code::SUCCEEDED; } + + virtual folly::SemiFuture processRemote(Code code) { return code; } + + virtual folly::SemiFuture processLocal(Code code) { return code; } + + virtual folly::Future getFinished() { return finished_.getFuture(); } + + virtual void finish() = 0; + + protected: + void setErrorCode(Code code) { + if (code_ == Code::SUCCEEDED) { + code_ = code; + } + } + + protected: + Code code_ = Code::SUCCEEDED; + folly::Promise finished_; +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainProcessorFactory.cpp b/src/storage/transaction/ChainProcessorFactory.cpp new file mode 100644 index 00000000000..23b81d9d4fa --- /dev/null +++ b/src/storage/transaction/ChainProcessorFactory.cpp @@ -0,0 +1,63 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#include "storage/transaction/ChainProcessorFactory.h" + +#include "storage/transaction/ConsistUtil.h" +#include "storage/transaction/ResumeAddEdgeProcessor.h" +#include "storage/transaction/ResumeAddEdgeRemoteProcessor.h" +#include "storage/transaction/ResumeUpdateProcessor.h" +#include "storage/transaction/ResumeUpdateRemoteProcessor.h" + +namespace nebula { +namespace storage { + +ChainBaseProcessor* ChainProcessorFactory::makeProcessor(StorageEnv* env, + const ResumeOptions& options) { + ChainBaseProcessor* ret = nullptr; + auto requestType = ConsistUtil::parseType(options.primeValue); + switch (requestType) { + case RequestType::INSERT: { + switch (options.resumeType) { + case ResumeType::RESUME_CHAIN: { + ret = ResumeAddEdgeProcessor::instance(env, options.primeValue); + break; + } + case ResumeType::RESUME_REMOTE: { + ret = ResumeAddEdgeRemoteProcessor::instance(env, options.primeValue); + break; + } + case ResumeType::UNKNOWN: { + LOG(FATAL) << "ResumeType::UNKNOWN: not supposed run here"; + } + } + break; + } + case RequestType::UPDATE: { + switch (options.resumeType) { + case ResumeType::RESUME_CHAIN: { + ret = ResumeUpdateProcessor::instance(env, options.primeValue); + break; + } + case ResumeType::RESUME_REMOTE: { + ret = ResumeUpdateRemoteProcessor::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"; + } + } + return ret; +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainProcessorFactory.h b/src/storage/transaction/ChainProcessorFactory.h new file mode 100644 index 00000000000..857b4b8a56b --- /dev/null +++ b/src/storage/transaction/ChainProcessorFactory.h @@ -0,0 +1,33 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#pragma once + +#include "storage/CommonUtils.h" +#include "storage/transaction/ChainBaseProcessor.h" + +namespace nebula { +namespace storage { + +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 ChainProcessorFactory { + public: + static ChainBaseProcessor* makeProcessor(StorageEnv* env, const ResumeOptions& options); +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainResumeProcessor.cpp b/src/storage/transaction/ChainResumeProcessor.cpp new file mode 100644 index 00000000000..903fee75ea2 --- /dev/null +++ b/src/storage/transaction/ChainResumeProcessor.cpp @@ -0,0 +1,63 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#include "storage/transaction/ChainResumeProcessor.h" + +#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ChainProcessorFactory.h" +#include "storage/transaction/ChainUpdateEdgeProcessorLocal.h" +#include "storage/transaction/ConsistUtil.h" +#include "storage/transaction/TransactionManager.h" + +namespace nebula { +namespace storage { + +void ChainResumeProcessor::process() { + std::unordered_map> leaders; + if (env_->kvstore_->allLeader(leaders) == 0) { + LOG(INFO) << "no leader found, skip any resume process"; + return; + } + // LOG(INFO) << "ChainResumeProcessor::process(), leaders.size() = " << leaders.size(); + std::unique_ptr iter; + for (auto& leader : leaders) { + auto spaceId = leader.first; + // LOG(INFO) << "leader.second.size() = " << leader.second.size() << ", spaceId = " << spaceId; + for (auto& partInfo : leader.second) { + auto partId = partInfo.get_part_id(); + auto prefix = ConsistUtil::primePrefix(partId); + // LOG(INFO) << "scanning prefix = " << folly::hexlify(prefix); + auto rc = env_->kvstore_->prefix(spaceId, partId, prefix, &iter); + if (rc != nebula::cpp2::ErrorCode::SUCCEEDED) { + break; + } + for (; iter->valid(); iter->next()) { + LOG(INFO) << "resume prime " << folly::hexlify(iter->key()); + ResumeOptions opt(ResumeType::RESUME_CHAIN, iter->val().str()); + auto* proc = ChainProcessorFactory::makeProcessor(env_, opt); + futs.emplace_back(proc->getFinished()); + env_->txnMan_->addChainTask(proc); + } + + prefix = ConsistUtil::doublePrimePrefix(partId); + rc = env_->kvstore_->prefix(spaceId, partId, prefix, &iter); + if (rc != nebula::cpp2::ErrorCode::SUCCEEDED) { + break; + } + for (; iter->valid(); iter->next()) { + LOG(INFO) << "resume double prime " << folly::hexlify(iter->key()); + ResumeOptions opt(ResumeType::RESUME_REMOTE, iter->val().str()); + auto* proc = ChainProcessorFactory::makeProcessor(env_, opt); + futs.emplace_back(proc->getFinished()); + env_->txnMan_->addChainTask(proc); + } + } + } + folly::collectAll(futs).get(); +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainResumeProcessor.h b/src/storage/transaction/ChainResumeProcessor.h new file mode 100644 index 00000000000..0611c29d44e --- /dev/null +++ b/src/storage/transaction/ChainResumeProcessor.h @@ -0,0 +1,33 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#pragma once + +#include "clients/storage/InternalStorageClient.h" +#include "common/utils/NebulaKeyUtils.h" +#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ChainBaseProcessor.h" +#include "storage/transaction/ChainUpdateEdgeProcessorLocal.h" +#include "storage/transaction/TransactionManager.h" + +namespace nebula { +namespace storage { + +class ChainResumeProcessor { + friend class ChainResumeProcessorTestHelper; + + public: + explicit ChainResumeProcessor(StorageEnv* env) : env_(env) {} + + void process(); + + private: + StorageEnv* env_{nullptr}; + std::vector> futs; +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainUpdateEdgeProcessorLocal.cpp b/src/storage/transaction/ChainUpdateEdgeProcessorLocal.cpp new file mode 100644 index 00000000000..b56609bdf77 --- /dev/null +++ b/src/storage/transaction/ChainUpdateEdgeProcessorLocal.cpp @@ -0,0 +1,269 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#include "storage/transaction/ChainUpdateEdgeProcessorLocal.h" + +#include + +#include "storage/StorageFlags.h" +#include "storage/mutate/UpdateEdgeProcessor.h" +#include "storage/transaction/ConsistUtil.h" +#include "storage/transaction/TransactionManager.h" + +namespace nebula { +namespace storage { + +void ChainUpdateEdgeProcessorLocal::process(const cpp2::UpdateEdgeRequest& req) { + if (!prepareRequest(req)) { + onFinished(); + } + + env_->txnMan_->addChainTask(this); +} + +bool ChainUpdateEdgeProcessorLocal::prepareRequest(const cpp2::UpdateEdgeRequest& req) { + req_ = req; + spaceId_ = req.get_space_id(); + partId_ = req_.get_part_id(); + + auto rc = getSpaceVidLen(spaceId_); + if (rc != nebula::cpp2::ErrorCode::SUCCEEDED) { + pushResultCode(rc, partId_); + 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_); + return false; + } + return true; +} + +/** + * 1. set mem lock + * 2. set edge prime + * */ +folly::SemiFuture ChainUpdateEdgeProcessorLocal::prepareLocal() { + if (!setLock()) { + LOG(INFO) << "set lock failed, return E_DATA_CONFLICT_ERROR"; + return Code::E_DATA_CONFLICT_ERROR; + } + + auto key = ConsistUtil::primeKey(spaceVidLen_, partId_, req_.get_edge_key()); + + std::string val; + apache::thrift::CompactSerializer::serialize(req_, &val); + val.append(ConsistUtil::updateIdentifier()); + + std::vector data{{key, val}}; + auto c = folly::makePromiseContract(); + env_->kvstore_->asyncMultiPut( + spaceId_, partId_, std::move(data), [p = std::move(c.first)](auto rc) mutable { + p.setValue(rc); + }); + return std::move(c.second); +} + +folly::SemiFuture ChainUpdateEdgeProcessorLocal::processRemote(Code code) { + LOG(INFO) << __func__ << "(), code = " << apache::thrift::util::enumNameSafe(code); + if (code != Code::SUCCEEDED) { + return code; + } + auto [pro, fut] = folly::makePromiseContract(); + doRpc(std::move(pro)); + return std::move(fut); +} + +folly::SemiFuture ChainUpdateEdgeProcessorLocal::processLocal(Code code) { + LOG(INFO) << __func__ << "(), 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; + } + + if (code == Code::E_RPC_FAILURE) { + appendDoublePrime(); + } + + if (code == Code::SUCCEEDED || code == Code::E_RPC_FAILURE) { + erasePrime(); + forwardToDelegateProcessor(); + } else { + abort(); + } + + // LOG(INFO) << "~processNormalLocal(), code_: " << apache::thrift::util::enumNameSafe(code); + return code_; +} + +void ChainUpdateEdgeProcessorLocal::doRpc(folly::Promise&& promise, int retry) noexcept { + if (retry > retryLimit_) { + promise.setValue(Code::E_LEADER_CHANGED); + return; + } + auto* iClient = env_->txnMan_->getInternalClient(); + folly::Promise p; + auto reversedReq = reverseRequest(req_); + iClient->chainUpdateEdge(reversedReq, termOfPrepare_, ver_, std::move(p)); + + auto f = p.getFuture(); + std::move(f).thenTry([=, p = std::move(promise)](auto&& t) mutable { + auto code = t.hasValue() ? t.value() : Code::E_RPC_FAILURE; + LOG(INFO) << "code = " << apache::thrift::util::enumNameSafe(code); + switch (code) { + case Code::E_LEADER_CHANGED: + doRpc(std::move(p), ++retry); + break; + default: + p.setValue(code); + break; + } + return code; + }); +} + +folly::SemiFuture ChainUpdateEdgeProcessorLocal::processResumeRemoteLocal(Code code) { + LOG(INFO) << __func__ << "(), code = " << apache::thrift::util::enumNameSafe(code); + if (code == Code::SUCCEEDED) { + auto partId = req_.get_part_id(); + auto key = ConsistUtil::doublePrime(spaceVidLen_, partId, req_.get_edge_key()); + kvErased_.emplace_back(key); + abort(); + } + return code; +} + +void ChainUpdateEdgeProcessorLocal::erasePrime() { + auto key = ConsistUtil::primeKey(spaceVidLen_, partId_, req_.get_edge_key()); + kvErased_.emplace_back(std::move(key)); +} + +void ChainUpdateEdgeProcessorLocal::appendDoublePrime() { + auto key = ConsistUtil::doublePrime(spaceVidLen_, partId_, 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() { + kUpdateEdgeCounters.init("update_edge"); + UpdateEdgeProcessor::ContextAdjuster fn = [=](EdgeContext& ctx) { + ctx.kvAppend = std::move(kvAppend_); + ctx.kvErased = std::move(kvErased_); + }; + + auto* proc = UpdateEdgeProcessor::instance(env_); + proc->adjustContext(std::move(fn)); + auto f = proc->getFuture(); + proc->process(req_); + auto resp = std::move(f).get(); + code_ = getErrorCode(resp); + std::swap(resp_, resp); +} + +Code ChainUpdateEdgeProcessorLocal::checkAndBuildContexts(const cpp2::UpdateEdgeRequest&) { + return Code::SUCCEEDED; +} + +std::string ChainUpdateEdgeProcessorLocal::sEdgeKey(const cpp2::UpdateEdgeRequest& req) { + return ConsistUtil::edgeKey(spaceVidLen_, req.get_part_id(), req.get_edge_key()); +} + +void ChainUpdateEdgeProcessorLocal::finish() { + LOG(INFO) << "ChainUpdateEdgeProcessorLocal::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()); + kvErased_.emplace_back(std::move(key)); + + folly::Baton baton; + env_->kvstore_->asyncMultiRemove( + req_.get_space_id(), req_.get_part_id(), std::move(kvErased_), [&](auto rc) mutable { + LOG_IF(WARNING, rc != Code::SUCCEEDED) << "error: " << static_cast(rc); + baton.post(); + }); + baton.wait(); +} + +cpp2::UpdateEdgeRequest ChainUpdateEdgeProcessorLocal::reverseRequest( + const cpp2::UpdateEdgeRequest& req) { + cpp2::UpdateEdgeRequest reversedRequest(req); + auto reversedEdgeKey = ConsistUtil::reverseEdgeKey(req.get_edge_key()); + reversedRequest.set_edge_key(reversedEdgeKey); + + auto partsNum = env_->metaClient_->partsNum(req.get_space_id()); + CHECK(partsNum.ok()); + auto srcVid = reversedRequest.get_edge_key().get_src().getStr(); + auto partId = env_->metaClient_->partId(partsNum.value(), srcVid); + // CHECK(partId.ok()); + // reversedRequest.set_part_id(partId.value()); + reversedRequest.set_part_id(partId); + + return reversedRequest; +} + +bool ChainUpdateEdgeProcessorLocal::setLock() { + auto spaceId = req_.get_space_id(); + auto* lockCore = env_->txnMan_->getLockCore(spaceId); + if (lockCore == nullptr) { + return false; + } + auto key = ConsistUtil::edgeKey(spaceVidLen_, req_.get_part_id(), req_.get_edge_key()); + lk_ = std::make_unique>(lockCore, key); + 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( + const cpp2::UpdateResponse& resp) { + auto& respCommon = resp.get_result(); + auto& parts = respCommon.get_failed_parts(); + if (parts.empty()) { + return nebula::cpp2::ErrorCode::SUCCEEDED; + } + return parts.front().get_code(); +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainUpdateEdgeProcessorLocal.h b/src/storage/transaction/ChainUpdateEdgeProcessorLocal.h new file mode 100644 index 00000000000..6980449c273 --- /dev/null +++ b/src/storage/transaction/ChainUpdateEdgeProcessorLocal.h @@ -0,0 +1,95 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#pragma once + +#include + +#include "common/utils/MemoryLockWrapper.h" +#include "storage/query/QueryBaseProcessor.h" +#include "storage/transaction/ChainBaseProcessor.h" +#include "storage/transaction/ConsistUtil.h" +#include "storage/transaction/TransactionManager.h" + +namespace nebula { +namespace storage { + +class ChainUpdateEdgeProcessorLocal + : public QueryBaseProcessor, + public ChainBaseProcessor { + friend struct ChainUpdateEdgeTestHelper; + + public: + using Code = ::nebula::cpp2::ErrorCode; + static ChainUpdateEdgeProcessorLocal* instance(StorageEnv* env) { + return new ChainUpdateEdgeProcessorLocal(env); + } + + void process(const cpp2::UpdateEdgeRequest& req) override; + + folly::SemiFuture prepareLocal() override; + + folly::SemiFuture processRemote(Code code) override; + + folly::SemiFuture processLocal(Code code) override; + + void onProcessFinished() override {} + + void finish() override; + + virtual ~ChainUpdateEdgeProcessorLocal() = default; + + protected: + explicit ChainUpdateEdgeProcessorLocal(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 processResumeRemoteLocal(Code code); + + folly::SemiFuture processNormalLocal(Code code); + + void abort(); + + bool prepareRequest(const cpp2::UpdateEdgeRequest& req); + + void erasePrime(); + + void appendDoublePrime(); + + void forwardToDelegateProcessor(); + + std::string sEdgeKey(const cpp2::UpdateEdgeRequest& req); + + cpp2::UpdateEdgeRequest reverseRequest(const cpp2::UpdateEdgeRequest& req); + + bool setLock(); + + int64_t getVersion(const cpp2::UpdateEdgeRequest& req); + + nebula::cpp2::ErrorCode getErrorCode(const cpp2::UpdateResponse& resp); + + Code checkAndBuildContexts(const cpp2::UpdateEdgeRequest& req) override; + + protected: + cpp2::UpdateEdgeRequest req_; + std::unique_ptr lk_; + PartitionID partId_; + int retryLimit_{10}; + TermID termOfPrepare_{-1}; + std::vector kvErased_; + std::vector kvAppend_; + folly::Optional ver_{folly::none}; +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainUpdateEdgeProcessorRemote.cpp b/src/storage/transaction/ChainUpdateEdgeProcessorRemote.cpp new file mode 100644 index 00000000000..7c194f45c6a --- /dev/null +++ b/src/storage/transaction/ChainUpdateEdgeProcessorRemote.cpp @@ -0,0 +1,67 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#include "storage/transaction/ChainUpdateEdgeProcessorRemote.h" + +#include "storage/mutate/UpdateEdgeProcessor.h" +#include "storage/transaction/ConsistUtil.h" +#include "storage/transaction/TransactionManager.h" + +namespace nebula { +namespace storage { + +using Code = ::nebula::cpp2::ErrorCode; + +void ChainUpdateEdgeProcessorRemote::process(const cpp2::ChainUpdateEdgeRequest& req) { + auto rc = Code::SUCCEEDED; + if (!checkTerm(req)) { + 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()); + } else { + updateEdge(req); + } + 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; +} + +// forward to UpdateEdgeProcessor +void ChainUpdateEdgeProcessorRemote::updateEdge(const cpp2::ChainUpdateEdgeRequest& req) { + auto* proc = UpdateEdgeProcessor::instance(env_, counters_); + auto f = proc->getFuture(); + proc->process(req.get_update_edge_request()); + auto resp = std::move(f).get(); + std::swap(resp_, resp); +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ChainUpdateEdgeProcessorRemote.h b/src/storage/transaction/ChainUpdateEdgeProcessorRemote.h new file mode 100644 index 00000000000..3001700d686 --- /dev/null +++ b/src/storage/transaction/ChainUpdateEdgeProcessorRemote.h @@ -0,0 +1,39 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#pragma once + +#include "common/utils/MemoryLockWrapper.h" +#include "storage/BaseProcessor.h" +#include "storage/transaction/TransactionManager.h" + +namespace nebula { +namespace storage { + +class ChainUpdateEdgeProcessorRemote : public BaseProcessor { + public: + static ChainUpdateEdgeProcessorRemote* instance(StorageEnv* env) { + return new ChainUpdateEdgeProcessorRemote(env); + } + + void process(const cpp2::ChainUpdateEdgeRequest& req); + + private: + explicit ChainUpdateEdgeProcessorRemote(StorageEnv* env) + : BaseProcessor(env) {} + + bool checkTerm(const cpp2::ChainUpdateEdgeRequest& req); + + bool checkVersion(const cpp2::ChainUpdateEdgeRequest& req); + + void updateEdge(const cpp2::ChainUpdateEdgeRequest& req); + + private: + std::unique_ptr lk_; +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ConsistUtil.cpp b/src/storage/transaction/ConsistUtil.cpp new file mode 100644 index 00000000000..6cd6f775a14 --- /dev/null +++ b/src/storage/transaction/ConsistUtil.cpp @@ -0,0 +1,207 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#include "storage/transaction/ConsistUtil.h" + +#include +#include +#include + +#include "common/utils/NebulaKeyUtils.h" +namespace nebula { +namespace storage { + +static const std::string kPrimeTable{"__prime__"}; // NOLINT +static const std::string kDoublePrimeTable{"__prime_prime__"}; // NOLINT +static const std::string kTempRequestTable{"__temp_request__"}; // NOLINT + +std::string ConsistUtil::primeTable() { return kPrimeTable; } + +std::string ConsistUtil::doublePrimeTable() { return kDoublePrimeTable; } + +// std::string ConsistUtil::tempRequestTable() { +// return kTempRequestTable; +// } + +std::string ConsistUtil::primePrefix(PartitionID partId) { + return kPrimeTable + NebulaKeyUtils::edgePrefix(partId); +} + +std::string ConsistUtil::doublePrimePrefix(PartitionID partId) { + return kDoublePrimeTable + NebulaKeyUtils::edgePrefix(partId); +} + +std::string ConsistUtil::primeKey(size_t vIdLen, PartitionID partId, const cpp2::EdgeKey& edgeKey) { + return kPrimeTable + NebulaKeyUtils::edgeKey(vIdLen, + partId, + edgeKey.get_src().getStr(), + edgeKey.get_edge_type(), + edgeKey.get_ranking(), + edgeKey.get_dst().getStr()); +} + +std::string ConsistUtil::doublePrime(size_t vIdLen, PartitionID partId, const cpp2::EdgeKey& key) { + return kDoublePrimeTable + NebulaKeyUtils::edgeKey(vIdLen, + partId, + key.get_src().getStr(), + key.get_edge_type(), + key.get_ranking(), + key.get_dst().getStr()); +} + +RequestType ConsistUtil::parseType(folly::StringPiece val) { + char identifier = val.str().back(); + switch (identifier) { + case 'u': + return RequestType::UPDATE; + case 'a': + return RequestType::INSERT; + default: + LOG(FATAL) << "shoule not happend, identifier is " << identifier; + // return RequestType::UNKNOWN; + } +} + +cpp2::UpdateEdgeRequest ConsistUtil::parseUpdateRequest(folly::StringPiece val) { + cpp2::UpdateEdgeRequest req; + apache::thrift::CompactSerializer::deserialize(val, req); + return req; +} + +cpp2::AddEdgesRequest ConsistUtil::parseAddRequest(folly::StringPiece val) { + cpp2::AddEdgesRequest req; + apache::thrift::CompactSerializer::deserialize(val, req); + return req; +} + +std::string ConsistUtil::strUUID() { + static boost::uuids::random_generator gen; + return boost::uuids::to_string(gen()); +} + +std::string ConsistUtil::ConsistUtil::edgeKey(size_t vIdLen, + PartitionID partId, + const cpp2::EdgeKey& key) { + return NebulaKeyUtils::edgeKey(vIdLen, + partId, + key.get_src().getStr(), + *key.edge_type_ref(), + *key.ranking_ref(), + (*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::makeDirectAddReq(const cpp2::ChainAddEdgesRequest& req) { + cpp2::AddEdgesRequest ret; + ret.set_space_id(req.get_space_id()); + ret.set_parts(req.get_parts()); + ret.set_prop_names(req.get_prop_names()); + ret.set_if_not_exists(req.get_if_not_exists()); + return ret; +} + +cpp2::EdgeKey ConsistUtil::reverseEdgeKey(const cpp2::EdgeKey& edgeKey) { + cpp2::EdgeKey reversedKey(edgeKey); + std::swap(*reversedKey.src_ref(), *reversedKey.dst_ref()); + *reversedKey.edge_type_ref() = 0 - edgeKey.get_edge_type(); + return reversedKey; +} + +void ConsistUtil::reverseEdgeKeyInplace(cpp2::EdgeKey& edgeKey) { + cpp2::EdgeKey reversedKey(edgeKey); + std::swap(*edgeKey.src_ref(), *edgeKey.dst_ref()); + *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; + + 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); + + return std::make_pair(ver, rc); +} + +std::string ConsistUtil::dumpAddEdgeReq(const cpp2::AddEdgesRequest& req) { + std::stringstream oss; + for (auto& part : req.get_parts()) { + for (auto& edge : part.second) { + oss << " edge: " << folly::hexlify(edge.get_key().get_src().toString()) << "->" + << folly::hexlify(edge.get_key().get_dst().toString()) << ", vals: "; + for (auto& val : edge.get_props()) { + oss << val.toString() << ", "; + } + oss << "\n"; + } + } + return oss.str(); +} + +std::string ConsistUtil::dumpParts(const Parts& parts) { + 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()) << ", vals: "; + for (auto& val : edge.get_props()) { + oss << val.toString() << ", "; + } + oss << "\n"; + } + } + return oss.str(); +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ConsistUtil.h b/src/storage/transaction/ConsistUtil.h new file mode 100644 index 00000000000..71f9284b29a --- /dev/null +++ b/src/storage/transaction/ConsistUtil.h @@ -0,0 +1,104 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#pragma once + +#include "common/time/WallClock.h" +#include "common/utils/MemoryLockWrapper.h" +#include "common/utils/NebulaKeyUtils.h" +#include "interface/gen-cpp2/storage_types.h" +#include "kvstore/KVStore.h" +#include "storage/CommonUtils.h" + +namespace nebula { +namespace storage { + +enum class RequestType { + UNKNOWN, + INSERT, + UPDATE, +}; + +enum class ChainProcessType { + NORMAL = 0, + RESUME_CHAIN = 1, + RESUME_REMOTE = 2, +}; + +class ConsistUtil final { + public: + static std::string primeTable(); + + static std::string doublePrimeTable(); + + 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); + + static std::string doublePrime(size_t vIdLen, PartitionID partId, const cpp2::EdgeKey& edgeKey); + + static std::string primePrefix(PartitionID partId); + + static std::string doublePrimePrefix(PartitionID partId); + + static std::string primeKey(size_t vIdLen, + PartitionID partId, + const VertexID& srcId, + EdgeType type, + EdgeRanking rank, + const VertexID& dstId); + + static std::string doublePrime(size_t vIdLen, + PartitionID partId, + const VertexID& srcId, + EdgeType type, + EdgeRanking rank, + const VertexID& dstId); + + static RequestType parseType(folly::StringPiece val); + + static cpp2::UpdateEdgeRequest parseUpdateRequest(folly::StringPiece val); + + static cpp2::AddEdgesRequest parseAddRequest(folly::StringPiece val); + + 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 makeDirectAddReq(const cpp2::ChainAddEdgesRequest& req); + + static cpp2::EdgeKey reverseEdgeKey(const cpp2::EdgeKey& edgeKey); + + static void reverseEdgeKeyInplace(cpp2::EdgeKey& edgeKey); + + static std::string insertIdentifier() noexcept { return "a"; } + + static std::string updateIdentifier() noexcept { return "u"; } + + static std::pair versionOfUpdateReq( + StorageEnv* env, const cpp2::UpdateEdgeRequest& req); + + static std::string dumpAddEdgeReq(const cpp2::AddEdgesRequest& req); + + using Parts = std::unordered_map>; + static std::string dumpParts(const Parts& parts); +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/GetValueProcessor.cpp b/src/storage/transaction/GetValueProcessor.cpp deleted file mode 100644 index b9e75a9c78c..00000000000 --- a/src/storage/transaction/GetValueProcessor.cpp +++ /dev/null @@ -1,39 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License, - * attached with Common Clause Condition 1.0, found in the LICENSES directory. - */ - -#include "storage/transaction/GetValueProcessor.h" - -#include - -#include "storage/StorageFlags.h" - -namespace nebula { -namespace storage { - -ProcessorCounters kGetValueCounters; - -void GetValueProcessor::process(const cpp2::GetValueRequest& req) { - CHECK_NOTNULL(env_->kvstore_); - auto spaceId = req.get_space_id(); - auto partId = req.get_part_id(); - auto key = req.get_key(); - - std::string value; - auto rc = env_->kvstore_->get(spaceId, partId, key, &value); - LOG_IF(INFO, FLAGS_trace_toss) << "getValue for partId=" << partId - << ", key=" << folly::hexlify(key) - << ", rc=" << static_cast(rc); - - if (rc != nebula::cpp2::ErrorCode::SUCCEEDED) { - handleErrorCode(rc, spaceId, partId); - } else { - resp_.set_value(std::move(value)); - } - this->onFinished(); -} - -} // namespace storage -} // namespace nebula diff --git a/src/storage/transaction/GetValueProcessor.h b/src/storage/transaction/GetValueProcessor.h deleted file mode 100644 index bc71d34fdec..00000000000 --- a/src/storage/transaction/GetValueProcessor.h +++ /dev/null @@ -1,35 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License, - * attached with Common Clause Condition 1.0, found in the LICENSES directory. - */ - -#ifndef STORAGE_TRANSACTION_GetValueProcessor_H_ -#define STORAGE_TRANSACTION_GetValueProcessor_H_ - -#include "common/base/Base.h" -#include "storage/BaseProcessor.h" - -namespace nebula { -namespace storage { - -extern ProcessorCounters kGetValueCounters; - -class GetValueProcessor : public BaseProcessor { - public: - static GetValueProcessor* instance(StorageEnv* env, - const ProcessorCounters* counters = &kGetValueCounters) { - return new GetValueProcessor(env, counters); - } - - void process(const cpp2::GetValueRequest& req); - - protected: - explicit GetValueProcessor(StorageEnv* env, const ProcessorCounters* counters) - : BaseProcessor(env, counters) {} -}; - -} // namespace storage -} // namespace nebula - -#endif // STORAGE_TRANSACTION_GetValueProcessor_H_ diff --git a/src/storage/transaction/ResumeAddEdgeProcessor.cpp b/src/storage/transaction/ResumeAddEdgeProcessor.cpp new file mode 100644 index 00000000000..4f8d650f56a --- /dev/null +++ b/src/storage/transaction/ResumeAddEdgeProcessor.cpp @@ -0,0 +1,72 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#include "storage/transaction/ResumeAddEdgeProcessor.h" + +namespace nebula { +namespace storage { + +ResumeAddEdgeProcessor::ResumeAddEdgeProcessor(StorageEnv* env, const std::string& val) + : ChainAddEdgesProcessorLocal(env) { + req_ = ConsistUtil::parseAddRequest(val); + LOG(WARNING) << this << ConsistUtil::dumpAddEdgeReq(req_); + ChainAddEdgesProcessorLocal::prepareRequest(req_); +} + +folly::SemiFuture ResumeAddEdgeProcessor::prepareLocal() { + if (code_ != Code::SUCCEEDED) { + return code_; + } + auto spaceId = req_.get_space_id(); + auto numOfPart = env_->metaClient_->partsNum(spaceId); + if (!numOfPart.ok()) { + return Code::E_SPACE_NOT_FOUND; + } + 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 ResumeAddEdgeProcessor::processRemote(Code code) { + return ChainAddEdgesProcessorLocal::processRemote(code); +} + +folly::SemiFuture ResumeAddEdgeProcessor::processLocal(Code code) { + LOG(INFO) << this << " " << __func__ << "(), code = " << apache::thrift::util::enumNameSafe(code); + setErrorCode(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; + } + + if (code == Code::E_RPC_FAILURE) { + kvAppend_ = ChainAddEdgesProcessorLocal::makeDoublePrime(); + } + + 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) + erasePrime(); + return ChainAddEdgesProcessorLocal::forwardToDelegateProcessor(); + } + + return code; +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ResumeAddEdgeProcessor.h b/src/storage/transaction/ResumeAddEdgeProcessor.h new file mode 100644 index 00000000000..8ce3cf3e377 --- /dev/null +++ b/src/storage/transaction/ResumeAddEdgeProcessor.h @@ -0,0 +1,33 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#pragma once + +#include "storage/transaction/ChainAddEdgesProcessorLocal.h" + +namespace nebula { +namespace storage { + +class ResumeAddEdgeProcessor : public ChainAddEdgesProcessorLocal { + public: + static ResumeAddEdgeProcessor* instance(StorageEnv* env, const std::string& val) { + return new ResumeAddEdgeProcessor(env, val); + } + + folly::SemiFuture prepareLocal() override; + + folly::SemiFuture processRemote(nebula::cpp2::ErrorCode code) override; + + folly::SemiFuture processLocal(nebula::cpp2::ErrorCode code) override; + + virtual ~ResumeAddEdgeProcessor() = default; + + protected: + ResumeAddEdgeProcessor(StorageEnv* env, const std::string& val); +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ResumeAddEdgeRemoteProcessor.cpp b/src/storage/transaction/ResumeAddEdgeRemoteProcessor.cpp new file mode 100644 index 00000000000..e8d2c263487 --- /dev/null +++ b/src/storage/transaction/ResumeAddEdgeRemoteProcessor.cpp @@ -0,0 +1,98 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#include "storage/transaction/ResumeAddEdgeRemoteProcessor.h" + +namespace nebula { +namespace storage { + +ResumeAddEdgeRemoteProcessor::ResumeAddEdgeRemoteProcessor(StorageEnv* env, const std::string& val) + : ChainAddEdgesProcessorLocal(env) { + req_ = ConsistUtil::parseAddRequest(val); + LOG(WARNING) << this << this << ConsistUtil::dumpAddEdgeReq(req_); + ChainAddEdgesProcessorLocal::prepareRequest(req_); +} + +folly::SemiFuture ResumeAddEdgeRemoteProcessor::prepareLocal() { + if (!lockEdges()) { + return Code::E_WRITE_WRITE_CONFLICT; + } + + 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 spaceId = req_.get_space_id(); + auto numOfPart = env_->metaClient_->partsNum(spaceId); + if (!numOfPart.ok()) { + return Code::E_SPACE_NOT_FOUND; + } + 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); +} + +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; + } + + if (code == Code::E_OUTDATED_TERM) { + // E_OUTDATED_TERM indicate this host is no longer the leader of curr part + // any following kv operation will fail + // due to not allowed to write from follower + return code; + } + + if (code == Code::E_RPC_FAILURE) { + // nothing to do, as we are already an rpc failure + } + + 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(); + return forwardToDelegateProcessor(); + } + + return code; +} + +bool ResumeAddEdgeRemoteProcessor::lockEdges() { + std::vector keys; + auto spaceId = req_.get_space_id(); + auto partId = req_.get_parts().begin()->first; + for (auto& edge : req_.get_parts().begin()->second) { + keys.emplace_back(ConsistUtil::edgeKey(spaceVidLen_, partId, edge.get_key())); + } + resumeLock_ = std::make_unique(env_->txnMan_, spaceId, keys[0]); + + return resumeLock_->isLocked(); +} + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ResumeAddEdgeRemoteProcessor.h b/src/storage/transaction/ResumeAddEdgeRemoteProcessor.h new file mode 100644 index 00000000000..4ab508a9e92 --- /dev/null +++ b/src/storage/transaction/ResumeAddEdgeRemoteProcessor.h @@ -0,0 +1,38 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#pragma once + +#include "storage/transaction/ChainAddEdgesProcessorLocal.h" +#include "storage/transaction/ResumeLockGuard.h" + +namespace nebula { +namespace storage { + +class ResumeAddEdgeRemoteProcessor : public ChainAddEdgesProcessorLocal { + public: + static ResumeAddEdgeRemoteProcessor* instance(StorageEnv* env, const std::string& val) { + return new ResumeAddEdgeRemoteProcessor(env, val); + } + + folly::SemiFuture prepareLocal() override; + + folly::SemiFuture processRemote(nebula::cpp2::ErrorCode code) override; + + folly::SemiFuture processLocal(nebula::cpp2::ErrorCode code) override; + + virtual ~ResumeAddEdgeRemoteProcessor() = default; + + protected: + ResumeAddEdgeRemoteProcessor(StorageEnv* env, const std::string& val); + + bool lockEdges(); + + std::unique_ptr resumeLock_; +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ResumeLockGuard.h b/src/storage/transaction/ResumeLockGuard.h new file mode 100644 index 00000000000..12f94fba972 --- /dev/null +++ b/src/storage/transaction/ResumeLockGuard.h @@ -0,0 +1,62 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#pragma once + +#include + +#include "common/utils/MemoryLockCore.h" + +namespace nebula { +namespace storage { + +// RAII style to easily control the lock acquire / release +class ResumeLockGuard { + public: + ResumeLockGuard(TransactionManager* txnMgr, GraphSpaceID spaceId, const std::string& key) + : key_(key) { + lockCore_ = txnMgr->getLockCore(spaceId); + if (txnMgr->takeDanglingEdge(spaceId, key)) { + locked_ = true; + } else { + locked_ = lockCore_->try_lock(key_); + } + } + + ResumeLockGuard(const ResumeLockGuard&) = delete; + + ResumeLockGuard(ResumeLockGuard&& lg) noexcept + : key_(std::move(lg.key_)), lockCore_(lg.lockCore_), locked_(lg.locked_) {} + + ResumeLockGuard& operator=(const ResumeLockGuard&) = delete; + + ResumeLockGuard& operator=(ResumeLockGuard&& lg) noexcept { + if (this != &lg) { + lockCore_ = lg.lockCore_; + key_ = std::move(lg.key_); + locked_ = lg.locked_; + } + return *this; + } + + ~ResumeLockGuard() { + if (locked_) { + lockCore_->unlock(key_); + } + } + + bool isLocked() const noexcept { return locked_; } + + operator bool() const noexcept { return isLocked(); } + + protected: + std::string key_; + MemoryLockCore* lockCore_; + bool locked_{false}; +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ResumeUpdateProcessor.cpp b/src/storage/transaction/ResumeUpdateProcessor.cpp new file mode 100644 index 00000000000..76f8eb81582 --- /dev/null +++ b/src/storage/transaction/ResumeUpdateProcessor.cpp @@ -0,0 +1,66 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#include "storage/transaction/ResumeUpdateProcessor.h" + +#include + +namespace nebula { +namespace storage { + +ResumeUpdateProcessor::ResumeUpdateProcessor(StorageEnv* env, const std::string& val) + : ChainUpdateEdgeProcessorLocal(env) { + req_ = ConsistUtil::parseUpdateRequest(val); + ChainUpdateEdgeProcessorLocal::prepareRequest(req_); +} + +folly::SemiFuture ResumeUpdateProcessor::prepareLocal() { + if (!setLock()) { + LOG(INFO) << "set lock failed, return E_DATA_CONFLICT_ERROR"; + return Code::E_DATA_CONFLICT_ERROR; + } + ver_ = getVersion(req_); + + return Code::SUCCEEDED; +} + +folly::SemiFuture ResumeUpdateProcessor::processRemote(Code code) { + return ChainUpdateEdgeProcessorLocal::processRemote(code); +} + +folly::SemiFuture ResumeUpdateProcessor::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; + } + + if (code == Code::E_RPC_FAILURE) { + appendDoublePrime(); + } + + 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()); + kvErased_.emplace_back(std::move(key)); + forwardToDelegateProcessor(); + return code_; + } + + return code; +} + +void ResumeUpdateProcessor::finish() { onFinished(); } + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ResumeUpdateProcessor.h b/src/storage/transaction/ResumeUpdateProcessor.h new file mode 100644 index 00000000000..ed9e8e0ff33 --- /dev/null +++ b/src/storage/transaction/ResumeUpdateProcessor.h @@ -0,0 +1,45 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#pragma once + +#include "storage/transaction/ChainUpdateEdgeProcessorLocal.h" +#include "storage/transaction/ResumeLockGuard.h" + +namespace nebula { +namespace storage { + +/** + * @brief + * if the TxnManager backgroud resume thread found a prime key + * it will create this processor to resume the complete update process + */ +class ResumeUpdateProcessor : public ChainUpdateEdgeProcessorLocal { + public: + static ResumeUpdateProcessor* instance(StorageEnv* env, const std::string& val) { + return new ResumeUpdateProcessor(env, val); + } + + folly::SemiFuture prepareLocal() override; + + folly::SemiFuture processRemote(nebula::cpp2::ErrorCode code) override; + + folly::SemiFuture processLocal(nebula::cpp2::ErrorCode code) override; + + void finish() override; + + virtual ~ResumeUpdateProcessor() = default; + + protected: + ResumeUpdateProcessor(StorageEnv* env, const std::string& val); + + bool lockEdge(); + + std::unique_ptr resumeLock_; +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ResumeUpdateRemoteProcessor.cpp b/src/storage/transaction/ResumeUpdateRemoteProcessor.cpp new file mode 100644 index 00000000000..8618d0a68c6 --- /dev/null +++ b/src/storage/transaction/ResumeUpdateRemoteProcessor.cpp @@ -0,0 +1,69 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#include "storage/transaction/ResumeUpdateRemoteProcessor.h" + +namespace nebula { +namespace storage { + +ResumeUpdateRemoteProcessor::ResumeUpdateRemoteProcessor(StorageEnv* env, const std::string& val) + : ChainUpdateEdgeProcessorLocal(env) { + req_ = ConsistUtil::parseUpdateRequest(val); + ChainUpdateEdgeProcessorLocal::prepareRequest(req_); +} + +folly::SemiFuture ResumeUpdateRemoteProcessor::prepareLocal() { + if (!lockEdge(req_)) { + LOG(INFO) << "set lock failed, return E_DATA_CONFLICT_ERROR"; + return Code::E_DATA_CONFLICT_ERROR; + } + ver_ = getVersion(req_); + + return Code::SUCCEEDED; +} + +folly::SemiFuture ResumeUpdateRemoteProcessor::processRemote(Code code) { + return ChainUpdateEdgeProcessorLocal::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; + } + + 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()); + kvErased_.emplace_back(std::move(key)); + forwardToDelegateProcessor(); + return code; + } else { + // we can't decide if the double prime shoule be deleted. + // so do nothing + } + + return code; +} + +bool ResumeUpdateRemoteProcessor::lockEdge(const cpp2::UpdateEdgeRequest& req) { + auto key = sEdgeKey(req); + resumeLock_ = std::make_unique(env_->txnMan_, req.get_space_id(), key); + return resumeLock_->isLocked(); +} + +void ResumeUpdateRemoteProcessor::finish() { onFinished(); } + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/ResumeUpdateRemoteProcessor.h b/src/storage/transaction/ResumeUpdateRemoteProcessor.h new file mode 100644 index 00000000000..b1cf9515ff0 --- /dev/null +++ b/src/storage/transaction/ResumeUpdateRemoteProcessor.h @@ -0,0 +1,45 @@ +/* Copyright (c) 2021 vesoft inc. All rights reserved. + * + * This source code is licensed under Apache 2.0 License, + * attached with Common Clause Condition 1.0, found in the LICENSES directory. + */ + +#pragma once + +#include "storage/transaction/ChainUpdateEdgeProcessorLocal.h" +#include "storage/transaction/ResumeLockGuard.h" + +namespace nebula { +namespace storage { + +/** + * @brief + * if the TxnManager backgroud resume thread found a prime key + * it will create this processor to resume the complete update process + */ +class ResumeUpdateRemoteProcessor : public ChainUpdateEdgeProcessorLocal { + public: + static ResumeUpdateRemoteProcessor* instance(StorageEnv* env, const std::string& val) { + return new ResumeUpdateRemoteProcessor(env, val); + } + + folly::SemiFuture prepareLocal() override; + + folly::SemiFuture processRemote(nebula::cpp2::ErrorCode code) override; + + folly::SemiFuture processLocal(nebula::cpp2::ErrorCode code) override; + + void finish() override; + + virtual ~ResumeUpdateRemoteProcessor() = default; + + protected: + ResumeUpdateRemoteProcessor(StorageEnv* env, const std::string& val); + + bool lockEdge(const cpp2::UpdateEdgeRequest& req); + + std::unique_ptr resumeLock_; +}; + +} // namespace storage +} // namespace nebula diff --git a/src/storage/transaction/TossEdgeIterator.h b/src/storage/transaction/TossEdgeIterator.h deleted file mode 100644 index 5c64de13701..00000000000 --- a/src/storage/transaction/TossEdgeIterator.h +++ /dev/null @@ -1,259 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License, - * attached with Common Clause Condition 1.0, found in the LICENSES directory. - */ - -#ifndef STORAGE_TRANSACTION_TOSSEDGEITERATOR_H_ -#define STORAGE_TRANSACTION_TOSSEDGEITERATOR_H_ - -#include -#include - -#include "common/base/Base.h" -#include "kvstore/KVIterator.h" -#include "storage/CommonUtils.h" -#include "storage/StorageFlags.h" -#include "storage/exec/StorageIterator.h" -#include "storage/transaction/TransactionManager.h" - -namespace nebula { -namespace storage { - -class TossEdgeIterator : public SingleEdgeIterator { - public: - TossEdgeIterator(RuntimeContext* context, - std::unique_ptr iter, - EdgeType edgeType, - const std::vector>* schemas, - const folly::Optional>* ttl, - bool stopAtFirstEdge) - : SingleEdgeIterator() { - context_ = context; - iter_ = std::move(iter); - edgeType_ = edgeType; - schemas_ = schemas; - stopAtFirstEdge_ = stopAtFirstEdge; - recoverEdgesIter_ = recoverEdges_.end(); - if (ttl->hasValue()) { - hasTtl_ = true; - ttlCol_ = ttl->value().first; - ttlDuration_ = ttl->value().second; - } - LOG_IF(INFO, FLAGS_trace_toss) - << "TossEdgeIterator::ctor() iter_->key=" << folly::hexlify(iter_->key()); - next(); - } - - folly::StringPiece key() const override { - if (iter_->valid()) { - return iter_->key(); - } else if (recoverEdgesIter_ != recoverEdges_.end()) { - auto rLockedPtr = (*recoverEdgesIter_)->rlock(); - return rLockedPtr->first; - } - return iter_->key(); - } - - bool valid() const override { - if (!iter_->valid() && recoverEdgesIter_ == recoverEdges_.end()) { - LOG_IF(INFO, FLAGS_trace_toss) << "TossEdgeIterator::valid() = false"; - return false; - } - auto ret = (!stopSearching_) && (reader_ != nullptr); - - if (FLAGS_trace_toss) { - std::string key; - std::string val; - if (iter_->valid()) { - key = iter_->key().str(); - val = iter_->val().str(); - } else { - (*recoverEdgesIter_)->withRLock([&](auto&& data) { - key = data.first; - val = data.second; - }); - } - LOG(INFO) << "TossEdgeIterator::valid() = " << ret << ", key=" << folly::hexlify(key) - << ", val=" << val; - } - return ret; - } - - /** - * @brief this iterator has two parts - * 1st. normal scan iterator from begin to end. - * 2nd. return the resume locks\edges. - * - * iter_ will invalid after step1. - */ - void next() override { - LOG_IF(INFO, FLAGS_trace_toss) - << "TossEdgeIterator::next() iter_->key=" << folly::hexlify(iter_->key()) - << ", iter_->valid()=" << iter_->valid(); - if (stopSearching_) { - return; - } - reader_.reset(); - - // Step 1: normal scan iterator from begin to end. - while (iter_->valid()) { - if (!calledByCtor_) { - iter_->next(); - } - SCOPE_EXIT { calledByCtor_ = false; }; - if (!iter_->valid()) { - break; - } - if (isEdge(iter_->key())) { - LOG_IF(INFO, FLAGS_trace_toss) - << "TossEdgeIterator::next(), found an edge, hex=" - << TransactionUtils::hexEdgeId(context_->vIdLen(), iter_->key()); - - if (stopAtFirstEdge_ && !calledByCtor_) { - stopSearching_ = true; - } - if (isLatestEdge(iter_->key()) && setReader(iter_->val())) { - lastRank_ = NebulaKeyUtils::getRank(context_->vIdLen(), iter_->key()); - lastDstId_ = NebulaKeyUtils::getDstId(context_->vIdLen(), iter_->key()).str(); - lastIsLock_ = false; - LOG_IF(INFO, FLAGS_trace_toss) - << "TossEdgeIterator::next(), return edge hex=" - << TransactionUtils::hexEdgeId(context_->vIdLen(), iter_->key()); - return; - } else { - LOG_IF(INFO, FLAGS_trace_toss) - << "edge, hex=" << TransactionUtils::hexEdgeId(context_->vIdLen(), iter_->key()) - << ", is not latest"; - } - - /** - * if meet a lock before, will delay this edge return. - */ - if (lastIsLock_) { - LOG_IF(INFO, FLAGS_trace_toss) - << "TossEdgeIterator::next(), prev is a lock, hexEdgeId=" - << TransactionUtils::hexEdgeId(context_->vIdLen(), iter_->key()) - << ", hex=" << folly::hexlify(iter_->key()); - auto tryLockData = recoverEdges_.back()->tryWLock(); - if (tryLockData && tryLockData->first.empty()) { - LOG_IF(INFO, FLAGS_trace_toss) - << "set edge val for key=" << folly::hexlify(iter_->key()); - tryLockData->first = iter_->key().str(); - tryLockData->second = iter_->val().str(); - } - lastIsLock_ = false; - continue; - } - } else if (NebulaKeyUtils::isLock(context_->vIdLen(), iter_->key())) { - LOG_IF(INFO, FLAGS_trace_toss) - << "TossEdgeIterator::next() found a lock, hex=" - << TransactionUtils::hexEdgeId(context_->vIdLen(), iter_->key()); - std::string rawKey = NebulaKeyUtils::toEdgeKey(iter_->key()); - auto rank = NebulaKeyUtils::getRank(context_->vIdLen(), rawKey); - auto dstId = NebulaKeyUtils::getDstId(context_->vIdLen(), rawKey).str(); - if (!lastIsLock_ && rank == lastRank_ && dstId == lastDstId_) { - continue; - } - if (rank != lastRank_ || dstId != lastDstId_) { - recoverEdges_.emplace_back( - std::make_shared>(std::make_pair("", ""))); - } - - resumeTasks_.emplace_back(context_->env()->txnMan_->resumeTransaction( - context_->vIdLen(), context_->spaceId(), iter_->key().str(), recoverEdges_.back())); - lastRank_ = NebulaKeyUtils::getRank(context_->vIdLen(), rawKey); - lastDstId_ = NebulaKeyUtils::getDstId(context_->vIdLen(), rawKey).str(); - lastIsLock_ = true; - continue; - } else { - LOG_IF(INFO, FLAGS_trace_toss) << "next to a weird record: \n" - << folly::hexDump(iter_->key().data(), iter_->key().size()) - << folly::hexDump(iter_->val().data(), iter_->val().size()); - } - } // end while(iter_->valid()) - LOG_IF(INFO, FLAGS_trace_toss) << "next(), no more iter to read"; - - // Step 2: return the resumed locks/edges. - // set recoverEdgesIter_ as begin() at first time. else ++recoverEdgesIter_ - if (needWaitResumeTask_) { - LOG_IF(INFO, FLAGS_trace_toss) << "next(), waiting resume finished"; - folly::collectAll(std::move(resumeTasks_)).wait(); - needWaitResumeTask_ = false; - recoverEdgesIter_ = recoverEdges_.begin(); - } else { - recoverEdgesIter_++; - } - - while (recoverEdgesIter_ != recoverEdges_.end()) { - auto data = (*recoverEdgesIter_)->copy(); - if (!data.second.empty()) { - if (setReader(data.second)) { - LOG_IF(INFO, FLAGS_trace_toss) << "setReader succeed, break"; - break; - } else { - LOG_IF(INFO, FLAGS_trace_toss) << "setReader failed, continue"; - } - } else { - LOG_IF(INFO, FLAGS_trace_toss) << "invalid lock, data.second.empty(), continue"; - } - recoverEdgesIter_++; - } - LOG_IF(INFO, FLAGS_trace_toss) << "next(), exit"; - } - - bool isEdge(const folly::StringPiece& key) { - return NebulaKeyUtils::isEdge(context_->vIdLen(), key); - } - - bool isLatestEdge(const folly::StringPiece& key) { - auto rank = NebulaKeyUtils::getRank(context_->vIdLen(), key); - auto dstId = NebulaKeyUtils::getDstId(context_->vIdLen(), key); - return !(rank == lastRank_ && dstId == lastDstId_); - } - - bool setReader(folly::StringPiece val) { - reader_.reset(); - if (!reader_) { - reader_.reset(*schemas_, val); - if (!reader_) { - context_->resultStat_ = ResultStatus::ILLEGAL_DATA; - return false; - } - } else if (!reader_->reset(*schemas_, val)) { - context_->resultStat_ = ResultStatus::ILLEGAL_DATA; - return false; - } - - if (hasTtl_ && CommonUtils::checkDataExpiredForTTL( - schemas_->back().get(), reader_.get(), ttlCol_, ttlDuration_)) { - reader_.reset(); - return false; - } - return true; - } - - private: - // using shared_ptr because this Iterator may be deleted if there is a limit - // in nGQL using folly::Synchronized because the scan thread and the resume - // thread may write same ptr - using TResultsItem = std::shared_ptr>; - bool needWaitResumeTask_{true}; - bool lastIsLock_{false}; - /** - * getNeighbors need to scan all edges, but update will stop at first edge, - * use stopAtFirstEdge_ to let caller tell if this need to stop early - * use stopSearching_ to judge inside. - */ - bool stopSearching_{false}; - bool stopAtFirstEdge_{false}; - bool calledByCtor_{true}; - std::list> resumeTasks_; - std::list recoverEdges_; - std::list::iterator recoverEdgesIter_; -}; - -} // namespace storage -} // namespace nebula - -#endif // STORAGE_TRANSACTION_TOSSEDGEITERATOR_H_ diff --git a/src/storage/transaction/TransactionManager.cpp b/src/storage/transaction/TransactionManager.cpp index 8edb19f6aaa..4f685db9652 100644 --- a/src/storage/transaction/TransactionManager.cpp +++ b/src/storage/transaction/TransactionManager.cpp @@ -8,414 +8,101 @@ #include -#include "clients/storage/InternalStorageClient.h" #include "codec/RowWriterV2.h" #include "common/utils/NebulaKeyUtils.h" #include "storage/CommonUtils.h" -#include "storage/StorageFlags.h" -#include "storage/mutate/AddEdgesProcessor.h" -#include "storage/transaction/TransactionUtils.h" +#include "storage/transaction/ChainResumeProcessor.h" namespace nebula { namespace storage { -/* - * edgeKey : thrift data structure - * rawKey : NebulaKeyUtils::edgeKey - * lockKey : rawKey + lock suffix - * */ +DEFINE_int32(resume_interval_secs, 10, "Resume interval"); + +ProcessorCounters kForwardTranxCounters; + TransactionManager::TransactionManager(StorageEnv* env) : env_(env) { exec_ = std::make_shared(10); - interClient_ = std::make_unique(exec_, env_->metaClient_); + iClient_ = env_->interClient_; + resumeThread_ = std::make_unique(); } -/* - * multi edges have same local partition and same remote partition - * will process as a batch - * *Important**Important**Important* - * normally, ver will be 0 - * */ -folly::Future TransactionManager::addSamePartEdges( - size_t vIdLen, - GraphSpaceID spaceId, - PartitionID localPart, - PartitionID remotePart, - std::vector& localEdges, - AddEdgesProcessor* processor, - folly::Optional optBatchGetter) { - int64_t txnId = TransactionUtils::getSnowFlakeUUID(); - if (FLAGS_trace_toss) { - for (auto& kv : localEdges) { - auto srcId = NebulaKeyUtils::getSrcId(vIdLen, kv.first); - auto dstId = NebulaKeyUtils::getDstId(vIdLen, kv.first); - LOG(INFO) << "begin txn hexSrcDst=" << folly::hexlify(srcId) << folly::hexlify(dstId) - << ", txnId=" << txnId; - } - } - // steps 1: lock edges in memory - bool setMemoryLock = true; - std::for_each(localEdges.begin(), localEdges.end(), [&](auto& kv) { - auto keyWoVer = NebulaKeyUtils::keyWithNoVersion(kv.first).str(); - if (!memLock_.insert(std::make_pair(keyWoVer, txnId)).second) { - setMemoryLock = false; - } - }); - - auto cleanup = [=] { - for (auto& kv : localEdges) { - auto keyWoVer = NebulaKeyUtils::keyWithNoVersion(kv.first).str(); - auto cit = memLock_.find(keyWoVer); - if (cit != memLock_.end() && cit->second == txnId) { - memLock_.erase(keyWoVer); - } - } - }; - - if (!setMemoryLock) { - LOG(ERROR) << "set memory lock failed, txnId=" << txnId; - cleanup(); - return folly::makeFuture(nebula::cpp2::ErrorCode::E_MUTATE_EDGE_CONFLICT); - } else { - LOG_IF(INFO, FLAGS_trace_toss) << "set memory lock succeeded, txnId=" << txnId; +TransactionManager::LockCore* TransactionManager::getLockCore(GraphSpaceID spaceId) { + auto it = memLocks_.find(spaceId); + if (it != memLocks_.end()) { + return it->second.get(); } - // steps 2: batch commit persist locks - std::string batch; - std::vector lockData = localEdges; - // insert don't have BatchGetter - if (!optBatchGetter) { - // insert don't have batch Getter - auto addEdgeErrorCode = nebula::cpp2::ErrorCode::SUCCEEDED; - std::transform(lockData.begin(), lockData.end(), lockData.begin(), [&](auto& kv) { - if (processor) { - processor->spaceId_ = spaceId; - processor->spaceVidLen_ = vIdLen; - std::vector data{std::make_pair(kv.first, kv.second)}; - auto optVal = processor->addEdges(localPart, data); - if (nebula::ok(optVal)) { - return std::make_pair(NebulaKeyUtils::toLockKey(kv.first), nebula::value(optVal)); - } else { - addEdgeErrorCode = nebula::cpp2::ErrorCode::E_ATOMIC_OP_FAILED; - return std::make_pair(NebulaKeyUtils::toLockKey(kv.first), std::string("")); - } - } else { - std::vector data{std::make_pair(kv.first, kv.second)}; - return std::make_pair(NebulaKeyUtils::toLockKey(kv.first), encodeBatch(std::move(data))); - } - }); - if (addEdgeErrorCode != nebula::cpp2::ErrorCode::SUCCEEDED) { - cleanup(); - return addEdgeErrorCode; - } - auto lockDataSink = lockData; - batch = encodeBatch(std::move(lockDataSink)); - } else { // only update should enter here - auto optBatch = (*optBatchGetter)(); - if (!optBatch) { - cleanup(); - return nebula::cpp2::ErrorCode::E_ATOMIC_OP_FAILED; - } - lockData.back().first = NebulaKeyUtils::toLockKey(localEdges.back().first); - batch = *optBatch; - auto decodeKV = kvstore::decodeBatchValue(batch); - localEdges.back().first = decodeKV.back().second.first.str(); - localEdges.back().second = decodeKV.back().second.second.str(); - - lockData.back().first = localEdges.back().first; - lockData.back().second = batch; - } - - auto c = folly::makePromiseContract(); - commitBatch(spaceId, localPart, std::move(batch)) - .via(exec_.get()) - .thenTry([=, p = std::move(c.first)](auto&& t) mutable { - auto code = nebula::cpp2::ErrorCode::SUCCEEDED; - if (!t.hasValue()) { - LOG(INFO) << "commitBatch throw ex=" << t.exception() << ", txnId=" << txnId; - code = nebula::cpp2::ErrorCode::E_UNKNOWN; - } else if (t.value() != nebula::cpp2::ErrorCode::SUCCEEDED) { - code = t.value(); - } - if (code != nebula::cpp2::ErrorCode::SUCCEEDED) { - LOG(INFO) << folly::sformat("commitBatch for ({},{},{}) failed, code={}, txnId={}", - spaceId, - localPart, - remotePart, - static_cast(code), - txnId); - p.setValue(code); - return; - } - - std::vector remoteEdges = localEdges; - std::transform(remoteEdges.begin(), remoteEdges.end(), remoteEdges.begin(), [=](auto& kv) { - auto key = TransactionUtils::reverseRawKey(vIdLen, remotePart, kv.first); - return std::make_pair(std::move(key), kv.second); - }); - auto remoteBatch = encodeBatch(std::move(remoteEdges)); - - // steps 3: multi put remote edges - LOG_IF(INFO, FLAGS_trace_toss) << "begin forwardTransaction, txnId=" << txnId; - interClient_->forwardTransaction(txnId, spaceId, remotePart, std::move(remoteBatch)) - .via(exec_.get()) - .thenTry([=, p = std::move(p)](auto&& _t) mutable { - auto _code = _t.hasValue() ? _t.value() : nebula::cpp2::ErrorCode::E_UNKNOWN; - LOG_IF(INFO, FLAGS_trace_toss) << folly::sformat( - "end forwardTransaction: txnId={}, spaceId={}, partId={}, " - "code={}", - txnId, - spaceId, - remotePart, - static_cast(_code)); - if (_code != nebula::cpp2::ErrorCode::SUCCEEDED) { - p.setValue(_code); - return; - } - - if (FLAGS_trace_toss) { - for (auto& kv : localEdges) { - LOG(INFO) << "key=" << folly::hexlify(kv.first) << ", txnId=" << txnId; - } - } - - // steps 4 & 5: multi put local edges & multi remove persist locks - kvstore::BatchHolder bat; - for (auto& lock : lockData) { - LOG_IF(INFO, FLAGS_trace_toss) - << "remove lock, hex=" << folly::hexlify(lock.first) << ", txnId=" << txnId; - bat.remove(std::move(lock.first)); - auto operations = kvstore::decodeBatchValue(lock.second); - for (auto& op : operations) { - auto opType = op.first; - auto& kv = op.second; - LOG_IF(INFO, FLAGS_trace_toss) - << "bat op=" << static_cast(opType) - << ", hex=" << folly::hexlify(kv.first) << ", txnId=" << txnId; - switch (opType) { - case kvstore::BatchLogType::OP_BATCH_PUT: - bat.put(kv.first.str(), kv.second.str()); - break; - case kvstore::BatchLogType::OP_BATCH_REMOVE: - bat.remove(kv.first.str()); - break; - default: - LOG(ERROR) << "unexpected opType: " << static_cast(opType); - } - } - } - auto _batch = kvstore::encodeBatchValue(bat.getBatch()); - commitBatch(spaceId, localPart, std::move(_batch)) - .via(exec_.get()) - .thenValue([=, p = std::move(p)](auto&& rc) mutable { - auto commitBatchCode = rc; - LOG_IF(INFO, FLAGS_trace_toss) - << "txnId=" << txnId - << " finished, code=" << static_cast(commitBatchCode); - p.setValue(commitBatchCode); - }); - }); - }) - .ensure([=]() { cleanup(); }); - return std::move(c.second).via(exec_.get()); + auto item = memLocks_.insert(spaceId, std::make_unique()); + return item.first->second.get(); } -folly::Future TransactionManager::updateEdgeAtomic( - size_t vIdLen, - GraphSpaceID spaceId, - PartitionID partId, - const cpp2::EdgeKey& edgeKey, - GetBatchFunc batchGetter) { - auto remotePart = env_->metaClient_->partId(spaceId, (*edgeKey.dst_ref()).getStr()); - auto localKey = TransactionUtils::edgeKey(vIdLen, partId, edgeKey); - - std::vector data{std::make_pair(localKey, "")}; - return addSamePartEdges(vIdLen, spaceId, partId, remotePart, data, nullptr, batchGetter); +StatusOr TransactionManager::getTerm(GraphSpaceID spaceId, PartitionID partId) { + return env_->metaClient_->getTermFromCache(spaceId, partId); } -folly::Future TransactionManager::resumeTransaction(size_t vIdLen, - GraphSpaceID spaceId, - std::string lockKey, - ResumedResult result) { - LOG_IF(INFO, FLAGS_trace_toss) << "begin resume txn from lock=" << folly::hexlify(lockKey); - // 1st, set memory lock - auto localKey = NebulaKeyUtils::toEdgeKey(lockKey); - CHECK(NebulaKeyUtils::isEdge(vIdLen, localKey)); - // int64_t ver = NebulaKeyUtils::getVersion(vIdLen, localKey); - auto ver = NebulaKeyUtils::getLockVersion(lockKey); - auto keyWoVer = NebulaKeyUtils::keyWithNoVersion(localKey).str(); - if (!memLock_.insert(std::make_pair(keyWoVer, ver)).second) { - return folly::makeFuture(nebula::cpp2::ErrorCode::E_MUTATE_EDGE_CONFLICT); +bool TransactionManager::checkTerm(GraphSpaceID spaceId, PartitionID partId, TermID term) { + auto termOfMeta = env_->metaClient_->getTermFromCache(spaceId, partId); + if (termOfMeta.ok()) { + if (term < termOfMeta.value()) { + LOG(WARNING) << "checkTerm() failed: " + << "spaceId=" << spaceId << ", partId=" << partId << ", expect term=" << term + << ", actual term=" << termOfMeta.value(); + return false; + } } - - auto localPart = NebulaKeyUtils::getPart(localKey); - // 2nd, get values from remote in-edge - auto spPromiseVal = std::make_shared(nebula::cpp2::ErrorCode::SUCCEEDED); - auto c = folly::makePromiseContract(); - - auto dst = NebulaKeyUtils::getDstId(vIdLen, localKey); - auto remotePartId = env_->metaClient_->partId(spaceId, dst.str()); - auto remoteKey = TransactionUtils::reverseRawKey(vIdLen, remotePartId, localKey); - - LOG_IF(INFO, FLAGS_trace_toss) << "try to get remote key=" << folly::hexlify(remoteKey) - << ", according to lock=" << folly::hexlify(lockKey); - interClient_->getValue(vIdLen, spaceId, remoteKey) - .via(exec_.get()) - .thenValue([=](auto&& errOrVal) mutable { - if (!nebula::ok(errOrVal)) { - LOG_IF(INFO, FLAGS_trace_toss) - << "get remote key failed, lock=" << folly::hexlify(lockKey); - *spPromiseVal = nebula::error(errOrVal); - return; - } - auto lockedPtr = result->wlock(); - if (!lockedPtr) { - *spPromiseVal = nebula::cpp2::ErrorCode::E_MUTATE_EDGE_CONFLICT; - return; - } - auto& kv = *lockedPtr; - kv.first = localKey; - kv.second = nebula::value(errOrVal); - LOG_IF(INFO, FLAGS_trace_toss) - << "got value=[" << kv.second << "] from remote key=" << folly::hexlify(remoteKey) - << ", according to lock=" << folly::hexlify(lockKey); - // 3rd, commit local key(indexes) - /* - * if mvcc enabled, get value will get exactly the same ver in-edge - * against lock which means we can trust the val of lock as the out-edge - * else, don't trust lock. - * */ - commitEdgeOut(spaceId, localPart, std::string(kv.first), std::string(kv.second)) - .via(exec_.get()) - .thenValue([=](auto&& rc) { *spPromiseVal = rc; }) - .thenError([=](auto&&) { *spPromiseVal = nebula::cpp2::ErrorCode::E_UNKNOWN; }); - }) - .thenValue([=](auto&&) { - // 4th, remove persist lock - LOG_IF(INFO, FLAGS_trace_toss) << "erase lock " << folly::hexlify(lockKey) - << ", *spPromiseVal=" << static_cast(*spPromiseVal); - if (*spPromiseVal == nebula::cpp2::ErrorCode::SUCCEEDED || - *spPromiseVal == nebula::cpp2::ErrorCode::E_KEY_NOT_FOUND || - *spPromiseVal == nebula::cpp2::ErrorCode::E_OUTDATED_LOCK) { - auto eraseRet = eraseKey(spaceId, localPart, lockKey); - eraseRet.wait(); - auto code = eraseRet.hasValue() ? eraseRet.value() : nebula::cpp2::ErrorCode::E_UNKNOWN; - *spPromiseVal = code; - } - }) - .thenError([=](auto&& ex) { - LOG(ERROR) << ex.what(); - *spPromiseVal = nebula::cpp2::ErrorCode::E_UNKNOWN; - }) - .ensure([=, p = std::move(c.first)]() mutable { - eraseMemoryLock(localKey, ver); - LOG_IF(INFO, FLAGS_trace_toss) - << "end resume *spPromiseVal=" << static_cast(*spPromiseVal); - p.setValue(*spPromiseVal); - }); - return std::move(c.second).via(exec_.get()); + auto partUUID = std::make_pair(spaceId, partId); + auto it = cachedTerms_.find(partUUID); + if (it != cachedTerms_.cend()) { + if (term < it->second) { + LOG(WARNING) << "term < it->second"; + return false; + } + } + cachedTerms_.assign(partUUID, term); + return true; } -// combine multi put and remove in a batch -// this may sometimes reduce some raft operation -folly::SemiFuture TransactionManager::commitBatch(GraphSpaceID spaceId, - PartitionID partId, - std::string&& batch) { - auto c = folly::makePromiseContract(); - env_->kvstore_->asyncAppendBatch( - spaceId, - partId, - std::move(batch), - [pro = std::move(c.first)](nebula::cpp2::ErrorCode rc) mutable { pro.setValue(rc); }); - return std::move(c.second); +void TransactionManager::resumeThread() { + SCOPE_EXIT { + resumeThread_->addDelayTask( + FLAGS_resume_interval_secs * 1000, &TransactionManager::resumeThread, this); + }; + ChainResumeProcessor proc(env_); + proc.process(); } -/* - * 1. use rawKey without version as in-memory lock key - * */ -folly::SemiFuture TransactionManager::commitEdgeOut(GraphSpaceID spaceId, - PartitionID partId, - std::string&& key, - std::string&& props) { - std::vector> indexes; - auto idxRet = env_->indexMan_->getEdgeIndexes(spaceId); - if (idxRet.ok()) { - indexes = std::move(idxRet).value(); - } - if (!indexes.empty()) { - std::vector data{{std::move(key), std::move(props)}}; - - auto c = folly::makePromiseContract(); - - auto atomic = [partId, edges = std::move(data), this]() -> folly::Optional { - auto* processor = AddEdgesProcessor::instance(env_); - auto ret = processor->addEdges(partId, edges); - if (nebula::ok(ret)) { - return nebula::value(ret); - } else { - return folly::Optional(); - } - }; - - auto cb = [pro = std::move(c.first)](nebula::cpp2::ErrorCode rc) mutable { pro.setValue(rc); }; - - env_->kvstore_->asyncAtomicOp(spaceId, partId, atomic, std::move(cb)); - return std::move(c.second); +bool TransactionManager::start() { + if (!resumeThread_->start()) { + LOG(ERROR) << "resume thread start failed"; + return false; } - return commitEdge(spaceId, partId, key, props); + resumeThread_->addDelayTask( + FLAGS_resume_interval_secs * 1000, &TransactionManager::resumeThread, this); + return true; } -folly::SemiFuture TransactionManager::commitEdge(GraphSpaceID spaceId, - PartitionID partId, - std::string& key, - std::string& props) { - std::vector data; - data.emplace_back(std::move(key), std::move(props)); - - auto c = folly::makePromiseContract(); - env_->kvstore_->asyncMultiPut( - spaceId, - partId, - std::move(data), - [pro = std::move(c.first)](nebula::cpp2::ErrorCode rc) mutable { pro.setValue(rc); }); - return std::move(c.second); +void TransactionManager::stop() { + resumeThread_->stop(); + resumeThread_->wait(); } -folly::SemiFuture TransactionManager::eraseKey(GraphSpaceID spaceId, - PartitionID partId, - const std::string& key) { - LOG_IF(INFO, FLAGS_trace_toss) << "eraseKey: " << folly::hexlify(key); - auto c = folly::makePromiseContract(); - env_->kvstore_->asyncRemove( - spaceId, partId, key, [pro = std::move(c.first)](nebula::cpp2::ErrorCode code) mutable { - LOG_IF(INFO, FLAGS_trace_toss) << "asyncRemove code=" << static_cast(code); - pro.setValue(code); - }); - return std::move(c.second); +std::unique_ptr TransactionManager::tryLock(GraphSpaceID spaceId, + folly::StringPiece key) { + return std::make_unique(getLockCore(spaceId), key.str()); } -void TransactionManager::eraseMemoryLock(const std::string& rawKey, int64_t ver) { - // auto lockKey = TransactionUtils::lockKeyWithoutVer(rawKey); - auto lockKey = NebulaKeyUtils::keyWithNoVersion(rawKey).str(); - auto cit = memLock_.find(lockKey); - if (cit != memLock_.end() && cit->second == ver) { - memLock_.erase(lockKey); - } -} - -meta::cpp2::IsolationLevel TransactionManager::getSpaceIsolationLvel(GraphSpaceID spaceId) { - auto ret = env_->metaClient_->getIsolationLevel(spaceId); - if (!ret.ok()) { - return meta::cpp2::IsolationLevel::DEFAULT; - } - return ret.value(); +void TransactionManager::markDanglingEdge(GraphSpaceID spaceId, const std::string& edge) { + auto key = std::to_string(spaceId) + edge; + danglingEdges_.insert(std::make_pair(key, 0)); } -std::string TransactionManager::encodeBatch(std::vector&& data) { - kvstore::BatchHolder bat; - for (auto& kv : data) { - bat.put(std::move(kv.first), std::move(kv.second)); +bool TransactionManager::takeDanglingEdge(GraphSpaceID spaceId, const std::string& edge) { + auto key = std::to_string(spaceId) + edge; + if (danglingEdges_.find(key) == danglingEdges_.cend()) { + return false; } - return encodeBatchValue(bat.getBatch()); + danglingEdges_.erase(key); + return true; } } // namespace storage diff --git a/src/storage/transaction/TransactionManager.h b/src/storage/transaction/TransactionManager.h index ad8f557ae8d..4363a5cab59 100644 --- a/src/storage/transaction/TransactionManager.h +++ b/src/storage/transaction/TransactionManager.h @@ -4,128 +4,93 @@ * attached with Common Clause Condition 1.0, found in the LICENSES directory. */ -#ifndef STORAGE_TRANSACTION_TRANSACTIONMGR_H_ -#define STORAGE_TRANSACTION_TRANSACTIONMGR_H_ +#pragma once -#include -#include -#include +#include +#include #include "clients/meta/MetaClient.h" -#include "clients/storage/GraphStorageClient.h" #include "clients/storage/InternalStorageClient.h" #include "common/meta/SchemaManager.h" #include "common/thrift/ThriftTypes.h" -#include "common/utils/NebulaKeyUtils.h" +#include "common/utils/MemoryLockCore.h" +#include "common/utils/MemoryLockWrapper.h" #include "interface/gen-cpp2/storage_types.h" #include "kvstore/KVStore.h" -#include "storage/mutate/AddEdgesProcessor.h" -#include "storage/transaction/TransactionUtils.h" +#include "storage/CommonUtils.h" namespace nebula { namespace storage { - -using KV = std::pair; -using RawKeys = std::vector; -using MemEdgeLocks = folly::ConcurrentHashMap; -using ResumedResult = std::shared_ptr>; -using GetBatchFunc = std::function()>; - class TransactionManager { + public: + FRIEND_TEST(ChainUpdateEdgeTest, updateTest1); + friend class FakeInternalStorageClient; + using LockGuard = MemoryLockGuard; + using LockCore = MemoryLockCore; + using UPtrLock = std::unique_ptr; + public: explicit TransactionManager(storage::StorageEnv* env); ~TransactionManager() = default; - /** - * @brief edges have same localPart and remotePart will share - * one signle RPC request - * @param localEdges - * . - * @param processor - * will set this if edge have index - * @param optBatchGetter - * get a batch of raft operations. - * used by updateNode, need to run this func after edge locked - * */ - folly::Future addSamePartEdges( - size_t vIdLen, - GraphSpaceID spaceId, - PartitionID localPart, - PartitionID remotePart, - std::vector& localEdges, - AddEdgesProcessor* processor = nullptr, - folly::Optional optBatchGetter = folly::none); + void addChainTask(ChainBaseProcessor* proc) { + folly::async([=] { + proc->prepareLocal() + .via(exec_.get()) + .thenValue([=](auto&& code) { return proc->processRemote(code); }) + .thenValue([=](auto&& code) { return proc->processLocal(code); }) + .ensure([=]() { proc->finish(); }); + }); + } - /** - * @brief update out-edge first, then in-edge - * @param batchGetter - * need to update index & edge together, and exactly the same verion - * which means we need a lock before doing anything. - * as this method will call addSamePartEdges(), - * and addSamePartEdges() will set a lock to edge, - * I would like to forward this function to addSamePartEdges() - * */ - folly::Future updateEdgeAtomic(size_t vIdLen, - GraphSpaceID spaceId, - PartitionID partId, - const cpp2::EdgeKey& edgeKey, - GetBatchFunc batchGetter); - - /* - * resume an unfinished add/update/upsert request - * 1. if in-edge commited, will commit out-edge - * else, will remove lock - * 2. if mvcc enabled, will commit the value of lock - * else, get props from in-edge, then re-check index and commit - * */ - folly::Future resumeTransaction(size_t vIdLen, - GraphSpaceID spaceId, - std::string lockKey, - ResumedResult result = nullptr); + folly::Executor* getExecutor() { return exec_.get(); } - folly::SemiFuture commitBatch(GraphSpaceID spaceId, - PartitionID partId, - std::string&& batch); + LockCore* getLockCore(GraphSpaceID spaceId); - bool enableToss(GraphSpaceID spaceId) { - return nebula::meta::cpp2::IsolationLevel::TOSS == getSpaceIsolationLvel(spaceId); - } + InternalStorageClient* getInternalClient() { return iClient_; } - folly::Executor* getExecutor() { return exec_.get(); } + std::unique_ptr tryLock(GraphSpaceID spaceId, + folly::StringPiece key); - // used for perf trace, will remove finally - std::unordered_map> timer_; + StatusOr getTerm(GraphSpaceID spaceId, PartitionID partId); - protected: - folly::SemiFuture commitEdgeOut(GraphSpaceID spaceId, - PartitionID partId, - std::string&& key, - std::string&& props); + bool checkTerm(GraphSpaceID spaceId, PartitionID partId, TermID term); - folly::SemiFuture commitEdge(GraphSpaceID spaceId, - PartitionID partId, - std::string& key, - std::string& encodedProp); + bool start(); - folly::SemiFuture eraseKey(GraphSpaceID spaceId, - PartitionID partId, - const std::string& key); + void stop(); - void eraseMemoryLock(const std::string& rawKey, int64_t ver); + // leave a record for prime edge, to let resume processor there is one dangling edge + void markDanglingEdge(GraphSpaceID spaceId, const std::string& edge); - nebula::meta::cpp2::IsolationLevel getSpaceIsolationLvel(GraphSpaceID spaceId); + // return false if there is no "edge" in danglingEdges_ + // true if there is, and also erase the edge from danglingEdges_. + bool takeDanglingEdge(GraphSpaceID spaceId, const std::string& edge); - std::string encodeBatch(std::vector&& data); + protected: + void resumeThread(); protected: + using PartUUID = std::pair; + using SEdgeKey = std::string; + StorageEnv* env_{nullptr}; std::shared_ptr exec_; - std::unique_ptr interClient_; - MemEdgeLocks memLock_; + InternalStorageClient* iClient_; + folly::ConcurrentHashMap memLocks_; + folly::ConcurrentHashMap cachedTerms_; + 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 + * */ + folly::ConcurrentHashMap danglingEdges_; }; } // namespace storage } // namespace nebula - -#endif // STORAGE_TRANSACTION_TRANSACTIONMGR_H_ diff --git a/src/storage/transaction/TransactionProcessor.cpp b/src/storage/transaction/TransactionProcessor.cpp deleted file mode 100644 index e42bbf12b0c..00000000000 --- a/src/storage/transaction/TransactionProcessor.cpp +++ /dev/null @@ -1,57 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License, - * attached with Common Clause Condition 1.0, found in the LICENSES directory. - */ - -#include "storage/transaction/TransactionProcessor.h" - -#include "storage/transaction/TransactionManager.h" -#include "storage/transaction/TransactionUtils.h" - -namespace nebula { -namespace storage { - -ProcessorCounters kForwardTranxCounters; - -void InterTxnProcessor::process(const cpp2::InternalTxnRequest& req) { - int64_t txnId = req.get_txn_id(); - auto spaceId = req.get_space_id(); - auto partId = req.get_part_id(); - - LOG_IF(INFO, FLAGS_trace_toss) << "process req, txnId=" << txnId << ", spaceId=" << spaceId - << ", partId=" << partId; - auto data = req.get_data()[req.get_position()].back(); - - env_->txnMan_->commitBatch(spaceId, partId, std::move(data)) - .via(env_->txnMan_->getExecutor()) - .thenValue([=](nebula::cpp2::ErrorCode rc) { - LOG_IF(INFO, FLAGS_trace_toss) - << "txnId=" << txnId << " commitBatch ret rc=" << static_cast(rc); - auto code = rc; - if (code == nebula::cpp2::ErrorCode::E_LEADER_CHANGED) { - handleLeaderChanged(spaceId, partId); - } else if (code != nebula::cpp2::ErrorCode::SUCCEEDED) { - pushResultCode(code, partId); - } - for (auto& p : codes_) { - if (p.leader_ref().has_value()) { - LOG(INFO) << "txnId=" << txnId << ", part=" << *p.part_id_ref() - << ", code=" << static_cast(*p.code_ref()) - << ", leader=" << *p.leader_ref(); - } else { - LOG(INFO) << "txnId=" << txnId << ", part=" << *p.part_id_ref() - << ", code=" << static_cast(*p.code_ref()); - } - } - onFinished(); - }) - .thenError([&](auto&& ex) { - LOG(ERROR) << "txnId=" << txnId << ", " << ex.what(); - pushResultCode(nebula::cpp2::ErrorCode::E_UNKNOWN, partId); - onFinished(); - }); -} - -} // namespace storage -} // namespace nebula diff --git a/src/storage/transaction/TransactionProcessor.h b/src/storage/transaction/TransactionProcessor.h deleted file mode 100644 index 2b1046ff5bd..00000000000 --- a/src/storage/transaction/TransactionProcessor.h +++ /dev/null @@ -1,39 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License, - * attached with Common Clause Condition 1.0, found in the LICENSES directory. - */ - -#ifndef STORAGE_TRANSACTION_TRANSACTIONPROCESSOR_H_ -#define STORAGE_TRANSACTION_TRANSACTIONPROCESSOR_H_ - -#include - -#include "common/base/Base.h" -#include "kvstore/LogEncoder.h" -#include "storage/BaseProcessor.h" -#include "storage/StorageFlags.h" - -namespace nebula { -namespace storage { - -extern ProcessorCounters kForwardTranxCounters; - -class InterTxnProcessor : public BaseProcessor { - public: - static InterTxnProcessor* instance(StorageEnv* env, - const ProcessorCounters* counters = &kForwardTranxCounters) { - return new InterTxnProcessor(env, counters); - } - - void process(const cpp2::InternalTxnRequest& req); - - private: - InterTxnProcessor(StorageEnv* env, const ProcessorCounters* counters) - : BaseProcessor(env, counters) {} -}; - -} // namespace storage -} // namespace nebula - -#endif // STORAGE_TRANSACTION_TRANSACTIONPROCESSOR_H_ diff --git a/src/storage/transaction/TransactionUtils.cpp b/src/storage/transaction/TransactionUtils.cpp deleted file mode 100644 index 4e6df13b3c4..00000000000 --- a/src/storage/transaction/TransactionUtils.cpp +++ /dev/null @@ -1,52 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License, - * attached with Common Clause Condition 1.0, found in the LICENSES directory. - */ - -#include "storage/transaction/TransactionUtils.h" - -#include "common/time/WallClock.h" -#include "common/utils/NebulaKeyUtils.h" - -namespace nebula { -namespace storage { - -std::string TransactionUtils::dumpKey(const cpp2::EdgeKey& key) { - return folly::sformat( - "dumpKey(): src={}, dst={}", (*key.src_ref()).toString(), (*key.dst_ref()).toString()); -} - -std::string TransactionUtils::edgeKey(size_t vIdLen, PartitionID partId, const cpp2::EdgeKey& key) { - return NebulaKeyUtils::edgeKey(vIdLen, - partId, - (*key.src_ref()).getStr(), - *key.edge_type_ref(), - *key.ranking_ref(), - (*key.dst_ref()).getStr()); -} - -std::string TransactionUtils::reverseRawKey(size_t vIdLen, - PartitionID partId, - const std::string& rawKey) { - return NebulaKeyUtils::edgeKey(vIdLen, - partId, - NebulaKeyUtils::getDstId(vIdLen, rawKey).str(), - 0 - NebulaKeyUtils::getEdgeType(vIdLen, rawKey), - NebulaKeyUtils::getRank(vIdLen, rawKey), - NebulaKeyUtils::getSrcId(vIdLen, rawKey).str()); -} - -int64_t TransactionUtils::getSnowFlakeUUID() { - auto ver = std::numeric_limits::max() - time::WallClock::slowNowInMicroSec(); - // Switch ver to big-endian, make sure the key is in ordered. - return folly::Endian::big(ver); -} - -std::string TransactionUtils::hexEdgeId(size_t vIdLen, folly::StringPiece key) { - return folly::hexlify(NebulaKeyUtils::getSrcId(vIdLen, key)) + - folly::hexlify(NebulaKeyUtils::getDstId(vIdLen, key)); -} - -} // namespace storage -} // namespace nebula diff --git a/src/storage/transaction/TransactionUtils.h b/src/storage/transaction/TransactionUtils.h deleted file mode 100644 index 98fe6a0a375..00000000000 --- a/src/storage/transaction/TransactionUtils.h +++ /dev/null @@ -1,63 +0,0 @@ -/* Copyright (c) 2020 vesoft inc. All rights reserved. - * - * This source code is licensed under Apache 2.0 License, - * attached with Common Clause Condition 1.0, found in the LICENSES directory. - */ - -#ifndef STORAGE_TRANSACTION_TRANSACTIONUTILS_H_ -#define STORAGE_TRANSACTION_TRANSACTIONUTILS_H_ - -#include -#include - -#include "interface/gen-cpp2/storage_types.h" -#include "kvstore/Common.h" - -namespace nebula { -namespace storage { - -class TransactionUtils { - public: - static std::string dumpKey(const cpp2::EdgeKey& key); - - // join hex of src & dst as en edge id - static std::string hexEdgeId(size_t vIdLen, folly::StringPiece key); - - static std::string reverseRawKey(size_t vIdLen, PartitionID partId, const std::string& rawKey); - - static int64_t getSnowFlakeUUID(); - - /** - * @brief a simple wrapper for NebulaKeyUtils::edgeKey - * allow cpp2::EdgeKey& as a parameter - */ - static std::string edgeKey(size_t vIdLen, PartitionID partId, const cpp2::EdgeKey& key); -}; - -/** - * @brief a simple time recoder used for perf optimize - */ -class Timer { - public: - Timer() : start_(std::chrono::high_resolution_clock::now()) {} - - void reset() { start_ = std::chrono::high_resolution_clock::now(); } - - int64_t elapsed_ms() const { - auto tpNow = std::chrono::high_resolution_clock::now(); - return std::chrono::duration_cast(tpNow - start_).count(); - } - - int64_t elapsed_micro() const { - auto tpNow = std::chrono::high_resolution_clock::now(); - return std::chrono::duration_cast(tpNow - start_).count(); - } - - private: - std::chrono::time_point start_; -}; - -} // namespace storage -} // namespace nebula - -#endif // STORAGE_TRANSACTION_TRANSACTIONUTILS_H_