diff --git a/curvefs/conf/client.conf b/curvefs/conf/client.conf index 755d624920..8ba0127ec2 100644 --- a/curvefs/conf/client.conf +++ b/curvefs/conf/client.conf @@ -85,6 +85,12 @@ fuseClient.getThreadPool=4 # it gurantee the consistent of file after rename, otherwise you should # disable it for performance. fuseClient.enableMultiMountPointRename=true + +# the rename transaction models are different between version 1 and version 2 +# the v2 version greatly improves the performance of rename, especially in concurrent scenarios. +# Node: v1 and v2 are incompatible and cannot be directly upgraded from a v1 cluster to v2. +fuseClient.txVersion=1 + # splice will bring higher performance in some cases # but there might be a kernel issue that will cause kernel panic when enabling it # see https://lore.kernel.org/all/CAAmZXrsGg2xsP1CK+cbuEMumtrqdvD-NKnWzhNcvn71RV3c1yw@mail.gmail.com/ diff --git a/curvefs/conf/metaserver.conf b/curvefs/conf/metaserver.conf index a16fd4d101..1140bef40d 100644 --- a/curvefs/conf/metaserver.conf +++ b/curvefs/conf/metaserver.conf @@ -258,11 +258,17 @@ storage.rocksdb.unordered_write_buffer_size=67108864 # for store inode which exclude its s3chunkinfo list (default: 3) storage.rocksdb.unordered_max_write_buffer_number=3 # rocksdb column family's write_buffer_size -# for store dentry and inode's s3chunkinfo list (unit: bytes, default: 128MB) +# for store dentry and inode's s3chunkinfo list (unit: bytes, default: 64MB) storage.rocksdb.ordered_write_buffer_size=67108864 # rocksdb column family's max_write_buffer_number # for store dentry and inode's s3chunkinfo list (default: 3) storage.rocksdb.ordered_max_write_buffer_number=3 +# rocksdb column family's write_buffer_size +# for store tx lock and write (unit: bytes, default: 64MB) +storage.rocksdb.tx_cf_write_buffer_size=67108864 +# rocksdb column family's max_write_buffer_number +# for store tx lock and write (default: 3) +storage.rocksdb.tx_cf_max_write_buffer_number=3 # The target number of write history bytes to hold in memory (default: 20MB) storage.rocksdb.max_write_buffer_size_to_maintain=20971520 # rocksdb memtable prefix bloom size ratio (size=write_buffer_size*memtable_prefix_bloom_size_ratio) @@ -286,6 +292,8 @@ storage.rocksdb.perf_sampling_ratio=0 # we will sending its with rpc streaming instead of # padding its into inode (default: 25000, about 25000 * 41 (byte) = 1MB) storage.s3_meta_inside_inode.limit_size=25000 +# TTL(millisecond) for tx lock +storage.tx_lock_ttl_ms=5000 # recycle options # metaserver scan recycle period, default 1h diff --git a/curvefs/proto/common.proto b/curvefs/proto/common.proto index a56402c744..25ba220f42 100644 --- a/curvefs/proto/common.proto +++ b/curvefs/proto/common.proto @@ -80,7 +80,7 @@ message PartitionInfo { // partition manage inodeid range [start, end] required uint64 start = 5; required uint64 end = 6; - required uint64 txId = 7; + optional uint64 txId = 7; optional uint64 nextId = 8; // status can change from READWRITE to READONLY, but can not chanage from READONLY to READWRITE // READWRITE/READONLY can change to DELETING, but DELETING can not change to READWRITE/READONLY diff --git a/curvefs/proto/mds.proto b/curvefs/proto/mds.proto index 70e4ecb3b5..f89c13e733 100644 --- a/curvefs/proto/mds.proto +++ b/curvefs/proto/mds.proto @@ -244,6 +244,14 @@ message SetClientMdsAddrsOverrideResponse { required FSStatusCode statusCode = 1; } +message TsoRequest {} + +message TsoResponse { + required FSStatusCode statusCode = 1; + optional uint64 ts = 2; // transaction sequence number + optional uint64 timestamp = 3; +} + service MdsService { // fs interface rpc CreateFs(CreateFsRequest) returns (CreateFsResponse); @@ -259,6 +267,8 @@ service MdsService { rpc GetLatestTxId(GetLatestTxIdRequest) returns (GetLatestTxIdResponse); rpc CommitTx(CommitTxRequest) returns (CommitTxResponse); + rpc Tso(TsoRequest) returns (TsoResponse); + // client lease rpc RefreshSession(RefreshSessionRequest) returns (RefreshSessionResponse); diff --git a/curvefs/proto/metaserver.proto b/curvefs/proto/metaserver.proto index a1ec4ba69d..d22246069e 100644 --- a/curvefs/proto/metaserver.proto +++ b/curvefs/proto/metaserver.proto @@ -49,6 +49,15 @@ enum MetaStatusCode { RPC_STREAM_ERROR = 25; INODE_S3_META_TOO_LARGE = 26; STORAGE_CLOSED = 27; + // tx v2 related + TX_FAILED = 28; + TX_WRITE_CONFLICT = 29; + TX_KEY_LOCKED = 30; + TX_COMMITTED = 31; + TX_ROLLBACKED = 32; + TX_TIMEOUT = 33; + TX_INPROGRESS = 34; + TX_MISMATCH = 35; } // dentry interface @@ -59,7 +68,7 @@ message GetDentryRequest { required uint32 fsId = 4; required uint64 parentInodeId = 5; required string name = 6; - required uint64 txId = 7; + optional uint64 txId = 7; optional uint64 appliedIndex = 8; } @@ -74,7 +83,8 @@ message Dentry { required uint64 inodeId = 2; required uint64 parentInodeId = 3; required string name = 4; - required uint64 txId = 5; + // reused txId as ts in tx v2 for compatibility in metaserver + optional uint64 txId = 5; optional uint32 flag = 6; optional FsFileType type = 7; optional uint64 txSequence = 8; @@ -88,6 +98,7 @@ message GetDentryResponse { required MetaStatusCode statusCode = 1; optional Dentry dentry = 2; optional uint64 appliedIndex = 3; + optional TxLock txLock = 4; } message ListDentryRequest { @@ -96,7 +107,7 @@ message ListDentryRequest { required uint32 partitionId = 3; required uint32 fsId = 4; required uint64 dirInodeId = 5; - required uint64 txId = 6; + optional uint64 txId = 6; optional string last = 7; // the name of last entry optional uint32 count = 8; // the number of entry required optional bool onlyDir = 9; @@ -107,6 +118,7 @@ message ListDentryResponse { required MetaStatusCode statusCode = 1; repeated Dentry dentrys = 2; optional uint64 appliedIndex = 3; + optional TxLock txLock = 4; } message CreateDentryRequest { @@ -120,6 +132,7 @@ message CreateDentryRequest { message CreateDentryResponse { required MetaStatusCode statusCode = 1; optional uint64 appliedIndex = 2; + optional TxLock txLock = 3; } message DeleteDentryRequest { @@ -127,7 +140,7 @@ message DeleteDentryRequest { required uint32 copysetId = 2; required uint32 partitionId = 3; required uint32 fsId = 4; - required uint64 txId = 5; + optional uint64 txId = 5; required uint64 parentInodeId = 6; required string name = 7; optional FsFileType type = 8; @@ -137,6 +150,7 @@ message DeleteDentryRequest { message DeleteDentryResponse { required MetaStatusCode statusCode = 1; optional uint64 appliedIndex = 2; + optional TxLock txLock = 3; } message PrepareRenameTxRequest { @@ -160,6 +174,85 @@ message PrepareRenameTxResponse { optional uint64 appliedIndex = 2; } +message TxLock { + required string primaryKey = 1; + required uint64 startTs = 2; + required uint64 timestamp = 3; + optional uint32 index = 4; + optional int32 ttl = 5; +} + +enum TxWriteKind { + Commit = 1; + Rollback = 2; +} + +message TS { + required uint64 ts = 1; +} + +message TxWrite { + required uint64 startTs = 1; + required TxWriteKind kind = 2; +} + +message PrewriteRenameTxRequest { + required uint32 poolId = 1; + required uint32 copysetId = 2; + required uint32 partitionId = 3; + repeated Dentry dentrys = 4; + required TxLock txLock = 5; +} + +message PrewriteRenameTxResponse { + required MetaStatusCode statusCode = 1; + repeated Dentry dentrys = 2; + optional TxLock txLock = 3; + optional uint64 appliedIndex = 4; +} + +message CheckTxStatusRequest { + required uint32 poolId = 1; + required uint32 copysetId = 2; + required uint32 partitionId = 3; + required string primaryKey = 4; + required uint64 startTs = 5; + required uint64 curTimestamp = 6; +} + +message CheckTxStatusResponse { + required MetaStatusCode statusCode = 1; + optional uint64 appliedIndex = 2; +} + +message ResolveTxLockRequest { + required uint32 poolId = 1; + required uint32 copysetId = 2; + required uint32 partitionId = 3; + required Dentry dentry = 4; + required uint64 startTs = 5; + required uint64 commitTs = 6; +} + +message ResolveTxLockResponse { + required MetaStatusCode statusCode = 1; + optional uint64 appliedIndex = 2; +} + +message CommitTxRequest { + required uint32 poolId = 1; + required uint32 copysetId = 2; + required uint32 partitionId = 3; + repeated Dentry dentrys = 4; + required uint64 startTs = 5; + required uint64 commitTs = 6; +} + +message CommitTxResponse { + required MetaStatusCode statusCode = 1; + optional uint64 appliedIndex = 2; +} + // inode interface message GetInodeRequest { required uint32 poolId = 1; @@ -539,6 +632,11 @@ service MetaServerService { rpc CreateDentry(CreateDentryRequest) returns (CreateDentryResponse); rpc DeleteDentry(DeleteDentryRequest) returns (DeleteDentryResponse); rpc PrepareRenameTx(PrepareRenameTxRequest) returns (PrepareRenameTxResponse); + // tx v2 + rpc PrewriteRenameTx(PrewriteRenameTxRequest) returns (PrewriteRenameTxResponse); + rpc CheckTxStatus(CheckTxStatusRequest) returns (CheckTxStatusResponse); + rpc ResolveTxLock(ResolveTxLockRequest) returns (ResolveTxLockResponse); + rpc CommitTx(CommitTxRequest) returns (CommitTxResponse); // inode interface rpc GetInode(GetInodeRequest) returns (GetInodeResponse); diff --git a/curvefs/proto/topology.proto b/curvefs/proto/topology.proto index c9d70682b5..f248655fca 100644 --- a/curvefs/proto/topology.proto +++ b/curvefs/proto/topology.proto @@ -61,7 +61,6 @@ message ClusterInfoData { required string clusterId = 1; // map partitionIndexs = 2; - } message PoolData { diff --git a/curvefs/src/client/BUILD b/curvefs/src/client/BUILD index 450e8b6ced..33a3b45089 100644 --- a/curvefs/src/client/BUILD +++ b/curvefs/src/client/BUILD @@ -99,6 +99,7 @@ cc_library( "//curvefs/src/common:metric_utils", "//curvefs/src/common:dynamic_vlog", "//curvefs/src/common:threading", + "//curvefs/src/metaserver:metaserver_storage_conv", "@com_google_absl//absl/memory", "@com_google_absl//absl/strings", "@com_google_absl//absl/synchronization", diff --git a/curvefs/src/client/client_operator.cpp b/curvefs/src/client/client_operator.cpp index 720ce65ee3..fc41375caf 100644 --- a/curvefs/src/client/client_operator.cpp +++ b/curvefs/src/client/client_operator.cpp @@ -20,18 +20,22 @@ * Author: Jingli Chen (Wine93) */ +#include "curvefs/src/client/client_operator.h" + #include -#include "src/common/uuid.h" -#include "curvefs/src/client/client_operator.h" #include "curvefs/src/client/filesystem/error.h" +#include "curvefs/src/metaserver/storage/converter.h" +#include "src/common/uuid.h" namespace curvefs { namespace client { using ::curve::common::UUIDGenerator; -using ::curvefs::metaserver::DentryFlag; -using ::curvefs::mds::topology::PartitionTxId; using ::curvefs::client::filesystem::ToFSError; +using ::curvefs::mds::topology::PartitionTxId; +using ::curvefs::metaserver::DentryFlag; +using ::curvefs::metaserver::TxLock; +using ::curvefs::metaserver::storage::Key4Dentry; #define LOG_ERROR(action, rc) \ LOG(ERROR) << action << " failed, retCode = " << rc \ @@ -60,6 +64,7 @@ RenameOperator::RenameOperator(uint32_t fsId, dstTxId_(0), oldInodeId_(0), oldInodeSize_(-1), + startTs_(0), dentryManager_(dentryManager), inodeManager_(inodeManager), metaClient_(metaClient), @@ -77,6 +82,7 @@ std::string RenameOperator::DebugString() { << ", srcPartitionId = " << srcPartitionId_ << ", dstPartitionId = " << dstPartitionId_ << ", srcTxId = " << srcTxId_ << ", dstTxId_ = " << dstTxId_ + << ", startTs = " << startTs_ << ", oldInodeId = " << oldInodeId_ << ", srcDentry = [" << srcDentry_.ShortDebugString() << "]" << ", dstDentry = [" << dstDentry_.ShortDebugString() << "]" @@ -194,7 +200,6 @@ CURVEFS_ERROR RenameOperator::RecordOldInodeInfo() { return CURVEFS_ERROR::NOT_EXIST; } } - return CURVEFS_ERROR::OK; } @@ -204,7 +209,6 @@ CURVEFS_ERROR RenameOperator::PrepareRenameTx( if (rc != MetaStatusCode::OK) { LOG_ERROR("PrepareRenameTx", rc); } - return ToFSError(rc); } @@ -272,6 +276,118 @@ CURVEFS_ERROR RenameOperator::CommitTx() { return CURVEFS_ERROR::OK; } +CURVEFS_ERROR RenameOperator::PrewriteRenameTx( + const std::vector& dentrys, const TxLock& txLockIn) { + TxLock txLockOut; + uint32_t dcount = 0; + auto rc = metaClient_->PrewriteRenameTx(dentrys, txLockIn, &txLockOut); + while (rc == MetaStatusCode::TX_KEY_LOCKED) { + dcount += txLockOut.index(); + auto rt = dentryManager_->CheckAndResolveTx(dentrys[dcount], + txLockOut, txLockIn.timestamp(), txLockIn.startts()); + if (rt != MetaStatusCode::OK) { + LOG_ERROR("CheckAndResolveTx", rt); + return CURVEFS_ERROR::INTERNAL; + } + if (dcount < dentrys.size()) { + rc = metaClient_->PrewriteRenameTx(std::vector( + dentrys.begin() + dcount, dentrys.end()), + txLockIn, &txLockOut); + } else { + break; + } + } + if (rc != MetaStatusCode::OK) { + LOG_ERROR("PrewriteRenameTx", rc); + return CURVEFS_ERROR::INTERNAL; + } + return CURVEFS_ERROR::OK; +} + +CURVEFS_ERROR RenameOperator::PrewriteTx() { + uint64_t timestamp; + auto rc = mdsClient_->Tso(&startTs_, ×tamp); + if (rc != FSStatusCode::OK) { + LOG_ERROR("start Tso", rc); + return CURVEFS_ERROR::INTERNAL; + } + + dentry_ = Dentry(srcDentry_); + dentry_.set_flag(DentryFlag::DELETE_MARK_FLAG); + dentry_.set_type(srcDentry_.type()); + dentry_.set_txid(startTs_); + + newDentry_ = Dentry(srcDentry_); + newDentry_.set_parentinodeid(newParentId_); + newDentry_.set_name(newname_); + newDentry_.set_type(srcDentry_.type()); + newDentry_.set_txid(startTs_); + + Key4Dentry key4Dentry( + dentry_.fsid(), dentry_.parentinodeid(), dentry_.name()); + std::string primaryKey = key4Dentry.SerializeToString(); + + TxLock txLockIn; + txLockIn.set_primarykey(primaryKey); + txLockIn.set_startts(startTs_); + txLockIn.set_timestamp(timestamp); + + if (!metaClient_->GetPartitionId(dentry_.fsid(), dentry_.parentinodeid(), + &srcPartitionId_) || !metaClient_->GetPartitionId(newDentry_.fsid(), + newDentry_.parentinodeid(), &dstPartitionId_)) { + LOG_ERROR("GetPartitionId", rc); + return CURVEFS_ERROR::INTERNAL; + } + + // note: do not prewrite concurrently, the tx write table clear logic based primary key prewrite first // NOLINT + CURVEFS_ERROR rt = CURVEFS_ERROR::OK; + std::vector dentrys{dentry_}; + if (srcPartitionId_ == dstPartitionId_) { + dentrys.push_back(newDentry_); + rt = PrewriteRenameTx(dentrys, txLockIn); + } else { + rt = PrewriteRenameTx(dentrys, txLockIn); + if (rt == CURVEFS_ERROR::OK) { + dentrys[0] = newDentry_; + rt = PrewriteRenameTx(dentrys, txLockIn); + } + } + if (rt != CURVEFS_ERROR::OK) { + LOG_ERROR("PrepPrewriteTxareTx", rc); + return rt; + } + return CURVEFS_ERROR::OK; +} + +CURVEFS_ERROR RenameOperator::CommitTxV2() { + uint64_t commitTs; + uint64_t timestamp; + auto rc = mdsClient_->Tso(&commitTs, ×tamp); + if (rc != FSStatusCode::OK) { + LOG_ERROR("CommitTxV2 Tso", rc); + return CURVEFS_ERROR::INTERNAL; + } + + MetaStatusCode rt = MetaStatusCode::OK; + std::vector dentrys{dentry_}; + if (srcPartitionId_ == dstPartitionId_) { + dentrys.push_back(newDentry_); + rt = metaClient_->CommitTx(dentrys, startTs_, commitTs); + } else { + rt = metaClient_->CommitTx(dentrys, startTs_, commitTs); + if (rt == MetaStatusCode::OK) { + dentrys[0] = newDentry_; + // do not need check second key commit result + metaClient_->CommitTx(dentrys, startTs_, commitTs); + } + } + if (rt != MetaStatusCode::OK) { + LOG_ERROR("CommitTx", rt); + return CURVEFS_ERROR::INTERNAL; + } + return CURVEFS_ERROR::OK; +} + CURVEFS_ERROR RenameOperator::LinkInode(uint64_t inodeId, uint64_t parent) { std::shared_ptr inodeWrapper; auto rc = inodeManager_->GetInode(inodeId, inodeWrapper); @@ -413,8 +529,6 @@ CURVEFS_ERROR RenameOperator::UpdateInodeCtime() { LOG_ERROR("UpdateInodeCtime", rc); return rc; } - - LOG(INFO) << "UpdateInodeCtime inodeid = " << srcDentry_.inodeid(); return rc; } diff --git a/curvefs/src/client/client_operator.h b/curvefs/src/client/client_operator.h index 0f073a67c4..0408fd1181 100644 --- a/curvefs/src/client/client_operator.h +++ b/curvefs/src/client/client_operator.h @@ -27,13 +27,15 @@ #include #include -#include "curvefs/src/client/inode_cache_manager.h" -#include "curvefs/src/client/dentry_cache_manager.h" +#include "curvefs/src/client/inode_manager.h" +#include "curvefs/src/client/dentry_manager.h" #include "curvefs/src/client/rpcclient/mds_client.h" namespace curvefs { namespace client { +using ::curvefs::metaserver::MetaStatusCode; +using ::curvefs::metaserver::TxLock; using rpcclient::MdsClient; class RenameOperator { @@ -56,6 +58,8 @@ class RenameOperator { CURVEFS_ERROR LinkDestParentInode(); CURVEFS_ERROR PrepareTx(); CURVEFS_ERROR CommitTx(); + CURVEFS_ERROR PrewriteTx(); + CURVEFS_ERROR CommitTxV2(); CURVEFS_ERROR UnlinkSrcParentInode(); void UnlinkOldInode(); CURVEFS_ERROR UpdateInodeParent(); @@ -85,6 +89,9 @@ class RenameOperator { CURVEFS_ERROR PrepareRenameTx(const std::vector& dentrys); + CURVEFS_ERROR PrewriteRenameTx( + const std::vector& dentrys, const TxLock& txLockIn); + CURVEFS_ERROR LinkInode(uint64_t inodeId, uint64_t parent = 0); CURVEFS_ERROR UnLinkInode(uint64_t inodeId, uint64_t parent = 0); @@ -107,6 +114,7 @@ class RenameOperator { // if dest exist, record the size and type of file or empty dir int64_t oldInodeSize_; FsFileType oldInodeType_; + uint64_t startTs_; // tx sequence number Dentry srcDentry_; Dentry dstDentry_; Dentry dentry_; diff --git a/curvefs/src/client/common/common.cpp b/curvefs/src/client/common/common.cpp index b50898a630..1210287fc3 100644 --- a/curvefs/src/client/common/common.cpp +++ b/curvefs/src/client/common/common.cpp @@ -45,6 +45,18 @@ std::ostream &operator<<(std::ostream &os, MetaServerOpType optype) { case MetaServerOpType::PrepareRenameTx: os << "PrepareRenameTx"; break; + case MetaServerOpType::PrewriteRenameTx: + os << "PrewriteRenameTx"; + break; + case MetaServerOpType::CheckTxStatus: + os << "CheckTxStatus"; + break; + case MetaServerOpType::ResolveTxLock: + os << "ResolveTxLock"; + break; + case MetaServerOpType::CommitTx: + os << "CommitTx"; + break; case MetaServerOpType::GetInode: os << "GetInode"; break; diff --git a/curvefs/src/client/common/common.h b/curvefs/src/client/common/common.h index a76312b117..2a5a13d931 100644 --- a/curvefs/src/client/common/common.h +++ b/curvefs/src/client/common/common.h @@ -50,6 +50,10 @@ enum class MetaServerOpType { CreateDentry, DeleteDentry, PrepareRenameTx, + PrewriteRenameTx, + CheckTxStatus, + ResolveTxLock, + CommitTx, GetInode, BatchGetInodeAttr, BatchGetXAttr, diff --git a/curvefs/src/client/common/config.cpp b/curvefs/src/client/common/config.cpp index dc80177ab3..6125a22af9 100644 --- a/curvefs/src/client/common/config.cpp +++ b/curvefs/src/client/common/config.cpp @@ -105,6 +105,8 @@ DEFINE_uint64(fuseClientBurstReadIopsSecs, 180, "the times that Read burst iops can continue"); DEFINE_validator(fuseClientBurstReadIopsSecs, &pass_uint64); +DEFINE_int32(TxVersion, 1, "tx version"); + void InitMdsOption(Configuration *conf, MdsOption *mdsOpt) { conf->GetValueFatalIfFail("mdsOpt.mdsMaxRetryMS", &mdsOpt->mdsMaxRetryMS); conf->GetValueFatalIfFail("mdsOpt.rpcRetryOpt.maxRPCTimeoutMS", @@ -428,6 +430,8 @@ void InitFuseClientOption(Configuration *conf, FuseClientOption *clientOption) { &clientOption->dummyServerStartPort); conf->GetValueFatalIfFail("fuseClient.enableMultiMountPointRename", &clientOption->enableMultiMountPointRename); + conf->GetIntValue("fuseClient.txVersion", + &FLAGS_TxVersion); conf->GetValueFatalIfFail("fuseClient.downloadMaxRetryTimes", &clientOption->downloadMaxRetryTimes); conf->GetValueFatalIfFail("fuseClient.warmupThreadsNum", diff --git a/curvefs/src/client/dentry_cache_manager.cpp b/curvefs/src/client/dentry_cache_manager.cpp deleted file mode 100644 index afb5e49eef..0000000000 --- a/curvefs/src/client/dentry_cache_manager.cpp +++ /dev/null @@ -1,160 +0,0 @@ -/* - * Copyright (c) 2021 NetEase Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -/* - * Project: curve - * Created Date: Thur May 27 2021 - * Author: xuchaojie - */ -#include "curvefs/src/client/dentry_cache_manager.h" - -#include -#include -#include -#include -#include -#include - -using ::curvefs::metaserver::MetaStatusCode_Name; - -namespace curvefs { -namespace client { -namespace common { -DECLARE_bool(enableCto); -} // namespace common -} // namespace client -} // namespace curvefs - -namespace curvefs { -namespace client { - -using curve::common::WriteLockGuard; -using NameLockGuard = ::curve::common::GenericNameLockGuard; -using ::curvefs::client::filesystem::ToFSError; - -CURVEFS_ERROR DentryCacheManagerImpl::GetDentry(uint64_t parent, - const std::string &name, - Dentry *out) { - std::string key = GetDentryCacheKey(parent, name); - NameLockGuard lock(nameLock_, key); - - MetaStatusCode ret = metaClient_->GetDentry(fsId_, parent, name, out); - if (ret != MetaStatusCode::OK) { - LOG_IF(ERROR, ret != MetaStatusCode::NOT_FOUND) - << "metaClient_ GetDentry failed, MetaStatusCode = " << ret - << ", MetaStatusCode_Name = " << MetaStatusCode_Name(ret) - << ", parent = " << parent << ", name = " << name; - return ToFSError(ret); - } - return CURVEFS_ERROR::OK; -} - -CURVEFS_ERROR DentryCacheManagerImpl::CreateDentry(const Dentry &dentry) { - std::string key = GetDentryCacheKey(dentry.parentinodeid(), dentry.name()); - NameLockGuard lock(nameLock_, key); - MetaStatusCode ret = metaClient_->CreateDentry(dentry); - if (ret != MetaStatusCode::OK) { - LOG(ERROR) << "metaClient_ CreateDentry failed, MetaStatusCode = " - << ret - << ", MetaStatusCode_Name = " << MetaStatusCode_Name(ret) - << ", parent = " << dentry.parentinodeid() - << ", name = " << dentry.name(); - return ToFSError(ret); - } - - return CURVEFS_ERROR::OK; -} - -CURVEFS_ERROR DentryCacheManagerImpl::DeleteDentry(uint64_t parent, - const std::string &name, - FsFileType type) { - std::string key = GetDentryCacheKey(parent, name); - NameLockGuard lock(nameLock_, key); - - MetaStatusCode ret = metaClient_->DeleteDentry(fsId_, parent, name, type); - if (ret != MetaStatusCode::OK && ret != MetaStatusCode::NOT_FOUND) { - LOG(ERROR) << "metaClient_ DeleteInode failed, MetaStatusCode = " << ret - << ", MetaStatusCode_Name = " << MetaStatusCode_Name(ret) - << ", parent = " << parent << ", name = " << name; - return ToFSError(ret); - } - return CURVEFS_ERROR::OK; -} - -CURVEFS_ERROR DentryCacheManagerImpl::ListDentry(uint64_t parent, - std::list *dentryList, - uint32_t limit, - bool onlyDir, - uint32_t nlink) { - dentryList->clear(); - // means no dir under this dir - if (onlyDir && nlink == 2) { - LOG(INFO) << "ListDentry parent = " << parent - << ", onlyDir = 1 and nlink = 2, return directly"; - return CURVEFS_ERROR::OK; - } - - MetaStatusCode ret = MetaStatusCode::OK; - bool perceed = true; - std::string last = ""; - do { - std::list part; - ret = metaClient_->ListDentry(fsId_, parent, last, limit, onlyDir, - &part); - VLOG(6) << "ListDentry fsId = " << fsId_ << ", parent = " << parent - << ", last = " << last << ", count = " << limit - << ", onlyDir = " << onlyDir - << ", ret = " << ret << ", part.size() = " << part.size(); - if (ret != MetaStatusCode::OK) { - LOG(ERROR) << "metaClient_ ListDentry failed" - << ", MetaStatusCode_Name = " << MetaStatusCode_Name(ret) - << ", parent = " << parent << ", last = " << last - << ", count = " << limit << ", onlyDir = " << onlyDir; - return ToFSError(ret); - } - - if (!onlyDir) { - if (part.size() < limit) { - perceed = false; - } - if (!part.empty()) { - last = part.back().name(); - dentryList->splice(dentryList->end(), part); - } - } else { - // means iterate over the range - if (part.empty()) { - perceed = false; - } else { - last = part.back().name(); - if (part.back().type() != FsFileType::TYPE_DIRECTORY) { - part.pop_back(); - } - dentryList->splice(dentryList->end(), part); - // means already get all the dir under this dir - if (nlink - dentryList->size() == 2) { - perceed = false; - } - } - } - } while (perceed); - - return CURVEFS_ERROR::OK; -} - -} // namespace client -} // namespace curvefs diff --git a/curvefs/src/client/dentry_manager.cpp b/curvefs/src/client/dentry_manager.cpp new file mode 100644 index 0000000000..9c5227ccfb --- /dev/null +++ b/curvefs/src/client/dentry_manager.cpp @@ -0,0 +1,287 @@ +/* + * Copyright (c) 2021 NetEase Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +/* + * Project: curve + * Created Date: Thur May 27 2021 + * Author: xuchaojie + */ +#include "curvefs/src/client/dentry_manager.h" + +#include +#include +#include +#include +#include +#include +#include "curvefs/src/metaserver/storage/converter.h" + +using ::curvefs::metaserver::MetaStatusCode_Name; +using ::curvefs::metaserver::storage::Key4Dentry; + +namespace curvefs { +namespace client { +namespace common { +DECLARE_bool(enableCto); +} // namespace common +} // namespace client +} // namespace curvefs + +namespace curvefs { +namespace client { + +using curve::common::WriteLockGuard; +using NameLockGuard = ::curve::common::GenericNameLockGuard; +using ::curvefs::client::filesystem::ToFSError; + +MetaStatusCode DentryCacheManagerImpl::CheckTxStatus( + const std::string primaryKey, uint64_t startTs, uint64_t curTimestamp) { + Key4Dentry key4Dentry; + if (!key4Dentry.ParseFromString(primaryKey)) { + LOG(ERROR) << "CheckTxStatus parse primary key failed, primaryKey = " + << primaryKey; + return MetaStatusCode::PARSE_FROM_STRING_FAILED; + } + return metaClient_->CheckTxStatus(key4Dentry.fsId, key4Dentry.parentInodeId, + primaryKey, startTs, curTimestamp); +} + +MetaStatusCode DentryCacheManagerImpl::ResolveTxLock(const Dentry &dentry, + uint64_t startTs, uint64_t commitTs) { + return metaClient_->ResolveTxLock(dentry, startTs, commitTs); +} + +MetaStatusCode DentryCacheManagerImpl::CheckAndResolveTx(const Dentry& dentry, + const TxLock& txLock, uint64_t timestamp, uint64_t commitTs) { + auto rt = CheckTxStatus(txLock.primarykey(), txLock.startts(), timestamp); + switch (rt) { + case MetaStatusCode::TX_COMMITTED: + return ResolveTxLock(dentry, txLock.startts(), commitTs); + case MetaStatusCode::TX_ROLLBACKED: + case MetaStatusCode::TX_TIMEOUT: + return ResolveTxLock(dentry, txLock.startts()); + default: + LOG(ERROR) << "CheckTxStatus unexpected rt = " + << MetaStatusCode_Name(rt); + return rt; + } + return MetaStatusCode::OK; +} + +CURVEFS_ERROR DentryCacheManagerImpl::GetDentry(uint64_t parent, + const std::string &name, + Dentry *out) { + std::string key = GetDentryCacheKey(parent, name); + NameLockGuard lock(nameLock_, key); + TxLock txLockOut; + MetaStatusCode ret = metaClient_->GetDentry(fsId_, parent, name, out, + &txLockOut); + while (ret == MetaStatusCode::TX_KEY_LOCKED) { + uint64_t ts = 0; + uint64_t timestamp = 0; + if (mdsClient_->Tso(&ts, ×tamp) != FSStatusCode::OK) { + LOG(ERROR) << "GetDentry Tso failed, parent = " << parent + << ", name = " << name; + return CURVEFS_ERROR::INTERNAL; + } + Dentry dentry; + dentry.set_fsid(fsId_); + dentry.set_parentinodeid(parent); + dentry.set_name(name); + MetaStatusCode rc = CheckAndResolveTx(dentry, txLockOut, timestamp, ts); + if (rc != MetaStatusCode::OK) { + LOG(ERROR) << "GetDentry CheckAndResolveTx failed, rc = " + << MetaStatusCode_Name(rc) + << ", parent = " << parent << ", name = " << name; + return CURVEFS_ERROR::INTERNAL; + } + ret = metaClient_->GetDentry(fsId_, parent, name, out, &txLockOut); + } + + if (ret != MetaStatusCode::OK) { + LOG_IF(ERROR, ret != MetaStatusCode::NOT_FOUND) + << "metaClient_ GetDentry failed, MetaStatusCode = " << ret + << ", MetaStatusCode_Name = " << MetaStatusCode_Name(ret) + << ", parent = " << parent << ", name = " << name; + return ToFSError(ret); + } + return CURVEFS_ERROR::OK; +} + +CURVEFS_ERROR DentryCacheManagerImpl::CreateDentry(const Dentry &dentry) { + std::string key = GetDentryCacheKey(dentry.parentinodeid(), dentry.name()); + NameLockGuard lock(nameLock_, key); + TxLock txLockOut; + MetaStatusCode ret = metaClient_->CreateDentry(dentry, &txLockOut); + while (ret == MetaStatusCode::TX_KEY_LOCKED) { + uint64_t ts = 0; + uint64_t timestamp = 0; + if (mdsClient_->Tso(&ts, ×tamp) != FSStatusCode::OK) { + LOG(ERROR) << "CreateDentry Tso failed, dentry = " + << dentry.ShortDebugString(); + return CURVEFS_ERROR::INTERNAL; + } + MetaStatusCode rc = CheckAndResolveTx(dentry, txLockOut, timestamp, ts); + if (rc != MetaStatusCode::OK) { + LOG(ERROR) << "CreateDentry CheckAndResolveTx failed, rc = " + << MetaStatusCode_Name(rc) << ", dentry = " + << dentry.ShortDebugString(); + return CURVEFS_ERROR::INTERNAL; + } + ret = metaClient_->CreateDentry(dentry, &txLockOut); + } + if (ret != MetaStatusCode::OK) { + LOG(ERROR) << "metaClient_ CreateDentry failed" + << ", MetaStatusCode_Name = " << MetaStatusCode_Name(ret) + << ", parent = " << dentry.parentinodeid() + << ", name = " << dentry.name(); + return ToFSError(ret); + } + return CURVEFS_ERROR::OK; +} + +CURVEFS_ERROR DentryCacheManagerImpl::DeleteDentry(uint64_t parent, + const std::string &name, + FsFileType type) { + std::string key = GetDentryCacheKey(parent, name); + NameLockGuard lock(nameLock_, key); + + TxLock txLockOut; + MetaStatusCode ret = metaClient_->DeleteDentry( + fsId_, parent, name, type, &txLockOut); + while (ret == MetaStatusCode::TX_KEY_LOCKED) { + uint64_t ts = 0; + uint64_t timestamp = 0; + if (mdsClient_->Tso(&ts, ×tamp) != FSStatusCode::OK) { + LOG(ERROR) << "DeleteDentry Tso failed, parent = " << parent + << ", name = " << name; + return CURVEFS_ERROR::INTERNAL; + } + Dentry dentry; + dentry.set_fsid(fsId_); + dentry.set_parentinodeid(parent); + dentry.set_name(name); + MetaStatusCode rc = CheckAndResolveTx(dentry, txLockOut, timestamp, ts); + if (rc != MetaStatusCode::OK) { + LOG(ERROR) << "DeleteDentry CheckAndResolveTx failed, rc = " + << MetaStatusCode_Name(rc) << ", parent = " << parent + << ", name = " << name; + return CURVEFS_ERROR::INTERNAL; + } + ret = metaClient_->DeleteDentry(fsId_, parent, name, type, &txLockOut); + } + + if (ret != MetaStatusCode::OK && ret != MetaStatusCode::NOT_FOUND) { + LOG(ERROR) << "metaClient_ DeleteInode failed, MetaStatusCode = " << ret + << ", MetaStatusCode_Name = " << MetaStatusCode_Name(ret) + << ", parent = " << parent << ", name = " << name; + return ToFSError(ret); + } + return CURVEFS_ERROR::OK; +} + +CURVEFS_ERROR DentryCacheManagerImpl::ListDentry(uint64_t parent, + std::list *dentryList, + uint32_t limit, + bool onlyDir, + uint32_t nlink) { + dentryList->clear(); + // means no dir under this dir + if (onlyDir && nlink == 2) { + LOG(INFO) << "ListDentry parent = " << parent + << ", onlyDir = 1 and nlink = 2, return directly"; + return CURVEFS_ERROR::OK; + } + + MetaStatusCode ret = MetaStatusCode::OK; + bool perceed = true; + std::string last = ""; + TxLock txLockOut; + do { + std::list part; + ret = metaClient_->ListDentry(fsId_, parent, last, limit, onlyDir, + &part, &txLockOut); + VLOG(6) << "ListDentry fsId = " << fsId_ << ", parent = " << parent + << ", last = " << last << ", count = " << limit + << ", onlyDir = " << onlyDir + << ", ret = " << ret << ", part.size() = " << part.size(); + if (ret == MetaStatusCode::TX_KEY_LOCKED) { + uint64_t ts = 0; + uint64_t timestamp = 0; + if (mdsClient_->Tso(&ts, ×tamp) != FSStatusCode::OK) { + LOG(ERROR) << "ListDentry Tso failed, parent = " << parent; + return CURVEFS_ERROR::INTERNAL; + } + Dentry dentry; + dentry.set_fsid(fsId_); + dentry.set_parentinodeid(parent); + if (part.empty()) { + LOG(ERROR) << "ListDentry tx key locked, but part is empty" + << ", parent = " << parent; + return CURVEFS_ERROR::INTERNAL; + } + dentry.set_name(part.back().name()); + part.pop_back(); + MetaStatusCode rc = CheckAndResolveTx( + dentry, txLockOut, timestamp, ts); + if (rc != MetaStatusCode::OK) { + LOG(ERROR) << "ListDentry CheckAndResolveTx failed, rc = " + << MetaStatusCode_Name(rc) + << ", parent = " << parent; + return CURVEFS_ERROR::INTERNAL; + } + } else if (ret != MetaStatusCode::OK) { + LOG(ERROR) << "metaClient_ ListDentry failed" + << ", MetaStatusCode_Name = " << MetaStatusCode_Name(ret) + << ", parent = " << parent << ", last = " << last + << ", count = " << limit << ", onlyDir = " << onlyDir; + return ToFSError(ret); + } + + if (!onlyDir) { + if (part.size() < limit && ret != MetaStatusCode::TX_KEY_LOCKED) { + perceed = false; + } + if (!part.empty()) { + last = part.back().name(); + dentryList->splice(dentryList->end(), part); + } + } else { + // means iterate over the range + if (part.empty() && ret != MetaStatusCode::TX_KEY_LOCKED) { + perceed = false; + } else { + if (!part.empty()) { + last = part.back().name(); + if (part.back().type() != FsFileType::TYPE_DIRECTORY) { + part.pop_back(); + } + dentryList->splice(dentryList->end(), part); + } + // means already get all the dir under this dir + if (nlink - dentryList->size() == 2) { + perceed = false; + } + } + } + } while (perceed); + + return CURVEFS_ERROR::OK; +} + +} // namespace client +} // namespace curvefs diff --git a/curvefs/src/client/dentry_cache_manager.h b/curvefs/src/client/dentry_manager.h similarity index 76% rename from curvefs/src/client/dentry_cache_manager.h rename to curvefs/src/client/dentry_manager.h index 84f9f20f53..0014a28349 100644 --- a/curvefs/src/client/dentry_cache_manager.h +++ b/curvefs/src/client/dentry_manager.h @@ -21,8 +21,8 @@ * Author: xuchaojie */ -#ifndef CURVEFS_SRC_CLIENT_DENTRY_CACHE_MANAGER_H_ -#define CURVEFS_SRC_CLIENT_DENTRY_CACHE_MANAGER_H_ +#ifndef CURVEFS_SRC_CLIENT_DENTRY_MANAGER_H_ +#define CURVEFS_SRC_CLIENT_DENTRY_MANAGER_H_ #include #include @@ -33,6 +33,7 @@ #include #include "curvefs/src/client/rpcclient/metaserver_client.h" +#include "curvefs/src/client/rpcclient/mds_client.h" #include "src/common/concurrent/concurrent.h" #include "src/common/concurrent/name_lock.h" #include "curvefs/src/client/filesystem/error.h" @@ -44,6 +45,7 @@ namespace client { using rpcclient::MetaServerClient; using rpcclient::MetaServerClientImpl; +using rpcclient::MdsClient; using ::curvefs::client::filesystem::CURVEFS_ERROR; static const char* kDentryKeyDelimiter = ":"; @@ -57,6 +59,8 @@ class DentryCacheManager { fsId_ = fsId; } + virtual void Init(std::shared_ptr mdsClient) = 0; + virtual CURVEFS_ERROR GetDentry(uint64_t parent, const std::string &name, Dentry *out) = 0; @@ -70,6 +74,9 @@ class DentryCacheManager { std::list *dentryList, uint32_t limit, bool onlyDir = false, uint32_t nlink = 0) = 0; + virtual MetaStatusCode CheckAndResolveTx(const Dentry& dentry, + const TxLock& txLock, uint64_t timestamp, uint64_t commitTs) = 0; + protected: uint32_t fsId_; }; @@ -83,6 +90,10 @@ class DentryCacheManagerImpl : public DentryCacheManager { const std::shared_ptr &metaClient) : metaClient_(metaClient) {} + void Init(std::shared_ptr mdsClient) override { + mdsClient_ = mdsClient; + } + CURVEFS_ERROR GetDentry(uint64_t parent, const std::string &name, Dentry *out) override; @@ -96,11 +107,22 @@ class DentryCacheManagerImpl : public DentryCacheManager { std::list *dentryList, uint32_t limit, bool dirOnly = false, uint32_t nlink = 0) override; + MetaStatusCode CheckAndResolveTx(const Dentry& dentry, const TxLock& txLock, + uint64_t timestamp, uint64_t commitTs) override; + std::string GetDentryCacheKey(uint64_t parent, const std::string &name) { return std::to_string(parent) + kDentryKeyDelimiter + name; } private: + MetaStatusCode CheckTxStatus(const std::string primaryKey, uint64_t startTs, + uint64_t curTimestamp); + + MetaStatusCode ResolveTxLock(const Dentry& dentry, uint64_t startTs, + uint64_t commitTs = 0); + + private: + std::shared_ptr mdsClient_; std::shared_ptr metaClient_; curve::common::GenericNameLock nameLock_; }; @@ -108,4 +130,4 @@ class DentryCacheManagerImpl : public DentryCacheManager { } // namespace client } // namespace curvefs -#endif // CURVEFS_SRC_CLIENT_DENTRY_CACHE_MANAGER_H_ +#endif // CURVEFS_SRC_CLIENT_DENTRY_MANAGER_H_ diff --git a/curvefs/src/client/filesystem/package.h b/curvefs/src/client/filesystem/package.h index 47f9fccc36..8eb7a14b23 100644 --- a/curvefs/src/client/filesystem/package.h +++ b/curvefs/src/client/filesystem/package.h @@ -25,8 +25,8 @@ #include -#include "curvefs/src/client/dentry_cache_manager.h" -#include "curvefs/src/client/inode_cache_manager.h" +#include "curvefs/src/client/dentry_manager.h" +#include "curvefs/src/client/inode_manager.h" #include "curvefs/src/client/xattr_manager.h" namespace curvefs { diff --git a/curvefs/src/client/fuse_client.cpp b/curvefs/src/client/fuse_client.cpp index 658f587351..f1e19721e1 100644 --- a/curvefs/src/client/fuse_client.cpp +++ b/curvefs/src/client/fuse_client.cpp @@ -104,6 +104,8 @@ DECLARE_uint64(fuseClientAvgReadBytes); DECLARE_uint64(fuseClientBurstReadBytes); DECLARE_uint64(fuseClientBurstReadBytesSecs); DECLARE_bool(fs_disableXattr); + +DECLARE_int32(TxVersion); } // namespace common } // namespace client } // namespace curvefs @@ -158,6 +160,8 @@ CURVEFS_ERROR FuseClient::Init(const FuseClientOption &option) { metaCache, mdsClient_, &enableSumInDir_); + dentryManager_->Init(mdsClient_); + xattrManager_ = std::make_shared(inodeManager_, dentryManager_, option_.listDentryLimit, option_.listDentryThreads); @@ -895,38 +899,63 @@ CURVEFS_ERROR FuseClient::FuseOpRename(fuse_req_t req, fuse_ino_t parent, return CURVEFS_ERROR::NAME_TOO_LONG; } - auto renameOp = - RenameOperator(fsInfo_->fsid(), fsInfo_->fsname(), - parent, name, newparent, newname, - dentryManager_, inodeManager_, metaClient_, mdsClient_, - option_.enableMultiMountPointRename); + auto renameOp = RenameOperator(fsInfo_->fsid(), fsInfo_->fsname(), parent, + name, newparent, newname, dentryManager_, inodeManager_, metaClient_, + mdsClient_, option_.enableMultiMountPointRename); - curve::common::LockGuard lg(renameMutex_); CURVEFS_ERROR rc = CURVEFS_ERROR::OK; - VLOG(3) << "FuseOpRename [start]: " << renameOp.DebugString(); - RETURN_IF_UNSUCCESS(GetTxId); - RETURN_IF_UNSUCCESS(Precheck); - RETURN_IF_UNSUCCESS(RecordOldInodeInfo); - // Do not move LinkDestParentInode behind CommitTx. - // If so, the nlink will be lost when the machine goes down - RETURN_IF_UNSUCCESS(LinkDestParentInode); - RETURN_IF_UNSUCCESS(PrepareTx); - RETURN_IF_UNSUCCESS(CommitTx); - VLOG(3) << "FuseOpRename [success]: " << renameOp.DebugString(); - // Do not check UnlinkSrcParentInode, beause rename is already success - renameOp.UnlinkSrcParentInode(); - renameOp.UnlinkOldInode(); - if (parent != newparent) { - renameOp.UpdateInodeParent(); - } - renameOp.UpdateInodeCtime(); - renameOp.UpdateCache(); + if (common::FLAGS_TxVersion == 1) { + curve::common::LockGuard lg(renameMutex_); + VLOG(3) << "FuseOpRename [start]: " << renameOp.DebugString(); + RETURN_IF_UNSUCCESS(GetTxId); + RETURN_IF_UNSUCCESS(Precheck); + RETURN_IF_UNSUCCESS(RecordOldInodeInfo); + // Do not move LinkDestParentInode behind CommitTx. + // If so, the nlink will be lost when the machine goes down + RETURN_IF_UNSUCCESS(LinkDestParentInode); + RETURN_IF_UNSUCCESS(PrepareTx); + RETURN_IF_UNSUCCESS(CommitTx); + VLOG(3) << "FuseOpRename [success]: " << renameOp.DebugString(); + // Do not check UnlinkSrcParentInode, beause rename is already success + renameOp.UnlinkSrcParentInode(); + renameOp.UnlinkOldInode(); + if (parent != newparent) { + renameOp.UpdateInodeParent(); + } + renameOp.UpdateInodeCtime(); + renameOp.UpdateCache(); - if (enableSumInDir_.load()) { - xattrManager_->UpdateParentXattrAfterRename( - parent, newparent, newname, &renameOp); - } + if (enableSumInDir_.load()) { + xattrManager_->UpdateParentXattrAfterRename( + parent, newparent, newname, &renameOp); + } + } else if (common::FLAGS_TxVersion == 2) { + VLOG(3) << "FuseOpRename [start]: " << renameOp.DebugString(); + RETURN_IF_UNSUCCESS(Precheck); + RETURN_IF_UNSUCCESS(RecordOldInodeInfo); + // Do not move LinkDestParentInode behind CommitTx. + // If so, the nlink will be lost when the machine goes down + RETURN_IF_UNSUCCESS(LinkDestParentInode); + RETURN_IF_UNSUCCESS(PrewriteTx); + RETURN_IF_UNSUCCESS(CommitTxV2); + VLOG(3) << "FuseOpRename [success]: " << renameOp.DebugString(); + // Do not check UnlinkSrcParentInode, beause rename is already success + renameOp.UnlinkSrcParentInode(); + renameOp.UnlinkOldInode(); + if (parent != newparent) { + renameOp.UpdateInodeParent(); + } + renameOp.UpdateInodeCtime(); + if (enableSumInDir_.load()) { + xattrManager_->UpdateParentXattrAfterRename( + parent, newparent, newname, &renameOp); + } + } else { + LOG(ERROR) << "FuseOpRename not support tx version: " + << common::FLAGS_TxVersion; + return CURVEFS_ERROR::NOT_SUPPORT; + } return rc; } diff --git a/curvefs/src/client/fuse_client.h b/curvefs/src/client/fuse_client.h index cad4af68c7..aae53041cb 100644 --- a/curvefs/src/client/fuse_client.h +++ b/curvefs/src/client/fuse_client.h @@ -41,13 +41,13 @@ #include "curvefs/src/client/client_operator.h" #include "curvefs/src/client/common/common.h" #include "curvefs/src/client/common/config.h" -#include "curvefs/src/client/dentry_cache_manager.h" +#include "curvefs/src/client/dentry_manager.h" #include "curvefs/src/client/dir_buffer.h" #include "curvefs/src/client/filesystem/error.h" #include "curvefs/src/client/filesystem/filesystem.h" #include "curvefs/src/client/filesystem/meta.h" #include "curvefs/src/client/fuse_common.h" -#include "curvefs/src/client/inode_cache_manager.h" +#include "curvefs/src/client/inode_manager.h" #include "curvefs/src/client/lease/lease_excutor.h" #include "curvefs/src/client/metric/client_metric.h" #include "curvefs/src/client/rpcclient/mds_client.h" diff --git a/curvefs/src/client/inode_cache_manager.cpp b/curvefs/src/client/inode_manager.cpp similarity index 99% rename from curvefs/src/client/inode_cache_manager.cpp rename to curvefs/src/client/inode_manager.cpp index b30dd4ee16..d3aa1c9fac 100644 --- a/curvefs/src/client/inode_cache_manager.cpp +++ b/curvefs/src/client/inode_manager.cpp @@ -21,7 +21,7 @@ * Author: xuchaojie */ -#include "curvefs/src/client/inode_cache_manager.h" +#include "curvefs/src/client/inode_manager.h" #include #include diff --git a/curvefs/src/client/inode_cache_manager.h b/curvefs/src/client/inode_manager.h similarity index 98% rename from curvefs/src/client/inode_cache_manager.h rename to curvefs/src/client/inode_manager.h index 49c642c0e3..093b5e750d 100644 --- a/curvefs/src/client/inode_cache_manager.h +++ b/curvefs/src/client/inode_manager.h @@ -21,8 +21,8 @@ * Author: xuchaojie */ -#ifndef CURVEFS_SRC_CLIENT_INODE_CACHE_MANAGER_H_ -#define CURVEFS_SRC_CLIENT_INODE_CACHE_MANAGER_H_ +#ifndef CURVEFS_SRC_CLIENT_INODE_MANAGER_H_ +#define CURVEFS_SRC_CLIENT_INODE_MANAGER_H_ #include #include @@ -242,4 +242,4 @@ class BatchGetInodeAttrAsyncDone : public BatchGetInodeAttrDone { } // namespace client } // namespace curvefs -#endif // CURVEFS_SRC_CLIENT_INODE_CACHE_MANAGER_H_ +#endif // CURVEFS_SRC_CLIENT_INODE_MANAGER_H_ diff --git a/curvefs/src/client/lease/lease_excutor.cpp b/curvefs/src/client/lease/lease_excutor.cpp index 7ab4746dba..13972db561 100644 --- a/curvefs/src/client/lease/lease_excutor.cpp +++ b/curvefs/src/client/lease/lease_excutor.cpp @@ -31,6 +31,10 @@ using curvefs::mds::topology::PartitionTxId; namespace curvefs { namespace client { +namespace common { +DECLARE_int32(TxVersion); +} // namespace common + LeaseExecutor::~LeaseExecutor() { if (task_) { task_->Stop(); @@ -71,10 +75,12 @@ void LeaseExecutor::Stop() { } bool LeaseExecutor::RefreshLease() { + // for tx v2 txIds and latestTxIdList will empty here // get partition txid list std::vector txIds; - metaCache_->GetAllTxIds(&txIds); - + if (common::FLAGS_TxVersion == 1) { + metaCache_->GetAllTxIds(&txIds); + } // refresh from mds std::vector latestTxIdList; std::string mdsAddrs = mdsCli_->GetMdsAddrs(); @@ -82,12 +88,12 @@ bool LeaseExecutor::RefreshLease() { FSStatusCode ret = mdsCli_->RefreshSession(txIds, &latestTxIdList, fsName_, mountpoint_, enableSumInDir_, mdsAddrs, &mdsAddrsOverride); + if (ret != FSStatusCode::OK) { LOG(ERROR) << "LeaseExecutor refresh session fail, ret = " << ret << ", errorName = " << FSStatusCode_Name(ret); return true; } - // update to metacache std::for_each(latestTxIdList.begin(), latestTxIdList.end(), [&](const PartitionTxId &item) { diff --git a/curvefs/src/client/metric/client_metric.h b/curvefs/src/client/metric/client_metric.h index 25c45acbd9..f5140b896d 100644 --- a/curvefs/src/client/metric/client_metric.h +++ b/curvefs/src/client/metric/client_metric.h @@ -55,6 +55,7 @@ struct MDSClientMetric { InterfaceMetric refreshSession; InterfaceMetric getLatestTxId; InterfaceMetric commitTx; + InterfaceMetric tso; InterfaceMetric allocOrGetMemcacheCluster; MDSClientMetric() @@ -70,6 +71,7 @@ struct MDSClientMetric { refreshSession(prefix, "refreshSession"), getLatestTxId(prefix, "getLatestTxId"), commitTx(prefix, "commitTx"), + tso(prefix, "tso"), allocOrGetMemcacheCluster(prefix, "allocOrGetMemcacheCluster") {} }; @@ -93,6 +95,10 @@ struct MetaServerClientMetric { // tnx InterfaceMetric prepareRenameTx; + InterfaceMetric prewriteRenameTx; + InterfaceMetric checkTxStatus; + InterfaceMetric resolveTxLock; + InterfaceMetric commitTx; // volume extent InterfaceMetric updateVolumeExtent; @@ -100,9 +106,11 @@ struct MetaServerClientMetric { InterfaceMetric updateDeallocatableBlockGroup; MetaServerClientMetric() - : getDentry(prefix, "getDentry"), listDentry(prefix, "listDentry"), + : getDentry(prefix, "getDentry"), + listDentry(prefix, "listDentry"), createDentry(prefix, "createDentry"), - deleteDentry(prefix, "deleteDentry"), getInode(prefix, "getInode"), + deleteDentry(prefix, "deleteDentry"), + getInode(prefix, "getInode"), batchGetInodeAttr(prefix, "batchGetInodeAttr"), batchGetXattr(prefix, "batchGetXattr"), createInode(prefix, "createInode"), @@ -110,10 +118,14 @@ struct MetaServerClientMetric { deleteInode(prefix, "deleteInode"), appendS3ChunkInfo(prefix, "appendS3ChunkInfo"), prepareRenameTx(prefix, "prepareRenameTx"), + prewriteRenameTx(prefix, "prewriteRenameTx"), + checkTxStatus(prefix, "checkTxStatus"), + resolveTxLock(prefix, "resolveTxLock"), + commitTx(prefix, "commitTx"), updateVolumeExtent(prefix, "updateVolumeExtent"), getVolumeExtent(prefix, "getVolumeExtent"), - updateDeallocatableBlockGroup(prefix, - "updateDeallocatableBlockGroup") {} + updateDeallocatableBlockGroup( + prefix, "updateDeallocatableBlockGroup") {} }; struct InflightGuard { diff --git a/curvefs/src/client/rpcclient/base_client.cpp b/curvefs/src/client/rpcclient/base_client.cpp index ca504f5201..a2f75ae177 100644 --- a/curvefs/src/client/rpcclient/base_client.cpp +++ b/curvefs/src/client/rpcclient/base_client.cpp @@ -167,6 +167,12 @@ void MDSBaseClient::CommitTx(const CommitTxRequest& request, stub.CommitTx(cntl, &request, response, nullptr); } +void MDSBaseClient::Tso(const TsoRequest& request, TsoResponse* response, + brpc::Controller* cntl, brpc::Channel* channel) { + curvefs::mds::MdsService_Stub stub(channel); + stub.Tso(cntl, &request, response, nullptr); +} + // TODO(all): do we really need pass `fsId` all the time? // each curve-fuse process only mount one filesystem void MDSBaseClient::AllocateVolumeBlockGroup( diff --git a/curvefs/src/client/rpcclient/base_client.h b/curvefs/src/client/rpcclient/base_client.h index f43d2854ee..7f3bd8161f 100644 --- a/curvefs/src/client/rpcclient/base_client.h +++ b/curvefs/src/client/rpcclient/base_client.h @@ -57,11 +57,11 @@ using curvefs::metaserver::GetInodeResponse; using curvefs::metaserver::Inode; using curvefs::metaserver::ListDentryRequest; using curvefs::metaserver::ListDentryResponse; -using curvefs::metaserver::PrepareRenameTxRequest; -using curvefs::metaserver::PrepareRenameTxResponse; +using curvefs::metaserver::ManageInodeType; +using curvefs::metaserver::PrewriteRenameTxRequest; +using curvefs::metaserver::PrewriteRenameTxResponse; using curvefs::metaserver::UpdateInodeRequest; using curvefs::metaserver::UpdateInodeResponse; -using curvefs::metaserver::ManageInodeType; using curvefs::common::FSType; using curvefs::common::PartitionInfo; @@ -84,6 +84,8 @@ using curvefs::mds::CommitTxRequest; using curvefs::mds::CommitTxResponse; using curvefs::mds::RefreshSessionRequest; using curvefs::mds::RefreshSessionResponse; +using curvefs::mds::TsoRequest; +using curvefs::mds::TsoResponse; using curvefs::mds::UmountFsRequest; using curvefs::mds::UmountFsResponse; @@ -197,6 +199,9 @@ class MDSBaseClient { brpc::Controller* cntl, brpc::Channel* channel); + virtual void Tso(const TsoRequest& request, TsoResponse* response, + brpc::Controller* cntl, brpc::Channel* channel); + virtual void AllocateVolumeBlockGroup(uint32_t fsId, uint32_t count, const std::string& owner, diff --git a/curvefs/src/client/rpcclient/mds_client.cpp b/curvefs/src/client/rpcclient/mds_client.cpp index 572a2cc03e..2f53aa3d7d 100644 --- a/curvefs/src/client/rpcclient/mds_client.cpp +++ b/curvefs/src/client/rpcclient/mds_client.cpp @@ -703,6 +703,38 @@ FSStatusCode MdsClientImpl::CommitTxWithLock( return CommitTx(request); } +FSStatusCode MdsClientImpl::Tso(uint64_t* ts, uint64_t* timestamp) { + auto task = RPCTask { + (void)addrindex; + (void)rpctimeoutMS; + mdsClientMetric_.tso.qps.count << 1; + LatencyUpdater updater(&mdsClientMetric_.tso.latency); + TsoRequest request; + TsoResponse response; + mdsbasecli_->Tso(request, &response, cntl, channel); + if (cntl->Failed()) { + mdsClientMetric_.tso.eps.count << 1; + LOG(WARNING) << "Tso Failed, errorcode = " << cntl->ErrorCode() + << ", error content:" << cntl->ErrorText() + << ", log id = " << cntl->log_id(); + return -cntl->ErrorCode(); + } + + FSStatusCode ret = response.statuscode(); + if (ret != FSStatusCode::OK) { + LOG(ERROR) << "Tso: errcode = " << ret + << ", errmsg = " << FSStatusCode_Name(ret); + return ret; + } else { + *ts = response.ts(); + *timestamp = response.timestamp(); + } + return ret; + }; + // for rpc error or failed/timeout, we will retry until success + return ReturnError(rpcexcutor_.DoRPCTask(task, 0)); +} + FSStatusCode MdsClientImpl::ReturnError(int retcode) { // rpc error convert to FSStatusCode::RPC_ERROR if (retcode < 0) { diff --git a/curvefs/src/client/rpcclient/mds_client.h b/curvefs/src/client/rpcclient/mds_client.h index ec2f2c2016..e74ec84c03 100644 --- a/curvefs/src/client/rpcclient/mds_client.h +++ b/curvefs/src/client/rpcclient/mds_client.h @@ -141,6 +141,8 @@ class MdsClient { const std::string& uuid, uint64_t sequence) = 0; + virtual FSStatusCode Tso(uint64_t* ts, uint64_t* timestamp) = 0; + // allocate block group virtual SpaceErrCode AllocateVolumeBlockGroup( uint32_t fsId, @@ -234,6 +236,8 @@ class MdsClientImpl : public MdsClient { const std::string& uuid, uint64_t sequence) override; + FSStatusCode Tso(uint64_t* ts, uint64_t* timestamp) override; + // allocate block group SpaceErrCode AllocateVolumeBlockGroup( uint32_t fsId, @@ -242,11 +246,9 @@ class MdsClientImpl : public MdsClient { std::vector *groups) override; // acquire block group - SpaceErrCode AcquireVolumeBlockGroup( - uint32_t fsId, - uint64_t blockGroupOffset, - const std::string &owner, - curvefs::mds::space::BlockGroup *groups) override; + SpaceErrCode AcquireVolumeBlockGroup(uint32_t fsId, + uint64_t blockGroupOffset, const std::string& owner, + curvefs::mds::space::BlockGroup* groups) override; // release block group SpaceErrCode ReleaseVolumeBlockGroup( diff --git a/curvefs/src/client/rpcclient/metacache.cpp b/curvefs/src/client/rpcclient/metacache.cpp index d3bd4fd294..4ca48b4935 100644 --- a/curvefs/src/client/rpcclient/metacache.cpp +++ b/curvefs/src/client/rpcclient/metacache.cpp @@ -102,7 +102,7 @@ bool MetaCache::GetTarget(uint32_t fsID, uint64_t inodeID, } if (!GetCopysetIDwithInodeID(inodeID, &target->groupID, - &target->partitionID, &target->txId)) { + &target->partitionID, &target->txId)) { LOG(ERROR) << "{fsid:" << fsID << ", inodeid:" << inodeID << "} do not find partition"; return false; @@ -496,9 +496,7 @@ bool MetaCache::SelectPartition(CopysetTarget *target) { } bool MetaCache::GetCopysetIDwithInodeID(uint64_t inodeID, - CopysetGroupID *groupID, - PartitionID *partitionID, - uint64_t *txId) { + CopysetGroupID* groupID, PartitionID* partitionID, uint64_t *txId) { ReadLockGuard rl(rwlock4Partitions_); for (auto iter = partitionInfos_.begin(); iter != partitionInfos_.end(); ++iter) { diff --git a/curvefs/src/client/rpcclient/metacache.h b/curvefs/src/client/rpcclient/metacache.h index c4c5589761..a9a1505d86 100644 --- a/curvefs/src/client/rpcclient/metacache.h +++ b/curvefs/src/client/rpcclient/metacache.h @@ -54,8 +54,15 @@ using ::curvefs::common::PartitionStatus; namespace curvefs { namespace client { + +namespace common { + DECLARE_int32(TxVersion); +} // namespace common + namespace rpcclient { +using curvefs::client::common::FLAGS_TxVersion; + struct CopysetGroupID { LogicPoolID poolID = 0; CopysetID copysetID = 0; @@ -83,8 +90,9 @@ struct CopysetTarget { bool IsValid() const { return groupID.poolID != 0 && groupID.copysetID != 0 && - partitionID != 0 && txId != 0 && metaServerID != 0 && - endPoint.ip != butil::IP_ANY && endPoint.port != 0; + partitionID != 0 && metaServerID != 0 && + endPoint.ip != butil::IP_ANY && endPoint.port != 0 && + (FLAGS_TxVersion != 1 || txId != 0); } void Reset() { @@ -179,8 +187,8 @@ class MetaCache { bool SelectPartition(CopysetTarget *target); // get info from partitionMap or copysetMap - bool GetCopysetIDwithInodeID(uint64_t inodeID, CopysetGroupID *groupID, - PartitionID *patitionID, uint64_t *txId); + bool GetCopysetIDwithInodeID(uint64_t inodeID, CopysetGroupID* groupID, + PartitionID* patitionID, uint64_t *txId); bool GetCopysetInfowithCopySetID(const CopysetGroupID &groupID, CopysetInfo *targetInfo); diff --git a/curvefs/src/client/rpcclient/metaserver_client.cpp b/curvefs/src/client/rpcclient/metaserver_client.cpp index 54d218f002..0f5d951134 100644 --- a/curvefs/src/client/rpcclient/metaserver_client.cpp +++ b/curvefs/src/client/rpcclient/metaserver_client.cpp @@ -45,8 +45,12 @@ using curvefs::metaserver::BatchGetInodeAttrRequest; using curvefs::metaserver::BatchGetInodeAttrResponse; using curvefs::metaserver::BatchGetXAttrRequest; using curvefs::metaserver::BatchGetXAttrResponse; +using curvefs::metaserver::CheckTxStatusRequest; +using curvefs::metaserver::CheckTxStatusResponse; using curvefs::metaserver::GetOrModifyS3ChunkInfoRequest; using curvefs::metaserver::GetOrModifyS3ChunkInfoResponse; +using curvefs::metaserver::ResolveTxLockRequest; +using curvefs::metaserver::ResolveTxLockResponse; namespace curvefs { namespace client { @@ -56,6 +60,10 @@ using GetDentryExcutor = TaskExecutor; using ListDentryExcutor = TaskExecutor; using DeleteDentryExcutor = TaskExecutor; using PrepareRenameTxExcutor = TaskExecutor; +using PrewriteRenameTxExcutor = TaskExecutor; +using CheckTxStatusExcutor = TaskExecutor; +using ResolveTxLockExcutor = TaskExecutor; +using CommitTxExcutor = TaskExecutor; using DeleteInodeExcutor = TaskExecutor; using UpdateInodeExcutor = TaskExecutor; using GetInodeExcutor = TaskExecutor; @@ -119,9 +127,15 @@ void MetaServerClientImpl::SetTxId(uint32_t partitionId, uint64_t txId) { metaCache_->SetTxId(partitionId, txId); } +bool MetaServerClientImpl::GetPartitionId(uint32_t fsId, uint64_t inodeId, + PartitionID *partitionId) { + return metaCache_->GetPartitionIdByInodeId(fsId, inodeId, partitionId); +} + MetaStatusCode MetaServerClientImpl::GetDentry(uint32_t fsId, uint64_t inodeid, const std::string &name, - Dentry *out) { + Dentry *out, + TxLock* txLockOut) { auto task = RPCTask { (void)taskExecutorDone; metric_.getDentry.qps.count << 1; @@ -150,6 +164,9 @@ MetaStatusCode MetaServerClientImpl::GetDentry(uint32_t fsId, uint64_t inodeid, MetaStatusCode ret = response.statuscode(); if (ret != MetaStatusCode::OK) { + if (ret == MetaStatusCode::TX_KEY_LOCKED) { + *txLockOut = response.txlock(); + } LOG_IF(WARNING, ret != MetaStatusCode::NOT_FOUND) << "GetDentry: fsId = " << fsId << ", inodeid = " << inodeid << ", name = " << name << ", errcode = " << ret @@ -171,8 +188,7 @@ MetaStatusCode MetaServerClientImpl::GetDentry(uint32_t fsId, uint64_t inodeid, }; auto taskCtx = std::make_shared(MetaServerOpType::GetDentry, - task, fsId, inodeid, false, - opt_.enableRenameParallel); + task, fsId, inodeid, false, opt_.enableRenameParallel); GetDentryExcutor excutor(opt_, metaCache_, channelManager_, std::move(taskCtx)); return ConvertToMetaStatusCode(excutor.DoRPCTask()); @@ -181,7 +197,8 @@ MetaStatusCode MetaServerClientImpl::GetDentry(uint32_t fsId, uint64_t inodeid, MetaStatusCode MetaServerClientImpl::ListDentry(uint32_t fsId, uint64_t inodeid, const std::string &last, uint32_t count, bool onlyDir, - std::list *dentryList) { + std::list *dentryList, + TxLock* txLockOut) { auto task = RPCTask { (void)taskExecutorDone; metric_.listDentry.qps.count << 1; @@ -212,6 +229,9 @@ MetaStatusCode MetaServerClientImpl::ListDentry(uint32_t fsId, uint64_t inodeid, MetaStatusCode ret = response.statuscode(); if (ret != MetaStatusCode::OK) { + if (ret == MetaStatusCode::TX_KEY_LOCKED) { + *txLockOut = response.txlock(); + } LOG(WARNING) << "ListDentry: fsId = " << fsId << ", inodeid = " << inodeid << ", last = " << last << ", count = " << count << ", onlyDir = " << onlyDir @@ -229,14 +249,14 @@ MetaStatusCode MetaServerClientImpl::ListDentry(uint32_t fsId, uint64_t inodeid, }; auto taskCtx = std::make_shared(MetaServerOpType::ListDentry, - task, fsId, inodeid, false, - opt_.enableRenameParallel); + task, fsId, inodeid, false, opt_.enableRenameParallel); ListDentryExcutor excutor(opt_, metaCache_, channelManager_, std::move(taskCtx)); return ConvertToMetaStatusCode(excutor.DoRPCTask()); } -MetaStatusCode MetaServerClientImpl::CreateDentry(const Dentry &dentry) { +MetaStatusCode MetaServerClientImpl::CreateDentry( + const Dentry &dentry, TxLock* txLockOut) { auto task = RPCTask { (void)taskExecutorDone; metric_.createDentry.qps.count << 1; @@ -274,7 +294,11 @@ MetaStatusCode MetaServerClientImpl::CreateDentry(const Dentry &dentry) { } MetaStatusCode ret = response.statuscode(); + if (ret != MetaStatusCode::OK) { + if (ret == MetaStatusCode::TX_KEY_LOCKED) { + *txLockOut = response.txlock(); + } LOG(WARNING) << "CreateDentry: dentry = " << dentry.DebugString() << ", errcode = " << ret << ", errmsg = " << MetaStatusCode_Name(ret); @@ -298,9 +322,8 @@ MetaStatusCode MetaServerClientImpl::CreateDentry(const Dentry &dentry) { } MetaStatusCode MetaServerClientImpl::DeleteDentry(uint32_t fsId, - uint64_t inodeid, - const std::string &name, - FsFileType type) { + uint64_t inodeid, const std::string &name, FsFileType type, + TxLock* txLockOut) { auto task = RPCTask { (void)taskExecutorDone; metric_.deleteDentry.qps.count << 1; @@ -330,6 +353,9 @@ MetaStatusCode MetaServerClientImpl::DeleteDentry(uint32_t fsId, MetaStatusCode ret = response.statuscode(); if (ret != MetaStatusCode::OK) { + if (ret == MetaStatusCode::TX_KEY_LOCKED) { + *txLockOut = response.txlock(); + } LOG(WARNING) << "DeleteDentry: fsid = " << fsId << ", inodeid = " << inodeid << ", name = " << name << ", errcode = " << ret @@ -342,22 +368,21 @@ MetaStatusCode MetaServerClientImpl::DeleteDentry(uint32_t fsId, }; auto taskCtx = std::make_shared(MetaServerOpType::DeleteDentry, - task, fsId, inodeid, false, - opt_.enableRenameParallel); + task, fsId, inodeid, false, opt_.enableRenameParallel); DeleteDentryExcutor excutor(opt_, metaCache_, channelManager_, std::move(taskCtx)); return ConvertToMetaStatusCode(excutor.DoRPCTask()); } -MetaStatusCode -MetaServerClientImpl::PrepareRenameTx(const std::vector &dentrys) { +MetaStatusCode MetaServerClientImpl::PrepareRenameTx( + const std::vector &dentrys) { auto task = RPCTask { (void)txId; (void)taskExecutorDone; metric_.prepareRenameTx.qps.count << 1; LatencyUpdater updater(&metric_.prepareRenameTx.latency); - PrepareRenameTxRequest request; - PrepareRenameTxResponse response; + curvefs::metaserver::PrepareRenameTxRequest request; + curvefs::metaserver::PrepareRenameTxResponse response; request.set_poolid(poolID); request.set_copysetid(copysetID); request.set_partitionid(partitionID); @@ -395,6 +420,167 @@ MetaServerClientImpl::PrepareRenameTx(const std::vector &dentrys) { return ConvertToMetaStatusCode(excutor.DoRPCTask()); } +MetaStatusCode MetaServerClientImpl::PrewriteRenameTx( + const std::vector& dentrys, + const TxLock& txLockIn, TxLock* txLockOut) { + auto task = RPCTask { + (void)txId; + (void)taskExecutorDone; + metric_.prewriteRenameTx.qps.count << 1; + LatencyUpdater updater(&metric_.prewriteRenameTx.latency); + PrewriteRenameTxRequest request; + PrewriteRenameTxResponse response; + request.set_poolid(poolID); + request.set_copysetid(copysetID); + request.set_partitionid(partitionID); + *request.mutable_dentrys() = {dentrys.begin(), dentrys.end()}; + *request.mutable_txlock() = txLockIn; + + curvefs::metaserver::MetaServerService_Stub stub(channel); + stub.PrewriteRenameTx(cntl, &request, &response, nullptr); + + if (cntl->Failed()) { + metric_.prewriteRenameTx.eps.count << 1; + LOG(WARNING) << "PrewriteRenameTx Failed, errorcode = " + << cntl->ErrorCode() + << ", error content:" << cntl->ErrorText() + << ", request = " << request.DebugString() + << ", log id = " << cntl->log_id(); + return -cntl->ErrorCode(); + } + + MetaStatusCode ret = response.statuscode(); + if (ret != MetaStatusCode::OK) { + LOG(WARNING) << "PrewriteRenameTx: ret = " << ret + << ", errmsg = " << MetaStatusCode_Name(ret); + } else { + *txLockOut = response.txlock(); + } + + VLOG(6) << "PrewriteRenameTx done, request: " << request.DebugString() + << "response: " << response.DebugString(); + return ret; + }; + auto taskCtx = std::make_shared( + MetaServerOpType::PrewriteRenameTx, task, dentrys[0].fsid(), + dentrys[0].parentinodeid()); + PrewriteRenameTxExcutor excutor(opt_, metaCache_, channelManager_, + std::move(taskCtx)); + return ConvertToMetaStatusCode(excutor.DoRPCTask()); +} + +MetaStatusCode MetaServerClientImpl::CheckTxStatus(uint32_t fsId, + uint64_t inodeId, const std::string& primaryKey, uint64_t startTs, + uint64_t curTimestamp) { + auto task = RPCTask { + (void)txId; + (void)taskExecutorDone; + metric_.checkTxStatus.qps.count << 1; + LatencyUpdater updater(&metric_.checkTxStatus.latency); + CheckTxStatusRequest request; + CheckTxStatusResponse response; + request.set_poolid(poolID); + request.set_copysetid(copysetID); + request.set_partitionid(partitionID); + request.set_primarykey(primaryKey); + request.set_startts(startTs); + request.set_curtimestamp(curTimestamp); + + curvefs::metaserver::MetaServerService_Stub stub(channel); + stub.CheckTxStatus(cntl, &request, &response, nullptr); + + if (cntl->Failed()) { + metric_.checkTxStatus.eps.count << 1; + LOG(WARNING) << "CheckTxStatus failed" + << ", errorCode = " << cntl->ErrorCode() + << ", errorText = " << cntl->ErrorText() + << ", request = " << request.DebugString() + << ", logId = " << cntl->log_id(); + return -cntl->ErrorCode(); + } + + return response.statuscode(); + }; + auto taskCtx = std::make_shared( + MetaServerOpType::CheckTxStatus, task, fsId, inodeId); + CheckTxStatusExcutor excutor( + opt_, metaCache_, channelManager_, std::move(taskCtx)); + return ConvertToMetaStatusCode(excutor.DoRPCTask()); +} + +MetaStatusCode MetaServerClientImpl::ResolveTxLock(const Dentry& dentry, + uint64_t startTs, uint64_t commitTs) { + auto task = RPCTask { + (void)txId; + (void)taskExecutorDone; + metric_.resolveTxLock.qps.count << 1; + LatencyUpdater updater(&metric_.resolveTxLock.latency); + ResolveTxLockRequest request; + ResolveTxLockResponse response; + request.set_poolid(poolID); + request.set_copysetid(copysetID); + request.set_partitionid(partitionID); + *request.mutable_dentry() = dentry; + request.set_startts(startTs); + request.set_committs(commitTs); + + curvefs::metaserver::MetaServerService_Stub stub(channel); + stub.ResolveTxLock(cntl, &request, &response, nullptr); + + if (cntl->Failed()) { + metric_.resolveTxLock.eps.count << 1; + LOG(WARNING) << "ResolveTxLock failed" + << ", errorCode = " << cntl->ErrorCode() + << ", errorText = " << cntl->ErrorText() + << ", logId = " << cntl->log_id(); + return -cntl->ErrorCode(); + } + return response.statuscode(); + }; + auto taskCtx = + std::make_shared(MetaServerOpType::ResolveTxLock, task, + dentry.fsid(), dentry.parentinodeid()); + ResolveTxLockExcutor excutor( + opt_, metaCache_, channelManager_, std::move(taskCtx)); + return ConvertToMetaStatusCode(excutor.DoRPCTask()); +} + +MetaStatusCode MetaServerClientImpl::CommitTx( + const std::vector& dentrys, uint64_t startTs, uint64_t commitTs) { + auto task = RPCTask { + (void)txId; + (void)taskExecutorDone; + metric_.commitTx.qps.count << 1; + LatencyUpdater updater(&metric_.commitTx.latency); + curvefs::metaserver::CommitTxRequest request; + curvefs::metaserver::CommitTxResponse response; + request.set_poolid(poolID); + request.set_copysetid(copysetID); + request.set_partitionid(partitionID); + *request.mutable_dentrys() = {dentrys.begin(), dentrys.end()}; + request.set_startts(startTs); + request.set_committs(commitTs); + + curvefs::metaserver::MetaServerService_Stub stub(channel); + stub.CommitTx(cntl, &request, &response, nullptr); + + if (cntl->Failed()) { + metric_.commitTx.eps.count << 1; + LOG(WARNING) << "CommitTx failed" + << ", errorCode = " << cntl->ErrorCode() + << ", errorText = " << cntl->ErrorText() + << ", logId = " << cntl->log_id(); + return -cntl->ErrorCode(); + } + return response.statuscode(); + }; + auto taskCtx = std::make_shared(MetaServerOpType::CommitTx, + task, dentrys[0].fsid(), dentrys[0].parentinodeid()); + CommitTxExcutor excutor( + opt_, metaCache_, channelManager_, std::move(taskCtx)); + return ConvertToMetaStatusCode(excutor.DoRPCTask()); +} + MetaStatusCode MetaServerClientImpl::GetInode(uint32_t fsId, uint64_t inodeid, Inode *out, bool *streaming) { auto task = RPCTask { diff --git a/curvefs/src/client/rpcclient/metaserver_client.h b/curvefs/src/client/rpcclient/metaserver_client.h index 747685ecf3..ddce3c64ba 100644 --- a/curvefs/src/client/rpcclient/metaserver_client.h +++ b/curvefs/src/client/rpcclient/metaserver_client.h @@ -45,17 +45,17 @@ #include "absl/types/optional.h" using ::curvefs::client::metric::MetaServerClientMetric; +using ::curvefs::common::StreamClient; +using ::curvefs::common::StreamStatus; +using ::curvefs::metaserver::DeallocatableBlockGroup; using ::curvefs::metaserver::Dentry; using ::curvefs::metaserver::FsFileType; using ::curvefs::metaserver::Inode; using ::curvefs::metaserver::InodeAttr; -using ::curvefs::metaserver::XAttr; -using ::curvefs::metaserver::MetaStatusCode; using ::curvefs::metaserver::S3ChunkInfoList; -using ::curvefs::metaserver::DeallocatableBlockGroup; -using ::curvefs::common::StreamStatus; -using ::curvefs::common::StreamClient; using ::curvefs::metaserver::Time; +using ::curvefs::metaserver::TxLock; +using ::curvefs::metaserver::XAttr; using DeallocatableBlockGroupMap = std::map; using S3ChunkInfoMap = google::protobuf::Map; @@ -94,22 +94,36 @@ class MetaServerClient { virtual void SetTxId(uint32_t partitionId, uint64_t txId) = 0; virtual MetaStatusCode GetDentry(uint32_t fsId, uint64_t inodeid, - const std::string &name, Dentry *out) = 0; + const std::string &name, Dentry *out, TxLock* txLockOut) = 0; virtual MetaStatusCode ListDentry(uint32_t fsId, uint64_t inodeid, const std::string &last, uint32_t count, bool onlyDir, - std::list *dentryList) = 0; + std::list *dentryList, + TxLock* txLockOut) = 0; - virtual MetaStatusCode CreateDentry(const Dentry &dentry) = 0; + virtual MetaStatusCode CreateDentry( + const Dentry &dentry, TxLock* txLockOut) = 0; virtual MetaStatusCode DeleteDentry(uint32_t fsId, uint64_t inodeid, - const std::string &name, - FsFileType type) = 0; + const std::string &name, FsFileType type, TxLock* txLockOut) = 0; virtual MetaStatusCode PrepareRenameTx(const std::vector &dentrys) = 0; + virtual MetaStatusCode PrewriteRenameTx(const std::vector& dentrys, + const TxLock& txLockIn, TxLock* txLockOut) = 0; + + virtual MetaStatusCode CheckTxStatus(uint32_t fsId, uint64_t inodeId, + const std::string& primaryKey, uint64_t startTs, + uint64_t curTimestamp) = 0; + + virtual MetaStatusCode ResolveTxLock(const Dentry& dentry, + uint64_t startTs, uint64_t commitTs) = 0; + + virtual MetaStatusCode CommitTx(const std::vector& dentry, + uint64_t startTs, uint64_t commitTs) = 0; + virtual MetaStatusCode GetInode(uint32_t fsId, uint64_t inodeid, Inode *out, bool* streaming) = 0; @@ -185,6 +199,9 @@ class MetaServerClient { virtual MetaStatusCode UpdateDeallocatableBlockGroup(uint32_t fsId, uint64_t inodeId, DeallocatableBlockGroupMap *statistic) = 0; + + virtual bool GetPartitionId(uint32_t fsId, uint64_t inodeId, + PartitionID* partitionId) = 0; }; class MetaServerClientImpl : public MetaServerClient { @@ -202,21 +219,35 @@ class MetaServerClientImpl : public MetaServerClient { void SetTxId(uint32_t partitionId, uint64_t txId) override; MetaStatusCode GetDentry(uint32_t fsId, uint64_t inodeid, - const std::string &name, Dentry *out) override; + const std::string &name, Dentry *out, TxLock* txLockOut) override; MetaStatusCode ListDentry(uint32_t fsId, uint64_t inodeid, const std::string &last, uint32_t count, bool onlyDir, - std::list *dentryList) override; + std::list *dentryList, + TxLock* txLockOut) override; - MetaStatusCode CreateDentry(const Dentry &dentry) override; + MetaStatusCode CreateDentry( + const Dentry &dentry, TxLock* txLockOut) override; MetaStatusCode DeleteDentry(uint32_t fsId, uint64_t inodeid, - const std::string &name, - FsFileType type) override; + const std::string &name, FsFileType type, TxLock* txLockOut) override; MetaStatusCode PrepareRenameTx(const std::vector &dentrys) override; + MetaStatusCode PrewriteRenameTx(const std::vector& dentrys, + const TxLock& txLockIn, TxLock* txLockOut) override; + + MetaStatusCode CheckTxStatus(uint32_t fsId, uint64_t inodeId, + const std::string& primaryKey, uint64_t startTs, + uint64_t curTimestamp) override; + + MetaStatusCode ResolveTxLock(const Dentry& dentry, + uint64_t startTs, uint64_t commitTs) override; + + MetaStatusCode CommitTx(const std::vector& dentrys, + uint64_t startTs, uint64_t commitTs) override; + MetaStatusCode GetInode(uint32_t fsId, uint64_t inodeid, Inode *out, bool* streaming) override; @@ -294,6 +325,9 @@ class MetaServerClientImpl : public MetaServerClient { uint32_t fsId, uint64_t inodeId, DeallocatableBlockGroupMap *statistic) override; + bool GetPartitionId(uint32_t fsId, uint64_t inodeId, + PartitionID *partitionId) override; + private: MetaStatusCode UpdateInode(const UpdateInodeRequest &request, bool internal = false); @@ -317,6 +351,7 @@ class MetaServerClientImpl : public MetaServerClient { StreamClient streamClient_; MetaServerClientMetric metric_; }; + } // namespace rpcclient } // namespace client } // namespace curvefs diff --git a/curvefs/src/client/rpcclient/task_excutor.cpp b/curvefs/src/client/rpcclient/task_excutor.cpp index 191cff6e08..6d5d939369 100644 --- a/curvefs/src/client/rpcclient/task_excutor.cpp +++ b/curvefs/src/client/rpcclient/task_excutor.cpp @@ -29,8 +29,8 @@ #include "curvefs/proto/metaserver.pb.h" #include "curvefs/src/common/define.h" -using ::curvefs::metaserver::MetaStatusCode; using ::curvefs::RECYCLEINODEID; +using ::curvefs::metaserver::MetaStatusCode; namespace curvefs { namespace client { @@ -160,6 +160,10 @@ bool TaskExecutor::OnReturn(int retCode) { needRetry = true; break; + case MetaStatusCode::TX_INPROGRESS: + needRetry = true; + break; + default: break; } @@ -233,9 +237,8 @@ int TaskExecutor::ExcuteTask(brpc::Channel *channel, task_->cntl_.Reset(); task_->cntl_.set_timeout_ms(task_->rpcTimeoutMs); return task_->rpctask(task_->target.groupID.poolID, - task_->target.groupID.copysetID, - task_->target.partitionID, task_->target.txId, - channel, &task_->cntl_, done); + task_->target.groupID.copysetID, task_->target.partitionID, + task_->target.txId, channel, &task_->cntl_, done); } void TaskExecutor::OnSuccess() {} diff --git a/curvefs/src/client/rpcclient/task_excutor.h b/curvefs/src/client/rpcclient/task_excutor.h index 18a69346e5..23df4d1938 100644 --- a/curvefs/src/client/rpcclient/task_excutor.h +++ b/curvefs/src/client/rpcclient/task_excutor.h @@ -45,13 +45,19 @@ using ::curvefs::client::common::ExcutorOpt; using ::curvefs::client::common::MetaserverID; using ::curvefs::client::common::MetaServerOpType; using ::curvefs::common::PartitionInfo; -using ::curvefs::metaserver::MetaStatusCode; -using ::google::protobuf::RepeatedPtrField; using ::curvefs::metaserver::Inode; using ::curvefs::metaserver::InodeAttr; +using ::curvefs::metaserver::MetaStatusCode; +using ::google::protobuf::RepeatedPtrField; +using ::curvefs::client::common::FLAGS_TxVersion; namespace curvefs { namespace client { + +namespace common { + DECLARE_int32(TxVersion); +} // namespace common + namespace rpcclient { class TaskExecutorDone; @@ -60,10 +66,9 @@ MetaStatusCode ConvertToMetaStatusCode(int retcode); class TaskContext { public: - using RpcFunc = std::function; + using RpcFunc = std::function; TaskContext() = default; TaskContext(MetaServerOpType type, @@ -77,7 +82,7 @@ class TaskContext { fsID(fsid), inodeID(inodeid), streaming(streaming), - refreshTxId(refreshTxId) {} + refreshTxId(FLAGS_TxVersion == 1 ? refreshTxId : false) {} std::string TaskContextStr() { std::ostringstream oss; diff --git a/curvefs/src/client/s3/client_s3_adaptor.h b/curvefs/src/client/s3/client_s3_adaptor.h index dcc25c3795..453efb8931 100644 --- a/curvefs/src/client/s3/client_s3_adaptor.h +++ b/curvefs/src/client/s3/client_s3_adaptor.h @@ -34,7 +34,7 @@ #include "curvefs/src/client/common/common.h" #include "curvefs/src/client/common/config.h" #include "curvefs/src/client/filesystem/error.h" -#include "curvefs/src/client/inode_cache_manager.h" +#include "curvefs/src/client/inode_manager.h" #include "curvefs/src/client/rpcclient/mds_client.h" #include "curvefs/src/client/s3/client_s3.h" #include "curvefs/src/client/s3/client_s3_cache_manager.h" diff --git a/curvefs/src/client/volume/default_volume_storage.cpp b/curvefs/src/client/volume/default_volume_storage.cpp index 4b2618b170..010b1e1902 100644 --- a/curvefs/src/client/volume/default_volume_storage.cpp +++ b/curvefs/src/client/volume/default_volume_storage.cpp @@ -31,7 +31,7 @@ #include "absl/meta/type_traits.h" #include "curvefs/src/client/filesystem/error.h" -#include "curvefs/src/client/inode_cache_manager.h" +#include "curvefs/src/client/inode_manager.h" #include "curvefs/src/client/inode_wrapper.h" #include "curvefs/src/client/volume/extent_cache.h" #include "curvefs/src/client/volume/utils.h" diff --git a/curvefs/src/client/warmup/warmup_manager.h b/curvefs/src/client/warmup/warmup_manager.h index f55752c801..e83772ed35 100644 --- a/curvefs/src/client/warmup/warmup_manager.h +++ b/curvefs/src/client/warmup/warmup_manager.h @@ -40,9 +40,9 @@ #include #include "curvefs/src/client/common/common.h" -#include "curvefs/src/client/dentry_cache_manager.h" +#include "curvefs/src/client/dentry_manager.h" #include "curvefs/src/client/fuse_common.h" -#include "curvefs/src/client/inode_cache_manager.h" +#include "curvefs/src/client/inode_manager.h" #include "curvefs/src/client/kvclient/kvclient_manager.h" #include "curvefs/src/client/metric/client_metric.h" #include "curvefs/src/client/rpcclient/metaserver_client.h" diff --git a/curvefs/src/mds/codec/codec.cpp b/curvefs/src/mds/codec/codec.cpp index 3bee130c98..1089533842 100644 --- a/curvefs/src/mds/codec/codec.cpp +++ b/curvefs/src/mds/codec/codec.cpp @@ -33,6 +33,7 @@ namespace codec { using ::curve::common::EncodeBigEndian; using ::curve::common::EncodeBigEndian_uint32; +using ::curve::common::DecodeBigEndian_uint32; using ::curvefs::mds::BLOCKGROUP_KEY_END; using ::curvefs::mds::BLOCKGROUP_KEY_PREFIX; using ::curvefs::mds::COMMON_PREFIX_LENGTH; diff --git a/curvefs/src/mds/common/storage_key.h b/curvefs/src/mds/common/storage_key.h index 8b2eb9f1b1..81a7ae85b3 100644 --- a/curvefs/src/mds/common/storage_key.h +++ b/curvefs/src/mds/common/storage_key.h @@ -45,26 +45,27 @@ const char BLOCKGROUP_KEY_PREFIX[] = "fs_04"; const char BLOCKGROUP_KEY_END[] = "fs_05"; const char FS_USAGE_KEY_PREFIX[] = "fs_05"; const char FS_USAGE_KEY_END[] = "fs_06"; +const char TS_INFO_KEY_PREFIX[] = "fs_07"; constexpr uint32_t COMMON_PREFIX_LENGTH = 5; -const char POOLKEYPREFIX[] = "fs_1001"; -const char POOLKEYEND[] = "fs_1002"; -const char ZONEKEYPREFIX[] = "fs_1002"; -const char ZONEKEYEND[] = "fs_1003"; -const char SERVERKEYPREFIX[] = "fs_1003"; -const char SERVERKEYEND[] = "fs_1004"; -const char METASERVERKEYPREFIX[] = "fs_1004"; -const char METASERVERKEYEND[] = "fs_1005"; -const char CLUSTERINFOKEY[] = "fs_1006"; -const char COPYSETKEYPREFIX[] = "fs_1007"; -const char COPYSETKEYEND[] = "fs_1008"; -const char PARTITIONKEYPREFIX[] = "fs_1008"; -const char PARTITIONKEYEND[] = "fs_1009"; -const char MEMCACHECLUSTERKEYPREFIX[] = "fs_1009"; -const char MEMCACHECLUSTERKEYEND[] = "fs_1010"; -const char FS2MEMCACHECLUSTERKEYPREFIX[] = "fs_1010"; -const char FS2MEMCACHECLUSTERKEYEND[] = "fs_1011"; +const char POOL_KEY_PREFIX[] = "fs_1001"; +const char POOL_KEY_END[] = "fs_1002"; +const char ZONE_KEY_PREFIX[] = "fs_1002"; +const char ZONE_KEY_END[] = "fs_1003"; +const char SERVER_KEY_PREFIX[] = "fs_1003"; +const char SERVER_KEY_END[] = "fs_1004"; +const char METASERVER_KEY_PREFIX[] = "fs_1004"; +const char METASERVER_KEY_END[] = "fs_1005"; +const char CLUSTER_KEY[] = "fs_1006"; +const char COPYSET_KEY_PREFIX[] = "fs_1007"; +const char COPYSET_KEY_END[] = "fs_1008"; +const char PARTITION_KEY_PREFIX[] = "fs_1008"; +const char PARTITION_KEY_END[] = "fs_1009"; +const char MEMCACHE_CLUSTER_KEY_PREFIX[] = "fs_1009"; +const char MEMCACHE_CLUSTER_KEY_END[] = "fs_1010"; +const char FS_2_MEMCACHE_CLUSTER_KEY_PREFIX[] = "fs_1010"; +const char FS_2_MEMCACHE_CLUSTER_KEY_END[] = "fs_1011"; constexpr uint32_t TOPOLOGY_PREFIX_LENGTH = 7; diff --git a/curvefs/src/mds/fs_manager.cpp b/curvefs/src/mds/fs_manager.cpp index 86ea1f80f2..7c6a1aa51a 100644 --- a/curvefs/src/mds/fs_manager.cpp +++ b/curvefs/src/mds/fs_manager.cpp @@ -1317,5 +1317,19 @@ void FsManager::SetClientMdsAddrsOverride(const std::string& addrs) { clientMdsAddrsOverride_ = addrsWithActiveMds; } +void FsManager::Tso(const TsoRequest* request, TsoResponse* response) { + uint64_t ts; + uint64_t timestamp; + auto ret = fsStorage_->Tso(&ts, ×tamp); + if (ret != FSStatusCode::OK) { + LOG(ERROR) << "Tso fail, ret = " << FSStatusCode_Name(ret); + response->set_statuscode(ret); + return; + } + response->set_ts(ts); + response->set_timestamp(timestamp); + response->set_statuscode(ret); +} + } // namespace mds } // namespace curvefs diff --git a/curvefs/src/mds/fs_manager.h b/curvefs/src/mds/fs_manager.h index 064ad6feee..c1f3ed1e2b 100644 --- a/curvefs/src/mds/fs_manager.h +++ b/curvefs/src/mds/fs_manager.h @@ -203,6 +203,8 @@ class FsManager { void CommitTx(const CommitTxRequest* request, CommitTxResponse* response); + void Tso(const TsoRequest* request, TsoResponse* response); + // periodically check if the mount point is alive void BackEndCheckMountPoint(); void CheckMountPoint(); diff --git a/curvefs/src/mds/fs_storage.cpp b/curvefs/src/mds/fs_storage.cpp index b38b9f9803..be5fc466ee 100644 --- a/curvefs/src/mds/fs_storage.cpp +++ b/curvefs/src/mds/fs_storage.cpp @@ -30,6 +30,7 @@ #include "curvefs/src/mds/codec/codec.h" #include "curvefs/src/mds/metric/fs_metric.h" +#include "src/common/timeutility.h" namespace curvefs { namespace mds { @@ -40,6 +41,7 @@ using ::curve::kvstorage::KVStorageClient; bool MemoryFsStorage::Init() { WriteLockGuard writeLockGuard(rwLock_); fsInfoMap_.clear(); + tsId_.store(1); return true; } @@ -186,6 +188,12 @@ FSStatusCode MemoryFsStorage::DeleteFsUsage(const std::string& fsName) { return FSStatusCode::OK; } +FSStatusCode MemoryFsStorage::Tso(uint64_t* ts, uint64_t* timestamp) { + *timestamp = curve::common::TimeUtility::GetTimeofDayMs(); + *ts = tsId_.fetch_add(1, std::memory_order_relaxed); + return FSStatusCode::OK; +} + PersisKVStorage::PersisKVStorage( const std::shared_ptr& storage) : storage_(storage), @@ -193,7 +201,8 @@ PersisKVStorage::PersisKVStorage( fsLock_(), fs_(), idToNameLock_(), - idToName_() {} + idToName_(), + tsIdGen_(new TsIdGenerator(storage_)) {} PersisKVStorage::~PersisKVStorage() = default; @@ -207,8 +216,11 @@ FSStatusCode PersisKVStorage::Get(uint64_t fsId, FsInfoWrapper* fsInfo) { } bool PersisKVStorage::Init() { - bool ret = LoadAllFs(); - return ret; + if (!LoadAllFs()) { + LOG(ERROR) << "Load all fs failed"; + return false; + } + return true; } void PersisKVStorage::Uninit() {} @@ -570,5 +582,14 @@ FSStatusCode PersisKVStorage::DeleteFsUsage(const std::string& fsName) { return FSStatusCode::OK; } +FSStatusCode PersisKVStorage::Tso(uint64_t* ts, uint64_t* timestamp) { + *timestamp = curve::common::TimeUtility::CLockRealTimeMs(); + if (tsIdGen_->GenTsId(ts)) { + return FSStatusCode::OK; + } + LOG(ERROR) << "Gen ts failed"; + return FSStatusCode::INTERNAL_ERROR; +} + } // namespace mds } // namespace curvefs diff --git a/curvefs/src/mds/fs_storage.h b/curvefs/src/mds/fs_storage.h index d2ed8d674d..fcf1105469 100644 --- a/curvefs/src/mds/fs_storage.h +++ b/curvefs/src/mds/fs_storage.h @@ -37,6 +37,7 @@ #include "src/common/concurrent/rw_lock.h" #include "src/idgenerator/etcd_id_generator.h" #include "src/kvstorageclient/etcd_client.h" +#include "curvefs/src/mds/idgenerator/ts_id_generator.h" namespace curvefs { namespace mds { @@ -76,6 +77,8 @@ class FsStorage { virtual FSStatusCode GetFsUsage( const std::string& fsName, FsUsage* fsUsage, bool fromCache) = 0; virtual FSStatusCode DeleteFsUsage(const std::string& fsName) = 0; + + virtual FSStatusCode Tso(uint64_t* ts, uint64_t* timestamp) = 0; }; class MemoryFsStorage : public FsStorage { @@ -182,6 +185,8 @@ class MemoryFsStorage : public FsStorage { const std::string& fsName, FsUsage*, bool fromCache) override; FSStatusCode DeleteFsUsage(const std::string& fsName) override; + FSStatusCode Tso(uint64_t* ts, uint64_t* timestamp) override; + private: std::unordered_map fsInfoMap_; curve::common::RWLock rwLock_; @@ -190,6 +195,8 @@ class MemoryFsStorage : public FsStorage { std::unordered_map fsUsageMap_; curve::common::RWLock fsUsedUsageLock_; + + std::atomic tsId_; }; // Persist all data to kvstorage and cache all fsinfo in memory @@ -225,6 +232,8 @@ class PersisKVStorage : public FsStorage { const std::string& fsName, FsUsage*, bool fromCache) override; FSStatusCode DeleteFsUsage(const std::string& fsName) override; + FSStatusCode Tso(uint64_t* ts, uint64_t* timestamp) override; + private: bool LoadAllFs(); @@ -259,6 +268,8 @@ class PersisKVStorage : public FsStorage { // fs usage cache map std::unordered_map fsUsageCache_; mutable RWLock fsUsageCacheMutex_; + + std::unique_ptr tsIdGen_; }; } // namespace mds diff --git a/curvefs/src/mds/idgenerator/BUILD b/curvefs/src/mds/idgenerator/BUILD index fdde4ecd6b..fc465923ab 100644 --- a/curvefs/src/mds/idgenerator/BUILD +++ b/curvefs/src/mds/idgenerator/BUILD @@ -18,7 +18,8 @@ load("//:copts.bzl", "CURVE_DEFAULT_COPTS") cc_library( name = "fs_mds_idgenerator", - hdrs = ["fs_id_generator.h"], + hdrs = ["fs_id_generator.h", + "ts_id_generator.h"], copts = CURVE_DEFAULT_COPTS, visibility = ["//visibility:public"], deps = [ diff --git a/curvefs/src/mds/idgenerator/ts_id_generator.h b/curvefs/src/mds/idgenerator/ts_id_generator.h new file mode 100644 index 0000000000..68060d1384 --- /dev/null +++ b/curvefs/src/mds/idgenerator/ts_id_generator.h @@ -0,0 +1,54 @@ +/* + * Copyright (c) 2023 NetEase Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Project: curve + * Created Date: 2023-12-07 + * Author: wanghai01 + */ + +#ifndef CURVEFS_SRC_MDS_IDGENERATOR_TS_ID_GENERATOR_H_ +#define CURVEFS_SRC_MDS_IDGENERATOR_TS_ID_GENERATOR_H_ + +#include "curvefs/src/mds/common/storage_key.h" +#include "src/idgenerator/etcd_id_generator.h" + +namespace curvefs { +namespace mds { + +class TsIdGenerator { + public: + explicit TsIdGenerator( + const std::shared_ptr& client) + : generator_(new curve::idgenerator::EtcdIdGenerator( + client, TS_INFO_KEY_PREFIX, TS_ID_INIT, TS_ID_ALLOCATE_BUNDLE)) {} + + bool GenTsId(uint64_t* id) { + return generator_->GenID(id); + } + + private: + static constexpr uint64_t TS_ID_INIT = 0; + static constexpr uint64_t TS_ID_ALLOCATE_BUNDLE = 100; + + private: + std::unique_ptr generator_; +}; + +} // namespace mds +} // namespace curvefs + +#endif // CURVEFS_SRC_MDS_IDGENERATOR_TS_ID_GENERATOR_H_ diff --git a/curvefs/src/mds/mds_service.cpp b/curvefs/src/mds/mds_service.cpp index e8b6461b00..17c33156c2 100644 --- a/curvefs/src/mds/mds_service.cpp +++ b/curvefs/src/mds/mds_service.cpp @@ -29,10 +29,9 @@ namespace mds { using mds::Mountpoint; -void MdsServiceImpl::CreateFs(::google::protobuf::RpcController *controller, - const ::curvefs::mds::CreateFsRequest *request, - ::curvefs::mds::CreateFsResponse *response, - ::google::protobuf::Closure *done) { +void MdsServiceImpl::CreateFs(::google::protobuf::RpcController* controller, + const CreateFsRequest* request, CreateFsResponse* response, + ::google::protobuf::Closure* done) { (void)controller; brpc::ClosureGuard doneGuard(done); const std::string &fsName = request->fsname(); @@ -142,10 +141,9 @@ void MdsServiceImpl::CreateFs(::google::protobuf::RpcController *controller, << ", capacity = " << request->capacity(); } -void MdsServiceImpl::MountFs(::google::protobuf::RpcController *controller, - const ::curvefs::mds::MountFsRequest *request, - ::curvefs::mds::MountFsResponse *response, - ::google::protobuf::Closure *done) { +void MdsServiceImpl::MountFs(::google::protobuf::RpcController* controller, + const MountFsRequest* request, MountFsResponse* response, + ::google::protobuf::Closure* done) { (void)controller; brpc::ClosureGuard doneGuard(done); const std::string &fsName = request->fsname(); @@ -169,10 +167,9 @@ void MdsServiceImpl::MountFs(::google::protobuf::RpcController *controller, << ", mps: " << response->mutable_fsinfo()->mountpoints_size(); } -void MdsServiceImpl::UmountFs(::google::protobuf::RpcController *controller, - const ::curvefs::mds::UmountFsRequest *request, - ::curvefs::mds::UmountFsResponse *response, - ::google::protobuf::Closure *done) { +void MdsServiceImpl::UmountFs(::google::protobuf::RpcController* controller, + const UmountFsRequest* request, UmountFsResponse* response, + ::google::protobuf::Closure* done) { (void)controller; brpc::ClosureGuard doneGuard(done); const std::string &fsName = request->fsname(); @@ -192,10 +189,9 @@ void MdsServiceImpl::UmountFs(::google::protobuf::RpcController *controller, << ", mountPoint = " << mount.ShortDebugString(); } -void MdsServiceImpl::GetFsInfo(::google::protobuf::RpcController *controller, - const ::curvefs::mds::GetFsInfoRequest *request, - ::curvefs::mds::GetFsInfoResponse *response, - ::google::protobuf::Closure *done) { +void MdsServiceImpl::GetFsInfo(::google::protobuf::RpcController* controller, + const GetFsInfoRequest* request, GetFsInfoResponse* response, + ::google::protobuf::Closure* done) { (void)controller; brpc::ClosureGuard doneGuard(done); @@ -227,10 +223,8 @@ void MdsServiceImpl::GetFsInfo(::google::protobuf::RpcController *controller, << response->ShortDebugString(); } -void MdsServiceImpl::UpdateFsInfo( - ::google::protobuf::RpcController* controller, - const ::curvefs::mds::UpdateFsInfoRequest* request, - ::curvefs::mds::UpdateFsInfoResponse* response, +void MdsServiceImpl::UpdateFsInfo(::google::protobuf::RpcController* controller, + const UpdateFsInfoRequest* request, UpdateFsInfoResponse* response, ::google::protobuf::Closure* done) { (void)controller; brpc::ClosureGuard doneGuard(done); @@ -261,10 +255,9 @@ void MdsServiceImpl::UpdateFsInfo( << response->ShortDebugString(); } -void MdsServiceImpl::DeleteFs(::google::protobuf::RpcController *controller, - const ::curvefs::mds::DeleteFsRequest *request, - ::curvefs::mds::DeleteFsResponse *response, - ::google::protobuf::Closure *done) { +void MdsServiceImpl::DeleteFs(::google::protobuf::RpcController* controller, + const DeleteFsRequest* request, DeleteFsResponse* response, + ::google::protobuf::Closure* done) { (void)controller; brpc::ClosureGuard doneGuard(done); const std::string &fsName = request->fsname(); @@ -281,10 +274,9 @@ void MdsServiceImpl::DeleteFs(::google::protobuf::RpcController *controller, } void MdsServiceImpl::AllocateS3Chunk( - ::google::protobuf::RpcController *controller, - const ::curvefs::mds::AllocateS3ChunkRequest *request, - ::curvefs::mds::AllocateS3ChunkResponse *response, - ::google::protobuf::Closure *done) { + ::google::protobuf::RpcController* controller, + const AllocateS3ChunkRequest* request, AllocateS3ChunkResponse* response, + ::google::protobuf::Closure* done) { (void)controller; brpc::ClosureGuard guard(done); @@ -318,10 +310,9 @@ void MdsServiceImpl::AllocateS3Chunk( } void MdsServiceImpl::ListClusterFsInfo( - ::google::protobuf::RpcController *controller, - const ::curvefs::mds::ListClusterFsInfoRequest *request, - ::curvefs::mds::ListClusterFsInfoResponse *response, - ::google::protobuf::Closure *done) { + ::google::protobuf::RpcController* controller, + const ListClusterFsInfoRequest* request, + ListClusterFsInfoResponse* response, ::google::protobuf::Closure* done) { (void)controller; (void)request; @@ -333,10 +324,9 @@ void MdsServiceImpl::ListClusterFsInfo( } void MdsServiceImpl::RefreshSession( - ::google::protobuf::RpcController *controller, - const ::curvefs::mds::RefreshSessionRequest *request, - ::curvefs::mds::RefreshSessionResponse *response, - ::google::protobuf::Closure *done) { + ::google::protobuf::RpcController* controller, + const RefreshSessionRequest* request, RefreshSessionResponse* response, + ::google::protobuf::Closure* done) { (void)controller; brpc::ClosureGuard guard(done); fsManager_->RefreshSession(request, response); @@ -380,5 +370,15 @@ void MdsServiceImpl::SetClientMdsAddrsOverride( << response->DebugString(); } +void MdsServiceImpl::Tso(::google::protobuf::RpcController* controller, + const TsoRequest* request, TsoResponse* response, + ::google::protobuf::Closure* done) { + (void)controller; + brpc::ClosureGuard guard(done); + VLOG(3) << "Tso [request]: " << request->DebugString(); + fsManager_->Tso(request, response); + VLOG(3) << "Tso [response]: " << response->DebugString(); +} + } // namespace mds } // namespace curvefs diff --git a/curvefs/src/mds/mds_service.h b/curvefs/src/mds/mds_service.h index 645db7285d..dbcfaaf6f4 100644 --- a/curvefs/src/mds/mds_service.h +++ b/curvefs/src/mds/mds_service.h @@ -76,26 +76,24 @@ class MdsServiceImpl : public MdsService { ::google::protobuf::Closure* done); void AllocateS3Chunk(::google::protobuf::RpcController* controller, - const ::curvefs::mds::AllocateS3ChunkRequest* request, - ::curvefs::mds::AllocateS3ChunkResponse* response, - ::google::protobuf::Closure* done); + const AllocateS3ChunkRequest* request, + AllocateS3ChunkResponse* response, ::google::protobuf::Closure* done); - void ListClusterFsInfo( - ::google::protobuf::RpcController* controller, - const ::curvefs::mds::ListClusterFsInfoRequest* request, - ::curvefs::mds::ListClusterFsInfoResponse* response, - ::google::protobuf::Closure* done); + void ListClusterFsInfo(::google::protobuf::RpcController* controller, + const ListClusterFsInfoRequest* request, + ListClusterFsInfoResponse* response, ::google::protobuf::Closure* done); - void RefreshSession(::google::protobuf::RpcController *controller, - const ::curvefs::mds::RefreshSessionRequest *request, - ::curvefs::mds::RefreshSessionResponse *response, - ::google::protobuf::Closure *done); + void RefreshSession(::google::protobuf::RpcController* controller, + const RefreshSessionRequest* request, RefreshSessionResponse* response, + ::google::protobuf::Closure* done); + // reserved for compatibility void GetLatestTxId(::google::protobuf::RpcController* controller, const GetLatestTxIdRequest* request, GetLatestTxIdResponse* response, ::google::protobuf::Closure* done); + // reserved for compatibility void CommitTx(::google::protobuf::RpcController* controller, const CommitTxRequest* request, CommitTxResponse* response, @@ -107,6 +105,10 @@ class MdsServiceImpl : public MdsService { SetClientMdsAddrsOverrideResponse* response, ::google::protobuf::Closure* done); + void Tso(::google::protobuf::RpcController* controller, + const TsoRequest* request, TsoResponse* response, + ::google::protobuf::Closure* done); + private: std::shared_ptr fsManager_; std::shared_ptr chunkIdAllocator_; diff --git a/curvefs/src/mds/topology/topology_storage_codec.cpp b/curvefs/src/mds/topology/topology_storage_codec.cpp index 717a0a624b..7ba8edb105 100644 --- a/curvefs/src/mds/topology/topology_storage_codec.cpp +++ b/curvefs/src/mds/topology/topology_storage_codec.cpp @@ -30,12 +30,12 @@ namespace curvefs { namespace mds { namespace topology { -using curvefs::mds::FS2MEMCACHECLUSTERKEYPREFIX; -using curvefs::mds::MEMCACHECLUSTERKEYEND; -using curvefs::mds::MEMCACHECLUSTERKEYPREFIX; +using curvefs::mds::FS_2_MEMCACHE_CLUSTER_KEY_PREFIX; +using curvefs::mds::MEMCACHE_CLUSTER_KEY_END; +using curvefs::mds::MEMCACHE_CLUSTER_KEY_PREFIX; std::string TopologyStorageCodec::EncodePoolKey(PoolIdType id) { - std::string key = POOLKEYPREFIX; + std::string key = POOL_KEY_PREFIX; size_t prefixLen = TOPOLOGY_PREFIX_LENGTH; key.resize(prefixLen + sizeof(uint64_t)); EncodeBigEndian(&(key[prefixLen]), id); @@ -53,7 +53,7 @@ bool TopologyStorageCodec::DecodePoolData(const std::string &value, } std::string TopologyStorageCodec::EncodeZoneKey(ZoneIdType id) { - std::string key = ZONEKEYPREFIX; + std::string key = ZONE_KEY_PREFIX; size_t prefixLen = TOPOLOGY_PREFIX_LENGTH; key.resize(prefixLen + sizeof(uint64_t)); EncodeBigEndian(&(key[prefixLen]), id); @@ -71,7 +71,7 @@ bool TopologyStorageCodec::DecodeZoneData(const std::string &value, } std::string TopologyStorageCodec::EncodeServerKey(ServerIdType id) { - std::string key = SERVERKEYPREFIX; + std::string key = SERVER_KEY_PREFIX; size_t prefixLen = TOPOLOGY_PREFIX_LENGTH; key.resize(prefixLen + sizeof(uint64_t)); EncodeBigEndian(&(key[prefixLen]), id); @@ -89,7 +89,7 @@ bool TopologyStorageCodec::DecodeServerData(const std::string &value, } std::string TopologyStorageCodec::EncodeMetaServerKey(MetaServerIdType id) { - std::string key = METASERVERKEYPREFIX; + std::string key = METASERVER_KEY_PREFIX; size_t prefixLen = TOPOLOGY_PREFIX_LENGTH; key.resize(prefixLen + sizeof(uint64_t)); EncodeBigEndian(&(key[prefixLen]), id); @@ -107,7 +107,7 @@ bool TopologyStorageCodec::DecodeMetaServerData(const std::string &value, } std::string TopologyStorageCodec::EncodeCopySetKey(const CopySetKey &id) { - std::string key = COPYSETKEYPREFIX; + std::string key = COPYSET_KEY_PREFIX; size_t prefixLen = TOPOLOGY_PREFIX_LENGTH; key.resize(prefixLen + sizeof(uint64_t) + sizeof(uint64_t)); EncodeBigEndian(&(key[prefixLen]), id.first); @@ -126,7 +126,7 @@ bool TopologyStorageCodec::DecodeCopySetData(const std::string &value, } std::string TopologyStorageCodec::EncodePartitionKey(PartitionIdType id) { - std::string key = PARTITIONKEYPREFIX; + std::string key = PARTITION_KEY_PREFIX; size_t prefixLen = TOPOLOGY_PREFIX_LENGTH; key.resize(prefixLen + sizeof(uint64_t)); EncodeBigEndian(&(key[prefixLen]), id); @@ -155,7 +155,7 @@ bool TopologyStorageCodec::DecodeClusterInfoData(const std::string &value, std::string TopologyStorageCodec::EncodeMemcacheClusterKey( MetaServerIdType id) { - std::string key = MEMCACHECLUSTERKEYPREFIX; + std::string key = MEMCACHE_CLUSTER_KEY_PREFIX; size_t prefixLen = TOPOLOGY_PREFIX_LENGTH; key.resize(prefixLen + sizeof(uint64_t)); EncodeBigEndian(&(key[prefixLen]), id); @@ -173,7 +173,7 @@ bool TopologyStorageCodec::DecodeMemcacheClusterData(const std::string& value, } std::string TopologyStorageCodec::EncodeFs2MemcacheClusterKey(FsIdType fsId) { - std::string key = FS2MEMCACHECLUSTERKEYPREFIX; + std::string key = FS_2_MEMCACHE_CLUSTER_KEY_PREFIX; size_t prefixLen = TOPOLOGY_PREFIX_LENGTH; key.resize(prefixLen + sizeof(uint64_t)); EncodeBigEndian(&(key[prefixLen]), fsId); diff --git a/curvefs/src/mds/topology/topology_storage_codec.h b/curvefs/src/mds/topology/topology_storage_codec.h index d5bdc3df24..a6e3f7bb86 100644 --- a/curvefs/src/mds/topology/topology_storage_codec.h +++ b/curvefs/src/mds/topology/topology_storage_codec.h @@ -34,18 +34,18 @@ namespace curvefs { namespace mds { namespace topology { -using curvefs::mds::POOLKEYPREFIX; -using curvefs::mds::POOLKEYEND; -using curvefs::mds::ZONEKEYPREFIX; -using curvefs::mds::ZONEKEYEND; -using curvefs::mds::SERVERKEYPREFIX; -using curvefs::mds::SERVERKEYEND; -using curvefs::mds::METASERVERKEYPREFIX; -using curvefs::mds::METASERVERKEYEND; -using curvefs::mds::CLUSTERINFOKEY; -using curvefs::mds::COPYSETKEYPREFIX; -using curvefs::mds::COPYSETKEYEND; using curve::common::EncodeBigEndian; +using curvefs::mds::CLUSTER_KEY; +using curvefs::mds::COPYSET_KEY_END; +using curvefs::mds::COPYSET_KEY_PREFIX; +using curvefs::mds::METASERVER_KEY_END; +using curvefs::mds::METASERVER_KEY_PREFIX; +using curvefs::mds::POOL_KEY_END; +using curvefs::mds::POOL_KEY_PREFIX; +using curvefs::mds::SERVER_KEY_END; +using curvefs::mds::SERVER_KEY_PREFIX; +using curvefs::mds::ZONE_KEY_END; +using curvefs::mds::ZONE_KEY_PREFIX; class TopologyStorageCodec { public: diff --git a/curvefs/src/mds/topology/topology_storge_etcd.cpp b/curvefs/src/mds/topology/topology_storge_etcd.cpp index b59eb23a09..c70aa7dd8d 100644 --- a/curvefs/src/mds/topology/topology_storge_etcd.cpp +++ b/curvefs/src/mds/topology/topology_storge_etcd.cpp @@ -40,7 +40,7 @@ bool TopologyStorageEtcd::LoadPool( std::vector out; poolMap->clear(); *maxPoolId = 0; - int errCode = client_->List(POOLKEYPREFIX, POOLKEYEND, &out); + int errCode = client_->List(POOL_KEY_PREFIX, POOL_KEY_END, &out); if (errCode == EtcdErrCode::EtcdKeyNotExist) { return true; } @@ -76,7 +76,7 @@ bool TopologyStorageEtcd::LoadZone( std::vector out; zoneMap->clear(); *maxZoneId = 0; - int errCode = client_->List(ZONEKEYPREFIX, ZONEKEYEND, &out); + int errCode = client_->List(ZONE_KEY_PREFIX, ZONE_KEY_END, &out); if (errCode == EtcdErrCode::EtcdKeyNotExist) { return true; } @@ -112,7 +112,7 @@ bool TopologyStorageEtcd::LoadServer( std::vector out; serverMap->clear(); *maxServerId = 0; - int errCode = client_->List(SERVERKEYPREFIX, SERVERKEYEND, &out); + int errCode = client_->List(SERVER_KEY_PREFIX, SERVER_KEY_END, &out); if (errCode == EtcdErrCode::EtcdKeyNotExist) { return true; } @@ -148,7 +148,8 @@ bool TopologyStorageEtcd::LoadMetaServer( std::vector out; metaServerMap->clear(); *maxMetaServerId = 0; - int errCode = client_->List(METASERVERKEYPREFIX, METASERVERKEYEND, &out); + int errCode = + client_->List(METASERVER_KEY_PREFIX, METASERVER_KEY_END, &out); if (errCode == EtcdErrCode::EtcdKeyNotExist) { return true; } @@ -185,7 +186,7 @@ bool TopologyStorageEtcd::LoadCopySet( std::vector out; copySetMap->clear(); copySetIdMaxMap->clear(); - int errCode = client_->List(COPYSETKEYPREFIX, COPYSETKEYEND, &out); + int errCode = client_->List(COPYSET_KEY_PREFIX, COPYSET_KEY_END, &out); if (errCode == EtcdErrCode::EtcdKeyNotExist) { return true; } @@ -223,7 +224,7 @@ bool TopologyStorageEtcd::LoadPartition( std::vector out; partitionMap->clear(); *maxPartitionId = 0; - int errCode = client_->List(PARTITIONKEYPREFIX, PARTITIONKEYEND, &out); + int errCode = client_->List(PARTITION_KEY_PREFIX, PARTITION_KEY_END, &out); if (errCode == EtcdErrCode::EtcdKeyNotExist) { return true; } @@ -508,7 +509,7 @@ bool TopologyStorageEtcd::UpdatePartitions( bool TopologyStorageEtcd::LoadClusterInfo( std::vector *info) { std::string value; - int errCode = client_->Get(CLUSTERINFOKEY, &value); + int errCode = client_->Get(CLUSTER_KEY, &value); if (errCode == EtcdErrCode::EtcdKeyNotExist) { return true; } @@ -528,7 +529,7 @@ bool TopologyStorageEtcd::LoadClusterInfo( } bool TopologyStorageEtcd::StorageClusterInfo(const ClusterInformation &info) { - std::string key = CLUSTERINFOKEY; + std::string key = CLUSTER_KEY; std::string value; if (codec_->EncodeClusterInfoData(info, &value) != true) { @@ -551,8 +552,8 @@ bool TopologyStorageEtcd::LoadMemcacheCluster( std::vector out; memcacheClusterMap->clear(); *maxMemCacheClusterId = 0; - int errCode = - client_->List(MEMCACHECLUSTERKEYPREFIX, MEMCACHECLUSTERKEYEND, &out); + int errCode = client_->List( + MEMCACHE_CLUSTER_KEY_PREFIX, MEMCACHE_CLUSTER_KEY_END, &out); if (errCode == EtcdErrCode::EtcdKeyNotExist) { return true; } @@ -617,8 +618,8 @@ bool TopologyStorageEtcd::LoadFs2MemcacheCluster( std::unordered_map* fs2MemcacheCluster) { std::vector> out; fs2MemcacheCluster->clear(); - int errCode = client_->List(FS2MEMCACHECLUSTERKEYPREFIX, - FS2MEMCACHECLUSTERKEYEND, &out); + int errCode = client_->List( + FS_2_MEMCACHE_CLUSTER_KEY_PREFIX, FS_2_MEMCACHE_CLUSTER_KEY_END, &out); if (errCode == EtcdErrCode::EtcdKeyNotExist) { return true; } diff --git a/curvefs/src/metaserver/BUILD b/curvefs/src/metaserver/BUILD index df1696159a..1de9d79033 100644 --- a/curvefs/src/metaserver/BUILD +++ b/curvefs/src/metaserver/BUILD @@ -27,6 +27,9 @@ cc_library( ["copyset/*.cpp"], ) + glob( ["storage/*.cpp"], + exclude = [ + "storage/converter.cpp", + ], ) + glob( ["streaming/*.cpp"], ) + glob( @@ -40,6 +43,9 @@ cc_library( ["copyset/*.h"], ) + glob( ["storage/*.h"], + exclude = [ + "storage/converter.h", + ], ) + glob( ["streaming/*.h"], ) + glob( @@ -51,6 +57,7 @@ cc_library( visibility = ["//visibility:public"], deps = [ ":metaserver_s3_lib", + ":metaserver_storage_conv", "//curvefs/proto:cli2_cc_proto", "//curvefs/proto:copyset_cc_proto", "//curvefs/proto:curvefs_heartbeat_cc_proto", @@ -77,6 +84,21 @@ cc_library( ], ) +cc_library( + name = "metaserver_storage_conv", + srcs = ["storage/converter.cpp"], + hdrs = ["storage/converter.h", + "storage/common.h",], + copts = CURVE_DEFAULT_COPTS, + visibility = ["//visibility:public"], + deps = [ + "//external:glog", + "//src/common:curve_common", + "//curvefs/proto:metaserver_cc_proto", + "@com_google_absl//absl/container:btree", + ], +) + cc_library( name = "metaserver_s3_lib", srcs = glob(["s3/*.cpp", "mdsclient/*.cpp"]), diff --git a/curvefs/src/metaserver/copyset/meta_operator.cpp b/curvefs/src/metaserver/copyset/meta_operator.cpp index 456fd9c361..356ab12d35 100644 --- a/curvefs/src/metaserver/copyset/meta_operator.cpp +++ b/curvefs/src/metaserver/copyset/meta_operator.cpp @@ -139,6 +139,7 @@ OPERATOR_CAN_BY_PASS_PROPOSE(GetInode); OPERATOR_CAN_BY_PASS_PROPOSE(BatchGetInodeAttr); OPERATOR_CAN_BY_PASS_PROPOSE(BatchGetXAttr); OPERATOR_CAN_BY_PASS_PROPOSE(GetVolumeExtent); +OPERATOR_CAN_BY_PASS_PROPOSE(CheckTxStatus); #undef OPERATOR_CAN_BY_PASS_PROPOSE @@ -184,6 +185,10 @@ OPERATOR_ON_APPLY(CreateManageInode); OPERATOR_ON_APPLY(CreatePartition); OPERATOR_ON_APPLY(DeletePartition); OPERATOR_ON_APPLY(PrepareRenameTx); +OPERATOR_ON_APPLY(PrewriteRenameTx); +OPERATOR_ON_APPLY(CheckTxStatus); +OPERATOR_ON_APPLY(ResolveTxLock); +OPERATOR_ON_APPLY(CommitTx); OPERATOR_ON_APPLY(UpdateVolumeExtent); OPERATOR_ON_APPLY(UpdateDeallocatableBlockGroup); @@ -311,6 +316,9 @@ OPERATOR_ON_APPLY_FROM_LOG(CreateManageInode); OPERATOR_ON_APPLY_FROM_LOG(CreatePartition); OPERATOR_ON_APPLY_FROM_LOG(DeletePartition); OPERATOR_ON_APPLY_FROM_LOG(PrepareRenameTx); +OPERATOR_ON_APPLY_FROM_LOG(PrewriteRenameTx); +OPERATOR_ON_APPLY_FROM_LOG(ResolveTxLock); +OPERATOR_ON_APPLY_FROM_LOG(CommitTx); OPERATOR_ON_APPLY_FROM_LOG(UpdateVolumeExtent); OPERATOR_ON_APPLY_FROM_LOG(UpdateDeallocatableBlockGroup); @@ -346,6 +354,7 @@ READONLY_OPERATOR_ON_APPLY_FROM_LOG(GetInode); READONLY_OPERATOR_ON_APPLY_FROM_LOG(BatchGetInodeAttr); READONLY_OPERATOR_ON_APPLY_FROM_LOG(BatchGetXAttr); READONLY_OPERATOR_ON_APPLY_FROM_LOG(GetVolumeExtent); +READONLY_OPERATOR_ON_APPLY_FROM_LOG(CheckTxStatus); #undef READONLY_OPERATOR_ON_APPLY_FROM_LOG @@ -371,6 +380,10 @@ OPERATOR_REDIRECT(CreateManageInode); OPERATOR_REDIRECT(CreatePartition); OPERATOR_REDIRECT(DeletePartition); OPERATOR_REDIRECT(PrepareRenameTx); +OPERATOR_REDIRECT(PrewriteRenameTx); +OPERATOR_REDIRECT(CheckTxStatus); +OPERATOR_REDIRECT(ResolveTxLock); +OPERATOR_REDIRECT(CommitTx); OPERATOR_REDIRECT(GetVolumeExtent); OPERATOR_REDIRECT(UpdateVolumeExtent); OPERATOR_REDIRECT(UpdateDeallocatableBlockGroup); @@ -398,6 +411,10 @@ OPERATOR_ON_FAILED(CreateManageInode); OPERATOR_ON_FAILED(CreatePartition); OPERATOR_ON_FAILED(DeletePartition); OPERATOR_ON_FAILED(PrepareRenameTx); +OPERATOR_ON_FAILED(PrewriteRenameTx); +OPERATOR_ON_FAILED(CheckTxStatus); +OPERATOR_ON_FAILED(ResolveTxLock); +OPERATOR_ON_FAILED(CommitTx); OPERATOR_ON_FAILED(GetVolumeExtent); OPERATOR_ON_FAILED(UpdateVolumeExtent); OPERATOR_ON_FAILED(UpdateDeallocatableBlockGroup); @@ -423,6 +440,10 @@ OPERATOR_HASH_CODE(DeleteInode); OPERATOR_HASH_CODE(CreateRootInode); OPERATOR_HASH_CODE(CreateManageInode); OPERATOR_HASH_CODE(PrepareRenameTx); +OPERATOR_HASH_CODE(PrewriteRenameTx); +OPERATOR_HASH_CODE(CheckTxStatus); +OPERATOR_HASH_CODE(ResolveTxLock); +OPERATOR_HASH_CODE(CommitTx); OPERATOR_HASH_CODE(DeletePartition); OPERATOR_HASH_CODE(GetVolumeExtent); OPERATOR_HASH_CODE(UpdateVolumeExtent); @@ -461,6 +482,10 @@ OPERATOR_TYPE(DeleteInode); OPERATOR_TYPE(CreateRootInode); OPERATOR_TYPE(CreateManageInode); OPERATOR_TYPE(PrepareRenameTx); +OPERATOR_TYPE(PrewriteRenameTx); +OPERATOR_TYPE(CheckTxStatus); +OPERATOR_TYPE(ResolveTxLock); +OPERATOR_TYPE(CommitTx); OPERATOR_TYPE(CreatePartition); OPERATOR_TYPE(DeletePartition); OPERATOR_TYPE(GetVolumeExtent); diff --git a/curvefs/src/metaserver/copyset/meta_operator.h b/curvefs/src/metaserver/copyset/meta_operator.h index b78aa3e78f..d861913999 100644 --- a/curvefs/src/metaserver/copyset/meta_operator.h +++ b/curvefs/src/metaserver/copyset/meta_operator.h @@ -470,6 +470,65 @@ class DeletePartitionOperator : public MetaOperator { void OnFailed(MetaStatusCode code) override; }; +class GetVolumeExtentOperator : public MetaOperator { + public: + using MetaOperator::MetaOperator; + + void OnApply(int64_t index, google::protobuf::Closure* done, + uint64_t startTimeUs) override; + + void OnApplyFromLog(int64_t index, uint64_t startTimeUs) override; + + uint64_t HashCode() const override; + + OperatorType GetOperatorType() const override; + + private: + void Redirect() override; + + void OnFailed(MetaStatusCode code) override; + + bool CanBypassPropose() const override; +}; + +class UpdateVolumeExtentOperator : public MetaOperator { + public: + using MetaOperator::MetaOperator; + + void OnApply(int64_t index, google::protobuf::Closure* done, + uint64_t startTimeUs) override; + + void OnApplyFromLog(int64_t index, uint64_t startTimeUs) override; + + uint64_t HashCode() const override; + + OperatorType GetOperatorType() const override; + + private: + void Redirect() override; + + void OnFailed(MetaStatusCode code) override; +}; + +class UpdateDeallocatableBlockGroupOperator : public MetaOperator { + public: + using MetaOperator::MetaOperator; + + void OnApply(int64_t index, google::protobuf::Closure* done, + uint64_t startTimeUs) override; + + void OnApplyFromLog(int64_t index, uint64_t startTimeUs) override; + + uint64_t HashCode() const override; + + OperatorType GetOperatorType() const override; + + private: + void Redirect() override; + + void OnFailed(MetaStatusCode code) override; +}; + class PrepareRenameTxOperator : public MetaOperator { public: using MetaOperator::MetaOperator; @@ -489,7 +548,26 @@ class PrepareRenameTxOperator : public MetaOperator { void OnFailed(MetaStatusCode code) override; }; -class GetVolumeExtentOperator : public MetaOperator { +class PrewriteRenameTxOperator : public MetaOperator { + public: + using MetaOperator::MetaOperator; + + void OnApply(int64_t index, google::protobuf::Closure* done, + uint64_t startTimeUs) override; + + void OnApplyFromLog(int64_t index, uint64_t startTimeUs) override; + + uint64_t HashCode() const override; + + OperatorType GetOperatorType() const override; + + private: + void Redirect() override; + + void OnFailed(MetaStatusCode code) override; +}; + +class CheckTxStatusOperator : public MetaOperator { public: using MetaOperator::MetaOperator; @@ -510,7 +588,7 @@ class GetVolumeExtentOperator : public MetaOperator { bool CanBypassPropose() const override; }; -class UpdateVolumeExtentOperator : public MetaOperator { +class ResolveTxLockOperator : public MetaOperator { public: using MetaOperator::MetaOperator; @@ -529,7 +607,7 @@ class UpdateVolumeExtentOperator : public MetaOperator { void OnFailed(MetaStatusCode code) override; }; -class UpdateDeallocatableBlockGroupOperator : public MetaOperator { +class CommitTxOperator : public MetaOperator { public: using MetaOperator::MetaOperator; diff --git a/curvefs/src/metaserver/copyset/operator_type.cpp b/curvefs/src/metaserver/copyset/operator_type.cpp index adcee67671..e7627f6712 100644 --- a/curvefs/src/metaserver/copyset/operator_type.cpp +++ b/curvefs/src/metaserver/copyset/operator_type.cpp @@ -68,6 +68,14 @@ const char* OperatorTypeName(OperatorType type) { return "UpdateVolumeExtent"; case OperatorType::UpdateDeallocatableBlockGroup: return "UpdateDeallocatableBlockGroup"; + case OperatorType::PrewriteRenameTx: + return "PrewriteRenameTx"; + case OperatorType::CheckTxStatus: + return "CheckTxStatus"; + case OperatorType::ResolveTxLock: + return "ResolveTxLock"; + case OperatorType::CommitTx: + return "CommitTx"; // Add new case before `OperatorType::OperatorTypeMax` case OperatorType::OperatorTypeMax: break; diff --git a/curvefs/src/metaserver/copyset/operator_type.h b/curvefs/src/metaserver/copyset/operator_type.h index c2e54c44bf..cd8a97d987 100644 --- a/curvefs/src/metaserver/copyset/operator_type.h +++ b/curvefs/src/metaserver/copyset/operator_type.h @@ -53,6 +53,10 @@ enum class OperatorType : uint32_t { UpdateVolumeExtent = 16, CreateManageInode = 17, UpdateDeallocatableBlockGroup = 18, + PrewriteRenameTx = 19, + CheckTxStatus = 20, + ResolveTxLock = 21, + CommitTx = 22, // NOTE: // Add new operator before `OperatorTypeMax` diff --git a/curvefs/src/metaserver/copyset/raft_log_codec.cpp b/curvefs/src/metaserver/copyset/raft_log_codec.cpp index d70799f25c..c1f36e5233 100644 --- a/curvefs/src/metaserver/copyset/raft_log_codec.cpp +++ b/curvefs/src/metaserver/copyset/raft_log_codec.cpp @@ -154,6 +154,18 @@ std::unique_ptr RaftLogCodec::Decode(CopysetNode* node, case OperatorType::PrepareRenameTx: return ParseFromRaftLog(node, type, meta); + case OperatorType::PrewriteRenameTx: + return ParseFromRaftLog(node, type, meta); + case OperatorType::CheckTxStatus: + return ParseFromRaftLog(node, type, meta); + case OperatorType::ResolveTxLock: + return ParseFromRaftLog(node, type, meta); + case OperatorType::CommitTx: + return ParseFromRaftLog( + node, type, meta); case OperatorType::GetOrModifyS3ChunkInfo: return ParseFromRaftLog( diff --git a/curvefs/src/metaserver/dentry_manager.cpp b/curvefs/src/metaserver/dentry_manager.cpp index 46b0102369..d1bb0a053c 100644 --- a/curvefs/src/metaserver/dentry_manager.cpp +++ b/curvefs/src/metaserver/dentry_manager.cpp @@ -76,10 +76,10 @@ void DentryManager::Log4Code(const std::string& request, MetaStatusCode rc) { } MetaStatusCode DentryManager::CreateDentry(const Dentry& dentry, - int64_t logIndex) { + int64_t logIndex, TxLock* txLock) { CHECK_APPLIED(); Log4Dentry("CreateDentry", dentry); - MetaStatusCode rc = dentryStorage_->Insert(dentry, logIndex); + MetaStatusCode rc = dentryStorage_->Insert(dentry, logIndex, txLock); Log4Code("CreateDentry", rc); return rc; } @@ -95,26 +95,28 @@ MetaStatusCode DentryManager::CreateDentry(const DentryVec& vec, bool merge, } MetaStatusCode DentryManager::DeleteDentry(const Dentry& dentry, - int64_t logIndex) { + int64_t logIndex, TxLock* txLock) { CHECK_APPLIED(); Log4Dentry("DeleteDentry", dentry); - MetaStatusCode rc = dentryStorage_->Delete(dentry, logIndex); + MetaStatusCode rc = dentryStorage_->Delete(dentry, logIndex, txLock); Log4Code("DeleteDentry", rc); return rc; } -MetaStatusCode DentryManager::GetDentry(Dentry* dentry) { +MetaStatusCode DentryManager::GetDentry(Dentry* dentry, TxLock* txLock) { Log4Dentry("GetDentry", *dentry); - MetaStatusCode rc = dentryStorage_->Get(dentry); + MetaStatusCode rc = dentryStorage_->Get(dentry, txLock); Log4Code("GetDentry", rc); return rc; } MetaStatusCode DentryManager::ListDentry(const Dentry& dentry, std::vector* dentrys, - uint32_t limit, bool onlyDir) { + uint32_t limit, bool onlyDir, + TxLock* txLock) { Log4Dentry("ListDentry", dentry); - MetaStatusCode rc = dentryStorage_->List(dentry, dentrys, limit, onlyDir); + MetaStatusCode rc = dentryStorage_->List( + dentry, dentrys, limit, onlyDir, txLock); Log4Code("ListDentry", rc); return rc; } @@ -134,5 +136,94 @@ MetaStatusCode DentryManager::HandleRenameTx(const std::vector& dentrys, return rc; } +MetaStatusCode DentryManager::PrewriteRenameTx( + const std::vector& dentrys, + const TxLock& txLock, int64_t logIndex, TxLock* out) { + std::stringstream ss; + for (const auto& dentry : dentrys) { + ss << dentry.ShortDebugString() << ", "; + } + VLOG(1) << "PrewriteRenameTx request, dentrys = (" << ss.str() + << "), txLock = ("<< txLock.ShortDebugString() << ")"; + auto rc = dentryStorage_->PrewriteTx(dentrys, txLock, logIndex, out); + if (rc != MetaStatusCode::OK && rc != MetaStatusCode::TX_WRITE_CONFLICT + && rc != MetaStatusCode::TX_KEY_LOCKED) { + LOG(ERROR) << "PrewriteRenameTx failed, dentrys = (" << ss.str() + << "), inLock = (" << txLock.ShortDebugString() + << "), retCode = " << MetaStatusCode_Name(rc); + } else { + VLOG(1) << "PrewriteRenameTx success, dentrys = (" << ss.str() + << "), inLock = ("<< txLock.ShortDebugString() << ")" + << ", outLock = (" << out->ShortDebugString() << ")" + << ", status = " << MetaStatusCode_Name(rc); + } + return rc; +} + +MetaStatusCode DentryManager::CheckTxStatus(const std::string& primaryKey, + uint64_t startTs, uint64_t curTimestamp, int64_t logIndex) { + LOG(INFO) << "CheckTxStatus request, primaryKey = " << primaryKey + << ", startTs = " << startTs + << ", curTimestamp = " << curTimestamp; + auto rc = dentryStorage_->CheckTxStatus(primaryKey, startTs, curTimestamp, + logIndex); + if (rc != MetaStatusCode::TX_COMMITTED && + rc != MetaStatusCode::TX_ROLLBACKED && + rc != MetaStatusCode::TX_INPROGRESS) { + LOG(ERROR) << "CheckTxStatus failed, primaryKey = " << primaryKey + << ", startTs = " << startTs + << ", curTimestamp = " << curTimestamp + << ", retCode = " << MetaStatusCode_Name(rc); + } else { + LOG(INFO) << "CheckTxStatus success, primaryKey = " << primaryKey + << ", startTs = " << startTs + << ", curTimestamp = " << curTimestamp; + } + return rc; +} + +MetaStatusCode DentryManager::ResolveTxLock(const Dentry& dentry, + uint64_t startTs, uint64_t commitTs, int64_t logIndex) { + LOG(INFO) << "ResolveTxLock request, dentry = (" + << dentry.ShortDebugString() << "), startTs = " << startTs + << ", commitTs = " << commitTs; + auto rc = dentryStorage_->ResolveTxLock( + dentry, startTs, commitTs, logIndex); + if (rc != MetaStatusCode::OK) { + LOG(ERROR) << "ResolveTxLock failed, dentry = (" + << dentry.ShortDebugString() << "), startTs = " << startTs + << ", commitTs = " << commitTs + << ", retCode = " << MetaStatusCode_Name(rc); + } else { + LOG(INFO) << "ResolveTxLock success, dentry = (" + << dentry.ShortDebugString() << "), startTs = " << startTs + << ", commitTs = " << commitTs; + } + return rc; +} + +MetaStatusCode DentryManager::CommitTx(const std::vector& dentrys, + uint64_t startTs, uint64_t commitTs, int64_t logIndex) { + std::stringstream ss; + for (const auto& dentry : dentrys) { + ss << dentry.ShortDebugString() << ", "; + } + VLOG(1) << "CommitTx request, dentrys = (" << ss.str() + << "), startTs = " << startTs + << ", commitTs = " << commitTs; + auto rc = dentryStorage_->CommitTx(dentrys, startTs, commitTs, logIndex); + if (rc != MetaStatusCode::OK) { + LOG(ERROR) << "CommitTx failed, dentrys = (" << ss.str() + << "), startTs = " << startTs + << ", commitTs = " << commitTs + << ", retCode = " << MetaStatusCode_Name(rc); + } else { + VLOG(1) << "CommitTx success, dentrys = (" << ss.str() + << "), startTs = " << startTs + << ", commitTs = " << commitTs; + } + return rc; +} + } // namespace metaserver } // namespace curvefs diff --git a/curvefs/src/metaserver/dentry_manager.h b/curvefs/src/metaserver/dentry_manager.h index ef5ca8305c..fbcf605473 100644 --- a/curvefs/src/metaserver/dentry_manager.h +++ b/curvefs/src/metaserver/dentry_manager.h @@ -42,25 +42,39 @@ class DentryManager { bool Init(); - MetaStatusCode CreateDentry(const Dentry& dentry, int64_t logIndex); + MetaStatusCode CreateDentry(const Dentry& dentry, int64_t logIndex, + TxLock* txLock = nullptr); // only invoked from snapshot loadding MetaStatusCode CreateDentry(const DentryVec& vec, bool merge, int64_t logIndex); - MetaStatusCode DeleteDentry(const Dentry& dentry, int64_t logIndex); + MetaStatusCode DeleteDentry(const Dentry& dentry, int64_t logIndex, + TxLock* txLock = nullptr); - MetaStatusCode GetDentry(Dentry* dentry); + MetaStatusCode GetDentry(Dentry* dentry, TxLock* txLock = nullptr); MetaStatusCode ListDentry(const Dentry& dentry, std::vector* dentrys, uint32_t limit, - bool onlyDir = false); + bool onlyDir = false, TxLock* txLock = nullptr); void ClearDentry(); MetaStatusCode HandleRenameTx(const std::vector& dentrys, int64_t logIndex); + MetaStatusCode PrewriteRenameTx(const std::vector& dentrys, + const TxLock& txLock, int64_t logIndex, TxLock* out); + + MetaStatusCode CheckTxStatus(const std::string& primaryKey, + uint64_t startTs, uint64_t curTimestamp, int64_t logIndex); + + MetaStatusCode ResolveTxLock(const Dentry& dentry, + uint64_t startTs, uint64_t commitTs, int64_t logIndex); + + MetaStatusCode CommitTx(const std::vector& dentrys, + uint64_t startTs, uint64_t commitTs, int64_t logIndex); + private: void Log4Dentry(const std::string& request, const Dentry& dentry); void Log4Code(const std::string& request, MetaStatusCode rc); diff --git a/curvefs/src/metaserver/dentry_storage.cpp b/curvefs/src/metaserver/dentry_storage.cpp index 57fc70c1cf..3023bbc9e9 100644 --- a/curvefs/src/metaserver/dentry_storage.cpp +++ b/curvefs/src/metaserver/dentry_storage.cpp @@ -38,17 +38,30 @@ namespace curvefs { namespace metaserver { +namespace storage { + DECLARE_int32(tx_lock_ttl_ms); +} + using ::curve::common::ReadLockGuard; using ::curve::common::StringStartWith; using ::curve::common::WriteLockGuard; using ::curvefs::metaserver::storage::Key4Dentry; using ::curvefs::metaserver::storage::Prefix4AllDentry; using ::curvefs::metaserver::storage::Prefix4SameParentDentry; +using ::curvefs::metaserver::storage::Prefix4TxWrite; +using ::curvefs::metaserver::storage::Key4TxWrite; using ::curvefs::metaserver::storage::Status; +using ::curvefs::metaserver::storage::FLAGS_tx_lock_ttl_ms; + +const char* DentryStorage::kDentryAppliedKey("dentry"); +const char* DentryStorage::kDentryCountKey("count"); +const char* DentryStorage::kHandleTxKey("handleTx"); +const char* DentryStorage::kPendingTxKey("pendingTx"); +const char* DentryStorage::kTxLatestCommit("latestCommit"); bool operator==(const Dentry& lhs, const Dentry& rhs) { return EQUAL(fsid) && EQUAL(parentinodeid) && EQUAL(name) && EQUAL(txid) && - EQUAL(inodeid) && EQUAL(flag); + EQUAL(inodeid); } bool operator<(const Dentry& lhs, const Dentry& rhs) { @@ -65,6 +78,9 @@ static bool HasDeleteMarkFlag(const Dentry& dentry) { return (dentry.flag() & DentryFlag::DELETE_MARK_FLAG) != 0; } +/* +* DentryVector is a wrapper of DentryVec +*/ DentryVector::DentryVector(DentryVec* vec) : vec_(vec), nPendingAdd_(0), nPendingDel_(0) {} @@ -114,6 +130,9 @@ void DentryVector::Confirm(uint64_t* count) { *count = *count + nPendingAdd_ - nPendingDel_; } +/* +* DentryList +*/ DentryList::DentryList(std::vector* list, uint32_t limit, const std::string& exclude, uint64_t maxTxId, bool onlyDir) @@ -124,7 +143,7 @@ DentryList::DentryList(std::vector* list, uint32_t limit, maxTxId_(maxTxId), onlyDir_(onlyDir) {} -void DentryList::PushBack(DentryVec* vec) { +void DentryList::PushBack(DentryVec* vec, bool* realEntry) { // NOTE: it's a cheap operation becacuse the size of // dentryVec must less than 2 BTree dentrys; @@ -148,28 +167,22 @@ void DentryList::PushBack(DentryVec* vec) { } return; } + *realEntry = true; list_->push_back(*last); VLOG(9) << "Push dentry, dentry = (" << last->ShortDebugString() << ")"; } -uint32_t DentryList::Size() { return size_; } - -bool DentryList::IsFull() { return limit_ != 0 && size_ >= limit_; } - -const char* DentryStorage::kDentryAppliedKey("dentry"); -const char* DentryStorage::kDentryCountKey("count"); -const char* DentryStorage::kHandleTxKey("handleTx"); -const char* DentryStorage::kPendingTxKey("pendingTx"); +uint32_t DentryList::Size() { + return size_; +} -bool DentryStorage::Init() { - auto s = GetDentryCount(&nDentry_); - if (s.ok() || s.IsNotFound()) { - s = GetHandleTxIndex(&handleTxIndex_); - return s.ok() || s.IsNotFound(); - } - return false; +bool DentryList::IsFull() { + return limit_ != 0 && size_ >= limit_; } +/* +* DentryStorage +*/ DentryStorage::DentryStorage(std::shared_ptr kvStorage, std::shared_ptr nameGenerator, uint64_t nDentry) @@ -178,135 +191,45 @@ DentryStorage::DentryStorage(std::shared_ptr kvStorage, table4AppliedIndex_(nameGenerator->GetAppliedIndexTableName()), table4Transaction_(nameGenerator->GetTransactionTableName()), table4DentryCount_(nameGenerator->GetDentryCountTableName()), + table4TxLock_(nameGenerator->GetTxLockTableName()), + table4TxWrite_(nameGenerator->GetTxWriteTableName()), handleTxIndex_(-1), nDentry_(nDentry), - conv_() { + conv_(), + latestCommit_(0) { // NOTE: for compatibility with older versions // we cannot ignore `nDentry` argument // try get dentry count for rocksdb // if we got it, replace old value } -std::string DentryStorage::DentryKey(const Dentry& dentry) { - Key4Dentry key(dentry.fsid(), dentry.parentinodeid(), dentry.name()); - return conv_.SerializeToString(key); -} - -bool DentryStorage::CompressDentry(storage::StorageTransaction* txn, - DentryVec* vec, BTree* dentrys, - uint64_t* outCount) { - DentryVector vector(vec); - std::vector deleted; - if (dentrys->size() == 2) { - deleted.push_back(*dentrys->begin()); +bool DentryStorage::Init() { + auto s = GetDentryCount(&nDentry_); + if (!s.ok() && !s.IsNotFound()) { + LOG(ERROR) << "Get dentry count failed, status = " << s.ToString(); + return false; } - if (HasDeleteMarkFlag(*dentrys->rbegin())) { - deleted.push_back(*dentrys->rbegin()); + s = GetHandleTxIndex(&handleTxIndex_); + if (!s.ok() && !s.IsNotFound()) { + LOG(ERROR) << "Get handle tx index failed, status = " << s.ToString(); + return false; } - for (const auto& dentry : deleted) { - vector.Delete(dentry); + s = GetLatestCommit(&latestCommit_); + if (!s.ok() && !s.IsNotFound()) { + LOG(ERROR) << "Get latest commit failed, status = " << s.ToString(); + return false; } - const char* step = "Compress dentry from transaction"; - Status s; - std::string skey = DentryKey(*dentrys->begin()); - do { - if (vec->dentrys_size() == 0) { // delete directly - s = txn->SDel(table4Dentry_, skey); - } else { - s = txn->SSet(table4Dentry_, skey, *vec); - } - if (!s.ok()) { - break; - } - uint64_t countCopy = *outCount; - vector.Confirm(&countCopy); - s = SetDentryCount(txn, countCopy); - if (!s.ok()) { - step = "Insert dentry count to transaction"; - break; - } - *outCount = countCopy; - return true; - } while (false); - LOG(ERROR) << step << " failed, status = " << s.ToString(); - return false; + return true; } -// NOTE: Find() return the dentry which has the latest txid, -// and it will clean the old txid's dentry if you specify compress to true -MetaStatusCode DentryStorage::Find(const Dentry& in, Dentry* out, - DentryVec* vec) { - std::string skey = DentryKey(in); - Status s = kvStorage_->SGet(table4Dentry_, skey, vec); - if (s.IsNotFound()) { - return MetaStatusCode::NOT_FOUND; - } else if (!s.ok()) { - return MetaStatusCode::STORAGE_INTERNAL_ERROR; - } - - // status = OK - BTree dentrys; - DentryVector vector(vec); - vector.Filter(in.txid(), &dentrys); - size_t size = dentrys.size(); - if (size > 2) { - LOG(ERROR) << "There are more than 2 dentrys"; - return MetaStatusCode::NOT_FOUND; - } else if (size == 0) { - return MetaStatusCode::NOT_FOUND; - } - - // size == 1 || size == 2 - MetaStatusCode rc; - if (HasDeleteMarkFlag(*dentrys.rbegin())) { - rc = MetaStatusCode::NOT_FOUND; - } else { - rc = MetaStatusCode::OK; - *out = *dentrys.rbegin(); - } - return rc; +std::string DentryStorage::DentryKey(const Dentry& dentry) { + Key4Dentry key(dentry.fsid(), dentry.parentinodeid(), dentry.name()); + return conv_.SerializeToString(key); } -// NOTE: Find() return the dentry which has the latest txid, -// and it will clean the old txid's dentry if you specify compressOutCount to -// non-nullptr compressOutCount must point to a variable that value is equal -// with `nDentry_` -MetaStatusCode DentryStorage::Find(storage::StorageTransaction* txn, - const Dentry& in, Dentry* out, - DentryVec* vec, uint64_t* compressOutCount) { - std::string skey = DentryKey(in); - Status s = txn->SGet(table4Dentry_, skey, vec); - if (s.IsNotFound()) { - return MetaStatusCode::NOT_FOUND; - } else if (!s.ok()) { - return MetaStatusCode::STORAGE_INTERNAL_ERROR; - } - // status = OK - BTree dentrys; - DentryVector vector(vec); - vector.Filter(in.txid(), &dentrys); - size_t size = dentrys.size(); - if (size > 2) { - LOG(ERROR) << "There are more than 2 dentrys"; - return MetaStatusCode::NOT_FOUND; - } else if (size == 0) { - return MetaStatusCode::NOT_FOUND; - } - - // size == 1 || size == 2 - MetaStatusCode rc; - if (HasDeleteMarkFlag(*dentrys.rbegin())) { - rc = MetaStatusCode::NOT_FOUND; - } else { - rc = MetaStatusCode::OK; - *out = *dentrys.rbegin(); - } - - if (compressOutCount != nullptr && - !CompressDentry(txn, vec, &dentrys, compressOutCount)) { - rc = MetaStatusCode::STORAGE_INTERNAL_ERROR; - } - return rc; +std::string DentryStorage::TxWriteKey(const Dentry& dentry, uint64_t ts) { + Key4TxWrite key(dentry.fsid(), dentry.parentinodeid(), dentry.name(), ts); + return conv_.SerializeToString(key); } MetaStatusCode DentryStorage::GetAppliedIndex(int64_t* index) { @@ -406,226 +329,147 @@ storage::Status DentryStorage::GetHandleTxIndex(int64_t* index) { return s; } -MetaStatusCode DentryStorage::Insert(const Dentry& dentry, int64_t logIndex) { - WriteLockGuard lg(rwLock_); - - Dentry out; - DentryVec vec; - std::shared_ptr txn; - storage::Status s; - const char* step = "Begin transaction"; +bool DentryStorage::CompressDentry(storage::StorageTransaction* txn, + DentryVec* vec, BTree* dentrys, + uint64_t* outCount) { + DentryVector vector(vec); + std::vector deleted; + if (dentrys->size() == 2) { + deleted.push_back(*dentrys->begin()); + } + if (HasDeleteMarkFlag(*dentrys->rbegin())) { + deleted.push_back(*dentrys->rbegin()); + } + for (const auto& dentry : deleted) { + vector.Delete(dentry); + } + const char* step = "Compress dentry from transaction"; + Status s; + std::string skey = DentryKey(*dentrys->begin()); do { - txn = kvStorage_->BeginTransaction(); - if (txn == nullptr) { - break; - } - uint64_t count = nDentry_; - s = SetAppliedIndex(txn.get(), logIndex); - if (!s.ok()) { - step = "Insert applied index to transaction"; - break; - } - MetaStatusCode rc = Find(txn.get(), dentry, &out, &vec, &count); - if (rc == MetaStatusCode::OK) { - auto s = txn->Commit(); - if (!s.ok()) { - step = "Commit compress dentry transaction"; - break; - } - // if compress is success - // we use output dentry count to replace old one - nDentry_ = count; - if (BelongSomeOne(out, dentry)) { - return MetaStatusCode::IDEMPOTENCE_OK; - } - return MetaStatusCode::DENTRY_EXIST; - } else if (rc != MetaStatusCode::NOT_FOUND) { - step = "Find dentry failed"; - break; + if (vec->dentrys_size() == 0) { // delete directly + s = txn->SDel(table4Dentry_, skey); + } else { + s = txn->SSet(table4Dentry_, skey, *vec); } - // rc == MetaStatusCode::NOT_FOUND - - // NOTE: `count` maybe already written by `Find()` in here - // so we continue use `count` in follow operations - DentryVector vector(&vec); - vector.Insert(dentry); - std::string skey = DentryKey(dentry); - s = txn->SSet(table4Dentry_, skey, vec); if (!s.ok()) { - step = "Insert dentry to transaction"; break; } - vector.Confirm(&count); - s = SetDentryCount(txn.get(), count); + uint64_t countCopy = *outCount; + vector.Confirm(&countCopy); + s = SetDentryCount(txn, countCopy); if (!s.ok()) { step = "Insert dentry count to transaction"; break; } - s = txn->Commit(); - if (!s.ok()) { - step = "Insert dentry"; - break; - } - nDentry_ = count; - return MetaStatusCode::OK; + *outCount = countCopy; + return true; } while (false); LOG(ERROR) << step << " failed, status = " << s.ToString(); - if (txn != nullptr && !txn->Rollback().ok()) { - LOG(ERROR) << "Rollback insert dentry transaction failed, status = " - << s.ToString(); - } - return MetaStatusCode::STORAGE_INTERNAL_ERROR; + return false; } -MetaStatusCode DentryStorage::Insert(const DentryVec& vec, bool merge, - int64_t logIndex) { - WriteLockGuard lg(rwLock_); - +// NOTE: Find() return the dentry which has the latest txid, +// and it will clean the old txid's dentry if you specify compressOutCount to +// non-nullptr compressOutCount must point to a variable that value is equal +// with `nDentry_` +MetaStatusCode DentryStorage::Find(storage::StorageTransaction* txn, + const Dentry& in, Dentry* out, + DentryVec* vec, uint64_t* compressOutCount, + TxLock* txLock) { + std::string skey = DentryKey(in); Status s; - DentryVec oldVec; - std::string skey = DentryKey(vec.dentrys(0)); - std::shared_ptr txn; - const char* step = "Begin transaction"; - do { - txn = kvStorage_->BeginTransaction(); - if (txn == nullptr) { - break; - } - if (merge) { // for old version dumpfile (v1) - s = txn->SGet(table4Dentry_, skey, &oldVec); - if (s.IsNotFound()) { - // do nothing - } else if (!s.ok()) { - step = "Find old version from transaction"; - break; - } - } - DentryVector vector(&oldVec); - vector.Merge(vec); - s = txn->SSet(table4Dentry_, skey, oldVec); - if (!s.ok()) { - step = "Insert dentry vector to tranasction"; - break; - } - s = SetAppliedIndex(txn.get(), logIndex); - if (!s.ok()) { - step = "Insert applied index to tranasction"; - break; - } - uint64_t count = nDentry_; - vector.Confirm(&count); - s = SetDentryCount(txn.get(), count); - if (!s.ok()) { - step = "Insert dentry count to transaction"; - break; - } - s = txn->Commit(); - if (!s.ok()) { - step = "Insert dentry vector"; - break; + // check tx lock on dentry + if (txLock != nullptr) { + s = txn->SGet(table4TxLock_, skey, txLock); + if (s.ok()) { + return MetaStatusCode::TX_KEY_LOCKED; + } else if (!s.IsNotFound()) { + return MetaStatusCode::STORAGE_INTERNAL_ERROR; } - nDentry_ = count; - return MetaStatusCode::OK; - } while (false); - LOG(ERROR) << step << " failed, status = " << s.ToString(); - if (txn != nullptr && !txn->Rollback().ok()) { - LOG(ERROR) << "Rollback insert dentry transaction failed, status = " - << s.ToString(); } - return MetaStatusCode::STORAGE_INTERNAL_ERROR; -} -MetaStatusCode DentryStorage::Delete(const Dentry& dentry, int64_t logIndex) { - WriteLockGuard lg(rwLock_); + s = txn->SGet(table4Dentry_, skey, vec); + if (s.IsNotFound()) { + return MetaStatusCode::NOT_FOUND; + } else if (!s.ok()) { + return MetaStatusCode::STORAGE_INTERNAL_ERROR; + } - Dentry out; - DentryVec vec; - const char* step = "Begin transaction"; - std::shared_ptr txn; - storage::Status s; - do { - txn = kvStorage_->BeginTransaction(); - if (txn == nullptr) { - break; - } - uint64_t count = nDentry_; - s = SetAppliedIndex(txn.get(), logIndex); - if (!s.ok()) { - step = "Insert applied index to transaction"; - break; - } - MetaStatusCode rc = Find(txn.get(), dentry, &out, &vec, &count); - if (rc == MetaStatusCode::NOT_FOUND) { - // NOTE: we should commit transaction - // even if rc is NOT_FOUND - // because Find() maybe write dentry count to rocksdb - s = txn->Commit(); - if (!s.ok()) { - step = "Commit transaction"; - break; - } - nDentry_ = count; - return MetaStatusCode::NOT_FOUND; - } else if (rc != MetaStatusCode::OK) { - step = "Find dentry"; - break; - } - DentryVector vector(&vec); - vector.Delete(out); - std::string skey = DentryKey(dentry); - if (vec.dentrys_size() == 0) { - s = txn->SDel(table4Dentry_, skey); - } else { - s = txn->SSet(table4Dentry_, skey, vec); - } - if (!s.ok()) { - step = "Delete dentry vector from transaction"; - break; - } - // NOTE: we should use count variable instead of nDentry_ - // (it means that we should not reset count to nDentry_) - // count is newest version of dentry count - vector.Confirm(&count); - s = SetDentryCount(txn.get(), count); - if (!s.ok()) { - step = "Insert applied index to transaction"; - break; - } - s = txn->Commit(); - if (!s.ok()) { - step = "Delete dentry vector"; - break; - } - nDentry_ = count; - return MetaStatusCode::OK; - } while (false); - LOG(ERROR) << step << " failed, status = " << s.ToString(); - if (txn != nullptr && !txn->Rollback().ok()) { - LOG(ERROR) << "Rollback transaction failed"; + // status = OK + // txId here means latest dentry version + uint64_t txId = latestCommit_ > 0 ? latestCommit_ : in.txid(); + BTree dentrys; + DentryVector vector(vec); + vector.Filter(txId, &dentrys); + size_t size = dentrys.size(); + + if (size > 2) { + LOG(ERROR) << "There are more than 2 dentrys"; + return MetaStatusCode::NOT_FOUND; + } else if (size == 0) { + return MetaStatusCode::NOT_FOUND; } - return MetaStatusCode::STORAGE_INTERNAL_ERROR; + + // size == 1 || size == 2 + MetaStatusCode rc; + if (HasDeleteMarkFlag(*dentrys.rbegin())) { + rc = MetaStatusCode::NOT_FOUND; + } else { + rc = MetaStatusCode::OK; + *out = *dentrys.rbegin(); + } + + if (compressOutCount != nullptr && + !CompressDentry(txn, vec, &dentrys, compressOutCount)) { + rc = MetaStatusCode::STORAGE_INTERNAL_ERROR; + } + return rc; } -MetaStatusCode DentryStorage::Get(Dentry* dentry) { +#define ON_ERROR(msg) \ + do \ + { \ + LOG(ERROR) << msg; \ + if (txn != nullptr && !txn->Rollback().ok()) { \ + LOG(ERROR) << "Rollback transaction fail"; \ + } \ + return rc; \ + } while (false) + +#define ON_COMMIT() \ + do \ + { \ + s = txn->Commit(); \ + if (!s.ok()) { \ + rc = MetaStatusCode::STORAGE_INTERNAL_ERROR; \ + ON_ERROR("Commit transaction failed, " + s.ToString()); \ + } \ + nDentry_ = count; \ + return rc; \ + } while (false) + + +MetaStatusCode DentryStorage::Get(Dentry* dentry, TxLock* txLock) { ReadLockGuard lg(rwLock_); - - Dentry out; - DentryVec vec; - MetaStatusCode rc = Find(*dentry, &out, &vec); - if (rc == MetaStatusCode::NOT_FOUND) { - return MetaStatusCode::NOT_FOUND; - } else if (rc != MetaStatusCode::OK) { - return MetaStatusCode::STORAGE_INTERNAL_ERROR; + Status s; + uint64_t count = nDentry_; + MetaStatusCode rc = MetaStatusCode::STORAGE_INTERNAL_ERROR; + std::shared_ptr txn; + txn = kvStorage_->BeginTransaction(); + if (txn == nullptr) { + ON_ERROR("Begin transaction failed"); } - // MetaStatusCode::OK - *dentry = out; - return MetaStatusCode::OK; + DentryVec vec; + rc = Find(txn.get(), *dentry, dentry, &vec, nullptr, txLock); + ON_COMMIT(); } MetaStatusCode DentryStorage::List(const Dentry& dentry, std::vector* dentrys, uint32_t limit, - bool onlyDir) { + bool onlyDir, + TxLock* txLock) { // TODO(all): consider store dir dentry and file dentry separately ReadLockGuard lg(rwLock_); @@ -643,7 +487,7 @@ MetaStatusCode DentryStorage::List(const Dentry& dentry, Prefix4SameParentDentry prefix(fsId, parentInodeId); std::string sprefix = conv_.SerializeToString(prefix); // "1:1:" Key4Dentry key(fsId, parentInodeId, name); - std::string lower = conv_.SerializeToString(key); // "1:1:", "1:1:/a/b/c" + std::string lower = conv_.SerializeToString(key); // "1:1:", "1:1:dir" // 3. iterator key/value pair one by one auto iterator = kvStorage_->SSeek(table4Dentry_, lower); @@ -652,8 +496,10 @@ MetaStatusCode DentryStorage::List(const Dentry& dentry, return MetaStatusCode::STORAGE_INTERNAL_ERROR; } + // get newest dentry version + uint64_t txId = latestCommit_ > 0 ? latestCommit_ : dentry.txid(); DentryVec current; - DentryList list(dentrys, limit, name, dentry.txid(), onlyDir); + DentryList list(dentrys, limit, name, txId, onlyDir); butil::Timer time; uint32_t seekTimes = 0; time.start(); @@ -667,81 +513,209 @@ MetaStatusCode DentryStorage::List(const Dentry& dentry, return MetaStatusCode::PARSE_FROM_STRING_FAILED; } - list.PushBack(¤t); + bool realEntry = false; + list.PushBack(¤t, &realEntry); + // check dentry tx lock + if (txLock != nullptr && realEntry) { + Status s = kvStorage_->SGet(table4TxLock_, skey, txLock); + if (s.ok()) { + return MetaStatusCode::TX_KEY_LOCKED; + } else if (!s.IsNotFound()) { + return MetaStatusCode::STORAGE_INTERNAL_ERROR; + } + } + if (list.IsFull()) { break; } } time.stop(); VLOG(1) << "ListDentry request: dentry = (" << dentry.ShortDebugString() - << ")" - << ", onlyDir = " << onlyDir << ", limit = " << limit + << "), onlyDir = " << onlyDir << ", limit = " << limit << ", lower key = " << lower << ", seekTimes = " << seekTimes << ", dentrySize = " << dentrys->size() << ", costUs = " << time.u_elapsed(); return MetaStatusCode::OK; } +MetaStatusCode DentryStorage::Insert( + const Dentry& dentry, int64_t logIndex, TxLock* txLock) { + WriteLockGuard lg(rwLock_); + storage::Status s; + uint64_t count = nDentry_; + MetaStatusCode rc = MetaStatusCode::STORAGE_INTERNAL_ERROR; + std::shared_ptr txn; + txn = kvStorage_->BeginTransaction(); + if (txn == nullptr) { + ON_ERROR("Begin transaction failed"); + } + // 1. set applied index + s = SetAppliedIndex(txn.get(), logIndex); + if (!s.ok()) { + ON_ERROR("Insert applied index to transaction"); + } + // find dentry + Dentry out; + DentryVec vec; + rc = Find(txn.get(), dentry, &out, &vec, &count, txLock); + if (rc == MetaStatusCode::TX_KEY_LOCKED) { + ON_COMMIT(); + } + if (rc == MetaStatusCode::OK) { + if (BelongSomeOne(out, dentry)) { + rc = MetaStatusCode::IDEMPOTENCE_OK; + } else { + rc = MetaStatusCode::DENTRY_EXIST; + } + ON_COMMIT(); + } else if (rc != MetaStatusCode::NOT_FOUND) { + ON_ERROR("Find dentry failed"); + } + // rc == MetaStatusCode::NOT_FOUND + DentryVector vector(&vec); + vector.Insert(dentry); + s = txn->SSet(table4Dentry_, DentryKey(dentry), vec); + if (!s.ok()) { + ON_ERROR("Insert dentry to transaction"); + } + vector.Confirm(&count); + s = SetDentryCount(txn.get(), count); + if (!s.ok()) { + ON_ERROR("Insert dentry count to transaction"); + } + rc = MetaStatusCode::OK; + ON_COMMIT(); +} + +MetaStatusCode DentryStorage::Insert(const DentryVec& vec, bool merge, + int64_t logIndex) { + WriteLockGuard lg(rwLock_); + storage::Status s; + uint64_t count = nDentry_; + MetaStatusCode rc = MetaStatusCode::STORAGE_INTERNAL_ERROR; + std::shared_ptr txn; + txn = kvStorage_->BeginTransaction(); + if (txn == nullptr) { + ON_ERROR("Begin transaction failed"); + } + + DentryVec oldVec; + std::string skey = DentryKey(vec.dentrys(0)); + if (merge) { // for old version dumpfile (v1) + s = txn->SGet(table4Dentry_, skey, &oldVec); + if (s.IsNotFound()) { + // do nothing + } else if (!s.ok()) { + ON_ERROR("Find old version from transaction"); + } + } + DentryVector vector(&oldVec); + vector.Merge(vec); + s = txn->SSet(table4Dentry_, skey, oldVec); + if (!s.ok()) { + ON_ERROR("Insert dentry vector to tranasction"); + } + s = SetAppliedIndex(txn.get(), logIndex); + if (!s.ok()) { + ON_ERROR("Insert applied index to tranasction"); + } + vector.Confirm(&count); + s = SetDentryCount(txn.get(), count); + if (!s.ok()) { + ON_ERROR("Insert dentry count to transaction"); + } + rc = MetaStatusCode::OK; + ON_COMMIT(); +} + +MetaStatusCode DentryStorage::Delete( + const Dentry& dentry, int64_t logIndex, TxLock* txLock) { + WriteLockGuard lg(rwLock_); + Status s; + uint64_t count = nDentry_; + MetaStatusCode rc = MetaStatusCode::STORAGE_INTERNAL_ERROR; + std::shared_ptr txn; + txn = kvStorage_->BeginTransaction(); + if (txn == nullptr) { + ON_ERROR("Begin transaction failed"); + } + s = SetAppliedIndex(txn.get(), logIndex); + if (!s.ok()) { + ON_ERROR("Insert applied index to transaction"); + } + Dentry out; + DentryVec vec; + rc = Find(txn.get(), dentry, &out, &vec, &count, txLock); + if (rc == MetaStatusCode::TX_KEY_LOCKED) { + ON_COMMIT(); + } + if (rc == MetaStatusCode::NOT_FOUND) { + ON_COMMIT(); + } else if (rc != MetaStatusCode::OK) { + ON_ERROR("Find dentry failed"); + } + // OK + DentryVector vector(&vec); + vector.Delete(out); + std::string skey = DentryKey(dentry); + if (vec.dentrys_size() == 0) { + s = txn->SDel(table4Dentry_, skey); + } else { + s = txn->SSet(table4Dentry_, skey, vec); + } + if (!s.ok()) { + ON_ERROR("Delete dentry vector from transaction"); + } + // NOTE: we should use count variable instead of nDentry_ + // (it means that we should not reset count to nDentry_) + // count is newest version of dentry count + vector.Confirm(&count); + s = SetDentryCount(txn.get(), count); + if (!s.ok()) { + ON_ERROR("Insert dentry count to transaction"); + } + rc = MetaStatusCode::OK; + ON_COMMIT(); +} + MetaStatusCode DentryStorage::PrepareTx( const std::vector& dentrys, const metaserver::TransactionRequest& txRequest, int64_t logIndex) { WriteLockGuard lg(rwLock_); - uint64_t count = nDentry_; Status s; - const char* step = "Begin transaction"; + uint64_t count = nDentry_; + MetaStatusCode rc = MetaStatusCode::STORAGE_INTERNAL_ERROR; std::shared_ptr txn; - do { - txn = kvStorage_->BeginTransaction(); - if (txn == nullptr) { - break; - } - bool quit = false; - for (const auto& dentry : dentrys) { - DentryVec vec; - DentryVector vector(&vec); - std::string skey = DentryKey(dentry); - s = txn->SGet(table4Dentry_, skey, &vec); - if (!s.ok() && !s.IsNotFound()) { - step = "Get dentry from transaction"; - quit = true; - break; - } - // OK || NOT_FOUND - vector.Insert(dentry); - s = txn->SSet(table4Dentry_, skey, vec); - if (!s.ok()) { - step = "Insert dentry to transaction"; - quit = true; - break; - } - vector.Confirm(&count); - } - if (quit) { - break; - } - s = SetAppliedIndex(txn.get(), logIndex); - if (!s.ok()) { - step = "Insert applied index to transaction"; - break; - } - s = SetPendingTx(txn.get(), txRequest); - if (!s.ok()) { - step = "Insert tx request to transaction"; - break; + txn = kvStorage_->BeginTransaction(); + if (txn == nullptr) { + ON_ERROR("Begin transaction failed"); + } + for (const auto& dentry : dentrys) { + DentryVec vec; + DentryVector vector(&vec); + std::string skey = DentryKey(dentry); + s = txn->SGet(table4Dentry_, skey, &vec); + if (!s.ok() && !s.IsNotFound()) { + ON_ERROR("Get dentry from transaction"); } - s = txn->Commit(); + // OK || NOT_FOUND + vector.Insert(dentry); + s = txn->SSet(table4Dentry_, skey, vec); if (!s.ok()) { - step = "Commit transaction"; - break; + ON_ERROR("Insert dentry to transaction"); } - nDentry_ = count; - return MetaStatusCode::OK; - } while (false); - LOG(ERROR) << step << " failed, status = " << s.ToString(); - if (txn != nullptr && !txn->Rollback().ok()) { - LOG(ERROR) << "Rollback transaction fail"; + vector.Confirm(&count); } - return MetaStatusCode::STORAGE_INTERNAL_ERROR; + s = SetAppliedIndex(txn.get(), logIndex); + if (!s.ok()) { + ON_ERROR("Insert applied index to transaction"); + } + s = SetPendingTx(txn.get(), txRequest); + if (!s.ok()) { + ON_ERROR("Insert tx request to transaction"); + } + rc = MetaStatusCode::OK; + ON_COMMIT(); } MetaStatusCode DentryStorage::CommitTx(const std::vector& dentrys, @@ -760,53 +734,31 @@ MetaStatusCode DentryStorage::CommitTx(const std::vector& dentrys, } WriteLockGuard lg(rwLock_); Status s; - const char* step = "Begin transaction"; + uint64_t count = nDentry_; + MetaStatusCode rc = MetaStatusCode::STORAGE_INTERNAL_ERROR; std::shared_ptr txn; - do { - txn = kvStorage_->BeginTransaction(); - if (txn == nullptr) { - break; - } - uint64_t count = nDentry_; - bool quit = false; - for (const auto& dentry : dentrys) { - Dentry out; - DentryVec vec; - std::string skey = DentryKey(dentry); - MetaStatusCode rc = MetaStatusCode::OK; - rc = Find(txn.get(), dentry, &out, &vec, &count); - if (rc != MetaStatusCode::OK && rc != MetaStatusCode::NOT_FOUND) { - step = "Find dentry from transaction"; - quit = true; - break; - } - } - if (quit) { - break; - } - s = SetHandleTxIndex(txn.get(), logIndex); - if (!s.ok()) { - step = "Insert handle tx index to transaction"; - break; - } - s = ClearPendingTx(txn.get()); - if (!s.ok()) { - step = "Delete pending tx from transaction"; - break; - } - s = txn->Commit(); - if (!s.ok()) { - step = "Commit transaction"; - break; + txn = kvStorage_->BeginTransaction(); + if (txn == nullptr) { + ON_ERROR("Begin transaction failed"); + } + for (const auto& dentry : dentrys) { + Dentry out; + DentryVec vec; + rc = Find(txn.get(), dentry, &out, &vec, &count, nullptr); + if (rc != MetaStatusCode::OK && rc != MetaStatusCode::NOT_FOUND) { + ON_ERROR("Find dentry from transaction"); } - nDentry_ = count; - return MetaStatusCode::OK; - } while (false); - LOG(ERROR) << step << " failed, status = " << s.ToString(); - if (txn != nullptr && !txn->Rollback().ok()) { - LOG(ERROR) << "Rollback transaction failed"; } - return MetaStatusCode::STORAGE_INTERNAL_ERROR; + s = SetHandleTxIndex(txn.get(), logIndex); + if (!s.ok()) { + ON_ERROR("Insert handle tx index to transaction"); + } + s = ClearPendingTx(txn.get()); + if (!s.ok()) { + ON_ERROR("Delete pending tx from transaction"); + } + rc = MetaStatusCode::OK; + ON_COMMIT(); } MetaStatusCode DentryStorage::RollbackTx(const std::vector& dentrys, @@ -825,70 +777,47 @@ MetaStatusCode DentryStorage::RollbackTx(const std::vector& dentrys, } WriteLockGuard lg(rwLock_); Status s; - const char* step = "Begin transaction"; + uint64_t count = nDentry_; + MetaStatusCode rc = MetaStatusCode::STORAGE_INTERNAL_ERROR; std::shared_ptr txn; - do { - txn = kvStorage_->BeginTransaction(); - if (txn == nullptr) { - break; - } - uint64_t count = nDentry_; - bool quit = false; - for (const auto& dentry : dentrys) { - DentryVec vec; - DentryVector vector(&vec); - std::string skey = DentryKey(dentry); - s = txn->SGet(table4Dentry_, skey, &vec); - if (!s.ok() && !s.IsNotFound()) { - step = "Find dentry"; - quit = true; - break; - } - // OK || NOT_FOUND - vector.Delete(dentry); - if (vec.dentrys_size() == 0) { // delete directly - s = txn->SDel(table4Dentry_, skey); - } else { - s = txn->SSet(table4Dentry_, skey, vec); - } - if (!s.ok()) { - step = "Delete dentry from transaction"; - quit = true; - break; - } - vector.Confirm(&count); - } - if (quit) { - break; - } - s = SetDentryCount(txn.get(), count); - if (!s.ok()) { - step = "Insert dentry count to transaction"; - break; - } - s = SetHandleTxIndex(txn.get(), logIndex); - if (!s.ok()) { - step = "Insert handle tx index to transaction"; - break; + txn = kvStorage_->BeginTransaction(); + if (txn == nullptr) { + ON_ERROR("Begin transaction failed"); + } + for (const auto& dentry : dentrys) { + DentryVec vec; + DentryVector vector(&vec); + std::string skey = DentryKey(dentry); + s = txn->SGet(table4Dentry_, skey, &vec); + if (!s.ok() && !s.IsNotFound()) { + ON_ERROR("Get dentry from transaction"); } - s = ClearPendingTx(txn.get()); - if (!s.ok()) { - step = "Delete pending tx from transaction"; - break; + // OK || NOT_FOUND + vector.Delete(dentry); + if (vec.dentrys_size() == 0) { // delete directly + s = txn->SDel(table4Dentry_, skey); + } else { + s = txn->SSet(table4Dentry_, skey, vec); } - s = txn->Commit(); if (!s.ok()) { - step = "Commit transaction"; - break; + ON_ERROR("Delete dentry from transaction"); } - nDentry_ = count; - return MetaStatusCode::OK; - } while (false); - LOG(ERROR) << step << " failed, status = " << s.ToString(); - if (txn != nullptr && !txn->Rollback().ok()) { - LOG(ERROR) << "Rollback transaction failed"; + vector.Confirm(&count); } - return MetaStatusCode::STORAGE_INTERNAL_ERROR; + s = SetDentryCount(txn.get(), count); + if (!s.ok()) { + ON_ERROR("Insert dentry count to transaction"); + } + s = SetHandleTxIndex(txn.get(), logIndex); + if (!s.ok()) { + ON_ERROR("Insert handle tx index to transaction"); + } + s = ClearPendingTx(txn.get()); + if (!s.ok()) { + ON_ERROR("Delete pending tx from transaction"); + } + rc = MetaStatusCode::OK; + ON_COMMIT(); } std::shared_ptr DentryStorage::GetAll() { @@ -932,6 +861,16 @@ MetaStatusCode DentryStorage::Clear() { LOG(ERROR) << "Clear dentry table failed, status = " << s.ToString(); return MetaStatusCode::STORAGE_INTERNAL_ERROR; } + s = kvStorage_->SClear(table4TxWrite_); + if (!s.ok()) { + LOG(ERROR) << "Clear tx write table failed, status = " << s.ToString(); + return MetaStatusCode::STORAGE_INTERNAL_ERROR; + } + s = kvStorage_->SClear(table4TxLock_); + if (!s.ok()) { + LOG(ERROR) << "Clear tx lock table failed, status = " << s.ToString(); + return MetaStatusCode::STORAGE_INTERNAL_ERROR; + } std::shared_ptr txn; const char* step = "Begin transaction"; do { @@ -974,5 +913,393 @@ MetaStatusCode DentryStorage::Clear() { return MetaStatusCode::STORAGE_INTERNAL_ERROR; } +MetaStatusCode DentryStorage::GetLastTxWriteTs(storage::StorageTransaction* txn, + const Dentry& dentry, uint64_t* commitTs) { + // 1. prepare seek lower key + Prefix4TxWrite prefix; + prefix.fsId = dentry.fsid(); + prefix.parentInodeId = dentry.parentinodeid(); + prefix.name = dentry.name(); + std::string sprefix = conv_.SerializeToString(prefix); // "1:1:name/" + + // 2. iterator key/value pair one by one + auto iterator = txn->SSeek(table4TxWrite_, sprefix); + if (iterator->Status() < 0) { + LOG(ERROR) << "failed to get iterator for prefix" << sprefix; + return MetaStatusCode::STORAGE_INTERNAL_ERROR; + } + + std::string lastWriteKey; + std::vector toDelete; + butil::Timer time; + uint32_t seekTimes = 0; + uint32_t compressCount = 0; + time.start(); + for (iterator->SeekToFirst(); iterator->Valid(); iterator->Next()) { + seekTimes++; + lastWriteKey = iterator->Key(); + TxWrite value; + if (!iterator->ParseFromValue(&value)) { + LOG(ERROR) << "parse value failed, key = " << lastWriteKey; + return MetaStatusCode::PARSE_FROM_STRING_FAILED; + } + if (value.kind() == TxWriteKind::Rollback) { + continue; + } + toDelete.push_back(lastWriteKey); + } + time.stop(); + + if (seekTimes == 0) { + *commitTs = 0; + return MetaStatusCode::OK; + } + Key4TxWrite key; + if (!conv_.ParseFromString(lastWriteKey, &key)) { + LOG(ERROR) << "parse key failed, key = " << lastWriteKey; + return MetaStatusCode::PARSE_FROM_STRING_FAILED; + } + *commitTs = key.ts; + compressCount = toDelete.size() == 0 ? 0 : toDelete.size() - 1; + for (int i = 0; i < compressCount; i++) { + auto s = txn->SDel(table4TxWrite_, toDelete[i]); + if (!s.ok()) { + LOG(ERROR) << "delete tx write failed, key = " << toDelete[i]; + return MetaStatusCode::STORAGE_INTERNAL_ERROR; + } + } + VLOG(1) << "GetLastTxWriteTs request: dentry = (" + << dentry.ShortDebugString() << ")" + << ", lower key = " << sprefix << ", seekTimes = " << seekTimes + << ", costUs = " << time.u_elapsed() + << ", compressCount = " << compressCount; + return MetaStatusCode::OK; +} + +storage::Status DentryStorage::GetLatestCommit(uint64_t* statTs) { + TS commitTs; + Status s = kvStorage_->SGet(table4TxWrite_, kTxLatestCommit, &commitTs); + if (s.ok()) { + *statTs = commitTs.ts(); + } + return s; +} + +storage::Status DentryStorage::SetLatestCommit( + storage::StorageTransaction* txn, uint64_t ts) { + if (latestCommit_ >= ts) { + return Status::OK(); + } + TS commitTs; + commitTs.set_ts(ts); + Status s = txn->SSet(table4TxWrite_, kTxLatestCommit, commitTs); + if (s.ok()) { + latestCommit_ = ts; + } + return s; +} + +// based on tx lock not exist +MetaStatusCode DentryStorage::CheckTxStatus(storage::StorageTransaction* txn, + const std::string& primaryKey, uint64_t ts) { + Key4Dentry key; + if (!key.ParseFromString(primaryKey)) { + return MetaStatusCode::PARSE_FROM_STRING_FAILED; + } + Key4TxWrite wkey(key.fsId, key.parentInodeId, key.name, ts); + std::string skey = wkey.SerializeToString(); + TxWrite txWrite; + Status s = txn->SGet(table4TxWrite_, skey, &txWrite); + if (s.IsNotFound()) { + return MetaStatusCode::TX_COMMITTED; + } else if (!s.ok()) { + return MetaStatusCode::STORAGE_INTERNAL_ERROR; + } + + // status = OK + if (txWrite.startts() == ts && txWrite.kind() == TxWriteKind::Rollback) { + return MetaStatusCode::TX_ROLLBACKED; + } + return MetaStatusCode::TX_COMMITTED; +} + +storage::Status DentryStorage::SetTxWrite(storage::StorageTransaction* txn, + const std::string& key, const TxWrite& txWrite) { + return txn->SSet(table4TxWrite_, key, txWrite); +} + +storage::Status DentryStorage::GetTxLock( + storage::StorageTransaction* txn, const std::string& key, TxLock* out) { + return txn->SGet(table4TxLock_, key, out); +} + +storage::Status DentryStorage::SetTxLock(storage::StorageTransaction* txn, + const std::string& key, const TxLock& txLock) { + return txn->SSet(table4TxLock_, key, txLock); +} + +storage::Status DentryStorage::DelTxLock( + storage::StorageTransaction* txn, const std::string& key) { + return txn->SDel(table4TxLock_, key); +} + +MetaStatusCode DentryStorage::WriteTx(storage::StorageTransaction* txn, + const Dentry& dentry, TxLock txLock, uint64_t* count) { + // 1. set tx lock + txLock.set_ttl(FLAGS_tx_lock_ttl_ms); + Status s = SetTxLock(txn, DentryKey(dentry), txLock); + if (!s.ok()) { + return MetaStatusCode::STORAGE_INTERNAL_ERROR; + } + // 2. set dentry data and compress old version data + Dentry out; + DentryVec vec; + auto rc = Find(txn, dentry, &out, &vec, count, nullptr); + if (rc != MetaStatusCode::OK && rc != MetaStatusCode::NOT_FOUND) { + return rc; + } + DentryVector vector(&vec); + VLOG(3) << "WriteTx before insert = " << vec.DebugString(); + vector.Insert(dentry); + VLOG(3) << "WriteTx after insert = " << vec.DebugString(); + s = txn->SSet(table4Dentry_, DentryKey(dentry), vec); + if (!s.ok()) { + return MetaStatusCode::STORAGE_INTERNAL_ERROR; + } + vector.Confirm(count); + s = SetDentryCount(txn, *count); + if (!s.ok()) { + return MetaStatusCode::STORAGE_INTERNAL_ERROR; + } + return MetaStatusCode::OK; +} + +MetaStatusCode DentryStorage::PrewriteTx(const std::vector& dentrys, + TxLock txLock, int64_t logIndex, TxLock* out) { + WriteLockGuard lg(rwLock_); + Status s; + uint64_t count = nDentry_; + MetaStatusCode rc = MetaStatusCode::STORAGE_INTERNAL_ERROR; + std::shared_ptr txn; + txn = kvStorage_->BeginTransaction(); + if (txn == nullptr) { + ON_ERROR("Begin transaction failed"); + } + // 1. set applied index + s = SetAppliedIndex(txn.get(), logIndex); + if (!s.ok()) { + ON_ERROR("Insert applied index to transaction failed"); + } + for (int i = 0; i < dentrys.size(); i++) { + // 2. check write confict + uint64_t commitTs = 0; + if (MetaStatusCode::OK != + GetLastTxWriteTs(txn.get(), dentrys[i], &commitTs)) { + ON_ERROR("Get last tx write ts failed"); + } + if (commitTs >= txLock.startts()) { + rc = MetaStatusCode::TX_WRITE_CONFLICT; + ON_ERROR("Tx write conflict"); + } + // 3. check tx lock + s = GetTxLock(txn.get(), DentryKey(dentrys[i]), out); + if (s.ok()) { + if (out->startts() == txLock.startts()) { + continue; + } + out->set_index(i); + rc = MetaStatusCode::TX_KEY_LOCKED; + ON_COMMIT(); + } else if (!s.IsNotFound()) { + ON_ERROR("Get tx lock failed"); + } + // 4. write tx + if (WriteTx(txn.get(), dentrys[i], txLock, &count) + != MetaStatusCode::OK) { + ON_ERROR("Write tx failed"); + } + } + rc = MetaStatusCode::OK; + ON_COMMIT(); +} + +MetaStatusCode DentryStorage::CheckTxStatus(const std::string& primaryKey, + uint64_t startTs, uint64_t curTimestamp, int64_t logIndex) { + WriteLockGuard lg(rwLock_); + Status s; + uint64_t count = nDentry_; + MetaStatusCode rc = MetaStatusCode::STORAGE_INTERNAL_ERROR; + std::shared_ptr txn; + txn = kvStorage_->BeginTransaction(); + if (txn == nullptr) { + ON_ERROR("Begin transaction failed"); + } + // 1. set applied index + s = SetAppliedIndex(txn.get(), logIndex); + if (!s.ok()) { + ON_ERROR("Insert applied index to transaction failed"); + } + // 2. check tx lock + TxLock txLock; + s = GetTxLock(txn.get(), primaryKey, &txLock); + if (s.ok()) { + // inprogress or timeout + if (curTimestamp > txLock.timestamp() + txLock.ttl()) { + rc = MetaStatusCode::TX_TIMEOUT; + ON_COMMIT(); + } else { + rc = MetaStatusCode::TX_INPROGRESS; + ON_COMMIT(); + } + } else if (s.IsNotFound()) { + // committed or rollbacked + rc = CheckTxStatus(txn.get(), primaryKey, startTs); + ON_COMMIT(); + } else { + ON_ERROR("Get tx lock failed"); + } + rc = MetaStatusCode::OK; + ON_COMMIT(); +} + +MetaStatusCode DentryStorage::ResolveTxLock(const Dentry& dentry, + uint64_t startTs, uint64_t commitTs, int64_t logIndex) { + WriteLockGuard lg(rwLock_); + Status s; + uint64_t count = nDentry_; + MetaStatusCode rc = MetaStatusCode::STORAGE_INTERNAL_ERROR; + std::shared_ptr txn; + txn = kvStorage_->BeginTransaction(); + if (txn == nullptr) { + ON_ERROR("Begin transaction failed"); + } + // 1. set applied index + s = SetAppliedIndex(txn.get(), logIndex); + if (!s.ok()) { + ON_ERROR("Insert applied index to transaction failed"); + } + TxLock outLock; + s = GetTxLock(txn.get(), DentryKey(dentry), &outLock); + if (s.IsNotFound()) { + rc = MetaStatusCode::OK; + ON_COMMIT(); + } else if (!s.ok()) { + ON_ERROR("Get tx lock failed"); + } + if (outLock.startts() != startTs) { + rc = MetaStatusCode::TX_MISMATCH; + ON_ERROR("tx lock mismatch"); + } + // roll forward + if (commitTs > 0) { + if (!DelTxLock(txn.get(), DentryKey(dentry)).ok()) { + ON_ERROR("Delete tx lock failed"); + } + TxWrite txWrite; + txWrite.set_startts(startTs); + txWrite.set_kind(TxWriteKind::Commit); + if (!SetTxWrite(txn.get(), + TxWriteKey(dentry, commitTs), txWrite).ok()) { + ON_ERROR("Set tx write failed"); + } + // update latest commit + if (!SetLatestCommit(txn.get(), commitTs).ok()) { + ON_ERROR("update latest commit failed"); + } + } else { + // 1. delete tx lock + if (!DelTxLock(txn.get(), DentryKey(dentry)).ok()) { + ON_ERROR("Delete tx lock failed"); + } + // 2. delete tx data with startTs + DentryVec vec; + DentryVector vector(&vec); + std::string skey = DentryKey(dentry); + s = txn->SGet(table4Dentry_, skey, &vec); + if (!s.ok() && !s.IsNotFound()) { + ON_ERROR("Get dentry from transaction failed"); + } + // OK || NOT_FOUND + Dentry preDentry(dentry); + preDentry.set_txid(startTs); + vector.Delete(preDentry); + if (vec.dentrys_size() == 0) { // delete directly + s = txn->SDel(table4Dentry_, skey); + } else { + s = txn->SSet(table4Dentry_, skey, vec); + } + if (!s.ok()) { + ON_ERROR("Delete dentry from transaction failed"); + } + vector.Confirm(&count); + // 3. set tx write + TxWrite txWrite; + txWrite.set_startts(startTs); + txWrite.set_kind(TxWriteKind::Rollback); + if (!SetTxWrite( + txn.get(), TxWriteKey(dentry, startTs), txWrite).ok()) { + ON_ERROR("Set tx write failed"); + } + } + rc = MetaStatusCode::OK; + ON_COMMIT(); +} + +MetaStatusCode DentryStorage::CommitTx(const std::vector& dentrys, + uint64_t startTs, uint64_t commitTs, int64_t logIndex) { + WriteLockGuard lg(rwLock_); + Status s; + uint64_t count = nDentry_; + MetaStatusCode rc = MetaStatusCode::STORAGE_INTERNAL_ERROR; + std::shared_ptr txn; + txn = kvStorage_->BeginTransaction(); + if (txn == nullptr) { + ON_ERROR("Begin transaction failed"); + } + // 1. set applied index + s = SetAppliedIndex(txn.get(), logIndex); + if (!s.ok()) { + ON_ERROR("Insert applied index to transaction failed"); + } + for (const auto& dentry : dentrys) { + // check tx lock + TxLock txLock; + s = GetTxLock(txn.get(), DentryKey(dentry), &txLock); + if (s.IsNotFound()) { + // commited or rollbacked + rc = CheckTxStatus(txn.get(), DentryKey(dentry), startTs); + if (rc == MetaStatusCode::TX_COMMITTED) { + continue; + } else { + ON_ERROR("tx have been rollbacked when commit"); + } + } else if (!s.ok()) { + ON_ERROR("Get tx lock failed"); + } + if (txLock.startts() != startTs) { + rc = MetaStatusCode::TX_MISMATCH; + ON_ERROR("tx lock mismatch"); + } + // set tx write + TxWrite txWrite; + txWrite.set_startts(startTs); + txWrite.set_kind(TxWriteKind::Commit); + if (!SetTxWrite( + txn.get(), TxWriteKey(dentry, commitTs), txWrite).ok()) { + ON_ERROR("Set tx write failed"); + } + // delete tx lock + if (!DelTxLock(txn.get(), DentryKey(dentry)).ok()) { + ON_ERROR("Delete tx lock failed"); + } + } + // update latest commit + if (!SetLatestCommit(txn.get(), startTs).ok()) { + ON_ERROR("update latest commit failed"); + } + rc = MetaStatusCode::OK; + ON_COMMIT(); +} + } // namespace metaserver } // namespace curvefs diff --git a/curvefs/src/metaserver/dentry_storage.h b/curvefs/src/metaserver/dentry_storage.h index 102df6737a..c2bc141ba3 100644 --- a/curvefs/src/metaserver/dentry_storage.h +++ b/curvefs/src/metaserver/dentry_storage.h @@ -81,7 +81,7 @@ class DentryList { DentryList(std::vector* list, uint32_t limit, const std::string& exclude, uint64_t maxTxId, bool onlyDir); - void PushBack(DentryVec* vec); + void PushBack(DentryVec* vec, bool* realEntry); uint32_t Size(); @@ -104,17 +104,19 @@ class DentryStorage { bool Init(); - MetaStatusCode Insert(const Dentry& dentry, int64_t logIndex); + MetaStatusCode Get(Dentry* dentry, TxLock* txLock = nullptr); - // only for loadding from snapshot - MetaStatusCode Insert(const DentryVec& vec, bool merge, int64_t logIndex); + MetaStatusCode List(const Dentry& dentry, std::vector* dentrys, + uint32_t limit, bool onlyDir = false, TxLock* txLock = nullptr); - MetaStatusCode Delete(const Dentry& dentry, int64_t logIndex); + MetaStatusCode Insert(const Dentry& dentry, int64_t logIndex, + TxLock* txLock = nullptr); - MetaStatusCode Get(Dentry* dentry); + // only for loadding from snapshot + MetaStatusCode Insert(const DentryVec& vec, bool merge, int64_t logIndex); - MetaStatusCode List(const Dentry& dentry, std::vector* dentrys, - uint32_t limit, bool onlyDir = false); + MetaStatusCode Delete(const Dentry& dentry, int64_t logIndex, + TxLock* txLock = nullptr); MetaStatusCode PrepareTx(const std::vector& dentrys, const metaserver::TransactionRequest& txRequest, @@ -138,17 +140,22 @@ class DentryStorage { MetaStatusCode GetAppliedIndex(int64_t* index); - private: - std::string DentryKey(const Dentry& entry); + MetaStatusCode PrewriteTx(const std::vector& dentrys, + TxLock txLock, int64_t logIndex, TxLock* out); - bool CompressDentry(storage::StorageTransaction* txn, DentryVec* vec, - BTree* dentrys, uint64_t* outCount); + MetaStatusCode CheckTxStatus(const std::string& primaryKey, + uint64_t startTs, uint64_t curTimestamp, int64_t logIndex); - MetaStatusCode Find(const Dentry& in, Dentry* out, DentryVec* vec); + MetaStatusCode ResolveTxLock(const Dentry& dentry, + uint64_t startTs, uint64_t commitTs, int64_t logIndex); - MetaStatusCode Find(storage::StorageTransaction* txn, const Dentry& in, - Dentry* out, DentryVec* vec, - uint64_t* compressOutCount); + MetaStatusCode CommitTx(const std::vector& dentrys, + uint64_t startTs, uint64_t commitTs, int64_t logIndex); + + private: + std::string DentryKey(const Dentry& daemon); + + std::string TxWriteKey(const Dentry& dentry, uint64_t ts); storage::Status SetAppliedIndex(storage::StorageTransaction* transaction, int64_t index); @@ -176,21 +183,60 @@ class DentryStorage { storage::Status GetHandleTxIndex(int64_t* count); + bool CompressDentry(storage::StorageTransaction* txn, DentryVec* vec, + BTree* dentrys, uint64_t* outCount); + + MetaStatusCode Find(storage::StorageTransaction* txn, const Dentry& in, + Dentry* out, DentryVec* vec, + uint64_t* compressOutCount, + TxLock* txLock); + + MetaStatusCode GetLastTxWriteTs(storage::StorageTransaction* transaction, + const Dentry& dentry, uint64_t* commitTs); + + storage::Status GetLatestCommit(uint64_t* statTs); + + storage::Status SetLatestCommit(storage::StorageTransaction* transaction, + uint64_t ts); + + MetaStatusCode CheckTxStatus(storage::StorageTransaction* transaction, + const std::string& primaryKey, uint64_t ts); + + storage::Status SetTxWrite(storage::StorageTransaction* transaction, + const std::string& key, const TxWrite& txWrite); + + storage::Status GetTxLock(storage::StorageTransaction* transaction, + const std::string& key, TxLock* out); + + storage::Status SetTxLock(storage::StorageTransaction* transaction, + const std::string& key, const TxLock& txLock); + + storage::Status DelTxLock(storage::StorageTransaction* transaction, + const std::string& key); + + MetaStatusCode WriteTx(storage::StorageTransaction* transaction, + const Dentry& dentry, TxLock txLock, uint64_t* count); + private: RWLock rwLock_; std::shared_ptr kvStorage_; std::string table4Dentry_; std::string table4AppliedIndex_; std::string table4Transaction_; + // record dentry total count std::string table4DentryCount_; + std::string table4TxLock_; + std::string table4TxWrite_; int64_t handleTxIndex_; uint64_t nDentry_; Converter conv_; + uint64_t latestCommit_; static const char* kDentryCountKey; static const char* kDentryAppliedKey; static const char* kHandleTxKey; static const char* kPendingTxKey; + static const char* kTxLatestCommit; }; } // namespace metaserver diff --git a/curvefs/src/metaserver/metaserver.cpp b/curvefs/src/metaserver/metaserver.cpp index 5798920f7d..d483e9d5a6 100644 --- a/curvefs/src/metaserver/metaserver.cpp +++ b/curvefs/src/metaserver/metaserver.cpp @@ -197,10 +197,8 @@ void InitExcutorOption(const std::shared_ptr& conf, &opts->minRetryTimesForceTimeoutBackoff); conf->GetValueFatalIfFail("excutorOpt.maxRetryTimesBeforeConsiderSuspend", &opts->maxRetryTimesBeforeConsiderSuspend); - conf->GetValueFatalIfFail("excutorOpt.batchInodeAttrLimit", - &opts->batchInodeAttrLimit); - conf->GetValueFatalIfFail("excutorOpt.enableMultiMountPointRename", - &opts->enableRenameParallel); + conf->GetValueFatalIfFail( + "excutorOpt.batchInodeAttrLimit", &opts->batchInodeAttrLimit); } void InitMetaCacheOption(const std::shared_ptr& conf, diff --git a/curvefs/src/metaserver/metaserver_service.cpp b/curvefs/src/metaserver/metaserver_service.cpp index 6cc3d9c914..b787c6587e 100644 --- a/curvefs/src/metaserver/metaserver_service.cpp +++ b/curvefs/src/metaserver/metaserver_service.cpp @@ -33,26 +33,30 @@ static bvar::LatencyRecorder g_oprequest_in_service_before_propose_latency( namespace curvefs { namespace metaserver { -using ::curvefs::metaserver::copyset::GetDentryOperator; -using ::curvefs::metaserver::copyset::ListDentryOperator; -using ::curvefs::metaserver::copyset::CreateDentryOperator; -using ::curvefs::metaserver::copyset::DeleteDentryOperator; -using ::curvefs::metaserver::copyset::GetInodeOperator; using ::curvefs::metaserver::copyset::BatchGetInodeAttrOperator; using ::curvefs::metaserver::copyset::BatchGetXAttrOperator; +using ::curvefs::metaserver::copyset::CheckTxStatusOperator; +using ::curvefs::metaserver::copyset::CommitTxOperator; +using ::curvefs::metaserver::copyset::CreateDentryOperator; using ::curvefs::metaserver::copyset::CreateInodeOperator; -using ::curvefs::metaserver::copyset::CreateRootInodeOperator; using ::curvefs::metaserver::copyset::CreateManageInodeOperator; -using ::curvefs::metaserver::copyset::UpdateInodeOperator; -using ::curvefs::metaserver::copyset::GetOrModifyS3ChunkInfoOperator; -using ::curvefs::metaserver::copyset::DeleteInodeOperator; -using ::curvefs::metaserver::copyset::UpdateInodeS3VersionOperator; using ::curvefs::metaserver::copyset::CreatePartitionOperator; +using ::curvefs::metaserver::copyset::CreateRootInodeOperator; +using ::curvefs::metaserver::copyset::DeleteDentryOperator; +using ::curvefs::metaserver::copyset::DeleteInodeOperator; using ::curvefs::metaserver::copyset::DeletePartitionOperator; -using ::curvefs::metaserver::copyset::PrepareRenameTxOperator; +using ::curvefs::metaserver::copyset::GetDentryOperator; +using ::curvefs::metaserver::copyset::GetInodeOperator; +using ::curvefs::metaserver::copyset::GetOrModifyS3ChunkInfoOperator; using ::curvefs::metaserver::copyset::GetVolumeExtentOperator; -using ::curvefs::metaserver::copyset::UpdateVolumeExtentOperator; +using ::curvefs::metaserver::copyset::ListDentryOperator; +using ::curvefs::metaserver::copyset::PrepareRenameTxOperator; +using ::curvefs::metaserver::copyset::PrewriteRenameTxOperator; +using ::curvefs::metaserver::copyset::ResolveTxLockOperator; using ::curvefs::metaserver::copyset::UpdateDeallocatableBlockGroupOperator; +using ::curvefs::metaserver::copyset::UpdateInodeOperator; +using ::curvefs::metaserver::copyset::UpdateInodeS3VersionOperator; +using ::curvefs::metaserver::copyset::UpdateVolumeExtentOperator; namespace { @@ -267,16 +271,6 @@ void MetaServerServiceImpl::DeletePartition( request->copysetid()); } -void MetaServerServiceImpl::PrepareRenameTx( - google::protobuf::RpcController* controller, - const PrepareRenameTxRequest* request, PrepareRenameTxResponse* response, - google::protobuf::Closure* done) { - OperatorHelper helper(copysetNodeManager_, inflightThrottle_); - helper.operator()(controller, request, response, - done, request->poolid(), - request->copysetid()); -} - void MetaServerServiceImpl::GetVolumeExtent( ::google::protobuf::RpcController* controller, const GetVolumeExtentRequest* request, @@ -310,5 +304,49 @@ void MetaServerServiceImpl::UpdateDeallocatableBlockGroup( request->copysetid()); } +void MetaServerServiceImpl::PrepareRenameTx( + google::protobuf::RpcController* controller, + const PrepareRenameTxRequest* request, PrepareRenameTxResponse* response, + google::protobuf::Closure* done) { + OperatorHelper helper(copysetNodeManager_, inflightThrottle_); + helper.operator()(controller, request, response, + done, request->poolid(), request->copysetid()); +} + +void MetaServerServiceImpl::PrewriteRenameTx( + google::protobuf::RpcController* controller, + const PrewriteRenameTxRequest* request, PrewriteRenameTxResponse* response, + google::protobuf::Closure* done) { + OperatorHelper helper(copysetNodeManager_, inflightThrottle_); + helper.operator()(controller, request, response, + done, request->poolid(), request->copysetid()); +} + +void MetaServerServiceImpl::CheckTxStatus( + google::protobuf::RpcController* controller, + const CheckTxStatusRequest* request, CheckTxStatusResponse* response, + google::protobuf::Closure* done) { + OperatorHelper helper(copysetNodeManager_, inflightThrottle_); + helper.operator()(controller, request, response, + done, request->poolid(), request->copysetid()); +} + +void MetaServerServiceImpl::ResolveTxLock( + google::protobuf::RpcController* controller, + const ResolveTxLockRequest* request, ResolveTxLockResponse* response, + google::protobuf::Closure* done) { + OperatorHelper helper(copysetNodeManager_, inflightThrottle_); + helper.operator()(controller, request, response, + done, request->poolid(), request->copysetid()); +} + +void MetaServerServiceImpl::CommitTx( + google::protobuf::RpcController* controller, const CommitTxRequest* request, + CommitTxResponse* response, google::protobuf::Closure* done) { + OperatorHelper helper(copysetNodeManager_, inflightThrottle_); + helper.operator()(controller, request, response, done, + request->poolid(), request->copysetid()); +} + } // namespace metaserver } // namespace curvefs diff --git a/curvefs/src/metaserver/metaserver_service.h b/curvefs/src/metaserver/metaserver_service.h index 7e19e20b20..a5fc8e6211 100644 --- a/curvefs/src/metaserver/metaserver_service.h +++ b/curvefs/src/metaserver/metaserver_service.h @@ -108,11 +108,6 @@ class MetaServerServiceImpl : public MetaServerService { DeletePartitionResponse* response, google::protobuf::Closure* done) override; - void PrepareRenameTx(google::protobuf::RpcController* controller, - const PrepareRenameTxRequest* request, - PrepareRenameTxResponse* response, - google::protobuf::Closure* done) override; - void GetVolumeExtent(::google::protobuf::RpcController* controller, const GetVolumeExtentRequest* request, GetVolumeExtentResponse* response, @@ -129,6 +124,29 @@ class MetaServerServiceImpl : public MetaServerService { UpdateDeallocatableBlockGroupResponse *response, ::google::protobuf::Closure *done) override; + // reserved for compatibility + void PrepareRenameTx(google::protobuf::RpcController* controller, + const PrepareRenameTxRequest* request, + PrepareRenameTxResponse* response, + google::protobuf::Closure* done) override; + + void PrewriteRenameTx(google::protobuf::RpcController* controller, + const PrewriteRenameTxRequest* request, + PrewriteRenameTxResponse* response, + google::protobuf::Closure* done) override; + + void CheckTxStatus(google::protobuf::RpcController* controller, + const CheckTxStatusRequest* request, CheckTxStatusResponse* response, + google::protobuf::Closure* done) override; + + void ResolveTxLock(google::protobuf::RpcController* controller, + const ResolveTxLockRequest* request, ResolveTxLockResponse* response, + google::protobuf::Closure* done) override; + + void CommitTx(google::protobuf::RpcController* controller, + const CommitTxRequest* request, CommitTxResponse* response, + google::protobuf::Closure* done) override; + private: CopysetNodeManager* copysetNodeManager_; InflightThrottle* inflightThrottle_; diff --git a/curvefs/src/metaserver/metastore.cpp b/curvefs/src/metaserver/metastore.cpp index a2fd20323c..81a289390a 100644 --- a/curvefs/src/metaserver/metastore.cpp +++ b/curvefs/src/metaserver/metastore.cpp @@ -377,10 +377,13 @@ MetaStatusCode MetaStoreImpl::CreateDentry(const CreateDentryRequest* request, GET_PARTITION_OR_RETURN(partition); Time tm; GET_TIME_FROM_REQUEST(tm); - MetaStatusCode status = - partition->CreateDentry(request->dentry(), tm, logIndex); - response->set_statuscode(status); - return status; + TxLock txLock; + auto rc = partition->CreateDentry(request->dentry(), tm, logIndex, &txLock); + response->set_statuscode(rc); + if (rc == MetaStatusCode::TX_KEY_LOCKED) { + *response->mutable_txlock() = std::move(txLock); + } + return rc; } MetaStatusCode MetaStoreImpl::GetDentry(const GetDentryRequest* request, @@ -403,10 +406,13 @@ MetaStatusCode MetaStoreImpl::GetDentry(const GetDentryRequest* request, dentry.set_name(name); dentry.set_txid(txId); - auto rc = partition->GetDentry(&dentry); + TxLock txLock; + auto rc = partition->GetDentry(&dentry, &txLock); response->set_statuscode(rc); if (rc == MetaStatusCode::OK) { *response->mutable_dentry() = std::move(dentry); + } else if (rc == MetaStatusCode::TX_KEY_LOCKED) { + *response->mutable_txlock() = std::move(txLock); } return rc; } @@ -432,8 +438,12 @@ MetaStatusCode MetaStoreImpl::DeleteDentry(const DeleteDentryRequest* request, Time tm; GET_TIME_FROM_REQUEST(tm); - auto rc = partition->DeleteDentry(dentry, tm, logIndex); + TxLock txLock; + auto rc = partition->DeleteDentry(dentry, tm, logIndex, &txLock); response->set_statuscode(rc); + if (rc == MetaStatusCode::TX_KEY_LOCKED) { + *response->mutable_txlock() = std::move(txLock); + } return rc; } @@ -464,12 +474,16 @@ MetaStatusCode MetaStoreImpl::ListDentry(const ListDentryRequest* request, } std::vector dentrys; - auto rc = - partition->ListDentry(dentry, &dentrys, request->count(), onlyDir); + TxLock txLock; + auto rc = partition->ListDentry( + dentry, &dentrys, request->count(), onlyDir, &txLock); response->set_statuscode(rc); if (rc == MetaStatusCode::OK && !dentrys.empty()) { *response->mutable_dentrys() = {dentrys.begin(), dentrys.end()}; } + if (rc == MetaStatusCode::TX_KEY_LOCKED) { + *response->mutable_txlock() = std::move(txLock); + } return rc; } @@ -491,6 +505,59 @@ MetaStatusCode MetaStoreImpl::PrepareRenameTx( return rc; } +MetaStatusCode MetaStoreImpl::PrewriteRenameTx( + const PrewriteRenameTxRequest* request, PrewriteRenameTxResponse* response, + int64_t logIndex) { + MetaStatusCode rc; + std::shared_ptr partition; + GET_PARTITION_OR_RETURN(partition); + TxLock txLock; + std::vector dentrys{request->dentrys().begin(), + request->dentrys().end()}; + rc = partition->PrewriteRenameTx( + dentrys, request->txlock(), logIndex, &txLock); + response->set_statuscode(rc); + if (rc == MetaStatusCode::TX_KEY_LOCKED) { + *response->mutable_txlock() = std::move(txLock); + } + return rc; +} + +MetaStatusCode MetaStoreImpl::CheckTxStatus(const CheckTxStatusRequest* request, + CheckTxStatusResponse* response, int64_t logIndex) { + MetaStatusCode rc; + std::shared_ptr partition; + GET_PARTITION_OR_RETURN(partition); + rc = partition->CheckTxStatus(request->primarykey(), request->startts(), + request->curtimestamp(), logIndex); + response->set_statuscode(rc); + return MetaStatusCode::OK; +} + +MetaStatusCode MetaStoreImpl::ResolveTxLock(const ResolveTxLockRequest* request, + ResolveTxLockResponse* response, int64_t logIndex) { + MetaStatusCode rc; + std::shared_ptr partition; + GET_PARTITION_OR_RETURN(partition); + rc = partition->ResolveTxLock(request->dentry(), + request->startts(), request->committs(), logIndex); + response->set_statuscode(rc); + return MetaStatusCode::OK; +} + +MetaStatusCode MetaStoreImpl::CommitTx(const CommitTxRequest* request, + CommitTxResponse* response, int64_t logIndex) { + MetaStatusCode rc; + std::shared_ptr partition; + GET_PARTITION_OR_RETURN(partition); + std::vector dentrys{request->dentrys().begin(), + request->dentrys().end()}; + rc = partition->CommitTx(dentrys, request->startts(), + request->committs(), logIndex); + response->set_statuscode(rc); + return MetaStatusCode::OK; +} + // inode MetaStatusCode MetaStoreImpl::CreateInode(const CreateInodeRequest* request, CreateInodeResponse* response, diff --git a/curvefs/src/metaserver/metastore.h b/curvefs/src/metaserver/metastore.h index a13c0a4980..9d906a62dc 100644 --- a/curvefs/src/metaserver/metastore.h +++ b/curvefs/src/metaserver/metastore.h @@ -155,6 +155,19 @@ class MetaStore { const PrepareRenameTxRequest* request, PrepareRenameTxResponse* response, int64_t logIndex) = 0; + virtual MetaStatusCode PrewriteRenameTx( + const PrewriteRenameTxRequest* request, + PrewriteRenameTxResponse* response, int64_t logIndex) = 0; + + virtual MetaStatusCode CheckTxStatus(const CheckTxStatusRequest* request, + CheckTxStatusResponse* response, int64_t logIndex) = 0; + + virtual MetaStatusCode ResolveTxLock(const ResolveTxLockRequest* request, + ResolveTxLockResponse* response, int64_t logIndex) = 0; + + virtual MetaStatusCode CommitTx(const CommitTxRequest* request, + CommitTxResponse* response, int64_t logIndex) = 0; + // inode virtual MetaStatusCode CreateInode(const CreateInodeRequest* request, CreateInodeResponse* response, @@ -261,6 +274,18 @@ class MetaStoreImpl : public MetaStore { PrepareRenameTxResponse* response, int64_t logIndex) override; + MetaStatusCode PrewriteRenameTx(const PrewriteRenameTxRequest* request, + PrewriteRenameTxResponse* response, int64_t logIndex) override; + + MetaStatusCode CheckTxStatus(const CheckTxStatusRequest* request, + CheckTxStatusResponse* response, int64_t logIndex) override; + + MetaStatusCode ResolveTxLock(const ResolveTxLockRequest* request, + ResolveTxLockResponse* response, int64_t logIndex) override; + + MetaStatusCode CommitTx(const CommitTxRequest* request, + CommitTxResponse* response, int64_t logIndex) override; + // inode MetaStatusCode CreateInode(const CreateInodeRequest* request, CreateInodeResponse* response, diff --git a/curvefs/src/metaserver/partition.cpp b/curvefs/src/metaserver/partition.cpp index f942000c3b..3ec98b8b18 100644 --- a/curvefs/src/metaserver/partition.cpp +++ b/curvefs/src/metaserver/partition.cpp @@ -119,9 +119,9 @@ Partition::Partition(PartitionInfo partition, } while (0) MetaStatusCode Partition::CreateDentry(const Dentry& dentry, const Time& tm, - int64_t logIndex) { + int64_t logIndex, TxLock* txLock) { PRECHECK(dentry.fsid(), dentry.parentinodeid()); - MetaStatusCode ret = dentryManager_->CreateDentry(dentry, logIndex); + MetaStatusCode ret = dentryManager_->CreateDentry(dentry, logIndex, txLock); if (MetaStatusCode::OK == ret) { if (dentry.has_type()) { return inodeManager_->UpdateInodeWhenCreateOrRemoveSubNode( @@ -163,11 +163,12 @@ MetaStatusCode Partition::LoadDentry(const DentryVec& vec, bool merge, return rc; } -MetaStatusCode Partition::DeleteDentry( - const Dentry& dentry, const Time& tm, int64_t logIndex) { - PRECHECK(dentry.fsid(), dentry.parentinodeid()); - MetaStatusCode ret = dentryManager_->DeleteDentry(dentry, logIndex); +MetaStatusCode Partition::DeleteDentry(const Dentry& dentry, + const Time& tm, int64_t logIndex, TxLock* txLock) { + PRECHECK(dentry.fsid(), dentry.parentinodeid()); + MetaStatusCode ret = dentryManager_->DeleteDentry( + dentry, logIndex, txLock); if (MetaStatusCode::OK == ret) { if (dentry.has_type()) { return inodeManager_->UpdateInodeWhenCreateOrRemoveSubNode( @@ -193,16 +194,18 @@ MetaStatusCode Partition::DeleteDentry( } } -MetaStatusCode Partition::GetDentry(Dentry* dentry) { +MetaStatusCode Partition::GetDentry(Dentry* dentry, TxLock* txLock) { PRECHECK(dentry->fsid(), dentry->parentinodeid()); - return dentryManager_->GetDentry(dentry); + return dentryManager_->GetDentry(dentry, txLock); } MetaStatusCode Partition::ListDentry(const Dentry& dentry, std::vector* dentrys, - uint32_t limit, bool onlyDir) { + uint32_t limit, bool onlyDir, + TxLock* txLock) { PRECHECK(dentry.fsid(), dentry.parentinodeid()); - return dentryManager_->ListDentry(dentry, dentrys, limit, onlyDir); + return dentryManager_->ListDentry( + dentry, dentrys, limit, onlyDir, txLock); } void Partition::ClearDentry() { dentryManager_->ClearDentry(); } @@ -257,6 +260,35 @@ bool Partition::FindPendingTx(PrepareRenameTxRequest* pendingTx) { return true; } +MetaStatusCode Partition::PrewriteRenameTx(const std::vector& dentrys, + const TxLock& txLock, int64_t logIndex, TxLock* out) { + for (const auto& it : dentrys) { + PRECHECK(it.fsid(), it.parentinodeid()); + } + return dentryManager_->PrewriteRenameTx(dentrys, txLock, logIndex, out); +} + +MetaStatusCode Partition::CheckTxStatus(const std::string& primaryKey, + uint64_t startTs, uint64_t curTimestamp, int64_t logIndex) { + return dentryManager_->CheckTxStatus(primaryKey, startTs, curTimestamp, + logIndex); +} + +MetaStatusCode Partition::ResolveTxLock(const Dentry& dentry, + uint64_t startTs, uint64_t commitTs, int64_t logIndex) { + PRECHECK(dentry.fsid(), dentry.parentinodeid()); + return dentryManager_->ResolveTxLock(dentry, startTs, + commitTs, logIndex); +} + +MetaStatusCode Partition::CommitTx(const std::vector& dentrys, + uint64_t startTs, uint64_t commitTs, int64_t logIndex) { + for (const auto& it : dentrys) { + PRECHECK(it.fsid(), it.parentinodeid()); + } + return dentryManager_->CommitTx(dentrys, startTs, commitTs, logIndex); +} + // inode MetaStatusCode Partition::CreateInode(const InodeParam& param, Inode* inode, int64_t logIndex) { diff --git a/curvefs/src/metaserver/partition.h b/curvefs/src/metaserver/partition.h index df930bda52..30bf94255a 100644 --- a/curvefs/src/metaserver/partition.h +++ b/curvefs/src/metaserver/partition.h @@ -56,22 +56,23 @@ class Partition { Partition(PartitionInfo partition, std::shared_ptr kvStorage, bool startCompact = true, bool startVolumeDeallocate = true); Partition() = default; + bool Init(); // dentry - MetaStatusCode CreateDentry( - const Dentry& dentry, const Time& tm, int64_t logIndex); + MetaStatusCode CreateDentry(const Dentry& dentry, + const Time& tm, int64_t logIndex, TxLock* txLock = nullptr); MetaStatusCode LoadDentry( const DentryVec& vec, bool merge, int64_t logIndex); - MetaStatusCode DeleteDentry( - const Dentry& dentry, const Time& tm, int64_t logIndex); + MetaStatusCode DeleteDentry(const Dentry& dentry, + const Time& tm, int64_t logIndex, TxLock* txLock = nullptr); - MetaStatusCode GetDentry(Dentry* dentry); + MetaStatusCode GetDentry(Dentry* dentry, TxLock* txLock = nullptr); MetaStatusCode ListDentry(const Dentry& dentry, std::vector* dentrys, uint32_t limit, - bool onlyDir = false); + bool onlyDir = false, TxLock* txLock = nullptr); void ClearDentry(); @@ -84,7 +85,17 @@ class Partition { void SerializeRenameTx(const RenameTx& in, PrepareRenameTxRequest* out); - bool Init(); + MetaStatusCode PrewriteRenameTx(const std::vector& dentrys, + const TxLock& txLock, int64_t logIndex, TxLock* out); + + MetaStatusCode CheckTxStatus(const std::string& primaryKey, + uint64_t startTs, uint64_t curTimestamp, int64_t logIndex); + + MetaStatusCode ResolveTxLock(const Dentry& dentry, uint64_t startTs, + uint64_t commitTs, int64_t logIndex); + + MetaStatusCode CommitTx(const std::vector& dentrys, + uint64_t startTs, uint64_t commitTs, int64_t logIndex); // inode MetaStatusCode CreateInode(const InodeParam& param, Inode* inode, diff --git a/curvefs/src/metaserver/recycle_cleaner.cpp b/curvefs/src/metaserver/recycle_cleaner.cpp index bc087a704a..647ab864cb 100644 --- a/curvefs/src/metaserver/recycle_cleaner.cpp +++ b/curvefs/src/metaserver/recycle_cleaner.cpp @@ -87,7 +87,8 @@ bool RecycleCleaner::DeleteNode(const Dentry& dentry) { LOG(INFO) << "RecycleCleaner DeleteNode, " << dentry.ShortDebugString(); // Code refers to the implementation of fuse_client.cpp DeleteNode() // 1. delete dentry - auto ret = metaClient_->DeleteDentry(fsId, parent, name, type); + TxLock txLockOut; + auto ret = metaClient_->DeleteDentry(fsId, parent, name, type, &txLockOut); if (ret != MetaStatusCode::OK) { LOG(WARNING) << "delete dentry fail, ret = " << MetaStatusCode_Name(ret) << ", dentry: " << dentry.ShortDebugString(); @@ -189,8 +190,9 @@ bool RecycleCleaner::DeleteDirRecursive(const Dentry& dentry) { while (true) { // 1. list dir std::list dentryList; + TxLock txLockOut; auto ret = metaClient_->ListDentry(fsId, inodeid, last, limit_, onlyDir, - &dentryList); + &dentryList, &txLockOut); if (ret != MetaStatusCode::OK) { LOG(WARNING) << "DeleteDirRecursive list dentry fail, ret = " << MetaStatusCode_Name(ret) @@ -305,14 +307,17 @@ bool RecycleCleaner::ScanRecycle() { Dentry dentry; dentry.set_fsid(partition_->GetFsId()); dentry.set_parentinodeid(RECYCLEINODEID); + // TODO(ALL): fix it when rewrite recycle + // get txid will not be needed in tx v2 dentry.set_txid(GetTxId()); uint32_t count = 0; uint32_t timeoutCount = 0; while (true) { dentry.set_name(last); std::vector tempDentrys; - auto ret = - partition_->ListDentry(dentry, &tempDentrys, limit_, onlyDir); + TxLock txLockOut; + auto ret = partition_->ListDentry( + dentry, &tempDentrys, limit_, onlyDir, &txLockOut); if (ret != MetaStatusCode::OK) { LOG(WARNING) << "Scan recycle, list dentry fail, dentry = " << dentry.ShortDebugString(); diff --git a/curvefs/src/metaserver/storage/converter.cpp b/curvefs/src/metaserver/storage/converter.cpp index 2e48b20d50..f5ce0328ac 100644 --- a/curvefs/src/metaserver/storage/converter.cpp +++ b/curvefs/src/metaserver/storage/converter.cpp @@ -46,6 +46,9 @@ using ::curve::common::StringToUll; using ::curvefs::common::PartitionInfo; static const char* const kDelimiter = ":"; +// Key4TxWrite: kTypeTxWrite:parentInodeId:name/ts +// if name contains ":" will not work if use ":" kDelimiter +static const char* const kTxDelimiter = "/"; static bool CompareType(const std::string& str, KEY_TYPE keyType) { uint32_t n; @@ -65,7 +68,9 @@ NameGenerator::NameGenerator(uint32_t partitionId) tableName4AppliedIndex_(Format(kTypeAppliedIndex, partitionId)), tableName4Transaction_(Format(kTypeTransaction, partitionId)), tableName4InodeCount_(Format(kTypeInodeCount, partitionId)), - tableName4DentryCount_(Format(kTypeDentryCount, partitionId)) {} + tableName4DentryCount_(Format(kTypeDentryCount, partitionId)), + tableName4TxLock_(Format(kTypeTxLock, partitionId)), + tableName4TxWrite_(Format(kTypeTxWrite, partitionId)) {} std::string NameGenerator::GetInodeTableName() const { return tableName4Inode_; @@ -111,10 +116,16 @@ std::string NameGenerator::GetDentryCountTableName() const { return tableName4DentryCount_; } +std::string NameGenerator::GetTxLockTableName() const { + return tableName4TxLock_; +} + +std::string NameGenerator::GetTxWriteTableName() const { + return tableName4TxWrite_; +} + size_t NameGenerator::GetFixedLength() { - size_t length = sizeof(kTypeInode) + sizeof(uint32_t) + strlen(kDelimiter); - LOG(INFO) << "Tablename fixed length is " << length; - return length; + return sizeof(kTypeInode) + sizeof(uint32_t) + strlen(kDelimiter); } std::string NameGenerator::Format(KEY_TYPE type, uint32_t partitionId) { @@ -123,6 +134,15 @@ std::string NameGenerator::Format(KEY_TYPE type, uint32_t partitionId) { return absl::StrCat(type, kDelimiter, absl::string_view(buf, sizeof(buf))); } +KEY_TYPE NameGenerator::DecodeKeyType(const std::string& name) { + std::vector items; + SplitString(name, kDelimiter, &items); + if (items.size() < 2) { + return KEY_TYPE::kTypeUnknown; + } + return static_cast(std::stoi(items[0])); +} + Key4Inode::Key4Inode() : fsId(0), inodeId(0) {} Key4Inode::Key4Inode(uint32_t fsId, uint64_t inodeId) @@ -136,23 +156,23 @@ bool Key4Inode::operator==(const Key4Inode& rhs) { } std::string Key4Inode::SerializeToString() const { - return absl::StrCat(keyType_, ":", fsId, ":", inodeId); + return absl::StrCat(keyType_, kDelimiter, fsId, kDelimiter, inodeId); } bool Key4Inode::ParseFromString(const std::string& value) { std::vector items; - SplitString(value, ":", &items); + SplitString(value, kDelimiter, &items); return items.size() == 3 && CompareType(items[0], keyType_) && StringToUl(items[1], &fsId) && StringToUll(items[2], &inodeId); } std::string Prefix4AllInode::SerializeToString() const { - return absl::StrCat(keyType_, ":"); + return absl::StrCat(keyType_, kDelimiter); } bool Prefix4AllInode::ParseFromString(const std::string& value) { std::vector items; - SplitString(value, ":", &items); + SplitString(value, kDelimiter, &items); return items.size() == 1 && CompareType(items[0], keyType_); } @@ -179,15 +199,16 @@ Key4S3ChunkInfoList::Key4S3ChunkInfoList(uint32_t fsId, uint64_t inodeId, size(size) {} std::string Key4S3ChunkInfoList::SerializeToString() const { - return absl::StrCat(keyType_, ":", fsId, ":", inodeId, ":", chunkIndex, ":", - absl::StrFormat("%020" PRIu64 "", firstChunkId), ":", - absl::StrFormat("%020" PRIu64 "", lastChunkId), ":", - size); + return absl::StrCat(keyType_, kDelimiter, fsId, kDelimiter, inodeId, + kDelimiter, chunkIndex, kDelimiter, + absl::StrFormat("%020" PRIu64 "", firstChunkId), kDelimiter, + absl::StrFormat("%020" PRIu64 "", lastChunkId), kDelimiter, + size); } bool Key4S3ChunkInfoList::ParseFromString(const std::string& value) { std::vector items; - SplitString(value, ":", &items); + SplitString(value, kDelimiter, &items); return items.size() == 7 && CompareType(items[0], keyType_) && StringToUl(items[1], &fsId) && StringToUll(items[2], &inodeId) && StringToUll(items[3], &chunkIndex) && @@ -203,14 +224,14 @@ Prefix4ChunkIndexS3ChunkInfoList::Prefix4ChunkIndexS3ChunkInfoList( : fsId(fsId), inodeId(inodeId), chunkIndex(chunkIndex) {} std::string Prefix4ChunkIndexS3ChunkInfoList::SerializeToString() const { - return absl::StrCat(keyType_, ":", fsId, ":", inodeId, ":", chunkIndex, - ":"); + return absl::StrCat(keyType_, kDelimiter, fsId, kDelimiter, inodeId, + kDelimiter, chunkIndex, kDelimiter); } bool Prefix4ChunkIndexS3ChunkInfoList::ParseFromString( const std::string& value) { std::vector items; - SplitString(value, ":", &items); + SplitString(value, kDelimiter, &items); return items.size() == 4 && CompareType(items[0], keyType_) && StringToUl(items[1], &fsId) && StringToUll(items[2], &inodeId) && StringToUll(items[3], &chunkIndex); @@ -224,23 +245,24 @@ Prefix4InodeS3ChunkInfoList::Prefix4InodeS3ChunkInfoList(uint32_t fsId, : fsId(fsId), inodeId(inodeId) {} std::string Prefix4InodeS3ChunkInfoList::SerializeToString() const { - return absl::StrCat(keyType_, ":", fsId, ":", inodeId, ":"); + return absl::StrCat(keyType_, kDelimiter, fsId, kDelimiter, + inodeId, kDelimiter); } bool Prefix4InodeS3ChunkInfoList::ParseFromString(const std::string& value) { std::vector items; - SplitString(value, ":", &items); + SplitString(value, kDelimiter, &items); return items.size() == 3 && CompareType(items[0], keyType_) && StringToUl(items[1], &fsId) && StringToUll(items[2], &inodeId); } std::string Prefix4AllS3ChunkInfoList::SerializeToString() const { - return absl::StrCat(kTypeS3ChunkInfo, ":"); + return absl::StrCat(kTypeS3ChunkInfo, kDelimiter); } bool Prefix4AllS3ChunkInfoList::ParseFromString(const std::string& value) { std::vector items; - SplitString(value, ":", &items); + SplitString(value, kDelimiter, &items); return items.size() == 1 && CompareType(items[0], keyType_); } @@ -255,7 +277,7 @@ std::string Key4Dentry::SerializeToString() const { bool Key4Dentry::ParseFromString(const std::string& value) { std::vector items; - SplitString(value, ":", &items); + SplitString(value, kDelimiter, &items); if (items.size() < 3 || !CompareType(items[0], keyType_) || !StringToUl(items[1], &fsId) || !StringToUll(items[2], &parentInodeId)) { @@ -282,21 +304,84 @@ std::string Prefix4SameParentDentry::SerializeToString() const { bool Prefix4SameParentDentry::ParseFromString(const std::string& value) { std::vector items; - SplitString(value, ":", &items); + SplitString(value, kDelimiter, &items); return items.size() == 3 && CompareType(items[0], keyType_) && StringToUl(items[1], &fsId) && StringToUll(items[2], &parentInodeId); } std::string Prefix4AllDentry::SerializeToString() const { - return absl::StrCat(keyType_, ":"); + return absl::StrCat(keyType_, kDelimiter); } bool Prefix4AllDentry::ParseFromString(const std::string& value) { std::vector items; - SplitString(value, ":", &items); + SplitString(value, kDelimiter, &items); return items.size() == 1 && CompareType(items[0], keyType_); } +std::string Key4TxWrite::SerializeToString() const { + return absl::StrCat(keyType_, kDelimiter, fsId, kDelimiter, parentInodeId, + kDelimiter, name, kTxDelimiter, ts); +} + +bool Key4TxWrite::ParseFromString(const std::string& value) { + // 1. split dentryKey and ts + std::vector keys; + SplitString(value, kTxDelimiter, &keys); + if (keys.size() != 2) { + return false; + } + if (!StringToUll(keys[1], &ts)) { + return false; + } + + // 2. decode dentryKey + std::vector items; + SplitString(keys[0], kDelimiter, &items); + if (items.size() < 3 || !CompareType(items[0], keyType_) || + !StringToUl(items[1], &fsId) || + !StringToUll(items[2], &parentInodeId)) { + return false; + } + + size_t prefixLength = items[0].size() + items[1].size() + items[2].size() + + 3 * strlen(kDelimiter); + if (keys[0].size() < prefixLength) { + return false; + } + name = keys[0].substr(prefixLength); + return true; +} + +std::string Prefix4TxWrite::SerializeToString() const { + return absl::StrCat(keyType_, kDelimiter, fsId, kDelimiter, parentInodeId, + kDelimiter, name, kTxDelimiter); +} + +bool Prefix4TxWrite::ParseFromString(const std::string& value) { + std::vector keys; + SplitString(value, kTxDelimiter, &keys); + if (keys.size() != 1) { + return false; + } + + std::vector items; + SplitString(value, kDelimiter, &items); + if (items.size() < 3 || !CompareType(items[0], keyType_) || + !StringToUl(items[1], &fsId) || + !StringToUll(items[2], &parentInodeId)) { + return false; + } + + size_t prefixLength = items[0].size() + items[1].size() + items[2].size() + + 3 * strlen(kDelimiter); + if (value.size() < prefixLength) { + return false; + } + name = value.substr(prefixLength); + return true; +} + Key4VolumeExtentSlice::Key4VolumeExtentSlice(uint32_t fsId, uint64_t inodeId, uint64_t offset) : fsId_(fsId), inodeId_(inodeId), offset_(offset) {} @@ -369,13 +454,13 @@ bool Key4DeallocatableBlockGroup::ParseFromString(const std::string& value) { } std::string Prefix4AllDeallocatableBlockGroup::SerializeToString() const { - return absl::StrCat(keyType_, ":"); + return absl::StrCat(keyType_, kDelimiter); } bool Prefix4AllDeallocatableBlockGroup::ParseFromString( const std::string& value) { std::vector items; - SplitString(value, ":", &items); + SplitString(value, kDelimiter, &items); return items.size() == 1 && CompareType(items[0], keyType_); } diff --git a/curvefs/src/metaserver/storage/converter.h b/curvefs/src/metaserver/storage/converter.h index 41870c6b67..ea045271e0 100644 --- a/curvefs/src/metaserver/storage/converter.h +++ b/curvefs/src/metaserver/storage/converter.h @@ -38,6 +38,7 @@ class MetaStoreFStream; namespace storage { enum KEY_TYPE : unsigned char { + kTypeUnknown = 0, kTypeInode = 1, kTypeS3ChunkInfo = 2, kTypeDentry = 3, @@ -49,7 +50,9 @@ enum KEY_TYPE : unsigned char { kTypeAppliedIndex = 9, kTypeTransaction = 10, kTypeInodeCount = 11, - kTypeDentryCount = 12 + kTypeDentryCount = 12, + kTypeTxLock = 13, + kTypeTxWrite = 14 }; // NOTE: you must generate all table name by NameGenerator class for @@ -83,8 +86,14 @@ class NameGenerator { std::string GetDentryCountTableName() const; + std::string GetTxLockTableName() const; + + std::string GetTxWriteTableName() const; + static size_t GetFixedLength(); + static KEY_TYPE DecodeKeyType(const std::string& name); + private: std::string Format(KEY_TYPE type, uint32_t partitionId); @@ -100,6 +109,8 @@ class NameGenerator { std::string tableName4Transaction_; std::string tableName4InodeCount_; std::string tableName4DentryCount_; + std::string tableName4TxLock_; + std::string tableName4TxWrite_; }; class StorageKey { @@ -126,6 +137,8 @@ class StorageKey { * Key4InodeAuxInfo : kTypeInodeAuxInfo:fsId:inodeId * Key4DeallocatableBlockGroup : kTypeBlockGroup:fsId:volumeOffset * Prefix4AllDeallocatableBlockGroup: kTypeBlockGroup: + * Key4TxWrite : kTypeTxWrite:parentInodeId:name/ts + * Prefix4TxWrite : kTypeTxWrite:parentInodeId:name/ */ class Key4Inode : public StorageKey { @@ -290,6 +303,41 @@ class Prefix4AllDentry : public StorageKey { static const KEY_TYPE keyType_ = kTypeDentry; }; +class Key4TxWrite : public Key4Dentry { + public: + Key4TxWrite() = default; + + Key4TxWrite(uint32_t fsId, uint64_t parentInodeId, + const std::string& name, uint64_t ts) : + Key4Dentry(fsId, parentInodeId, name), ts(ts) {} + + std::string SerializeToString() const override; + + bool ParseFromString(const std::string& value) override; + + public: + uint64_t ts; + + private: + static const KEY_TYPE keyType_ = kTypeTxWrite; +}; + +class Prefix4TxWrite : public Key4Dentry { + public: + Prefix4TxWrite() = default; + + Prefix4TxWrite(uint32_t fsId, uint64_t parentInodeId, + const std::string& name) : + Key4Dentry(fsId, parentInodeId, name) {} + + std::string SerializeToString() const override; + + bool ParseFromString(const std::string& value) override; + + private: + static const KEY_TYPE keyType_ = kTypeTxWrite; +}; + class Key4VolumeExtentSlice : public StorageKey { public: Key4VolumeExtentSlice() = default; diff --git a/curvefs/src/metaserver/storage/rocksdb_options.cpp b/curvefs/src/metaserver/storage/rocksdb_options.cpp index 9ee96cac4e..50631cd75c 100644 --- a/curvefs/src/metaserver/storage/rocksdb_options.cpp +++ b/curvefs/src/metaserver/storage/rocksdb_options.cpp @@ -98,6 +98,14 @@ DEFINE_int32(rocksdb_ordered_cf_max_write_buffer_number, 2, "Number of writer buffer for ordered column family"); +DEFINE_int64(rocksdb_tx_cf_write_buffer_size, + 64ULL << 20, + "Writer buffer size for tx column family"); + +DEFINE_int32(rocksdb_tx_cf_max_write_buffer_number, + 2, + "Number of writer buffer for tx column family"); + DEFINE_int32(rocksdb_max_write_buffer_size_to_maintain, 20ULL << 20, "The target number of write history bytes to hold in memory"); @@ -106,6 +114,10 @@ DEFINE_int32(rocksdb_stats_dump_period_sec, 180, "Dump rocksdb.stats to LOG every stats_dump_period_sec"); +DEFINE_int32(tx_lock_ttl_ms, + 5000, + "tx lock timeout after ttl ms"); + namespace { std::shared_ptr rocksdbBlockCache; @@ -113,6 +125,7 @@ std::shared_ptr rocksdbWriteBufferManager; std::shared_ptr metricEventListener; const char* const kOrderedColumnFamilyName = "ordered_column_family"; +const char* const kTxColumnFamilyName = "tx_column_family"; void CreateBlockCacheAndWriterBufferManager() { static std::once_flag createBlockCache; @@ -208,10 +221,18 @@ void InitRocksdbOptions( unorderedCfOptions.max_write_buffer_number = FLAGS_rocksdb_unordered_cf_max_write_buffer_number; + rocksdb::ColumnFamilyOptions txCfOptions = defaultCfOptions; + txCfOptions.write_buffer_size = + FLAGS_rocksdb_tx_cf_write_buffer_size; + txCfOptions.max_write_buffer_number = + FLAGS_rocksdb_tx_cf_max_write_buffer_number; + columnFamilies->push_back(rocksdb::ColumnFamilyDescriptor{ rocksdb::kDefaultColumnFamilyName, unorderedCfOptions}); columnFamilies->push_back(rocksdb::ColumnFamilyDescriptor{ kOrderedColumnFamilyName, orderedCfOptions}); + columnFamilies->push_back(rocksdb::ColumnFamilyDescriptor{ + kTxColumnFamilyName, txCfOptions}); } void ParseRocksdbOptions(curve::common::Configuration* conf) { @@ -260,6 +281,14 @@ void ParseRocksdbOptions(curve::common::Configuration* conf) { "storage.rocksdb.ordered_max_write_buffer_number", &FLAGS_rocksdb_ordered_cf_max_write_buffer_number, /*fatalIfMissing*/ false); + dummy.Load(conf, "rocksdb_tx_cf_write_buffer_size", + "storage.rocksdb.tx_cf_write_buffer_size", + &FLAGS_rocksdb_tx_cf_write_buffer_size, + /*fatalIfMissing*/ false); + dummy.Load(conf, "rocksdb_tx_cf_max_write_buffer_number", + "storage.rocksdb.tx_cf_max_write_buffer_number", + &FLAGS_rocksdb_tx_cf_max_write_buffer_number, + /*fatalIfMissing*/ false); dummy.Load(conf, "rocksdb_max_write_buffer_size_to_maintain", "storage.rocksdb.max_write_buffer_size_to_maintain", &FLAGS_rocksdb_max_write_buffer_size_to_maintain, @@ -267,6 +296,9 @@ void ParseRocksdbOptions(curve::common::Configuration* conf) { dummy.Load(conf, "rocksdb_stats_dump_period_sec", "storage.rocksdb.stats_dump_period_sec", &FLAGS_rocksdb_stats_dump_period_sec, /*fatalIfMissing*/ false); + dummy.Load(conf, "tx_lock_ttl_ms", + "storage.tx_lock_ttl_ms", + &FLAGS_tx_lock_ttl_ms, /*fatalIfMissing*/ false); } } // namespace storage diff --git a/curvefs/src/metaserver/storage/rocksdb_storage.cpp b/curvefs/src/metaserver/storage/rocksdb_storage.cpp index 5875ba6817..a94fed47c6 100644 --- a/curvefs/src/metaserver/storage/rocksdb_storage.cpp +++ b/curvefs/src/metaserver/storage/rocksdb_storage.cpp @@ -29,7 +29,6 @@ #include "src/common/timeutility.h" #include "curvefs/src/metaserver/storage/utils.h" #include "curvefs/src/metaserver/storage/storage.h" -#include "curvefs/src/metaserver/storage/converter.h" #include "curvefs/src/metaserver/storage/rocksdb_perf.h" #include "curvefs/src/metaserver/storage/rocksdb_storage.h" #include "curvefs/src/metaserver/storage/rocksdb_options.h" @@ -57,8 +56,8 @@ size_t RocksDBStorage::GetKeyPrefixLength() { static const size_t length = []() { const std::string tableName = std::string(NameGenerator::GetFixedLength(), '0'); - const std::string iname = - RocksDBStorage::ToInternalName(tableName, true, true); + const std::string iname = RocksDBStorage::ToInternalName(tableName, + ColumnFamilyType::kUnordered, true); return iname.size(); }(); @@ -162,32 +161,29 @@ bool RocksDBStorage::Close() { return true; } -inline ColumnFamilyHandle* RocksDBStorage::GetColumnFamilyHandle(bool ordered) { - return ordered ? handles_[1] : handles_[0]; -} - /* NOTE: * 1. we use suffix 0/1 to determine the key range: - * [ordered:name:0, ordered:name:1) + * [type:name:0, type:name:1) * 2. please gurantee the length of name is fixed for * we can determine the rocksdb's prefix key */ std::string RocksDBStorage::ToInternalName(const std::string& name, - bool ordered, + ColumnFamilyType type, bool start) { std::ostringstream oss; - oss << ordered << kDelimiter_ << name << kDelimiter_ << (start ? "0" : "1"); + oss << static_cast(type) << kDelimiter_ << name + << kDelimiter_ << (start ? "0" : "1"); return oss.str(); } std::string RocksDBStorage::ToInternalKey(const std::string& name, const std::string& key, - bool ordered) { - std::string iname = ToInternalName(name, ordered, true); + ColumnFamilyType type) { + std::string iname = ToInternalName(name, type, true); std::ostringstream oss; oss << iname << kDelimiter_ << key; std::string ikey = oss.str(); - VLOG(9) << "ikey = " << ikey << " (ordered = " << ordered + VLOG(9) << "ikey = " << ikey << " (type = " << static_cast(type) << ", name = " << name << ", key = " << key << ")" << ", size = " << ikey.size(); return ikey; @@ -198,18 +194,61 @@ std::string RocksDBStorage::ToUserKey(const std::string& ikey) { return ikey.substr(GetKeyPrefixLength() + kDelimiter_.size()); } +ColumnFamilyType Table2FamilyType(const std::string& tableName) { + auto tableKey = NameGenerator::DecodeKeyType(tableName); + switch (tableKey) { + case kTypeInode: + case kTypeInodeAuxInfo: + case kTypeDeallocatableInode: + case kTypeDeallocatableBlockGroup: + return ColumnFamilyType::kUnordered; + case kTypeS3ChunkInfo: + case kTypeDentry: + case kTypeVolumeExtent: + case kTypeAppliedIndex: + case kTypeTransaction: + case kTypeInodeCount: + case kTypeDentryCount: + return ColumnFamilyType::kOrdered; + case kTypeTxLock: + case kTypeTxWrite: + return ColumnFamilyType::kTx; + default: + break; + } + return ColumnFamilyType::kUnknown; +} + +ColumnFamilyHandle* RocksDBStorage::GetColumnFamilyHandle( + ColumnFamilyType type) { + if (type == ColumnFamilyType::kUnknown) { + return nullptr; + } + // handle index is same as dbCfDescriptors_ + // 0: kUnordered; 1: kOrdered; 2: kTxn + return handles_[static_cast(type)]; +} + +#define CHECK_COLUMN_TYPE(name) \ + auto type = Table2FamilyType(name); \ + do { \ + if (ColumnFamilyType::kUnknown == type) { \ + return Status::NotSupported(); \ + } \ + } while (0) + Status RocksDBStorage::Get(const std::string& name, const std::string& key, - ValueType* value, - bool ordered) { + ValueType* value) { if (!inited_) { return Status::DBClosed(); } + CHECK_COLUMN_TYPE(name); ROCKSDB_NAMESPACE::Status s; std::string svalue; - std::string ikey = ToInternalKey(name, key, ordered); - auto handle = GetColumnFamilyHandle(ordered); + std::string ikey = ToInternalKey(name, key, type); + auto handle = GetColumnFamilyHandle(type); { RocksDBPerfGuard guard(OP_GET); s = InTransaction_ ? txn_->Get(dbReadOptions_, handle, ikey, &svalue) : @@ -223,17 +262,17 @@ Status RocksDBStorage::Get(const std::string& name, Status RocksDBStorage::Set(const std::string& name, const std::string& key, - const ValueType& value, - bool ordered) { + const ValueType& value) { std::string svalue; if (!inited_) { return Status::DBClosed(); } else if (!value.SerializeToString(&svalue)) { return Status::SerializedFailed(); } + CHECK_COLUMN_TYPE(name); - auto handle = GetColumnFamilyHandle(ordered); - std::string ikey = ToInternalKey(name, key, ordered); + auto handle = GetColumnFamilyHandle(type); + std::string ikey = ToInternalKey(name, key, type); RocksDBPerfGuard guard(OP_PUT); ROCKSDB_NAMESPACE::Status s = InTransaction_ ? txn_->Put(handle, ikey, svalue) : @@ -242,14 +281,14 @@ Status RocksDBStorage::Set(const std::string& name, } Status RocksDBStorage::Del(const std::string& name, - const std::string& key, - bool ordered) { + const std::string& key) { if (!inited_) { return Status::DBClosed(); } + CHECK_COLUMN_TYPE(name); - std::string ikey = ToInternalKey(name, key, ordered); - auto handle = GetColumnFamilyHandle(ordered); + std::string ikey = ToInternalKey(name, key, type); + auto handle = GetColumnFamilyHandle(type); RocksDBPerfGuard guard(OP_DELETE); ROCKSDB_NAMESPACE::Status s = InTransaction_ ? txn_->Delete(handle, ikey) : @@ -259,22 +298,23 @@ Status RocksDBStorage::Del(const std::string& name, std::shared_ptr RocksDBStorage::Seek(const std::string& name, const std::string& prefix) { - int status = inited_ ? 0 : -1; - std::string ikey = ToInternalKey(name, prefix, true); + auto type = Table2FamilyType(name); + int status = (inited_ && ColumnFamilyType::kUnknown != type) ? 0 : -1; + std::string ikey = ToInternalKey(name, prefix, type); return std::make_shared( - this, ikey, 0, status, true); + this, std::move(ikey), 0, status, type); } -std::shared_ptr RocksDBStorage::GetAll(const std::string& name, - bool ordered) { - int status = inited_ ? 0 : -1; - std::string ikey = ToInternalKey(name, "", ordered); +std::shared_ptr RocksDBStorage::GetAll(const std::string& name) { + auto type = Table2FamilyType(name); + int status = (inited_ && ColumnFamilyType::kUnknown != type) ? 0 : -1; + std::string ikey = ToInternalKey(name, "", type); return std::make_shared( - this, std::move(ikey), 0, status, ordered); + this, std::move(ikey), 0, status, type); } -size_t RocksDBStorage::Size(const std::string& name, bool ordered) { - auto iterator = GetAll(name, ordered); +size_t RocksDBStorage::Size(const std::string& name) { + auto iterator = GetAll(name); if (iterator->Status() != 0) { return 0; } @@ -286,7 +326,7 @@ size_t RocksDBStorage::Size(const std::string& name, bool ordered) { return size; } -Status RocksDBStorage::Clear(const std::string& name, bool ordered) { +Status RocksDBStorage::Clear(const std::string& name) { if (!inited_) { return Status::DBClosed(); } else if (InTransaction_) { @@ -294,6 +334,7 @@ Status RocksDBStorage::Clear(const std::string& name, bool ordered) { // maybe we can implement `Clear` by "iterate and delete" return Status::NotSupported(); } + CHECK_COLUMN_TYPE(name); // TODO(all): Maybe we should let `Clear` just do nothing, because it's only // called when recover state machine from raft snapshot, and in this case, @@ -301,13 +342,14 @@ Status RocksDBStorage::Clear(const std::string& name, bool ordered) { // database's checkpoint in raft snapshot // But, currently, many unittest cases depend it - auto handle = GetColumnFamilyHandle(ordered); - std::string lower = ToInternalName(name, ordered, true); - std::string upper = ToInternalName(name, ordered, false); + auto handle = GetColumnFamilyHandle(type); + std::string lower = ToInternalName(name, type, true); + std::string upper = ToInternalName(name, type, false); RocksDBPerfGuard guard(OP_DELETE_RANGE); ROCKSDB_NAMESPACE::Status s = db_->DeleteRange( dbWriteOptions_, handle, lower, upper); - LOG(INFO) << "Clear(), tablename = " << name << ", ordered = " << ordered + LOG(INFO) << "Clear(), tablename = " << name << ", type = " + << static_cast(type) << ", lower key = " << lower << ", upper key = " << upper; return ToStorageStatus(s); } diff --git a/curvefs/src/metaserver/storage/rocksdb_storage.h b/curvefs/src/metaserver/storage/rocksdb_storage.h index e0023dd8e2..2432da6988 100644 --- a/curvefs/src/metaserver/storage/rocksdb_storage.h +++ b/curvefs/src/metaserver/storage/rocksdb_storage.h @@ -43,7 +43,7 @@ #include "curvefs/src/metaserver/storage/utils.h" #include "curvefs/src/metaserver/storage/storage.h" #include "curvefs/src/metaserver/storage/rocksdb_perf.h" -#include "curvefs/src/metaserver/storage/rocksdb_storage.h" +#include "curvefs/src/metaserver/storage/converter.h" namespace curvefs { namespace metaserver { @@ -65,6 +65,15 @@ using ROCKSDB_NAMESPACE::NewFixedPrefixTransform; using ROCKSDB_NAMESPACE::NewBlockBasedTableFactory; using STORAGE_TYPE = KVStorage::STORAGE_TYPE; +enum class ColumnFamilyType : uint8_t { + kUnordered = 0, + kOrdered = 1, + kTx = 2, + + // unknown type + kUnknown = 255, +}; + // NOTE: The HSize() and SSize() is an expensive operation for rocksdb storage, // you should only invoke it in test cases. class RocksDBStorage : public KVStorage, public StorageTransaction { @@ -132,33 +141,30 @@ class RocksDBStorage : public KVStorage, public StorageTransaction { bool Recover(const std::string& dir) override; private: - ColumnFamilyHandle* GetColumnFamilyHandle(bool ordered); + ColumnFamilyHandle* GetColumnFamilyHandle(ColumnFamilyType type); static size_t GetKeyPrefixLength(); static std::string ToInternalName(const std::string& name, - bool ordered, + ColumnFamilyType type, bool start); std::string ToInternalKey(const std::string& name, const std::string& key, - bool ordered); + ColumnFamilyType type); std::string ToUserKey(const std::string& ikey); Status Get(const std::string& name, const std::string& key, - ValueType* value, - bool ordered); + ValueType* value); Status Set(const std::string& name, const std::string& key, - const ValueType& value, - bool ordered); + const ValueType& value); Status Del(const std::string& name, - const std::string& key, - bool ordered); + const std::string& key); std::shared_ptr Seek(const std::string& name, const std::string& prefix); @@ -166,11 +172,11 @@ class RocksDBStorage : public KVStorage, public StorageTransaction { // TODO(@Wine93): We do not support transactions for the // below 3 methods, maybe we should return Status::NotSupported // when user invoke it in transaction. - std::shared_ptr GetAll(const std::string& name, bool ordered); + std::shared_ptr GetAll(const std::string& name); - size_t Size(const std::string& name, bool ordered); + size_t Size(const std::string& name); - Status Clear(const std::string& name, bool ordered); + Status Clear(const std::string& name); private: friend class RocksDBStorageIterator; @@ -210,48 +216,48 @@ class RocksDBStorage : public KVStorage, public StorageTransaction { inline Status RocksDBStorage::HGet(const std::string& name, const std::string& key, ValueType* value) { - return Get(name, key, value, false); + return Get(name, key, value); } inline Status RocksDBStorage::HSet(const std::string& name, const std::string& key, const ValueType& value) { - return Set(name, key, value, false); + return Set(name, key, value); } inline Status RocksDBStorage::HDel(const std::string& name, const std::string& key) { - return Del(name, key, false); + return Del(name, key); } inline std::shared_ptr RocksDBStorage::HGetAll( const std::string& name) { - return GetAll(name, false); + return GetAll(name); } inline size_t RocksDBStorage::HSize(const std::string& name) { - return Size(name, false); + return Size(name); } inline Status RocksDBStorage::HClear(const std::string& name) { - return Clear(name, false); + return Clear(name); } inline Status RocksDBStorage::SGet(const std::string& name, const std::string& key, ValueType* value) { - return Get(name, key, value, true); + return Get(name, key, value); } inline Status RocksDBStorage::SSet(const std::string& name, const std::string& key, const ValueType& value) { - return Set(name, key, value, true); + return Set(name, key, value); } inline Status RocksDBStorage::SDel(const std::string& name, const std::string& key) { - return Del(name, key, true); + return Del(name, key); } inline std::shared_ptr RocksDBStorage::SSeek( @@ -261,15 +267,15 @@ inline std::shared_ptr RocksDBStorage::SSeek( inline std::shared_ptr RocksDBStorage::SGetAll( const std::string& name) { - return GetAll(name, true); + return GetAll(name); } inline size_t RocksDBStorage::SSize(const std::string& name) { - return Size(name, true); + return Size(name); } inline Status RocksDBStorage::SClear(const std::string& name) { - return Clear(name, true); + return Clear(name); } class RocksDBStorageIterator : public Iterator { @@ -278,13 +284,13 @@ class RocksDBStorageIterator : public Iterator { std::string prefix, size_t size, int status, - bool ordered) + ColumnFamilyType type) : storage_(storage), prefix_(std::move(prefix)), size_(size), status_(status), prefixChecking_(true), - ordered_(ordered), + type_(type), iter_(nullptr) { RocksDBPerfGuard guard(OP_GET_SNAPSHOT); if (status_ == 0) { @@ -324,7 +330,7 @@ class RocksDBStorageIterator : public Iterator { } void SeekToFirst() { - auto handler = storage_->GetColumnFamilyHandle(ordered_); + auto handler = storage_->GetColumnFamilyHandle(type_); { RocksDBPerfGuard guard(OP_GET_ITERATOR); if (storage_->InTransaction_) { @@ -379,7 +385,7 @@ class RocksDBStorageIterator : public Iterator { uint64_t size_; int status_; bool prefixChecking_; - bool ordered_; + ColumnFamilyType type_; std::unique_ptr iter_; rocksdb::ReadOptions readOptions_; }; diff --git a/curvefs/test/client/client_operator_test.cpp b/curvefs/test/client/client_operator_test.cpp index 45574a56c6..41dda5c9fa 100644 --- a/curvefs/test/client/client_operator_test.cpp +++ b/curvefs/test/client/client_operator_test.cpp @@ -25,8 +25,8 @@ #include #include "curvefs/src/client/client_operator.h" -#include "curvefs/test/client/mock_dentry_cache_mamager.h" -#include "curvefs/test/client/mock_inode_cache_manager.h" +#include "curvefs/test/client/mock_dentry_mamager.h" +#include "curvefs/test/client/mock_inode_manager.h" #include "curvefs/test/client/mock_metaserver_client.h" #include "curvefs/test/client/rpcclient/mock_mds_client.h" @@ -191,5 +191,85 @@ TEST_F(ClientOperatorTest, CommitTx) { ASSERT_EQ(rc, CURVEFS_ERROR::OK); } +TEST_F(ClientOperatorTest, PrewriteTx) { + CURVEFS_ERROR rc = CURVEFS_ERROR::OK; + // 1. tso failed + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::UNKNOWN_ERROR)); + rc = renameOp_->PrewriteTx(); + ASSERT_EQ(rc, CURVEFS_ERROR::INTERNAL); + // 2. GetPartitionId failed + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::OK)) + .WillOnce(Return(FSStatusCode::OK)); + EXPECT_CALL(*metaClient_, GetPartitionId(_, _, _)) + .WillOnce(Return(false)) + .WillOnce(Return(true)) + .WillOnce(Return(false)); + rc = renameOp_->PrewriteTx(); + ASSERT_EQ(rc, CURVEFS_ERROR::INTERNAL); + rc = renameOp_->PrewriteTx(); + ASSERT_EQ(rc, CURVEFS_ERROR::INTERNAL); + // 3. PrewriteRenameTx failed + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::OK)); + EXPECT_CALL(*metaClient_, GetPartitionId(_, _, _)) + .WillOnce(DoAll(SetArgPointee<2>(1), Return(true))) + .WillOnce(DoAll(SetArgPointee<2>(2), Return(true))); + EXPECT_CALL(*metaClient_, PrewriteRenameTx(_, _, _)) + .WillOnce(Return(MetaStatusCode::STORAGE_INTERNAL_ERROR)); + rc = renameOp_->PrewriteTx(); + ASSERT_EQ(rc, CURVEFS_ERROR::INTERNAL); + // 4. PrewriteRenameTx key is locked and CheckAndResolveTx failed + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::OK)); + EXPECT_CALL(*metaClient_, GetPartitionId(_, _, _)) + .WillOnce(DoAll(SetArgPointee<2>(1), Return(true))) + .WillOnce(DoAll(SetArgPointee<2>(2), Return(true))); + EXPECT_CALL(*metaClient_, PrewriteRenameTx(_, _, _)) + .WillOnce(Return(MetaStatusCode::TX_KEY_LOCKED)); + EXPECT_CALL(*dentryManager_, CheckAndResolveTx(_, _, _, _)) + .WillOnce(Return(MetaStatusCode::STORAGE_INTERNAL_ERROR)); + rc = renameOp_->PrewriteTx(); + ASSERT_EQ(rc, CURVEFS_ERROR::INTERNAL); + // 5. PrewriteRenameTx key is locked and CheckAndResolveTx success + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::OK)); + EXPECT_CALL(*metaClient_, GetPartitionId(_, _, _)) + .WillOnce(DoAll(SetArgPointee<2>(1), Return(true))) + .WillOnce(DoAll(SetArgPointee<2>(2), Return(true))); + EXPECT_CALL(*metaClient_, PrewriteRenameTx(_, _, _)) + .WillOnce(Return(MetaStatusCode::TX_KEY_LOCKED)) + .WillOnce(Return(MetaStatusCode::OK)) + .WillOnce(Return(MetaStatusCode::OK)); + EXPECT_CALL(*dentryManager_, CheckAndResolveTx(_, _, _, _)) + .WillOnce(Return(MetaStatusCode::OK)); + rc = renameOp_->PrewriteTx(); + ASSERT_EQ(rc, CURVEFS_ERROR::OK); +} + +TEST_F(ClientOperatorTest, CommitTxV2) { + CURVEFS_ERROR rc = CURVEFS_ERROR::OK; + // 1. tso failed + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::UNKNOWN_ERROR)); + rc = renameOp_->CommitTxV2(); + ASSERT_EQ(rc, CURVEFS_ERROR::INTERNAL); + // 2. CommitTx failed + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::OK)); + EXPECT_CALL(*metaClient_, CommitTx(_, _, _)) + .WillOnce(Return(MetaStatusCode::STORAGE_INTERNAL_ERROR)); + rc = renameOp_->CommitTxV2(); + ASSERT_EQ(rc, CURVEFS_ERROR::INTERNAL); + // 3. CommitTx success + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::OK)); + EXPECT_CALL(*metaClient_, CommitTx(_, _, _)) + .WillOnce(Return(MetaStatusCode::OK)); + rc = renameOp_->CommitTxV2(); + ASSERT_EQ(rc, CURVEFS_ERROR::OK); +} + } // namespace client } // namespace curvefs diff --git a/curvefs/test/client/client_prefetch_test.cpp b/curvefs/test/client/client_prefetch_test.cpp index 712fbd208d..896467fcc0 100644 --- a/curvefs/test/client/client_prefetch_test.cpp +++ b/curvefs/test/client/client_prefetch_test.cpp @@ -33,7 +33,7 @@ #include "curvefs/test/client/mock_disk_cache_manager.h" #include "curvefs/test/client/mock_disk_cache_read.h" #include "curvefs/test/client/mock_disk_cache_write.h" -#include "curvefs/test/client/mock_inode_cache_manager.h" +#include "curvefs/test/client/mock_inode_manager.h" #include "curvefs/test/client/mock_kvclient.h" #include "curvefs/test/client/mock_test_posix_wapper.h" #include "src/common/concurrent/task_thread_pool.h" diff --git a/curvefs/test/client/client_s3_adaptor_Integration.cpp b/curvefs/test/client/client_s3_adaptor_Integration.cpp index 1b4ee2a280..1127dda78c 100644 --- a/curvefs/test/client/client_s3_adaptor_Integration.cpp +++ b/curvefs/test/client/client_s3_adaptor_Integration.cpp @@ -30,7 +30,7 @@ #include "curvefs/src/client/kvclient/kvclient_manager.h" #include "src/common/curve_define.h" #include "curvefs/test/client/mock_client_s3.h" -#include "curvefs/test/client/mock_inode_cache_manager.h" +#include "curvefs/test/client/mock_inode_manager.h" #include "curvefs/test/client/mock_metaserver_service.h" #include "curvefs/test/client/mock_kvclient.h" #include "curvefs/test/client/rpcclient/mock_mds_client.h" diff --git a/curvefs/test/client/client_s3_adaptor_test.cpp b/curvefs/test/client/client_s3_adaptor_test.cpp index 7430a14856..d062d5e674 100644 --- a/curvefs/test/client/client_s3_adaptor_test.cpp +++ b/curvefs/test/client/client_s3_adaptor_test.cpp @@ -30,7 +30,7 @@ #include "curvefs/test/client/mock_client_s3.h" #include "curvefs/test/client/mock_client_s3_cache_manager.h" #include "curvefs/test/client/mock_disk_cache_manager.h" -#include "curvefs/test/client/mock_inode_cache_manager.h" +#include "curvefs/test/client/mock_inode_manager.h" #include "curvefs/test/client/mock_metaserver_service.h" #include "curvefs/test/client/rpcclient/mock_mds_client.h" #include "src/common/curve_define.h" diff --git a/curvefs/test/client/file_cache_manager_test.cpp b/curvefs/test/client/file_cache_manager_test.cpp index 013e64b6f8..d9549611f7 100644 --- a/curvefs/test/client/file_cache_manager_test.cpp +++ b/curvefs/test/client/file_cache_manager_test.cpp @@ -26,7 +26,7 @@ #include "curvefs/src/client/s3/client_s3_adaptor.h" #include "curvefs/src/client/s3/client_s3_cache_manager.h" #include "curvefs/test/client/mock_client_s3_cache_manager.h" -#include "curvefs/test/client/mock_inode_cache_manager.h" +#include "curvefs/test/client/mock_inode_manager.h" #include "curvefs/test/client/mock_client_s3.h" #include "src/common/concurrent/task_thread_pool.h" diff --git a/curvefs/test/client/filesystem/helper/builder.h b/curvefs/test/client/filesystem/helper/builder.h index 2370dad311..31178733af 100644 --- a/curvefs/test/client/filesystem/helper/builder.h +++ b/curvefs/test/client/filesystem/helper/builder.h @@ -32,8 +32,8 @@ #include "curvefs/src/client/filesystem/meta.h" #include "curvefs/src/client/filesystem/filesystem.h" #include "curvefs/test/client/mock_metaserver_client.h" -#include "curvefs/test/client/mock_inode_cache_manager.h" -#include "curvefs/test/client/mock_dentry_cache_mamager.h" +#include "curvefs/test/client/mock_inode_manager.h" +#include "curvefs/test/client/mock_dentry_mamager.h" namespace curvefs { namespace client { diff --git a/curvefs/test/client/filesystem/helper/expect.h b/curvefs/test/client/filesystem/helper/expect.h index 5daad73d9c..a3df7b5b91 100644 --- a/curvefs/test/client/filesystem/helper/expect.h +++ b/curvefs/test/client/filesystem/helper/expect.h @@ -26,8 +26,8 @@ #include #include "curvefs/test/client/mock_metaserver_client.h" -#include "curvefs/test/client/mock_inode_cache_manager.h" -#include "curvefs/test/client/mock_dentry_cache_mamager.h" +#include "curvefs/test/client/mock_inode_manager.h" +#include "curvefs/test/client/mock_dentry_mamager.h" namespace curvefs { namespace client { diff --git a/curvefs/test/client/mock_dentry_cache_mamager.h b/curvefs/test/client/mock_dentry_mamager.h similarity index 78% rename from curvefs/test/client/mock_dentry_cache_mamager.h rename to curvefs/test/client/mock_dentry_mamager.h index 2e5e8ac41e..f20a3217c4 100644 --- a/curvefs/test/client/mock_dentry_cache_mamager.h +++ b/curvefs/test/client/mock_dentry_mamager.h @@ -20,14 +20,15 @@ * Author: xuchaojie */ -#ifndef CURVEFS_TEST_CLIENT_MOCK_DENTRY_CACHE_MAMAGER_H_ -#define CURVEFS_TEST_CLIENT_MOCK_DENTRY_CACHE_MAMAGER_H_ +#ifndef CURVEFS_TEST_CLIENT_MOCK_DENTRY_MAMAGER_H_ +#define CURVEFS_TEST_CLIENT_MOCK_DENTRY_MAMAGER_H_ #include #include #include #include -#include "curvefs/src/client/dentry_cache_manager.h" +#include +#include "curvefs/src/client/dentry_manager.h" namespace curvefs { namespace client { @@ -37,6 +38,8 @@ class MockDentryCacheManager : public DentryCacheManager { MockDentryCacheManager() {} ~MockDentryCacheManager() {} + MOCK_METHOD(void, Init, (std::shared_ptr mdsClient), (override)); + MOCK_METHOD3(GetDentry, CURVEFS_ERROR(uint64_t parent, const std::string &name, Dentry *out)); @@ -51,10 +54,14 @@ class MockDentryCacheManager : public DentryCacheManager { uint32_t limit, bool onlyDir, uint32_t nlink)); + + MOCK_METHOD(MetaStatusCode, CheckAndResolveTx, (const Dentry& dentry, + const TxLock& txLock, uint64_t timestamp, uint64_t commitTs), + (override)); }; } // namespace client } // namespace curvefs -#endif // CURVEFS_TEST_CLIENT_MOCK_DENTRY_CACHE_MAMAGER_H_ +#endif // CURVEFS_TEST_CLIENT_MOCK_DENTRY_MAMAGER_H_ diff --git a/curvefs/test/client/mock_inode_cache_manager.h b/curvefs/test/client/mock_inode_manager.h similarity index 91% rename from curvefs/test/client/mock_inode_cache_manager.h rename to curvefs/test/client/mock_inode_manager.h index 39c332ce04..adc043d4a5 100644 --- a/curvefs/test/client/mock_inode_cache_manager.h +++ b/curvefs/test/client/mock_inode_manager.h @@ -20,8 +20,8 @@ * Author: xuchaojie */ -#ifndef CURVEFS_TEST_CLIENT_MOCK_INODE_CACHE_MANAGER_H_ -#define CURVEFS_TEST_CLIENT_MOCK_INODE_CACHE_MANAGER_H_ +#ifndef CURVEFS_TEST_CLIENT_MOCK_INODE_MANAGER_H_ +#define CURVEFS_TEST_CLIENT_MOCK_INODE_MANAGER_H_ #include #include @@ -30,7 +30,7 @@ #include #include -#include "curvefs/src/client/inode_cache_manager.h" +#include "curvefs/src/client/inode_manager.h" namespace curvefs { namespace client { @@ -77,4 +77,4 @@ class MockInodeCacheManager : public InodeCacheManager { } // namespace curvefs -#endif // CURVEFS_TEST_CLIENT_MOCK_INODE_CACHE_MANAGER_H_ +#endif // CURVEFS_TEST_CLIENT_MOCK_INODE_MANAGER_H_ diff --git a/curvefs/test/client/mock_metaserver_base_client.h b/curvefs/test/client/mock_metaserver_base_client.h deleted file mode 100644 index dec0dbcdba..0000000000 --- a/curvefs/test/client/mock_metaserver_base_client.h +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Copyright (c) 2021 NetEase Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/* - * Project: curve - * Created Date: Thur Jun 16 2021 - * Author: lixiaocui - */ - - -#ifndef CURVEFS_TEST_CLIENT_MOCK_METASERVER_BASE_CLIENT_H_ -#define CURVEFS_TEST_CLIENT_MOCK_METASERVER_BASE_CLIENT_H_ - -#include -#include -#include -#include "curvefs/src/client/base_client.h" - -namespace curvefs { -namespace client { - -class MockMetaServerBaseClient : public MetaServerBaseClient { - public: - MockMetaServerBaseClient() : MetaServerBaseClient() {} - ~MockMetaServerBaseClient() = default; - - MOCK_METHOD6(GetDentry, - void(uint32_t fsId, uint64_t inodeid, const std::string &name, - GetDentryResponse *response, brpc::Controller *cntl, - brpc::Channel *channel)); - - MOCK_METHOD7(ListDentry, - void(uint32_t fsId, uint64_t inodeid, const std::string &last, - uint32_t count, ListDentryResponse *response, - brpc::Controller *cntl, brpc::Channel *channel)); - - MOCK_METHOD4(CreateDentry, - void(const Dentry &dentry, CreateDentryResponse *response, - brpc::Controller *cntl, brpc::Channel *channel)); - - MOCK_METHOD6(DeleteDentry, - void(uint32_t fsId, uint64_t inodeid, const std::string &name, - DeleteDentryResponse *response, brpc::Controller *cntl, - brpc::Channel *channel)); - - MOCK_METHOD4(PrepareRenameTx, void(const std::vector& dentrys, - PrepareRenameTxResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel)); - - MOCK_METHOD5(GetInode, - void(uint32_t fsId, uint64_t inodeid, - GetInodeResponse *response, brpc::Controller *cntl, - brpc::Channel *channel)); - - MOCK_METHOD4(UpdateInode, - void(const Inode &inode, UpdateInodeResponse *response, - brpc::Controller *cntl, brpc::Channel *channel)); - - MOCK_METHOD4(CreateInode, - void(const InodeParam ¶m, CreateInodeResponse *response, - brpc::Controller *cntl, brpc::Channel *channel)); - - MOCK_METHOD5(DeleteInode, - void(uint32_t fsId, uint64_t inodeid, - DeleteInodeResponse *response, brpc::Controller *cntl, - brpc::Channel *channel)); -}; -} // namespace client -} // namespace curvefs - -#endif // CURVEFS_TEST_CLIENT_MOCK_METASERVER_BASE_CLIENT_H_ diff --git a/curvefs/test/client/mock_metaserver_client.h b/curvefs/test/client/mock_metaserver_client.h index b0fac7b1fe..db2cb6f432 100644 --- a/curvefs/test/client/mock_metaserver_client.h +++ b/curvefs/test/client/mock_metaserver_client.h @@ -55,18 +55,19 @@ class MockMetaServerClient : public MetaServerClient { MOCK_METHOD2(SetTxId, void(uint32_t partitionId, uint64_t txId)); - MOCK_METHOD4(GetDentry, MetaStatusCode(uint32_t fsId, uint64_t inodeid, - const std::string &name, Dentry *out)); + MOCK_METHOD(MetaStatusCode, GetDentry, (uint32_t fsId, uint64_t inodeid, + const std::string &name, Dentry *out, TxLock* txLockOut), (override)); - MOCK_METHOD6(ListDentry, MetaStatusCode(uint32_t fsId, uint64_t inodeid, + MOCK_METHOD(MetaStatusCode, ListDentry, (uint32_t fsId, uint64_t inodeid, const std::string &last, uint32_t count, bool onlyDir, - std::list *dentryList)); + std::list *dentryList, TxLock* txLockOut), (override)); - MOCK_METHOD1(CreateDentry, MetaStatusCode(const Dentry &dentry)); + MOCK_METHOD(MetaStatusCode, CreateDentry, ( + const Dentry &dentry, TxLock* txLockOut), (override)); - MOCK_METHOD4(DeleteDentry, MetaStatusCode( + MOCK_METHOD(MetaStatusCode, DeleteDentry, ( uint32_t fsId, uint64_t inodeid, const std::string &name, - FsFileType type)); + FsFileType type, TxLock* txLockOut), (override)); MOCK_METHOD1(PrepareRenameTx, MetaStatusCode(const std::vector& dentrys)); @@ -161,6 +162,23 @@ class MockMetaServerClient : public MetaServerClient { MOCK_METHOD3(UpdateDeallocatableBlockGroup, MetaStatusCode(uint32_t, uint64_t, DeallocatableBlockGroupMap *)); + + MOCK_METHOD(MetaStatusCode, PrewriteRenameTx, + (const std::vector& dentrys, + const TxLock& txLockIn, TxLock* txLockOut), (override)); + + MOCK_METHOD(MetaStatusCode, CheckTxStatus, (uint32_t fsId, uint64_t inodeId, + const std::string& primaryKey, uint64_t startTs, uint64_t curTimestamp), + (override)); + + MOCK_METHOD(MetaStatusCode, ResolveTxLock, (const Dentry& dentry, + uint64_t startTs, uint64_t commitTs), (override)); + + MOCK_METHOD(MetaStatusCode, CommitTx, (const std::vector& dentrys, + uint64_t startTs, uint64_t commitTs), (override)); + + MOCK_METHOD(bool, GetPartitionId, (uint32_t fsId, uint64_t inodeId, + PartitionID *partitionId), (override)); }; } // namespace rpcclient diff --git a/curvefs/test/client/rpcclient/mds_client_test.cpp b/curvefs/test/client/rpcclient/mds_client_test.cpp index d6b292e11c..33e9221b44 100644 --- a/curvefs/test/client/rpcclient/mds_client_test.cpp +++ b/curvefs/test/client/rpcclient/mds_client_test.cpp @@ -1056,6 +1056,51 @@ TEST_F(MdsClientImplTest, test_SetMdsAddrs) { ASSERT_EQ(mdsclient_.GetMdsAddrs(), addr_new); } +TEST_F(MdsClientImplTest, Tso) { + curvefs::mds::TsoResponse response; + + // CASE 1: Tso success + response.set_statuscode(FSStatusCode::OK); + response.set_ts(1); + response.set_timestamp(100); + EXPECT_CALL(mockmdsbasecli_, Tso(_, _, _, _)) + .WillOnce(SetArgPointee<1>(response)); + + uint64_t ts; + uint64_t timestamp; + auto rc = mdsclient_.Tso(&ts, ×tamp); + ASSERT_EQ(rc, FSStatusCode::OK); + ASSERT_EQ(ts, 1); + ASSERT_EQ(timestamp, 100); + + // CASE 2: Tso fail + response.set_statuscode(FSStatusCode::UNKNOWN_ERROR); + EXPECT_CALL(mockmdsbasecli_, Tso(_, _, _, _)) + .WillOnce(SetArgPointee<1>(response)); + + rc = mdsclient_.Tso(&ts, ×tamp); + ASSERT_EQ(rc, FSStatusCode::UNKNOWN_ERROR); + + // CASE 3: RPC error, retry until success + int count = 0; + EXPECT_CALL(mockmdsbasecli_, Tso(_, _, _, _)) + .Times(6) + .WillRepeatedly( + Invoke([&](const TsoRequest& request, + TsoResponse *response, + brpc::Controller *cntl, + brpc::Channel *channel) { + if (++count <= 5) { + cntl->SetFailed(112, "Not connected to"); + } else { + response->set_statuscode(FSStatusCode::OK); + } + })); + + rc = mdsclient_.Tso(&ts, ×tamp); + ASSERT_EQ(rc, FSStatusCode::OK); +} + } // namespace rpcclient } // namespace client } // namespace curvefs diff --git a/curvefs/test/client/rpcclient/metaserver_client_test.cpp b/curvefs/test/client/rpcclient/metaserver_client_test.cpp index bfffefce54..3cd751cbe4 100644 --- a/curvefs/test/client/rpcclient/metaserver_client_test.cpp +++ b/curvefs/test/client/rpcclient/metaserver_client_test.cpp @@ -63,6 +63,12 @@ using ::curvefs::metaserver::BatchGetXAttrRequest; using ::curvefs::metaserver::BatchGetXAttrResponse; using ::curvefs::metaserver::UpdateDeallocatableBlockGroupRequest; using ::curvefs::metaserver::UpdateDeallocatableBlockGroupResponse; +using ::curvefs::metaserver::PrepareRenameTxRequest; +using ::curvefs::metaserver::PrepareRenameTxResponse; +using ::curvefs::metaserver::CheckTxStatusRequest; +using ::curvefs::metaserver::CheckTxStatusResponse; +using ::curvefs::metaserver::ResolveTxLockRequest; +using ::curvefs::metaserver::ResolveTxLockResponse; using ::curvefs::common::StreamServer; using ::curvefs::common::StreamOptions; using ::curvefs::common::StreamConnection; @@ -134,6 +140,7 @@ TEST_F(MetaServerClientImplTest, test_GetDentry) { // out Dentry out; uint64_t txID = 1; + TxLock txLockOut; // set response curvefs::metaserver::GetDentryResponse response; @@ -151,8 +158,11 @@ TEST_F(MetaServerClientImplTest, test_GetDentry) { Invoke(SetRpcService)); EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); + EXPECT_CALL(*mockMetacache_.get(), GetTargetLeader(_, _)) + .WillRepeatedly(Return(true)); - MetaStatusCode status = metaserverCli_.GetDentry(fsID, inodeID, name, &out); + MetaStatusCode status = metaserverCli_.GetDentry( + fsID, inodeID, name, &out, &txLockOut); ASSERT_EQ(MetaStatusCode::RPC_ERROR, status); // test1: get dentry ok @@ -165,7 +175,7 @@ TEST_F(MetaServerClientImplTest, test_GetDentry) { EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) .WillOnce(DoAll(SetArgPointee<2>(target_), Return(true))); - status = metaserverCli_.GetDentry(fsID, inodeID, name, &out); + status = metaserverCli_.GetDentry(fsID, inodeID, name, &out, &txLockOut); ASSERT_EQ(MetaStatusCode::OK, status); ASSERT_TRUE(google::protobuf::util::MessageDifferencer::Equals(out, *d)) << "out:\n" @@ -175,7 +185,7 @@ TEST_F(MetaServerClientImplTest, test_GetDentry) { // test2: get dentry get target fail EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) .WillRepeatedly(Return(false)); - status = metaserverCli_.GetDentry(fsID, inodeID, name, &out); + status = metaserverCli_.GetDentry(fsID, inodeID, name, &out, &txLockOut); ASSERT_EQ(MetaStatusCode::RPC_ERROR, status); // test3: get dentry over load and fail retry ok @@ -192,12 +202,32 @@ TEST_F(MetaServerClientImplTest, test_GetDentry) { DoAll(SetArgPointee<2>(response), Invoke(SetRpcService))); - status = metaserverCli_.GetDentry(fsID, inodeID, name, &out); + status = metaserverCli_.GetDentry(fsID, inodeID, name, &out, &txLockOut); ASSERT_EQ(MetaStatusCode::OK, status); - // test4: test response do not have dentry - response.clear_dentry(); + // test4: test dentry has tx lock + TxLock txLock; + txLock.set_primarykey("key"); + txLock.set_startts(1); + txLock.set_timestamp(100); + response.set_statuscode(MetaStatusCode::TX_KEY_LOCKED); + *response.mutable_txlock() = txLock; + EXPECT_CALL(mockMetaServerService_, GetDentry(_, _, _, _)) + .WillOnce( + DoAll(SetArgPointee<2>(response), + Invoke(SetRpcService))); + + EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) + .WillOnce(DoAll(SetArgPointee<2>(target_), Return(true))); + + status = metaserverCli_.GetDentry(fsID, inodeID, name, &out, &txLockOut); + ASSERT_EQ(MetaStatusCode::TX_KEY_LOCKED, status); + ASSERT_TRUE(google::protobuf::util::MessageDifferencer::Equals(txLockOut, + txLock)); + // test5: test response do not have dentry + response.set_statuscode(MetaStatusCode::OK); + response.clear_dentry(); EXPECT_CALL(mockMetaServerService_, GetDentry(_, _, _, _)) .WillRepeatedly( DoAll(SetArgPointee<2>(response), @@ -206,7 +236,7 @@ TEST_F(MetaServerClientImplTest, test_GetDentry) { EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); - status = metaserverCli_.GetDentry(fsID, inodeID, name, &out); + status = metaserverCli_.GetDentry(fsID, inodeID, name, &out, &txLockOut); ASSERT_EQ(MetaStatusCode::RPC_ERROR, status); } @@ -220,6 +250,7 @@ TEST_F(MetaServerClientImplTest, test_ListDentry) { // out std::list out; uint64_t txID = 10; + TxLock txLockOut; curvefs::metaserver::ListDentryResponse response; auto *d = response.add_dentrys(); @@ -235,12 +266,34 @@ TEST_F(MetaServerClientImplTest, test_ListDentry) { Invoke(SetRpcService)); EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); + EXPECT_CALL(*mockMetacache_.get(), GetTargetLeader(_, _)) + .WillRepeatedly(Return(true)); - MetaStatusCode status = - metaserverCli_.ListDentry(fsID, inodeID, last, count, onlyDir, &out); + MetaStatusCode status = metaserverCli_.ListDentry( + fsID, inodeID, last, count, onlyDir, &out, &txLockOut); ASSERT_EQ(MetaStatusCode::RPC_ERROR, status); + // test: dentry has tx lock + TxLock txLock; + txLock.set_primarykey("key"); + txLock.set_startts(1); + txLock.set_timestamp(100); + response.set_statuscode(MetaStatusCode::TX_KEY_LOCKED); + *response.mutable_txlock() = txLock; + EXPECT_CALL(mockMetaServerService_, ListDentry(_, _, _, _)) + .WillOnce(DoAll( + SetArgPointee<2>(response), + Invoke(SetRpcService))); + EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) + .WillOnce(DoAll(SetArgPointee<2>(target_), Return(true))); + + status = metaserverCli_.ListDentry( + fsID, inodeID, last, count, onlyDir, &out, &txLockOut); + ASSERT_EQ(MetaStatusCode::TX_KEY_LOCKED, status); + ASSERT_TRUE( + google::protobuf::util::MessageDifferencer::Equals(txLockOut, txLock)); + // test1: list dentry ok response.set_statuscode(MetaStatusCode::OK); EXPECT_CALL(mockMetaServerService_, ListDentry(_, _, _, _)) @@ -251,7 +304,7 @@ TEST_F(MetaServerClientImplTest, test_ListDentry) { .WillOnce(DoAll(SetArgPointee<2>(target_), Return(true))); status = metaserverCli_.ListDentry( - fsID, inodeID, last, count, onlyDir, &out); + fsID, inodeID, last, count, onlyDir, &out, &txLockOut); ASSERT_EQ(MetaStatusCode::OK, status); ASSERT_EQ(1, out.size()); ASSERT_TRUE( @@ -274,7 +327,7 @@ TEST_F(MetaServerClientImplTest, test_ListDentry) { SetArgPointee<2>(response), Invoke(SetRpcService))); status = metaserverCli_.ListDentry( - fsID, inodeID, last, count, onlyDir, &out); + fsID, inodeID, last, count, onlyDir, &out, &txLockOut); ASSERT_EQ(MetaStatusCode::OK, status); // test3: test response do not have dentrys @@ -288,7 +341,7 @@ TEST_F(MetaServerClientImplTest, test_ListDentry) { .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); status = metaserverCli_.ListDentry( - fsID, inodeID, last, count, onlyDir, &out); + fsID, inodeID, last, count, onlyDir, &out, &txLockOut); ASSERT_EQ(MetaStatusCode::OK, status); } @@ -302,8 +355,7 @@ TEST_F(MetaServerClientImplTest, test_CreateDentry_rpc_error) { d.set_txid(10); // out - butil::EndPoint target; - butil::str2endpoint(addr_.c_str(), &target); + TxLock txLockOut; curvefs::metaserver::CreateDentryResponse response; @@ -316,11 +368,11 @@ TEST_F(MetaServerClientImplTest, test_CreateDentry_rpc_error) { .Times(1 + opt_.maxRetry) .WillRepeatedly(Return(true)); - MetaStatusCode status = metaserverCli_.CreateDentry(d); + MetaStatusCode status = metaserverCli_.CreateDentry(d, &txLockOut); ASSERT_EQ(MetaStatusCode::RPC_ERROR, status); } -TEST_F(MetaServerClientImplTest, test_CreateDentry_create_dentry_ok) { +TEST_F(MetaServerClientImplTest, test_CreateDentry_ok) { // in Dentry d; d.set_fsid(1); @@ -330,8 +382,7 @@ TEST_F(MetaServerClientImplTest, test_CreateDentry_create_dentry_ok) { d.set_txid(10); // out - butil::EndPoint target; - butil::str2endpoint(addr_.c_str(), &target); + TxLock txLockOut; curvefs::metaserver::CreateDentryResponse response; @@ -343,10 +394,44 @@ TEST_F(MetaServerClientImplTest, test_CreateDentry_create_dentry_ok) { EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) .WillOnce(DoAll(SetArgPointee<2>(target_), Return(true))); - auto status = metaserverCli_.CreateDentry(d); + auto status = metaserverCli_.CreateDentry(d, &txLockOut); ASSERT_EQ(MetaStatusCode::OK, status); } +TEST_F(MetaServerClientImplTest, test_CreateDentry_tx_locked) { + // in + Dentry d; + d.set_fsid(1); + d.set_inodeid(2); + d.set_parentinodeid(1); + d.set_name("test11"); + d.set_txid(10); + + // out + TxLock txLockOut; + + curvefs::metaserver::CreateDentryResponse response; + + TxLock txLock; + txLock.set_primarykey("key"); + txLock.set_startts(1); + txLock.set_timestamp(100); + response.set_statuscode(MetaStatusCode::TX_KEY_LOCKED); + *response.mutable_txlock() = txLock; + + EXPECT_CALL(mockMetaServerService_, CreateDentry(_, _, _, _)) + .WillOnce(DoAll( + SetArgPointee<2>(response), + Invoke(SetRpcService))); + EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) + .WillOnce(DoAll(SetArgPointee<2>(target_), Return(true))); + + auto status = metaserverCli_.CreateDentry(d, &txLockOut); + ASSERT_EQ(MetaStatusCode::TX_KEY_LOCKED, status); + ASSERT_TRUE( + google::protobuf::util::MessageDifferencer::Equals(txLockOut, txLock)); +} + TEST_F(MetaServerClientImplTest, test_CreateDentry_copyset_not_exist) { // in Dentry d; @@ -357,8 +442,7 @@ TEST_F(MetaServerClientImplTest, test_CreateDentry_copyset_not_exist) { d.set_txid(10); // out - butil::EndPoint target; - butil::str2endpoint(addr_.c_str(), &target); + TxLock txLockOut; curvefs::metaserver::CreateDentryResponse response; @@ -380,7 +464,7 @@ TEST_F(MetaServerClientImplTest, test_CreateDentry_copyset_not_exist) { EXPECT_CALL(*mockMetacache_.get(), GetTargetLeader(_, _)) .WillOnce(Return(true)); - auto status = metaserverCli_.CreateDentry(d); + auto status = metaserverCli_.CreateDentry(d, &txLockOut); ASSERT_EQ(MetaStatusCode::OK, status); } @@ -390,10 +474,7 @@ TEST_F(MetaServerClientImplTest, test_DeleteDentry) { uint64_t inodeid = 2; std::string name = "test"; - // out - butil::EndPoint target; - butil::str2endpoint(addr_.c_str(), &target); - + TxLock txLockOut; curvefs::metaserver::DeleteDentryResponse response; // test1: delete dentry ok @@ -405,9 +486,11 @@ TEST_F(MetaServerClientImplTest, test_DeleteDentry) { Invoke(SetRpcService))); EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) .WillOnce(DoAll(SetArgPointee<2>(target_), Return(true))); + EXPECT_CALL(*mockMetacache_.get(), GetTargetLeader(_, _)) + .WillRepeatedly(Return(true)); MetaStatusCode status = metaserverCli_.DeleteDentry( - fsid, inodeid, name, FsFileType::TYPE_FILE); + fsid, inodeid, name, FsFileType::TYPE_FILE, &txLockOut); ASSERT_EQ(MetaStatusCode::OK, status); // test2: rpc error @@ -418,7 +501,7 @@ TEST_F(MetaServerClientImplTest, test_DeleteDentry) { .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); status = metaserverCli_.DeleteDentry( - fsid, inodeid, name, FsFileType::TYPE_FILE); + fsid, inodeid, name, FsFileType::TYPE_FILE, &txLockOut); ASSERT_EQ(MetaStatusCode::RPC_ERROR, status); // test3: delete response with unknown error @@ -430,8 +513,29 @@ TEST_F(MetaServerClientImplTest, test_DeleteDentry) { EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); status = metaserverCli_.DeleteDentry( - fsid, inodeid, name, FsFileType::TYPE_FILE); + fsid, inodeid, name, FsFileType::TYPE_FILE, &txLockOut); ASSERT_EQ(MetaStatusCode::UNKNOWN_ERROR, status); + + // test: delete dentry with tx lock + TxLock txLock; + txLock.set_primarykey("key"); + txLock.set_startts(1); + txLock.set_timestamp(100); + response.set_statuscode(MetaStatusCode::TX_KEY_LOCKED); + *response.mutable_txlock() = txLock; + + EXPECT_CALL(mockMetaServerService_, DeleteDentry(_, _, _, _)) + .WillOnce(DoAll( + SetArgPointee<2>(response), + Invoke(SetRpcService))); + EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) + .WillOnce(DoAll(SetArgPointee<2>(target_), Return(true))); + + status = metaserverCli_.DeleteDentry( + fsid, inodeid, name, FsFileType::TYPE_FILE, &txLockOut); + ASSERT_EQ(MetaStatusCode::TX_KEY_LOCKED, status); + ASSERT_TRUE( + google::protobuf::util::MessageDifferencer::Equals(txLockOut, txLock)); } TEST_F(MetaServerClientImplTest, PrepareRenameTx) { @@ -443,6 +547,8 @@ TEST_F(MetaServerClientImplTest, PrepareRenameTx) { dentry.set_name("A"); dentry.set_txid(4); + EXPECT_CALL(*mockMetacache_.get(), GetTargetLeader(_, _)) + .WillRepeatedly(Return(true)); EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); @@ -478,6 +584,190 @@ TEST_F(MetaServerClientImplTest, PrepareRenameTx) { ASSERT_EQ(rc, MetaStatusCode::RPC_ERROR); } +TEST_F(MetaServerClientImplTest, PrewriteRenameTx) { + curvefs::metaserver::PrewriteRenameTxResponse response; + Dentry dentry; + dentry.set_fsid(1); + dentry.set_inodeid(2); + dentry.set_parentinodeid(3); + dentry.set_name("A"); + TxLock txLockIn; + TxLock txLockOut; + txLockIn.set_primarykey("key"); + txLockIn.set_startts(1); + txLockIn.set_timestamp(100); + + EXPECT_CALL(*mockMetacache_.get(), GetTargetLeader(_, _)) + .WillRepeatedly(Return(true)); + EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) + .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); + + // CASE 1: PrewriteRenameTx success + response.set_statuscode(MetaStatusCode::OK); + EXPECT_CALL(mockMetaServerService_, PrewriteRenameTx(_, _, _, _)) + .WillOnce(DoAll(SetArgPointee<2>(response), + Invoke(SetRpcService))); + + auto dentrys = std::vector{dentry}; + auto rc = metaserverCli_.PrewriteRenameTx(dentrys, txLockIn, &txLockOut); + ASSERT_EQ(rc, MetaStatusCode::OK); + + // CASE 2: PrewriteRenameTx fail + response.set_statuscode(MetaStatusCode::STORAGE_INTERNAL_ERROR); + EXPECT_CALL(mockMetaServerService_, PrewriteRenameTx(_, _, _, _)) + .WillOnce(DoAll(SetArgPointee<2>(response), + Invoke(SetRpcService))); + + dentrys = std::vector{dentry}; + rc = metaserverCli_.PrewriteRenameTx(dentrys, txLockIn, &txLockOut); + ASSERT_EQ(rc, MetaStatusCode::STORAGE_INTERNAL_ERROR); + + // CASE 3: RPC error + EXPECT_CALL(mockMetaServerService_, PrewriteRenameTx(_, _, _, _)) + .WillRepeatedly(Invoke(SetRpcService)); + + dentrys = std::vector{dentry}; + rc = metaserverCli_.PrewriteRenameTx(dentrys, txLockIn, &txLockOut); + ASSERT_EQ(rc, MetaStatusCode::RPC_ERROR); +} + +TEST_F(MetaServerClientImplTest, CheckTxStatus) { + curvefs::metaserver::CheckTxStatusResponse response; + uint32_t poolId = 1; + uint64_t inodeId = 1; + std::string primaryKey = "key"; + uint64_t startTs = 1; + uint64_t curTimestamp = 100; + + EXPECT_CALL(*mockMetacache_.get(), GetTargetLeader(_, _)) + .WillRepeatedly(Return(true)); + EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) + .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); + + // CASE 1: CheckTxStatus success + response.set_statuscode(MetaStatusCode::OK); + EXPECT_CALL(mockMetaServerService_, CheckTxStatus(_, _, _, _)) + .WillOnce(DoAll(SetArgPointee<2>(response), + Invoke(SetRpcService))); + + auto rc = metaserverCli_.CheckTxStatus(poolId, inodeId, primaryKey, + startTs, curTimestamp); + ASSERT_EQ(rc, MetaStatusCode::OK); + + // CASE 2: CheckTxStatus fail + response.set_statuscode(MetaStatusCode::STORAGE_INTERNAL_ERROR); + EXPECT_CALL(mockMetaServerService_, CheckTxStatus(_, _, _, _)) + .WillOnce(DoAll(SetArgPointee<2>(response), + Invoke(SetRpcService))); + + rc = metaserverCli_.CheckTxStatus(poolId, inodeId, primaryKey, startTs, + curTimestamp); + ASSERT_EQ(rc, MetaStatusCode::STORAGE_INTERNAL_ERROR); + + // CASE 3: RPC error + EXPECT_CALL(mockMetaServerService_, CheckTxStatus(_, _, _, _)) + .WillRepeatedly(Invoke(SetRpcService)); + + rc = metaserverCli_.CheckTxStatus(poolId, inodeId, primaryKey, startTs, + curTimestamp); + ASSERT_EQ(rc, MetaStatusCode::RPC_ERROR); +} + +TEST_F(MetaServerClientImplTest, ResolveTxLock) { + curvefs::metaserver::ResolveTxLockResponse response; + Dentry dentry; + dentry.set_fsid(1); + dentry.set_inodeid(2); + dentry.set_parentinodeid(3); + dentry.set_name("A"); + uint64_t startTs = 1; + + EXPECT_CALL(*mockMetacache_.get(), GetTargetLeader(_, _)) + .WillRepeatedly(Return(true)); + EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) + .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); + + // CASE 1: ResolveTxLock success + response.set_statuscode(MetaStatusCode::OK); + EXPECT_CALL(mockMetaServerService_, ResolveTxLock(_, _, _, _)) + .WillOnce(DoAll(SetArgPointee<2>(response), + Invoke(SetRpcService))); + + auto rc = metaserverCli_.ResolveTxLock(dentry, startTs, true); + ASSERT_EQ(rc, MetaStatusCode::OK); + + // CASE 2: ResolveTxLock fail + response.set_statuscode(MetaStatusCode::STORAGE_INTERNAL_ERROR); + EXPECT_CALL(mockMetaServerService_, ResolveTxLock(_, _, _, _)) + .WillOnce(DoAll(SetArgPointee<2>(response), + Invoke(SetRpcService))); + + rc = metaserverCli_.ResolveTxLock(dentry, startTs, true); + ASSERT_EQ(rc, MetaStatusCode::STORAGE_INTERNAL_ERROR); + + // CASE 3: RPC error + EXPECT_CALL(mockMetaServerService_, ResolveTxLock(_, _, _, _)) + .WillRepeatedly(Invoke(SetRpcService)); + + rc = metaserverCli_.ResolveTxLock(dentry, startTs, true); + ASSERT_EQ(rc, MetaStatusCode::RPC_ERROR); +} + +TEST_F(MetaServerClientImplTest, CommitTx) { + curvefs::metaserver::CommitTxResponse response; + Dentry dentry; + dentry.set_fsid(1); + dentry.set_inodeid(2); + dentry.set_parentinodeid(3); + dentry.set_name("A"); + uint64_t startTs = 1; + uint64_t commitTs = 2; + + EXPECT_CALL(*mockMetacache_.get(), GetTargetLeader(_, _)) + .WillRepeatedly(Return(true)); + EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) + .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); + + // CASE 1: CommitTx success + response.set_statuscode(MetaStatusCode::OK); + EXPECT_CALL(mockMetaServerService_, CommitTx(_, _, _, _)) + .WillOnce(DoAll(SetArgPointee<2>(response), + Invoke(SetRpcService))); + + auto dentrys = std::vector{dentry}; + auto rc = metaserverCli_.CommitTx(dentrys, startTs, commitTs); + ASSERT_EQ(rc, MetaStatusCode::OK); + + // CASE 2: CommitTx fail + response.set_statuscode(MetaStatusCode::STORAGE_INTERNAL_ERROR); + EXPECT_CALL(mockMetaServerService_, CommitTx(_, _, _, _)) + .WillOnce(DoAll(SetArgPointee<2>(response), + Invoke(SetRpcService))); + + rc = metaserverCli_.CommitTx(dentrys, startTs, commitTs); + ASSERT_EQ(rc, MetaStatusCode::STORAGE_INTERNAL_ERROR); + + // CASE 3: RPC error + EXPECT_CALL(mockMetaServerService_, CommitTx(_, _, _, _)) + .WillRepeatedly(Invoke(SetRpcService< + curvefs::metaserver::CommitTxRequest, + curvefs::metaserver::CommitTxResponse, true>)); + + rc = metaserverCli_.CommitTx(dentrys, startTs, commitTs); + ASSERT_EQ(rc, MetaStatusCode::RPC_ERROR); +} + TEST_F(MetaServerClientImplTest, test_GetInode) { // in uint32_t fsid = 1; @@ -514,6 +804,8 @@ TEST_F(MetaServerClientImplTest, test_GetInode) { Invoke(SetRpcService)); EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); + EXPECT_CALL(*mockMetacache_.get(), GetTargetLeader(_, _)) + .WillRepeatedly(Return(true)); MetaStatusCode status = metaserverCli_.GetInode( fsid, inodeid, &out, &streaming); @@ -587,6 +879,8 @@ TEST_F(MetaServerClientImplTest, test_UpdateInodeAttr) { SetRpcService)); EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); + EXPECT_CALL(*mockMetacache_.get(), GetTargetLeader(_, _)) + .WillRepeatedly(Return(true)); MetaStatusCode status = metaserverCli_.UpdateInodeAttr( inode.fsid(), inode.inodeid(), ToInodeAttr(inode)); @@ -786,6 +1080,8 @@ TEST_F(MetaServerClientImplTest, test_CreateInode) { SetRpcService)); EXPECT_CALL(*mockMetacache_.get(), SelectTarget(_, _)) .WillRepeatedly(Return(true)); + EXPECT_CALL(*mockMetacache_.get(), GetTargetLeader(_, _)) + .WillRepeatedly(Return(true)); MetaStatusCode status = metaserverCli_.CreateInode(inode, &out); ASSERT_EQ(MetaStatusCode::RPC_ERROR, status); @@ -844,6 +1140,8 @@ TEST_F(MetaServerClientImplTest, test_DeleteInode) { SetRpcService)); EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); + EXPECT_CALL(*mockMetacache_.get(), GetTargetLeader(_, _)) + .WillRepeatedly(Return(true)); MetaStatusCode status = metaserverCli_.DeleteInode(fsId, inodeid); ASSERT_EQ(MetaStatusCode::RPC_ERROR, status); @@ -917,6 +1215,8 @@ TEST_F(MetaServerClientImplTest, test_BatchGetInodeAttr) { BatchGetInodeAttrResponse, true>)); EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); + EXPECT_CALL(*mockMetacache_.get(), GetTargetLeader(_, _)) + .WillRepeatedly(Return(true)); MetaStatusCode status = metaserverCli_.BatchGetInodeAttr( fsid, inodeIds, &attr); @@ -993,6 +1293,8 @@ TEST_F(MetaServerClientImplTest, test_BatchGetXAttr) { BatchGetXAttrResponse, true>)); EXPECT_CALL(*mockMetacache_.get(), GetTarget(_, _, _, _)) .WillRepeatedly(DoAll(SetArgPointee<2>(target_), Return(true))); + EXPECT_CALL(*mockMetacache_.get(), GetTargetLeader(_, _)) + .WillRepeatedly(Return(true)); MetaStatusCode status = metaserverCli_.BatchGetXAttr( fsid, inodeIds, &xattr); diff --git a/curvefs/test/client/rpcclient/mock_mds_base_client.h b/curvefs/test/client/rpcclient/mock_mds_base_client.h index 6f06d72dd0..6aa7b99e88 100644 --- a/curvefs/test/client/rpcclient/mock_mds_base_client.h +++ b/curvefs/test/client/rpcclient/mock_mds_base_client.h @@ -123,6 +123,9 @@ class MockMDSBaseClient : public MDSBaseClient { void(uint32_t fsId, AllocOrGetMemcacheClusterResponse* response, brpc::Controller* cntl, brpc::Channel* channel)); + + MOCK_METHOD(void, Tso, (const TsoRequest& request, TsoResponse* response, + brpc::Controller* cntl, brpc::Channel* channel), (override)); }; } // namespace rpcclient } // namespace client diff --git a/curvefs/test/client/rpcclient/mock_mds_client.h b/curvefs/test/client/rpcclient/mock_mds_client.h index 1f066896fd..c77a2296cc 100644 --- a/curvefs/test/client/rpcclient/mock_mds_client.h +++ b/curvefs/test/client/rpcclient/mock_mds_client.h @@ -138,6 +138,9 @@ class MockMdsClient : public MdsClient { MOCK_METHOD2(AllocOrGetMemcacheCluster, bool(uint32_t, curvefs::mds::topology::MemcacheClusterInfo*)); + + MOCK_METHOD(FSStatusCode, Tso, (uint64_t*, uint64_t*), + (override)); }; } // namespace rpcclient } // namespace client diff --git a/curvefs/test/client/rpcclient/mock_metaserver_service.h b/curvefs/test/client/rpcclient/mock_metaserver_service.h index f40dd6b15f..961ce9229f 100644 --- a/curvefs/test/client/rpcclient/mock_metaserver_service.h +++ b/curvefs/test/client/rpcclient/mock_metaserver_service.h @@ -66,6 +66,30 @@ class MockMetaServerService : public curvefs::metaserver::MetaServerService { ::curvefs::metaserver::PrepareRenameTxResponse* response, ::google::protobuf::Closure* done)); + MOCK_METHOD(void, PrewriteRenameTx, + (::google::protobuf::RpcController* controller, + const ::curvefs::metaserver::PrewriteRenameTxRequest* request, + ::curvefs::metaserver::PrewriteRenameTxResponse* response, + ::google::protobuf::Closure* done), (override)); + + MOCK_METHOD(void, CheckTxStatus, + (::google::protobuf::RpcController* controller, + const ::curvefs::metaserver::CheckTxStatusRequest* request, + ::curvefs::metaserver::CheckTxStatusResponse* response, + ::google::protobuf::Closure* done), (override)); + + MOCK_METHOD(void, ResolveTxLock, + (::google::protobuf::RpcController* controller, + const ::curvefs::metaserver::ResolveTxLockRequest* request, + ::curvefs::metaserver::ResolveTxLockResponse* response, + ::google::protobuf::Closure* done), (override)); + + MOCK_METHOD(void, CommitTx, + (::google::protobuf::RpcController* controller, + const ::curvefs::metaserver::CommitTxRequest* request, + ::curvefs::metaserver::CommitTxResponse* response, + ::google::protobuf::Closure* done), (override)); + MOCK_METHOD4(GetInode, void(::google::protobuf::RpcController *controller, const ::curvefs::metaserver::GetInodeRequest *request, diff --git a/curvefs/test/client/test_dentry_cache_manager.cpp b/curvefs/test/client/test_dentry_cache_manager.cpp index d67feff8bc..2f80e82b00 100644 --- a/curvefs/test/client/test_dentry_cache_manager.cpp +++ b/curvefs/test/client/test_dentry_cache_manager.cpp @@ -25,8 +25,9 @@ #include #include +#include "curvefs/test/client/rpcclient/mock_mds_client.h" #include "curvefs/test/client/mock_metaserver_client.h" -#include "curvefs/src/client/dentry_cache_manager.h" +#include "curvefs/src/client/dentry_manager.h" namespace curvefs { namespace client { @@ -47,6 +48,7 @@ using ::testing::DoAll; using ::testing::Invoke; using rpcclient::MockMetaServerClient; +using rpcclient::MockMdsClient; class TestDentryCacheManager : public ::testing::Test { protected: @@ -54,8 +56,10 @@ class TestDentryCacheManager : public ::testing::Test { ~TestDentryCacheManager() {} virtual void SetUp() { + mdsClient_ = std::make_shared(); metaClient_ = std::make_shared(); dCacheManager_ = std::make_shared(metaClient_); + dCacheManager_->Init(mdsClient_); dCacheManager_->SetFsId(fsId_); } @@ -67,10 +71,82 @@ class TestDentryCacheManager : public ::testing::Test { protected: std::shared_ptr dCacheManager_; std::shared_ptr metaClient_; + std::shared_ptr mdsClient_; uint32_t fsId_ = 888; uint32_t timeout_ = 3; }; +TEST_F(TestDentryCacheManager, CheckAndResolveTx) { + // In + std::string primaryKey = "3:1:1:A"; + std::string fakePrimaryKey = "ABC"; + uint64_t startTs = 1; + uint64_t commitTs = 2; + uint64_t curTimestamp = 100; + Dentry dentry; + TxLock txLock; + txLock.set_startts(startTs); + // 1. check tx status failed + // case: check tx status parse primary key failed + txLock.set_primarykey(fakePrimaryKey); + ASSERT_EQ(MetaStatusCode::PARSE_FROM_STRING_FAILED, + dCacheManager_->CheckAndResolveTx( + dentry, txLock, curTimestamp, commitTs)); + // case: check failed + txLock.set_primarykey(primaryKey); + EXPECT_CALL(*metaClient_, CheckTxStatus(_, _, _, _, _)) + .WillOnce(Return(MetaStatusCode::TX_MISMATCH)); + ASSERT_EQ(MetaStatusCode::TX_MISMATCH, + dCacheManager_->CheckAndResolveTx( + dentry, txLock, curTimestamp, commitTs)); + + // 2. check tx status success and resolve tx failed + EXPECT_CALL(*metaClient_, CheckTxStatus(_, _, _, _, _)) + .WillOnce(Return(MetaStatusCode::TX_COMMITTED)); + EXPECT_CALL(*metaClient_, ResolveTxLock(_, startTs, commitTs)) + .WillOnce(Return(MetaStatusCode::STORAGE_INTERNAL_ERROR)); + ASSERT_EQ(MetaStatusCode::STORAGE_INTERNAL_ERROR, + dCacheManager_->CheckAndResolveTx( + dentry, txLock, curTimestamp, commitTs)); + + EXPECT_CALL(*metaClient_, CheckTxStatus(_, _, _, _, _)) + .WillOnce(Return(MetaStatusCode::TX_ROLLBACKED)); + EXPECT_CALL(*metaClient_, ResolveTxLock(_, startTs, 0)) + .WillOnce(Return(MetaStatusCode::STORAGE_INTERNAL_ERROR)); + ASSERT_EQ(MetaStatusCode::STORAGE_INTERNAL_ERROR, + dCacheManager_->CheckAndResolveTx( + dentry, txLock, curTimestamp, commitTs)); + + EXPECT_CALL(*metaClient_, CheckTxStatus(_, _, _, _, _)) + .WillOnce(Return(MetaStatusCode::TX_TIMEOUT)); + EXPECT_CALL(*metaClient_, ResolveTxLock(_, startTs, 0)) + .WillOnce(Return(MetaStatusCode::STORAGE_INTERNAL_ERROR)); + ASSERT_EQ(MetaStatusCode::STORAGE_INTERNAL_ERROR, + dCacheManager_->CheckAndResolveTx( + dentry, txLock, curTimestamp, commitTs)); + + EXPECT_CALL(*metaClient_, CheckTxStatus(_, _, _, _, _)) + .WillOnce(Return(MetaStatusCode::TX_INPROGRESS)); + ASSERT_EQ(MetaStatusCode::TX_INPROGRESS, + dCacheManager_->CheckAndResolveTx( + dentry, txLock, curTimestamp, commitTs)); + + EXPECT_CALL(*metaClient_, CheckTxStatus(_, _, _, _, _)) + .WillOnce(Return(MetaStatusCode::TX_FAILED)); + ASSERT_EQ(MetaStatusCode::TX_FAILED, + dCacheManager_->CheckAndResolveTx( + dentry, txLock, curTimestamp, commitTs)); + + // 3. check tx status success and resolve tx success + EXPECT_CALL(*metaClient_, CheckTxStatus(_, _, _, _, _)) + .WillOnce(Return(MetaStatusCode::TX_COMMITTED)); + EXPECT_CALL(*metaClient_, ResolveTxLock(_, startTs, commitTs)) + .WillOnce(Return(MetaStatusCode::OK)); + ASSERT_EQ(MetaStatusCode::OK, + dCacheManager_->CheckAndResolveTx( + dentry, txLock, curTimestamp, commitTs)); +} + TEST_F(TestDentryCacheManager, GetDentry) { curvefs::client::common::FLAGS_enableCto = false; uint64_t parent = 99; @@ -84,7 +160,7 @@ TEST_F(TestDentryCacheManager, GetDentry) { dentryExp.set_parentinodeid(parent); dentryExp.set_inodeid(inodeid); - EXPECT_CALL(*metaClient_, GetDentry(fsId_, parent, name, _)) + EXPECT_CALL(*metaClient_, GetDentry(fsId_, parent, name, _, _)) .WillOnce(Return(MetaStatusCode::NOT_FOUND)) .WillOnce(DoAll(SetArgPointee<3>(dentryExp), Return(MetaStatusCode::OK))) @@ -106,16 +182,51 @@ TEST_F(TestDentryCacheManager, GetDentry) { curvefs::client::common::FLAGS_enableCto = true; EXPECT_CALL(*metaClient_, DeleteDentry( - fsId_, parent, name, FsFileType::TYPE_FILE)) + fsId_, parent, name, FsFileType::TYPE_FILE, _)) .WillOnce(Return(MetaStatusCode::OK)); dCacheManager_->DeleteDentry(parent, name, FsFileType::TYPE_FILE); - EXPECT_CALL(*metaClient_, GetDentry(fsId_, parent, name, _)) + EXPECT_CALL(*metaClient_, GetDentry(fsId_, parent, name, _, _)) .WillOnce( DoAll(SetArgPointee<3>(dentryExp), Return(MetaStatusCode::OK))); ret = dCacheManager_->GetDentry(parent, name, &out); ASSERT_EQ(CURVEFS_ERROR::OK, ret); ASSERT_TRUE( google::protobuf::util::MessageDifferencer::Equals(dentryExp, out)); + + // get dentry but dentry tx key is locked + // 1. Tso failed + EXPECT_CALL(*metaClient_, GetDentry(fsId_, parent, name, _, _)) + .WillOnce(Return(MetaStatusCode::TX_KEY_LOCKED)); + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::UNKNOWN_ERROR)); + ret = dCacheManager_->GetDentry(parent, name, &out); + ASSERT_EQ(CURVEFS_ERROR::INTERNAL, ret); + // 2. CheckAndResolveTx failed + EXPECT_CALL(*metaClient_, GetDentry(fsId_, parent, name, _, _)) + .WillOnce(Return(MetaStatusCode::TX_KEY_LOCKED)); + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::OK)); + ret = dCacheManager_->GetDentry(parent, name, &out); + ASSERT_EQ(CURVEFS_ERROR::INTERNAL, ret); + // 3. success + TxLock txLock; + txLock.set_primarykey("3:1:1:A"); + txLock.set_startts(1); + EXPECT_CALL(*metaClient_, GetDentry(fsId_, parent, name, _, _)) + .WillOnce(DoAll(SetArgPointee<4>(txLock), + Return(MetaStatusCode::TX_KEY_LOCKED))) + .WillOnce(DoAll(SetArgPointee<3>(dentryExp), + Return(MetaStatusCode::OK))); + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::OK)); + EXPECT_CALL(*metaClient_, CheckTxStatus(_, _, _, _, _)) + .WillOnce(Return(MetaStatusCode::TX_COMMITTED)); + EXPECT_CALL(*metaClient_, ResolveTxLock(_, _, _)) + .WillOnce(Return(MetaStatusCode::OK)); + ret = dCacheManager_->GetDentry(parent, name, &out); + ASSERT_EQ(CURVEFS_ERROR::OK, ret); + ASSERT_TRUE( + google::protobuf::util::MessageDifferencer::Equals(dentryExp, out)); } TEST_F(TestDentryCacheManager, CreateAndGetDentry) { @@ -131,11 +242,11 @@ TEST_F(TestDentryCacheManager, CreateAndGetDentry) { dentryExp.set_parentinodeid(parent); dentryExp.set_inodeid(inodeid); - EXPECT_CALL(*metaClient_, CreateDentry(_)) + EXPECT_CALL(*metaClient_, CreateDentry(_, _)) .WillOnce(Return(MetaStatusCode::UNKNOWN_ERROR)) .WillOnce(Return(MetaStatusCode::OK)); - EXPECT_CALL(*metaClient_, GetDentry(fsId_, parent, name, _)) + EXPECT_CALL(*metaClient_, GetDentry(fsId_, parent, name, _, _)) .WillOnce(DoAll(SetArgPointee<3>(dentryExp), Return(MetaStatusCode::OK))); @@ -152,12 +263,12 @@ TEST_F(TestDentryCacheManager, CreateAndGetDentry) { curvefs::client::common::FLAGS_enableCto = true; EXPECT_CALL(*metaClient_, DeleteDentry( - fsId_, parent, name, FsFileType::TYPE_FILE)) + fsId_, parent, name, FsFileType::TYPE_FILE, _)) .WillOnce(Return(MetaStatusCode::OK)); dCacheManager_->DeleteDentry(parent, name, FsFileType::TYPE_FILE); - EXPECT_CALL(*metaClient_, CreateDentry(_)) + EXPECT_CALL(*metaClient_, CreateDentry(_, _)) .WillOnce(Return(MetaStatusCode::OK)); - EXPECT_CALL(*metaClient_, GetDentry(fsId_, parent, name, _)) + EXPECT_CALL(*metaClient_, GetDentry(fsId_, parent, name, _, _)) .WillOnce( DoAll(SetArgPointee<3>(dentryExp), Return(MetaStatusCode::OK))); @@ -167,6 +278,38 @@ TEST_F(TestDentryCacheManager, CreateAndGetDentry) { ASSERT_EQ(CURVEFS_ERROR::OK, ret); ASSERT_TRUE( google::protobuf::util::MessageDifferencer::Equals(dentryExp, out)); + + // create dentry but dentry tx key is locked + // 1. Tso failed + EXPECT_CALL(*metaClient_, CreateDentry(_, _)) + .WillOnce(Return(MetaStatusCode::TX_KEY_LOCKED)); + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::UNKNOWN_ERROR)); + ret = dCacheManager_->CreateDentry(dentryExp); + ASSERT_EQ(CURVEFS_ERROR::INTERNAL, ret); + // 2. CheckAndResolveTx failed + EXPECT_CALL(*metaClient_, CreateDentry(_, _)) + .WillOnce(Return(MetaStatusCode::TX_KEY_LOCKED)); + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::OK)); + ret = dCacheManager_->CreateDentry(dentryExp); + ASSERT_EQ(CURVEFS_ERROR::INTERNAL, ret); + // 3. success + TxLock txLock; + txLock.set_primarykey("3:1:1:A"); + txLock.set_startts(1); + EXPECT_CALL(*metaClient_, CreateDentry(_, _)) + .WillOnce(DoAll(SetArgPointee<1>(txLock), + Return(MetaStatusCode::TX_KEY_LOCKED))) + .WillOnce(Return(MetaStatusCode::OK)); + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::OK)); + EXPECT_CALL(*metaClient_, CheckTxStatus(_, _, _, _, _)) + .WillOnce(Return(MetaStatusCode::TX_COMMITTED)); + EXPECT_CALL(*metaClient_, ResolveTxLock(_, _, _)) + .WillOnce(Return(MetaStatusCode::OK)); + ret = dCacheManager_->CreateDentry(dentryExp); + ASSERT_EQ(CURVEFS_ERROR::OK, ret); } TEST_F(TestDentryCacheManager, DeleteDentry) { @@ -174,7 +317,7 @@ TEST_F(TestDentryCacheManager, DeleteDentry) { const std::string name = "test"; EXPECT_CALL(*metaClient_, DeleteDentry( - fsId_, parent, name, FsFileType::TYPE_FILE)) + fsId_, parent, name, FsFileType::TYPE_FILE, _)) .WillOnce(Return(MetaStatusCode::NOT_FOUND)) .WillOnce(Return(MetaStatusCode::OK)); @@ -184,6 +327,41 @@ TEST_F(TestDentryCacheManager, DeleteDentry) { ret = dCacheManager_->DeleteDentry(parent, name, FsFileType::TYPE_FILE); ASSERT_EQ(CURVEFS_ERROR::OK, ret); + + // create dentry but dentry tx key is locked + // 1. Tso failed + EXPECT_CALL(*metaClient_, DeleteDentry( + fsId_, parent, name, FsFileType::TYPE_FILE, _)) + .WillOnce(Return(MetaStatusCode::TX_KEY_LOCKED)); + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::UNKNOWN_ERROR)); + ret = dCacheManager_->DeleteDentry(parent, name, FsFileType::TYPE_FILE); + ASSERT_EQ(CURVEFS_ERROR::INTERNAL, ret); + // 2. CheckAndResolveTx failed + EXPECT_CALL(*metaClient_, DeleteDentry( + fsId_, parent, name, FsFileType::TYPE_FILE, _)) + .WillOnce(Return(MetaStatusCode::TX_KEY_LOCKED)); + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::OK)); + ret = dCacheManager_->DeleteDentry(parent, name, FsFileType::TYPE_FILE); + ASSERT_EQ(CURVEFS_ERROR::INTERNAL, ret); + // 3. success + TxLock txLock; + txLock.set_primarykey("3:1:1:A"); + txLock.set_startts(1); + EXPECT_CALL(*metaClient_, DeleteDentry( + fsId_, parent, name, FsFileType::TYPE_FILE, _)) + .WillOnce(DoAll(SetArgPointee<4>(txLock), + Return(MetaStatusCode::TX_KEY_LOCKED))) + .WillOnce(Return(MetaStatusCode::OK)); + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::OK)); + EXPECT_CALL(*metaClient_, CheckTxStatus(_, _, _, _, _)) + .WillOnce(Return(MetaStatusCode::TX_COMMITTED)); + EXPECT_CALL(*metaClient_, ResolveTxLock(_, _, _)) + .WillOnce(Return(MetaStatusCode::OK)); + ret = dCacheManager_->DeleteDentry(parent, name, FsFileType::TYPE_FILE); + ASSERT_EQ(CURVEFS_ERROR::OK, ret); } TEST_F(TestDentryCacheManager, ListDentryNomal) { @@ -194,7 +372,7 @@ TEST_F(TestDentryCacheManager, ListDentryNomal) { part1.resize(limit); part2.resize(limit - 1); - EXPECT_CALL(*metaClient_, ListDentry(fsId_, parent, _, _, _, _)) + EXPECT_CALL(*metaClient_, ListDentry(fsId_, parent, _, _, _, _, _)) .WillOnce(DoAll(SetArgPointee<5>(part1), Return(MetaStatusCode::OK))) .WillOnce(DoAll(SetArgPointee<5>(part2), @@ -209,7 +387,7 @@ TEST_F(TestDentryCacheManager, ListDentryNomal) { TEST_F(TestDentryCacheManager, ListDentryEmpty) { uint64_t parent = 99; - EXPECT_CALL(*metaClient_, ListDentry(fsId_, parent, _, _, _, _)) + EXPECT_CALL(*metaClient_, ListDentry(fsId_, parent, _, _, _, _, _)) .WillOnce(Return(MetaStatusCode::OK)); std::list out; @@ -229,7 +407,7 @@ TEST_F(TestDentryCacheManager, ListDentryOnlyDir) { TEST_F(TestDentryCacheManager, ListDentryFailed) { uint64_t parent = 99; - EXPECT_CALL(*metaClient_, ListDentry(fsId_, parent, _, _, _, _)) + EXPECT_CALL(*metaClient_, ListDentry(fsId_, parent, _, _, _, _, _)) .WillOnce(Return(MetaStatusCode::UNKNOWN_ERROR)); std::list out; @@ -238,6 +416,57 @@ TEST_F(TestDentryCacheManager, ListDentryFailed) { ASSERT_EQ(0, out.size()); } +TEST_F(TestDentryCacheManager, ListDentry_txLocked) { + uint64_t parent = 99; + std::list out; + std::list part; + // 1. Tso failed + EXPECT_CALL(*metaClient_, ListDentry(fsId_, parent, _, _, _, _, _)) + .WillOnce(Return(MetaStatusCode::TX_KEY_LOCKED)); + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::UNKNOWN_ERROR)); + CURVEFS_ERROR ret = dCacheManager_->ListDentry(parent, &out, 100); + ASSERT_EQ(CURVEFS_ERROR::INTERNAL, ret); + // 2. tx key locked but part empty + EXPECT_CALL(*metaClient_, ListDentry(fsId_, parent, _, _, _, _, _)) + .WillOnce(Return(MetaStatusCode::TX_KEY_LOCKED)); + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::OK)); + ret = dCacheManager_->ListDentry(parent, &out, 100); + ASSERT_EQ(CURVEFS_ERROR::INTERNAL, ret); + // 3. CheckAndResolveTx failed + EXPECT_CALL(*metaClient_, ListDentry(fsId_, parent, _, _, _, _, _)) + .WillOnce(DoAll(SetArgPointee<5>(part), + Return(MetaStatusCode::TX_KEY_LOCKED))); + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::OK)); + Dentry dentry; + dentry.set_fsid(fsId_); + dentry.set_name("test"); + dentry.set_parentinodeid(parent); + dentry.set_inodeid(100); + part.emplace_back(dentry); + ret = dCacheManager_->ListDentry(parent, &out, 100); + ASSERT_EQ(CURVEFS_ERROR::INTERNAL, ret); + // 4. success + TxLock txLock; + txLock.set_primarykey("3:1:1:A"); + txLock.set_startts(1); + EXPECT_CALL(*metaClient_, ListDentry(fsId_, parent, _, _, _, _, _)) + .WillOnce(DoAll(SetArgPointee<5>(part), SetArgPointee<6>(txLock), + Return(MetaStatusCode::TX_KEY_LOCKED))) + .WillOnce(DoAll(SetArgPointee<5>(part), + Return(MetaStatusCode::OK))); + EXPECT_CALL(*mdsClient_, Tso(_, _)) + .WillOnce(Return(FSStatusCode::OK)); + EXPECT_CALL(*metaClient_, CheckTxStatus(_, _, _, _, _)) + .WillOnce(Return(MetaStatusCode::TX_COMMITTED)); + EXPECT_CALL(*metaClient_, ResolveTxLock(_, _, _)) + .WillOnce(Return(MetaStatusCode::OK)); + ret = dCacheManager_->ListDentry(parent, &out, 100); + ASSERT_EQ(CURVEFS_ERROR::OK, ret); +} + TEST_F(TestDentryCacheManager, GetTimeOutDentry) { curvefs::client::common::FLAGS_enableCto = false; uint64_t parent = 99; @@ -251,10 +480,10 @@ TEST_F(TestDentryCacheManager, GetTimeOutDentry) { dentryExp.set_parentinodeid(parent); dentryExp.set_inodeid(inodeid); - EXPECT_CALL(*metaClient_, CreateDentry(_)) + EXPECT_CALL(*metaClient_, CreateDentry(_, _)) .WillOnce(Return(MetaStatusCode::OK)); - EXPECT_CALL(*metaClient_, GetDentry(fsId_, parent, name, _)) + EXPECT_CALL(*metaClient_, GetDentry(fsId_, parent, name, _, _)) .WillOnce(DoAll(SetArgPointee<3>(dentryExp), Return(MetaStatusCode::OK))); @@ -269,7 +498,7 @@ TEST_F(TestDentryCacheManager, GetTimeOutDentry) { // get from metaserver when timeout sleep(timeout_); - EXPECT_CALL(*metaClient_, GetDentry(fsId_, parent, name, _)) + EXPECT_CALL(*metaClient_, GetDentry(fsId_, parent, name, _, _)) .WillOnce(Return(MetaStatusCode::OK)); ret = dCacheManager_->GetDentry(parent, name, &out); ASSERT_EQ(CURVEFS_ERROR::OK, ret); diff --git a/curvefs/test/client/test_fuse_s3_client.cpp b/curvefs/test/client/test_fuse_s3_client.cpp index 422c3c140f..c948fdf25e 100644 --- a/curvefs/test/client/test_fuse_s3_client.cpp +++ b/curvefs/test/client/test_fuse_s3_client.cpp @@ -41,9 +41,9 @@ #include "curvefs/src/common/define.h" #include "curvefs/test/client/mock_client_s3.h" #include "curvefs/test/client/mock_client_s3_adaptor.h" -#include "curvefs/test/client/mock_dentry_cache_mamager.h" +#include "curvefs/test/client/mock_dentry_mamager.h" #include "curvefs/test/client/mock_disk_cache_manager.h" -#include "curvefs/test/client/mock_inode_cache_manager.h" +#include "curvefs/test/client/mock_inode_manager.h" #include "curvefs/test/client/mock_metaserver_client.h" #include "curvefs/test/client/rpcclient/mock_mds_client.h" #include "fuse3/fuse_lowlevel.h" diff --git a/curvefs/test/client/test_fuse_volume_client.cpp b/curvefs/test/client/test_fuse_volume_client.cpp index a2601dcbf3..4aafa17fee 100644 --- a/curvefs/test/client/test_fuse_volume_client.cpp +++ b/curvefs/test/client/test_fuse_volume_client.cpp @@ -27,8 +27,8 @@ #include "curvefs/proto/metaserver.pb.h" #include "curvefs/src/client/fuse_volume_client.h" #include "curvefs/src/common/define.h" -#include "curvefs/test/client/mock_dentry_cache_mamager.h" -#include "curvefs/test/client/mock_inode_cache_manager.h" +#include "curvefs/test/client/mock_dentry_mamager.h" +#include "curvefs/test/client/mock_inode_manager.h" #include "curvefs/test/client/rpcclient/mock_mds_client.h" #include "curvefs/test/client/mock_metaserver_client.h" #include "curvefs/test/client/mock_volume_storage.h" @@ -602,6 +602,11 @@ TEST_F(TestFuseVolumeClient, FuseOpRmDir) { EXPECT_CALL(*dentryManager_, GetDentry(parent, name, _)) .WillOnce(DoAll(SetArgPointee<2>(dentry), Return(CURVEFS_ERROR::OK))); + std::list dentryList; + EXPECT_CALL(*dentryManager_, ListDentry(inodeid, _, _, _, _)) + .WillOnce(DoAll(SetArgPointee<1>(dentryList), + Return(CURVEFS_ERROR::OK))); + EXPECT_CALL(*dentryManager_, DeleteDentry(parent, name, FsFileType::TYPE_DIRECTORY)) .WillOnce(Return(CURVEFS_ERROR::OK)); @@ -705,12 +710,15 @@ TEST_F(TestFuseVolumeClient, FuseOpUnlinkFailed) { EXPECT_CALL(*metaClient_, UpdateInodeAttr(_, _, _)) .WillOnce(Return(MetaStatusCode::UNKNOWN_ERROR)); + // get dentry internal failed CURVEFS_ERROR ret = client_->FuseOpUnlink(req, parent, name.c_str()); ASSERT_EQ(CURVEFS_ERROR::INTERNAL, ret); + // delete dentry internal failed ret = client_->FuseOpUnlink(req, parent, name.c_str()); ASSERT_EQ(CURVEFS_ERROR::INTERNAL, ret); + // get inode internal failed ret = client_->FuseOpUnlink(req, parent, name.c_str()); ASSERT_EQ(CURVEFS_ERROR::INTERNAL, ret); diff --git a/curvefs/test/client/test_inode_cache_manager.cpp b/curvefs/test/client/test_inode_cache_manager.cpp index 6617d3c390..2499913c9a 100644 --- a/curvefs/test/client/test_inode_cache_manager.cpp +++ b/curvefs/test/client/test_inode_cache_manager.cpp @@ -30,7 +30,7 @@ #include "curvefs/src/client/inode_wrapper.h" #include "curvefs/src/client/rpcclient/metaserver_client.h" #include "curvefs/test/client/mock_metaserver_client.h" -#include "curvefs/src/client/inode_cache_manager.h" +#include "curvefs/src/client/inode_manager.h" #include "curvefs/src/common/define.h" #include "curvefs/src/client/filesystem/defer_sync.h" #include "curvefs/src/client/filesystem/openfile.h" diff --git a/curvefs/test/client/volume/default_volume_storage_test.cpp b/curvefs/test/client/volume/default_volume_storage_test.cpp index 008b0d2004..879addb5f2 100644 --- a/curvefs/test/client/volume/default_volume_storage_test.cpp +++ b/curvefs/test/client/volume/default_volume_storage_test.cpp @@ -27,7 +27,7 @@ #include "curvefs/src/client/filesystem/error.h" #include "curvefs/src/client/filesystem/meta.h" -#include "curvefs/test/client/mock_inode_cache_manager.h" +#include "curvefs/test/client/mock_inode_manager.h" #include "curvefs/test/client/mock_metaserver_client.h" #include "curvefs/test/volume/mock/mock_block_device_client.h" #include "curvefs/test/volume/mock/mock_space_manager.h" diff --git a/curvefs/test/mds/fs_manager_test2.cpp b/curvefs/test/mds/fs_manager_test2.cpp index 08c6c40d10..fb3ee29ec6 100644 --- a/curvefs/test/mds/fs_manager_test2.cpp +++ b/curvefs/test/mds/fs_manager_test2.cpp @@ -436,5 +436,25 @@ TEST_F(FsManagerTest2, checkFsName) { EXPECT_FALSE(FsManager::CheckFsName("curve-test--01")); } +TEST_F(FsManagerTest2, test_tso) { + // Tso success + TsoRequest request; + TsoResponse response; + EXPECT_CALL(*storage_, Tso(_, _)). + WillOnce(DoAll(SetArgPointee<0>(1), + SetArgPointee<1>(100), + Return(FSStatusCode::OK))); + fsManager_->Tso(&request, &response); + ASSERT_EQ(response.statuscode(), FSStatusCode::OK); + ASSERT_EQ(response.ts(), 1); + ASSERT_EQ(response.timestamp(), 100); + + // Tso failed + EXPECT_CALL(*storage_, Tso(_, _)). + WillOnce(Return(FSStatusCode::INTERNAL_ERROR)); + fsManager_->Tso(&request, &response); + ASSERT_EQ(response.statuscode(), FSStatusCode::INTERNAL_ERROR); +} + } // namespace mds } // namespace curvefs diff --git a/curvefs/test/mds/fs_storage_test.cpp b/curvefs/test/mds/fs_storage_test.cpp index c2d56b4618..f293434628 100644 --- a/curvefs/test/mds/fs_storage_test.cpp +++ b/curvefs/test/mds/fs_storage_test.cpp @@ -23,6 +23,7 @@ #include #include #include +#include "curvefs/test/mds/mock/mock_kvstorage_client.h" using ::testing::AtLeast; using ::testing::StrEq; @@ -50,6 +51,8 @@ class FSStorageTest : public ::testing::Test { TEST_F(FSStorageTest, test1) { MemoryFsStorage storage; + ASSERT_TRUE(storage.Init()); + common::Volume volume; uint32_t fsId = 1; uint64_t rootInodeId = 1; @@ -115,6 +118,15 @@ TEST_F(FSStorageTest, test1) { ASSERT_EQ(FSStatusCode::NOT_FOUND, storage.Delete(fs1.GetFsName())); ASSERT_EQ(FSStatusCode::OK, storage.Delete(fs5.GetFsName())); ASSERT_EQ(FSStatusCode::NOT_FOUND, storage.Delete(fs5.GetFsName())); + + // test tso + uint64_t ts; + uint64_t timestamp; + for (int i = 1; i < 5; i++) { + ASSERT_EQ(FSStatusCode::OK, storage.Tso(&ts, ×tamp)); + ASSERT_EQ(ts, i); + } } + } // namespace mds } // namespace curvefs diff --git a/curvefs/test/mds/mds_service_test.cpp b/curvefs/test/mds/mds_service_test.cpp index bdae7f7691..cb5a60b765 100644 --- a/curvefs/test/mds/mds_service_test.cpp +++ b/curvefs/test/mds/mds_service_test.cpp @@ -952,5 +952,21 @@ TEST_F(MdsServiceTest, test_update_fsinfo_parameter_error) { } } +TEST_F(MdsServiceTest, test_tso) { + TsoRequest tsoRequest; + TsoResponse tsoResponse; + for (int i = 1; i < 5; i++) { + cntl.Reset(); + stub_->Tso(&cntl, &tsoRequest, &tsoResponse, nullptr); + if (!cntl.Failed()) { + ASSERT_EQ(tsoResponse.statuscode(), FSStatusCode::OK); + ASSERT_EQ(tsoResponse.ts(), i); + } else { + LOG(ERROR) << "error = " << cntl.ErrorText(); + ASSERT_TRUE(false); + } + } +} + } // namespace mds } // namespace curvefs diff --git a/curvefs/test/mds/mock/mock_fs_stroage.h b/curvefs/test/mds/mock/mock_fs_stroage.h index c3d08e4ea9..aa178c5526 100644 --- a/curvefs/test/mds/mock/mock_fs_stroage.h +++ b/curvefs/test/mds/mock/mock_fs_stroage.h @@ -61,6 +61,8 @@ class MockFsStorage : public FsStorage { MOCK_METHOD3( GetFsUsage, FSStatusCode(const std::string&, FsUsage*, bool fromCache)); MOCK_METHOD1(DeleteFsUsage, FSStatusCode(const std::string&)); + MOCK_METHOD(FSStatusCode, Tso, (uint64_t*, uint64_t*), + (override)); }; } // namespace mds diff --git a/curvefs/test/mds/persist_kvstorage_test.cpp b/curvefs/test/mds/persist_kvstorage_test.cpp index c5d25b0760..8595fd86ee 100644 --- a/curvefs/test/mds/persist_kvstorage_test.cpp +++ b/curvefs/test/mds/persist_kvstorage_test.cpp @@ -122,6 +122,17 @@ class PersistKVStorageTest : public ::testing::Test { std::shared_ptr storageCli_; }; +#define DO_INIT(storage, storageCli_) \ + do { \ + std::vector> encoded = \ + PrepareFsInfoSamples(); \ + EXPECT_CALL(*storageCli_, List(_, _, Matcher(_))) \ + .WillOnce( \ + DoAll(SetArgPointee<2>(encoded), Return(EtcdErrCode::EtcdOK)));\ + EXPECT_CALL(*storageCli_, Get(_, _)) \ + .WillRepeatedly(Return(EtcdErrCode::EtcdOK)); \ + } while (false) + TEST_F(PersistKVStorageTest, TestInit) { // list from storage failed { @@ -152,14 +163,7 @@ TEST_F(PersistKVStorageTest, TestInit) { { PersisKVStorage storage(storageCli_); - - std::vector> encoded = - PrepareFsInfoSamples(); - - EXPECT_CALL(*storageCli_, List(_, _, Matcher(_))) - .WillOnce( - DoAll(SetArgPointee<2>(encoded), Return(EtcdErrCode::EtcdOK))); - + DO_INIT(storage, storageCli_); EXPECT_TRUE(storage.Init()); EXPECT_TRUE(storage.Exist(1)); @@ -185,6 +189,8 @@ TEST_F(PersistKVStorageTest, TestGetAndExist) { EXPECT_CALL(*storageCli_, List(_, _, Matcher(_))) .WillOnce( DoAll(SetArgPointee<2>(encoded), Return(EtcdErrCode::EtcdOK))); + EXPECT_CALL(*storageCli_, Get(_, _)) + .WillRepeatedly(Return(EtcdErrCode::EtcdOK)); EXPECT_TRUE(storage.Init()); EXPECT_FALSE(storage.Exist(1)); @@ -197,14 +203,9 @@ TEST_F(PersistKVStorageTest, TestGetAndExist) { { PersisKVStorage storage(storageCli_); - std::vector> encoded = - PrepareFsInfoSamples(); - - EXPECT_CALL(*storageCli_, List(_, _, Matcher(_))) - .WillOnce( - DoAll(SetArgPointee<2>(encoded), Return(EtcdErrCode::EtcdOK))); - + DO_INIT(storage, storageCli_); EXPECT_TRUE(storage.Init()); + EXPECT_TRUE(storage.Exist(1)); EXPECT_TRUE(storage.Exist("hello")); EXPECT_TRUE(storage.Exist(2)); @@ -290,17 +291,12 @@ TEST_F(PersistKVStorageTest, TestInsert) { // fs already exists { PersisKVStorage storage(storageCli_); - std::vector> encoded = - PrepareFsInfoSamples(); + DO_INIT(storage, storageCli_); + EXPECT_TRUE(storage.Init()); - EXPECT_CALL(*storageCli_, List(_, _, Matcher(_))) - .WillOnce( - DoAll(SetArgPointee<2>(encoded), Return(EtcdErrCode::EtcdOK))); EXPECT_CALL(*storageCli_, Put(_, _)) .Times(0); - EXPECT_TRUE(storage.Init()); - FsInfoWrapper wrapper; EXPECT_EQ(FSStatusCode::OK, storage.Get("hello", &wrapper)); @@ -310,18 +306,13 @@ TEST_F(PersistKVStorageTest, TestInsert) { // kvstorage error { - PersisKVStorage storage(storageCli_); - std::vector> encoded = - PrepareFsInfoSamples(); + PersisKVStorage storage(storageCli_); + DO_INIT(storage, storageCli_); + EXPECT_TRUE(storage.Init()); - EXPECT_CALL(*storageCli_, List(_, _, Matcher(_))) - .WillOnce( - DoAll(SetArgPointee<2>(encoded), Return(EtcdErrCode::EtcdOK))); EXPECT_CALL(*storageCli_, Put(_, _)) .WillOnce(Return(EtcdErrCode::EtcdInternal)); - EXPECT_TRUE(storage.Init()); - FsInfoWrapper wrapper; EXPECT_EQ(FSStatusCode::OK, storage.Get("hello", &wrapper)); @@ -337,17 +328,12 @@ TEST_F(PersistKVStorageTest, TestInsert) { // kvstorage persist ok { PersisKVStorage storage(storageCli_); - std::vector> encoded = - PrepareFsInfoSamples(); + DO_INIT(storage, storageCli_); + EXPECT_TRUE(storage.Init()); - EXPECT_CALL(*storageCli_, List(_, _, Matcher(_))) - .WillOnce( - DoAll(SetArgPointee<2>(encoded), Return(EtcdErrCode::EtcdOK))); EXPECT_CALL(*storageCli_, Put(_, _)) .WillOnce(Return(EtcdErrCode::EtcdOK)); - EXPECT_TRUE(storage.Init()); - FsInfoWrapper wrapper; EXPECT_EQ(FSStatusCode::OK, storage.Get("hello", &wrapper)); @@ -370,17 +356,12 @@ TEST_F(PersistKVStorageTest, TestUpdate) { // fs not found { PersisKVStorage storage(storageCli_); - std::vector> encoded = - PrepareFsInfoSamples(); + DO_INIT(storage, storageCli_); + EXPECT_TRUE(storage.Init()); - EXPECT_CALL(*storageCli_, List(_, _, Matcher(_))) - .WillOnce( - DoAll(SetArgPointee<2>(encoded), Return(EtcdErrCode::EtcdOK))); EXPECT_CALL(*storageCli_, Put(_, _)) .Times(0); - EXPECT_TRUE(storage.Init()); - FsInfoWrapper wrapper; EXPECT_EQ(FSStatusCode::OK, storage.Get("hello", &wrapper)); @@ -395,17 +376,12 @@ TEST_F(PersistKVStorageTest, TestUpdate) { // fs id mismatch { PersisKVStorage storage(storageCli_); - std::vector> encoded = - PrepareFsInfoSamples(); + DO_INIT(storage, storageCli_); + EXPECT_TRUE(storage.Init()); - EXPECT_CALL(*storageCli_, List(_, _, Matcher(_))) - .WillOnce( - DoAll(SetArgPointee<2>(encoded), Return(EtcdErrCode::EtcdOK))); EXPECT_CALL(*storageCli_, Put(_, _)) .Times(0); - EXPECT_TRUE(storage.Init()); - FsInfoWrapper wrapper; EXPECT_EQ(FSStatusCode::OK, storage.Get("hello", &wrapper)); @@ -420,17 +396,12 @@ TEST_F(PersistKVStorageTest, TestUpdate) { // storage failed { PersisKVStorage storage(storageCli_); - std::vector> encoded = - PrepareFsInfoSamples(); + DO_INIT(storage, storageCli_); + EXPECT_TRUE(storage.Init()); - EXPECT_CALL(*storageCli_, List(_, _, Matcher(_))) - .WillOnce( - DoAll(SetArgPointee<2>(encoded), Return(EtcdErrCode::EtcdOK))); EXPECT_CALL(*storageCli_, Put(_, _)) .WillOnce(Return(EtcdErrCode::EtcdInternal)); - EXPECT_TRUE(storage.Init()); - FsInfoWrapper wrapper; EXPECT_EQ(FSStatusCode::OK, storage.Get("hello", &wrapper)); @@ -449,17 +420,12 @@ TEST_F(PersistKVStorageTest, TestUpdate) { // storage ok { PersisKVStorage storage(storageCli_); - std::vector> encoded = - PrepareFsInfoSamples(); + DO_INIT(storage, storageCli_); + EXPECT_TRUE(storage.Init()); - EXPECT_CALL(*storageCli_, List(_, _, Matcher(_))) - .WillOnce( - DoAll(SetArgPointee<2>(encoded), Return(EtcdErrCode::EtcdOK))); EXPECT_CALL(*storageCli_, Put(_, _)) .WillOnce(Return(EtcdErrCode::EtcdOK)); - EXPECT_TRUE(storage.Init()); - FsInfoWrapper wrapper; EXPECT_EQ(FSStatusCode::OK, storage.Get("hello", &wrapper)); @@ -480,34 +446,22 @@ TEST_F(PersistKVStorageTest, TestDelete) { // fs not found { PersisKVStorage storage(storageCli_); - std::vector> encoded = - PrepareFsInfoSamples(); + DO_INIT(storage, storageCli_); + EXPECT_TRUE(storage.Init()); - EXPECT_CALL(*storageCli_, List(_, _, Matcher(_))) - .WillOnce( - DoAll(SetArgPointee<2>(encoded), Return(EtcdErrCode::EtcdOK))); EXPECT_CALL(*storageCli_, Delete(_)) .Times(0); - - EXPECT_TRUE(storage.Init()); - EXPECT_EQ(FSStatusCode::NOT_FOUND, storage.Delete("bvar")); } // storage failed { PersisKVStorage storage(storageCli_); - std::vector> encoded = - PrepareFsInfoSamples(); + DO_INIT(storage, storageCli_); + EXPECT_TRUE(storage.Init()); - EXPECT_CALL(*storageCli_, List(_, _, Matcher(_))) - .WillOnce( - DoAll(SetArgPointee<2>(encoded), Return(EtcdErrCode::EtcdOK))); EXPECT_CALL(*storageCli_, Delete(_)) .WillOnce(Return(EtcdErrCode::EtcdInternal)); - - EXPECT_TRUE(storage.Init()); - EXPECT_EQ(FSStatusCode::STORAGE_ERROR, storage.Delete("hello")); EXPECT_TRUE(storage.Exist("hello")); EXPECT_TRUE(storage.Exist(1)); @@ -516,17 +470,11 @@ TEST_F(PersistKVStorageTest, TestDelete) { // storage ok { PersisKVStorage storage(storageCli_); - std::vector> encoded = - PrepareFsInfoSamples(); + DO_INIT(storage, storageCli_); + EXPECT_TRUE(storage.Init()); - EXPECT_CALL(*storageCli_, List(_, _, Matcher(_))) - .WillOnce( - DoAll(SetArgPointee<2>(encoded), Return(EtcdErrCode::EtcdOK))); EXPECT_CALL(*storageCli_, Delete(_)) .WillOnce(Return(EtcdErrCode::EtcdOK)); - - EXPECT_TRUE(storage.Init()); - EXPECT_EQ(FSStatusCode::OK, storage.Delete("hello")); EXPECT_FALSE(storage.Exist("hello")); EXPECT_FALSE(storage.Exist(1)); @@ -535,5 +483,51 @@ TEST_F(PersistKVStorageTest, TestDelete) { EXPECT_TRUE(storage.Exist(2)); } } + +TEST_F(PersistKVStorageTest, TestTso) { + uint64_t ts; + uint64_t timestamp; + // tsid not exist + { + PersisKVStorage storage(storageCli_); + // get failed + EXPECT_CALL(*storageCli_, Get(_, _)) + .WillOnce(Return(EtcdErrCode::EtcdUnknown)); + ASSERT_EQ(FSStatusCode::INTERNAL_ERROR, + storage.Tso(&ts, ×tamp)); + // CompareAndSwap failed + EXPECT_CALL(*storageCli_, Get(_, _)) + .WillOnce(Return(EtcdErrCode::EtcdKeyNotExist)); + EXPECT_CALL(*storageCli_, CompareAndSwap(_, _, _)) + .WillOnce(Return(EtcdErrCode::EtcdUnknown)); + ASSERT_EQ(FSStatusCode::INTERNAL_ERROR, + storage.Tso(&ts, ×tamp)); + // success + EXPECT_CALL(*storageCli_, Get(_, _)) + .WillOnce(Return(EtcdErrCode::EtcdKeyNotExist)); + EXPECT_CALL(*storageCli_, CompareAndSwap(_, _, _)) + .WillOnce(Return(EtcdErrCode::EtcdOK)); + for (int i = 1; i < 5; i++) { + ASSERT_EQ(FSStatusCode::OK, storage.Tso(&ts, ×tamp)); + ASSERT_EQ(ts, i); + } + } + // txid exist + { + PersisKVStorage storage(storageCli_); + uint64_t tsId = 10000; + // success + EXPECT_CALL(*storageCli_, Get(_, _)) + .WillOnce(DoAll(SetArgPointee<1>(std::to_string(tsId)), + Return(EtcdErrCode::EtcdOK))); + EXPECT_CALL(*storageCli_, CompareAndSwap(_, _, _)) + .WillOnce(Return(EtcdErrCode::EtcdOK)); + for (int i = 1; i < 5; i++) { + ASSERT_EQ(FSStatusCode::OK, storage.Tso(&ts, ×tamp)); + ASSERT_EQ(ts, i + tsId); + } + } +} + } // namespace mds } // namespace curvefs diff --git a/curvefs/test/metaserver/dentry_manager_test.cpp b/curvefs/test/metaserver/dentry_manager_test.cpp index f113eca4dc..905ef307c1 100644 --- a/curvefs/test/metaserver/dentry_manager_test.cpp +++ b/curvefs/test/metaserver/dentry_manager_test.cpp @@ -35,6 +35,10 @@ namespace curvefs { namespace metaserver { +namespace storage { + DECLARE_int32(tx_lock_ttl_ms); +} + using ::curvefs::metaserver::storage::KVStorage; using ::curvefs::metaserver::storage::RandomStoragePath; using ::curvefs::metaserver::storage::RocksDBStorage; @@ -198,5 +202,141 @@ TEST_F(DentryManagerTest, HandleRenameTx) { ASSERT_EQ(dentryStorage_->Size(), 1); } +TEST_F(DentryManagerTest, PrewriteRenameTx) { + TxLock txLockIn; + TxLock txLockOut; + int64_t logIndex = 1; + uint64_t startTs = 2; + uint64_t commitTs = 3; + Dentry dentryA = GenDentry(1, 0, "A", startTs, 1, false); + // 1. prewrite success + std::vector dentrys = std::vector{dentryA}; + txLockIn.set_primarykey(storage::Key4Dentry(1, 0, "A").SerializeToString()); + txLockIn.set_startts(startTs); + txLockIn.set_timestamp(100); + auto rc = dentryManager_->PrewriteRenameTx(dentrys, txLockIn, logIndex, + &txLockOut); + ASSERT_EQ(rc, MetaStatusCode::OK); + // 2. tx locked + txLockIn.set_startts(1); + rc = dentryManager_->PrewriteRenameTx(dentrys, txLockIn, logIndex, + &txLockOut); + ASSERT_EQ(rc, MetaStatusCode::TX_KEY_LOCKED); + ASSERT_EQ(txLockOut.startts(), startTs); + ASSERT_EQ(txLockOut.primarykey(), txLockIn.primarykey()); + // 3. tx write conflict + rc = dentryManager_->CommitTx(dentrys, startTs, commitTs, logIndex++); + ASSERT_EQ(rc, MetaStatusCode::OK); + rc = dentryManager_->PrewriteRenameTx(dentrys, txLockIn, logIndex, + &txLockOut); + ASSERT_EQ(rc, MetaStatusCode::TX_WRITE_CONFLICT); +} + +TEST_F(DentryManagerTest, CheckTxStatus) { + storage::FLAGS_tx_lock_ttl_ms = 100; + + TxLock txLockIn; + TxLock txLockOut; + int64_t logIndex = 1; + uint64_t startTs = 2; + uint64_t commitTs = 3; + Dentry dentryA = GenDentry(1, 0, "A", startTs, 1, false); + std::vector dentrys = std::vector{dentryA}; + txLockIn.set_primarykey(storage::Key4Dentry(1, 0, "A").SerializeToString()); + txLockIn.set_startts(startTs); + txLockIn.set_timestamp(1000); + auto rc = dentryManager_->PrewriteRenameTx(dentrys, txLockIn, logIndex, + &txLockOut); + ASSERT_EQ(rc, MetaStatusCode::OK); + + // timeout + rc = dentryManager_->CheckTxStatus(txLockIn.primarykey(), startTs, 1500, + logIndex); + ASSERT_EQ(rc, MetaStatusCode::TX_TIMEOUT); + // inprogress + rc = dentryManager_->CheckTxStatus(txLockIn.primarykey(), startTs, 1050, + logIndex); + ASSERT_EQ(rc, MetaStatusCode::TX_INPROGRESS); + // commited + rc = dentryManager_->CommitTx(dentrys, startTs, commitTs, logIndex++); + ASSERT_EQ(rc, MetaStatusCode::OK); + rc = dentryManager_->CheckTxStatus(txLockIn.primarykey(), startTs, 1500, + logIndex); + ASSERT_EQ(rc, MetaStatusCode::TX_COMMITTED); +} + + +TEST_F(DentryManagerTest, ResolveTxLock) { + TxLock txLockIn; + TxLock txLockOut; + int64_t logIndex = 1; + uint64_t startTs = 2; + uint64_t commitTs = 3; + Dentry dentryA = GenDentry(1, 0, "A", startTs, 1, false); + std::vector dentrys = std::vector{dentryA}; + txLockIn.set_primarykey(storage::Key4Dentry(1, 0, "A").SerializeToString()); + txLockIn.set_startts(startTs); + txLockIn.set_timestamp(1000); + + // 1. tx lock not exist + auto rc = dentryManager_->ResolveTxLock(dentryA, startTs, commitTs, + logIndex++); + ASSERT_EQ(rc, MetaStatusCode::OK); + // 2. tx lock exist, but startts not match + rc = dentryManager_->PrewriteRenameTx(dentrys, txLockIn, logIndex++, + &txLockOut); + ASSERT_EQ(rc, MetaStatusCode::OK); + rc = dentryManager_->ResolveTxLock(dentryA, startTs + 1, commitTs, + logIndex++); + ASSERT_EQ(rc, MetaStatusCode::TX_MISMATCH); + // 3. roll forward success + rc = dentryManager_->ResolveTxLock(dentryA, startTs, commitTs, logIndex++); + ASSERT_EQ(rc, MetaStatusCode::OK); + rc = dentryManager_->CheckTxStatus(txLockIn.primarykey(), startTs, 1500, + logIndex++); + ASSERT_EQ(rc, MetaStatusCode::TX_COMMITTED); + // 4. roll back success + dentrys[0].set_txid(startTs + 2); + txLockIn.set_startts(startTs + 2); + commitTs++; + rc = dentryManager_->PrewriteRenameTx(dentrys, txLockIn, logIndex++, + &txLockOut); + ASSERT_EQ(rc, MetaStatusCode::OK); + rc = dentryManager_->ResolveTxLock(dentryA, startTs + 2, 0, logIndex++); + ASSERT_EQ(rc, MetaStatusCode::OK); + rc = dentryManager_->CheckTxStatus(txLockIn.primarykey(), startTs + 2, 1500, + logIndex++); + ASSERT_EQ(rc, MetaStatusCode::TX_ROLLBACKED); +} + +TEST_F(DentryManagerTest, CommitTx) { + TxLock txLockIn; + TxLock txLockOut; + int64_t logIndex = 1; + uint64_t startTs = 2; + uint64_t commitTs = 3; + Dentry dentryA = GenDentry(1, 0, "A", startTs, 1, false); + std::vector dentrys = std::vector{dentryA}; + + // 1. tx lock not exist + auto rc = dentryManager_->CommitTx(dentrys, startTs, commitTs, logIndex++); + ASSERT_EQ(rc, MetaStatusCode::OK); + // 2. tx lock exist, but startts not match + txLockIn.set_primarykey(storage::Key4Dentry(1, 0, "A").SerializeToString()); + txLockIn.set_startts(startTs); + txLockIn.set_timestamp(1000); + rc = dentryManager_->PrewriteRenameTx(dentrys, txLockIn, logIndex++, + &txLockOut); + ASSERT_EQ(rc, MetaStatusCode::OK); + rc = dentryManager_->CommitTx(dentrys, startTs + 1, commitTs, logIndex++); + ASSERT_EQ(rc, MetaStatusCode::TX_MISMATCH); + // 3. commit success + rc = dentryManager_->CommitTx(dentrys, startTs, commitTs, logIndex++); + ASSERT_EQ(rc, MetaStatusCode::OK); + rc = dentryManager_->CheckTxStatus(txLockIn.primarykey(), startTs, 1500, + logIndex++); + ASSERT_EQ(rc, MetaStatusCode::TX_COMMITTED); +} + } // namespace metaserver } // namespace curvefs diff --git a/curvefs/test/metaserver/dentry_storage_test.cpp b/curvefs/test/metaserver/dentry_storage_test.cpp index 4464f20c76..218248bca8 100644 --- a/curvefs/test/metaserver/dentry_storage_test.cpp +++ b/curvefs/test/metaserver/dentry_storage_test.cpp @@ -29,6 +29,7 @@ #include "curvefs/src/metaserver/storage/rocksdb_storage.h" #include "curvefs/test/metaserver/storage/utils.h" #include "src/fs/ext4_filesystem_impl.h" +#include "src/common/timeutility.h" namespace curvefs { namespace metaserver { @@ -38,6 +39,9 @@ using ::curvefs::metaserver::storage::NameGenerator; using ::curvefs::metaserver::storage::RandomStoragePath; using ::curvefs::metaserver::storage::RocksDBStorage; using ::curvefs::metaserver::storage::StorageOptions; +using ::curvefs::metaserver::storage::Status; +using ::curvefs::metaserver::storage::Key4Dentry; +using ::curvefs::metaserver::storage::Key4TxWrite; namespace { auto localfs = curve::fs::Ext4FileSystemImpl::getInstance(); @@ -55,6 +59,8 @@ class DentryStorageTest : public ::testing::Test { kvStorage_ = std::make_shared(options); ASSERT_TRUE(kvStorage_->Open()); logIndex_ = 0; + table4TxWrite_ = nameGenerator_->GetTxWriteTableName(); + table4TxLock_ = nameGenerator_->GetTxLockTableName(); } void TearDown() override { @@ -108,11 +114,25 @@ class DentryStorageTest : public ::testing::Test { ASSERT_EQ(lhs, rhs); } + std::string DentryKey(const Dentry& dentry) { + Key4Dentry key(dentry.fsid(), dentry.parentinodeid(), dentry.name()); + return conv_.SerializeToString(key); + } + + std::string TxWriteKey(const Dentry& dentry, uint64_t ts) { + Key4TxWrite key(dentry.fsid(), dentry.parentinodeid(), + dentry.name(), ts); + return conv_.SerializeToString(key); + } + protected: std::string dataDir_; std::shared_ptr nameGenerator_; std::shared_ptr kvStorage_; int64_t logIndex_; + Converter conv_; + std::string table4TxWrite_; + std::string table4TxLock_; }; TEST_F(DentryStorageTest, Insert) { @@ -573,5 +593,254 @@ TEST_F(DentryStorageTest, HandleTx) { ASSERT_EQ(dentry.inodeid(), 1); } +TEST_F(DentryStorageTest, PrewriteTx) { + DentryStorage storage(kvStorage_, nameGenerator_, 0); + ASSERT_TRUE(storage.Init()); + + // 1. prepare original dentry + // { fsId, parentId, name, txId, inodeId, deleteMarkFlag } + Dentry dentry = GenDentry(1, 1, "A", 0, 2, false); + ASSERT_EQ(storage.Insert(dentry, logIndex_++), MetaStatusCode::OK); + ASSERT_EQ(storage.Size(), 1); + + // 2. prepare prewrite dentry + uint64_t startTs = 2; + Dentry dentryA = GenDentry(1, 1, "A", startTs, 2, true); + Dentry dentryB = GenDentry(1, 1, "B", startTs, 3, false); + std::vector dentrys = {dentryA, dentryB}; + TxLock txLock; + txLock.set_primarykey(DentryKey(dentryA)); + txLock.set_startts(startTs); + txLock.set_timestamp(curve::common::TimeUtility::GetTimeofDayMs()); + + // 2.1 write conflict + TxLock outLock; + TxWrite txWrite; + txWrite.set_startts(startTs); + txWrite.set_kind(TxWriteKind::Commit); + Status s = kvStorage_->SSet(table4TxWrite_, + TxWriteKey(dentry, startTs + 1), txWrite); + ASSERT_TRUE(s.ok()); + ASSERT_EQ(storage.PrewriteTx(dentrys, txLock, logIndex_++, &outLock), + MetaStatusCode::TX_WRITE_CONFLICT); + s = kvStorage_->SDel(table4TxWrite_, TxWriteKey(dentry, startTs + 1)); + ASSERT_TRUE(s.ok()); + + // 2.2 key locked and IDEMPOTENCE OK + s = kvStorage_->SSet(table4TxLock_, DentryKey(dentryA), txLock); + ASSERT_TRUE(s.ok()); + ASSERT_EQ(storage.PrewriteTx(dentrys, txLock, logIndex_++, &outLock), + MetaStatusCode::OK); + s = kvStorage_->SDel(table4TxLock_, DentryKey(dentryA)); + ASSERT_TRUE(s.ok()); + + // 2.3 key locked + TxLock preLock(txLock); + preLock.set_startts(startTs + 1); + s = kvStorage_->SSet(table4TxLock_, DentryKey(dentryA), preLock); + ASSERT_TRUE(s.ok()); + ASSERT_EQ(storage.PrewriteTx(dentrys, txLock, logIndex_++, &outLock), + MetaStatusCode::TX_KEY_LOCKED); + s = kvStorage_->SDel(table4TxLock_, DentryKey(dentryA)); + ASSERT_TRUE(s.ok()); + + // 2.4 prewrite success + ASSERT_EQ(storage.PrewriteTx( + std::vector(dentrys.begin() + outLock.index(), dentrys.end()), + txLock, logIndex_++, &outLock), MetaStatusCode::OK); + ASSERT_EQ(storage.Size(), 3); + Dentry entryOut; + entryOut.set_fsid(1); + entryOut.set_parentinodeid(1); + entryOut.set_name("A"); + ASSERT_EQ(storage.Get(&entryOut), MetaStatusCode::OK); + ASSERT_TRUE(dentry == entryOut); + entryOut.set_name("B"); + ASSERT_EQ(storage.Get(&entryOut), MetaStatusCode::NOT_FOUND); +} + +TEST_F(DentryStorageTest, CheckTxStatus) { + DentryStorage storage(kvStorage_, nameGenerator_, 0); + ASSERT_TRUE(storage.Init()); + + // 1. tx lock exist, tx timeout + uint64_t startTs = 1; + uint64_t now = curve::common::TimeUtility::GetTimeofDayMs(); + Dentry dentry = GenDentry(1, 1, "A", startTs, 2, false); + TxLock txLock; + txLock.set_primarykey(DentryKey(dentry)); + txLock.set_startts(startTs); + txLock.set_timestamp(now - 10); + txLock.set_ttl(5); + Status s = kvStorage_->SSet(table4TxLock_, DentryKey(dentry), txLock); + ASSERT_TRUE(s.ok()); + ASSERT_EQ( + storage.CheckTxStatus(DentryKey(dentry), startTs, now, logIndex_++), + MetaStatusCode::TX_TIMEOUT); + + // 2. tx lock exist, tx in progress + txLock.set_timestamp(now); + s = kvStorage_->SSet(table4TxLock_, DentryKey(dentry), txLock); + ASSERT_TRUE(s.ok()); + ASSERT_EQ( + storage.CheckTxStatus(DentryKey(dentry), startTs, now, logIndex_++), + MetaStatusCode::TX_INPROGRESS); + s = kvStorage_->SDel(table4TxLock_, DentryKey(dentry)); + ASSERT_TRUE(s.ok()); + + // 3. tx lock not exist, tx write not exit, committed + ASSERT_EQ( + storage.CheckTxStatus(DentryKey(dentry), startTs, now, logIndex_++), + MetaStatusCode::TX_COMMITTED); + TxWrite txWrite; + txWrite.set_startts(startTs); + txWrite.set_kind(TxWriteKind::Commit); + s = kvStorage_->SSet(table4TxWrite_, + TxWriteKey(dentry, startTs + 1), txWrite); + ASSERT_TRUE(s.ok()); + ASSERT_EQ( + storage.CheckTxStatus(DentryKey(dentry), startTs, now, logIndex_++), + MetaStatusCode::TX_COMMITTED); + + // 4. tx lock not exist, rollbacked + txWrite.set_kind(TxWriteKind::Rollback); + s = kvStorage_->SSet(table4TxWrite_, TxWriteKey(dentry, startTs), txWrite); + ASSERT_TRUE(s.ok()); + ASSERT_EQ( + storage.CheckTxStatus(DentryKey(dentry), startTs, now, logIndex_++), + MetaStatusCode::TX_ROLLBACKED); +} + +TEST_F(DentryStorageTest, ResolveTxLock) { + DentryStorage storage(kvStorage_, nameGenerator_, 0); + ASSERT_TRUE(storage.Init()); + + uint64_t preTxStartTs = 1; + uint64_t startTs = 10; + uint64_t commitTs = 11; + uint64_t now = curve::common::TimeUtility::GetTimeofDayMs(); + Dentry dentry = GenDentry(1, 1, "A", startTs, 2, false); + + // 1. tx lock not exist + ASSERT_EQ(storage.ResolveTxLock(dentry, preTxStartTs, commitTs, + logIndex_++), MetaStatusCode::OK); + + // 2. roll forward + // 2.1 tx lock exist but startts mismatch + TxLock preTxLock; + preTxLock.set_primarykey(DentryKey(dentry)); + preTxLock.set_startts(preTxStartTs + 1); + preTxLock.set_timestamp(now-100); + Status s = kvStorage_->SSet(table4TxLock_, DentryKey(dentry), preTxLock); + ASSERT_TRUE(s.ok()); + ASSERT_EQ(storage.ResolveTxLock(dentry, preTxStartTs, commitTs, + logIndex_++), MetaStatusCode::TX_MISMATCH); + // 2.2 success + preTxLock.set_startts(preTxStartTs); + s = kvStorage_->SSet(table4TxLock_, DentryKey(dentry), preTxLock); + ASSERT_TRUE(s.ok()); + ASSERT_EQ(storage.ResolveTxLock(dentry, preTxStartTs, commitTs, + logIndex_++), MetaStatusCode::OK); + TxLock lockOut; + s = kvStorage_->SGet(table4TxLock_, DentryKey(dentry), &lockOut); + ASSERT_TRUE(s.IsNotFound()); + TxWrite txWriteOut; + s = kvStorage_->SGet(table4TxWrite_, TxWriteKey(dentry, commitTs), + &txWriteOut); + ASSERT_TRUE(s.ok()); + ASSERT_EQ(txWriteOut.kind(), TxWriteKind::Commit); + ASSERT_EQ(txWriteOut.startts(), preTxStartTs); + TS tsOut; + s = kvStorage_->SGet(table4TxWrite_, "latestCommit", &tsOut); + ASSERT_TRUE(s.ok()); + ASSERT_EQ(tsOut.ts(), commitTs); + + // 3. roll backward + // prepare rollback site + TxLock txLock(preTxLock); + txLock.set_startts(startTs); + s = kvStorage_->SSet(table4TxLock_, DentryKey(dentry), txLock); + ASSERT_TRUE(s.ok()); + dentry.set_txid(startTs); + ASSERT_EQ(storage.Insert(dentry, logIndex_++), MetaStatusCode::OK); + ASSERT_EQ(storage.Size(), 1); + ASSERT_EQ(storage.ResolveTxLock(dentry, startTs, 0, + logIndex_++), MetaStatusCode::OK); + s = kvStorage_->SGet(table4TxLock_, DentryKey(dentry), &lockOut); + ASSERT_TRUE(s.IsNotFound()); + ASSERT_EQ(storage.Size(), 0); + s = kvStorage_->SGet(table4TxWrite_, TxWriteKey(dentry, startTs), + &txWriteOut); + ASSERT_TRUE(s.ok()); + ASSERT_EQ(txWriteOut.kind(), TxWriteKind::Rollback); + ASSERT_EQ(txWriteOut.startts(), startTs); +} + +TEST_F(DentryStorageTest, CommitTx) { + DentryStorage storage(kvStorage_, nameGenerator_, 0); + ASSERT_TRUE(storage.Init()); + + uint64_t startTs = 1; + uint64_t commitTs = 2; + uint64_t now = curve::common::TimeUtility::GetTimeofDayMs(); + Dentry dentryA = GenDentry(1, 1, "A", startTs, 2, true); + Dentry dentryB = GenDentry(1, 1, "B", startTs, 2, false); + + // 1. tx lock not exist + ASSERT_EQ(storage.CommitTx( + {dentryA, dentryB}, startTs, commitTs, logIndex_++), + MetaStatusCode::OK); + TS tsOut; + Status s = kvStorage_->SGet(table4TxWrite_, "latestCommit", &tsOut); + ASSERT_TRUE(s.ok()); + ASSERT_EQ(tsOut.ts(), startTs); + // 2. tx lock exist, but startts mismatch + TxLock txLock; + txLock.set_primarykey(DentryKey(dentryA)); + txLock.set_startts(startTs + 1); + txLock.set_timestamp(now - 100); + s = kvStorage_->SSet(table4TxLock_, DentryKey(dentryA), txLock); + ASSERT_TRUE(s.ok()); + ASSERT_EQ(storage.CommitTx( + {dentryA, dentryB}, startTs, commitTs, logIndex_++), + MetaStatusCode::TX_MISMATCH); + // 3. commit success + TxLock txLockA; + txLockA.set_primarykey(DentryKey(dentryA)); + txLockA.set_startts(startTs); + txLockA.set_timestamp(now - 100); + s = kvStorage_->SSet(table4TxLock_, DentryKey(dentryA), txLockA); + ASSERT_TRUE(s.ok()); + TxLock txLockB; + txLockB.set_primarykey(DentryKey(dentryB)); + txLockB.set_startts(startTs); + txLockB.set_timestamp(now - 100); + s = kvStorage_->SSet(table4TxLock_, DentryKey(dentryB), txLockB); + ASSERT_TRUE(s.ok()); + ASSERT_EQ(storage.CommitTx( + {dentryA, dentryB}, startTs, commitTs, logIndex_++), + MetaStatusCode::OK); + TxLock lockOut; + s = kvStorage_->SGet(table4TxLock_, DentryKey(dentryA), &lockOut); + ASSERT_TRUE(s.IsNotFound()); + s = kvStorage_->SGet(table4TxLock_, DentryKey(dentryB), &lockOut); + ASSERT_TRUE(s.IsNotFound()); + TxWrite txWriteOut; + s = kvStorage_->SGet(table4TxWrite_, TxWriteKey(dentryA, commitTs), + &txWriteOut); + ASSERT_TRUE(s.ok()); + ASSERT_EQ(txWriteOut.kind(), TxWriteKind::Commit); + ASSERT_EQ(txWriteOut.startts(), startTs); + s = kvStorage_->SGet(table4TxWrite_, TxWriteKey(dentryB, commitTs), + &txWriteOut); + ASSERT_TRUE(s.ok()); + ASSERT_EQ(txWriteOut.kind(), TxWriteKind::Commit); + ASSERT_EQ(txWriteOut.startts(), startTs); + s = kvStorage_->SGet(table4TxWrite_, "latestCommit", &tsOut); + ASSERT_TRUE(s.ok()); + ASSERT_EQ(tsOut.ts(), startTs); +} + + } // namespace metaserver } // namespace curvefs diff --git a/curvefs/test/metaserver/metaserver_service_test2.cpp b/curvefs/test/metaserver/metaserver_service_test2.cpp index e7376b2e4f..6bb900b171 100644 --- a/curvefs/test/metaserver/metaserver_service_test2.cpp +++ b/curvefs/test/metaserver/metaserver_service_test2.cpp @@ -92,6 +92,10 @@ TEST_F(MetaServerServiceTest2, ServiceOverload) { TEST_SERVICE_OVERLOAD(CreatePartition); TEST_SERVICE_OVERLOAD(DeletePartition); TEST_SERVICE_OVERLOAD(PrepareRenameTx); + TEST_SERVICE_OVERLOAD(PrewriteRenameTx); + TEST_SERVICE_OVERLOAD(CheckTxStatus); + TEST_SERVICE_OVERLOAD(ResolveTxLock); + TEST_SERVICE_OVERLOAD(CommitTx); TEST_SERVICE_OVERLOAD(GetVolumeExtent); TEST_SERVICE_OVERLOAD(UpdateVolumeExtent); TEST_SERVICE_OVERLOAD(UpdateDeallocatableBlockGroup); @@ -130,6 +134,10 @@ TEST_F(MetaServerServiceTest2, CopysetNodeNotFound) { TEST_COPYSETNODE_NOTFOUND(CreatePartition); TEST_COPYSETNODE_NOTFOUND(DeletePartition); TEST_COPYSETNODE_NOTFOUND(PrepareRenameTx); + TEST_COPYSETNODE_NOTFOUND(PrewriteRenameTx); + TEST_COPYSETNODE_NOTFOUND(CheckTxStatus); + TEST_COPYSETNODE_NOTFOUND(ResolveTxLock); + TEST_COPYSETNODE_NOTFOUND(CommitTx); TEST_COPYSETNODE_NOTFOUND(GetVolumeExtent); TEST_COPYSETNODE_NOTFOUND(UpdateVolumeExtent); TEST_COPYSETNODE_NOTFOUND(UpdateDeallocatableBlockGroup); diff --git a/curvefs/test/metaserver/mock/mock_metastore.h b/curvefs/test/metaserver/mock/mock_metastore.h index a8c923d3aa..13188aae72 100644 --- a/curvefs/test/metaserver/mock/mock_metastore.h +++ b/curvefs/test/metaserver/mock/mock_metastore.h @@ -122,6 +122,21 @@ class MockMetaStore : public curvefs::metaserver::MetaStore { MetaStatusCode(const UpdateDeallocatableBlockGroupRequest*, UpdateDeallocatableBlockGroupResponse*, int64_t logIndex)); + + MOCK_METHOD(MetaStatusCode, PrewriteRenameTx, + (const PrewriteRenameTxRequest* request, + PrewriteRenameTxResponse* response, int64_t logIndex), (override)); + + MOCK_METHOD(MetaStatusCode, CheckTxStatus, + (const CheckTxStatusRequest* request, + CheckTxStatusResponse* response, int64_t logIndex), (override)); + + MOCK_METHOD(MetaStatusCode, ResolveTxLock, + (const ResolveTxLockRequest* request, + ResolveTxLockResponse* response, int64_t logIndex), (override)); + + MOCK_METHOD(MetaStatusCode, CommitTx, (const CommitTxRequest* request, + CommitTxResponse* response, int64_t logIndex), (override)); }; } // namespace mock diff --git a/curvefs/test/metaserver/recycle_cleaner_test.cpp b/curvefs/test/metaserver/recycle_cleaner_test.cpp index f7b6481ac4..3a596303b4 100644 --- a/curvefs/test/metaserver/recycle_cleaner_test.cpp +++ b/curvefs/test/metaserver/recycle_cleaner_test.cpp @@ -213,7 +213,7 @@ TEST_F(RecycleCleanerTest, delete_node_test) { // delete dentry fail { - EXPECT_CALL(*metaClient_, DeleteDentry(_, _, _, _)) + EXPECT_CALL(*metaClient_, DeleteDentry(_, _, _, _, _)) .WillOnce(Return(MetaStatusCode::UNKNOWN_ERROR)); ASSERT_FALSE(cleaner_->DeleteNode(dentry)); @@ -221,7 +221,7 @@ TEST_F(RecycleCleanerTest, delete_node_test) { // get parent inode fail { - EXPECT_CALL(*metaClient_, DeleteDentry(_, _, _, _)) + EXPECT_CALL(*metaClient_, DeleteDentry(_, _, _, _, _)) .WillOnce(Return(MetaStatusCode::OK)); EXPECT_CALL(*metaClient_, GetInode(_, _, _, _)) .WillOnce(Return(MetaStatusCode::UNKNOWN_ERROR)); @@ -231,7 +231,7 @@ TEST_F(RecycleCleanerTest, delete_node_test) { // update parent inode fail { - EXPECT_CALL(*metaClient_, DeleteDentry(_, _, _, _)) + EXPECT_CALL(*metaClient_, DeleteDentry(_, _, _, _, _)) .WillOnce(Return(MetaStatusCode::OK)); EXPECT_CALL(*metaClient_, GetInode(_, _, _, _)) .WillOnce(Return(MetaStatusCode::OK)); @@ -243,7 +243,7 @@ TEST_F(RecycleCleanerTest, delete_node_test) { // get inode fail { - EXPECT_CALL(*metaClient_, DeleteDentry(_, _, _, _)) + EXPECT_CALL(*metaClient_, DeleteDentry(_, _, _, _, _)) .WillOnce(Return(MetaStatusCode::OK)); EXPECT_CALL(*metaClient_, GetInode(_, _, _, _)) .WillOnce(Return(MetaStatusCode::OK)) @@ -258,7 +258,7 @@ TEST_F(RecycleCleanerTest, delete_node_test) { { Inode inode; inode.set_nlink(0); - EXPECT_CALL(*metaClient_, DeleteDentry(_, _, _, _)) + EXPECT_CALL(*metaClient_, DeleteDentry(_, _, _, _, _)) .WillOnce(Return(MetaStatusCode::OK)); EXPECT_CALL(*metaClient_, GetInode(_, _, _, _)) .WillOnce(Return(MetaStatusCode::OK)) @@ -274,7 +274,7 @@ TEST_F(RecycleCleanerTest, delete_node_test) { { Inode inode; inode.set_nlink(1); - EXPECT_CALL(*metaClient_, DeleteDentry(_, _, _, _)) + EXPECT_CALL(*metaClient_, DeleteDentry(_, _, _, _, _)) .WillOnce(Return(MetaStatusCode::OK)); EXPECT_CALL(*metaClient_, GetInode(_, _, _, _)) .WillOnce(Return(MetaStatusCode::OK)) @@ -292,7 +292,7 @@ TEST_F(RecycleCleanerTest, delete_node_test) { { Inode inode; inode.set_nlink(1); - EXPECT_CALL(*metaClient_, DeleteDentry(_, _, _, _)) + EXPECT_CALL(*metaClient_, DeleteDentry(_, _, _, _, _)) .WillOnce(Return(MetaStatusCode::OK)); EXPECT_CALL(*metaClient_, GetInode(_, _, _, _)) .WillOnce(Return(MetaStatusCode::OK)) @@ -406,7 +406,7 @@ TEST_F(RecycleCleanerTest, scan_recycle_test6) { LOG(INFO) << "create dentry1 " << dentry1.ShortDebugString(); LOG(INFO) << "create dentry2 " << dentry2.ShortDebugString(); - EXPECT_CALL(*metaClient_, ListDentry(_, _, _, _, _, _)) + EXPECT_CALL(*metaClient_, ListDentry(_, _, _, _, _, _, _)) .WillOnce(Return(MetaStatusCode::OK)); ASSERT_FALSE(cleaner_->ScanRecycle()); diff --git a/curvefs/test/metaserver/storage/rocksdb_storage_test.cpp b/curvefs/test/metaserver/storage/rocksdb_storage_test.cpp index 2d4425671e..b2772735f2 100644 --- a/curvefs/test/metaserver/storage/rocksdb_storage_test.cpp +++ b/curvefs/test/metaserver/storage/rocksdb_storage_test.cpp @@ -116,32 +116,32 @@ TEST_F(RocksDBStorageTest, OpenCloseTest) { ASSERT_TRUE(kvStorage_->Close()); - s = kvStorage_->HSet("partition:1", "key1", Value("value1")); + s = kvStorage_->HSet("1:1", "key1", Value("value1")); ASSERT_TRUE(s.IsDBClosed()); - s = kvStorage_->HGet("partition:1", "key1", &value); + s = kvStorage_->HGet("1:1", "key1", &value); ASSERT_TRUE(s.IsDBClosed()); - s = kvStorage_->HDel("partition:1", "key1"); + s = kvStorage_->HDel("1:1", "key1"); ASSERT_TRUE(s.IsDBClosed()); - iterator = kvStorage_->HGetAll("partition:1"); + iterator = kvStorage_->HGetAll("1:1"); ASSERT_EQ(iterator->Status(), -1); - size = kvStorage_->HSize("partition:1"); + size = kvStorage_->HSize("1:1"); ASSERT_EQ(size, 0); - s = kvStorage_->HClear("partition:1"); + s = kvStorage_->HClear("1:1"); ASSERT_TRUE(s.IsDBClosed()); - s = kvStorage_->SSet("partition:1", "key1", Value("value1")); + s = kvStorage_->SSet("3:1", "key1", Value("value1")); ASSERT_TRUE(s.IsDBClosed()); - s = kvStorage_->SGet("partition:1", "key1", &value); + s = kvStorage_->SGet("3:1", "key1", &value); ASSERT_TRUE(s.IsDBClosed()); - s = kvStorage_->SDel("partition:1", "key1"); + s = kvStorage_->SDel("3:1", "key1"); ASSERT_TRUE(s.IsDBClosed()); - iterator = kvStorage_->SGetAll("partition:1"); + iterator = kvStorage_->SGetAll("3:1"); ASSERT_EQ(iterator->Status(), -1); - iterator = kvStorage_->SSeek("partition:1", "key1"); + iterator = kvStorage_->SSeek("3:1", "key1"); ASSERT_EQ(iterator->Status(), -1); - size = kvStorage_->SSize("partition:1"); + size = kvStorage_->SSize("3:1"); ASSERT_EQ(size, 0); - s = kvStorage_->SClear("partition:1"); + s = kvStorage_->SClear("3:1"); ASSERT_TRUE(s.IsDBClosed()); } @@ -250,14 +250,14 @@ TEST_F(RocksDBStorageTest, TestCheckpointAndRecover) { ASSERT_TRUE(kvStorage_->Open()); // put some values - auto s = kvStorage_->SSet("1", "1", Value("1")); - s = kvStorage_->SSet("2", "2", Value("2")); - s = kvStorage_->SSet("3", "3", Value("3")); - s = kvStorage_->SSet("4", "4", Value("4")); - s = kvStorage_->SSet("5", "5", Value("5")); - s = kvStorage_->SSet("6", "6", Value("6")); - s = kvStorage_->SSet("7", "7", Value("7")); - s = kvStorage_->SDel("3", "3"); + auto s = kvStorage_->SSet("1:1", "1", Value("1")); + s = kvStorage_->SSet("1:2", "2", Value("2")); + s = kvStorage_->SSet("1:3", "3", Value("3")); + s = kvStorage_->SSet("1:4", "4", Value("4")); + s = kvStorage_->SSet("1:5", "5", Value("5")); + s = kvStorage_->SSet("1:6", "6", Value("6")); + s = kvStorage_->SSet("1:7", "7", Value("7")); + s = kvStorage_->SDel("1:3", "3"); ASSERT_TRUE(s.ok()) << s.ToString(); @@ -269,28 +269,28 @@ TEST_F(RocksDBStorageTest, TestCheckpointAndRecover) { // get values that checkpoint should have Dentry dummyDentry; - kvStorage_->SGet("1", "1", &dummyDentry); + kvStorage_->SGet("1:1", "1", &dummyDentry); EXPECT_EQ(Value("1"), dummyDentry) << "Expect: " << Value("1").ShortDebugString() << ", actual: " << dummyDentry.ShortDebugString(); - kvStorage_->SGet("2", "2", &dummyDentry); + kvStorage_->SGet("1:2", "2", &dummyDentry); EXPECT_EQ(Value("2"), dummyDentry); // "3" is deleted - s = kvStorage_->SGet("3", "3", &dummyDentry); + s = kvStorage_->SGet("1:3", "3", &dummyDentry); EXPECT_TRUE(s.IsNotFound()) << s.ToString(); - kvStorage_->SGet("4", "4", &dummyDentry); + kvStorage_->SGet("1:4", "4", &dummyDentry); EXPECT_EQ(Value("4"), dummyDentry); - kvStorage_->SGet("5", "5", &dummyDentry); + kvStorage_->SGet("1:5", "5", &dummyDentry); EXPECT_EQ(Value("5"), dummyDentry); - kvStorage_->SGet("6", "6", &dummyDentry); + kvStorage_->SGet("1:6", "6", &dummyDentry); EXPECT_EQ(Value("6"), dummyDentry); - kvStorage_->SGet("7", "7", &dummyDentry); + kvStorage_->SGet("1:7", "7", &dummyDentry); EXPECT_EQ(Value("7"), dummyDentry); } diff --git a/curvefs/test/metaserver/storage/storage_test.cpp b/curvefs/test/metaserver/storage/storage_test.cpp index d47f41a1ee..e897727352 100644 --- a/curvefs/test/metaserver/storage/storage_test.cpp +++ b/curvefs/test/metaserver/storage/storage_test.cpp @@ -783,7 +783,7 @@ void TestMixOperator(std::shared_ptr kvStorage) { // CASE 1: get s = kvStorage->HGet(TableName(1), "key1", &value); ASSERT_TRUE(s.IsNotFound()); - s = kvStorage->SGet(TableName(1), "key1", &value); + s = kvStorage->SGet(TableName(2), "key1", &value); ASSERT_TRUE(s.IsNotFound()); // CASE 2: set @@ -793,25 +793,25 @@ void TestMixOperator(std::shared_ptr kvStorage) { s = kvStorage->HGet(TableName(1), "key1", &value); ASSERT_TRUE(s.ok()); ASSERT_EQ(value, Value("value1")); - s = kvStorage->SGet(TableName(1), "key1", &value); + s = kvStorage->SGet(TableName(2), "key1", &value); ASSERT_TRUE(s.IsNotFound()); // CASE 3: del s = kvStorage->HDel(TableName(1), "key1"); ASSERT_TRUE(s.ok()); - s = kvStorage->SDel(TableName(1), "key1"); + s = kvStorage->SDel(TableName(2), "key1"); ASSERT_TRUE(s.ok()); - s = kvStorage->SSet(TableName(1), "key1", Value("value1")); + s = kvStorage->SSet(TableName(2), "key1", Value("value1")); ASSERT_TRUE(s.ok()); s = kvStorage->HGet(TableName(1), "key1", &value); ASSERT_TRUE(s.IsNotFound()); - s = kvStorage->SGet(TableName(1), "key1", &value); + s = kvStorage->SGet(TableName(2), "key1", &value); ASSERT_TRUE(s.ok()); ASSERT_EQ(value, Value("value1")); // CASE 4: range - iterator = kvStorage->SSeek(TableName(1), "key1"); + iterator = kvStorage->SSeek(TableName(2), "key1"); ASSERT_EQ(iterator->Status(), 0); size = 0; for (iterator->SeekToFirst(); iterator->Valid(); iterator->Next()) { @@ -825,22 +825,22 @@ void TestMixOperator(std::shared_ptr kvStorage) { // CASE 5: clear s = kvStorage->HClear(TableName(1)); ASSERT_TRUE(s.ok()); - s = kvStorage->SClear(TableName(1)); + s = kvStorage->SClear(TableName(2)); ASSERT_TRUE(s.ok()); s = kvStorage->HGet(TableName(1), "key1", &value); ASSERT_TRUE(s.IsNotFound()); - s = kvStorage->SGet(TableName(1), "key1", &value); + s = kvStorage->SGet(TableName(2), "key1", &value); ASSERT_TRUE(s.IsNotFound()); // CASE 6: size s = kvStorage->HSet(TableName(1), "key1", Value("value1")); ASSERT_TRUE(s.ok()); - s = kvStorage->SSet(TableName(1), "key2", Value("value2")); + s = kvStorage->SSet(TableName(2), "key2", Value("value2")); ASSERT_TRUE(s.ok()); ASSERT_EQ(kvStorage->HSize(TableName(1)), 1); - ASSERT_EQ(kvStorage->SSize(TableName(1)), 1); + ASSERT_EQ(kvStorage->SSize(TableName(2)), 1); } void TestTransaction(std::shared_ptr kvStorage) { diff --git a/curvefs/test/metaserver/trash_test.cpp b/curvefs/test/metaserver/trash_test.cpp index 9d6ea82e14..36a4de1eca 100644 --- a/curvefs/test/metaserver/trash_test.cpp +++ b/curvefs/test/metaserver/trash_test.cpp @@ -46,6 +46,9 @@ using ::testing::Invoke; namespace curvefs { namespace metaserver { +DECLARE_uint32(trash_scanPeriodSec); +DECLARE_uint32(trash_expiredAfterSec); + namespace { auto localfs = curve::fs::Ext4FileSystemImpl::getInstance(); } @@ -84,6 +87,7 @@ class TestTrash : public ::testing::Test { } void TearDown() override { + trashManager_->Fini(); inodeStorage_ = nullptr; trashManager_ = nullptr; ASSERT_TRUE(kvStorage_->Close()); @@ -223,8 +227,6 @@ TEST_F(TestTrash, testAdd3ItemAndDelete) { ASSERT_EQ(0, trashManager_->Size()); ASSERT_EQ(inodeStorage_->Size(), 0); - - trashManager_->Fini(); } } // namespace metaserver diff --git a/docs/cn/curvefs_improve_rename_design.md b/docs/cn/curvefs_improve_rename_design.md index c2a33a2b65..5e83cf121a 100644 --- a/docs/cn/curvefs_improve_rename_design.md +++ b/docs/cn/curvefs_improve_rename_design.md @@ -1,267 +1,193 @@ -# rename 接口实现优化 +# Rename 优化方案 -## 背景 +## 背景描述 在目前 CurveFS 的 rename 实现中,我们保证了该操作的原子性,但是仍然存在以下 2 个问题: -* 在多挂载的场景下,我们为了保证 txid 的正确性,在 MDS 中加了一把分布式锁来保证所有的事务都是串行执行,这严重影响了 rename 接口的性能,即事务不能并发执行 -* 同样地,为了保证其他操作接口(如 GetDentry、DeleteDentry 等)能拿到正确版本的 dentry,在这些接口执行前都要去 MDS 获取最新的 txid,对于这些接口来说,多了一次 RPC 请求,降低了整体的元数据性能 +1. 在多挂载的场景下,我们为了保证 txid 的正确性,在 MDS 中加了一把分布式锁来保证同一 FS 上所有的事务都是串行执行,这严重影响了 rename 接口的性能,即事务不能并发执行。 +2. 同样地,为了保证其他操作接口(如 GetDentry、DeleteDentry 等)能拿到正确版本的 dentry,在这些接口执行前都要去 MDS 获取最新的 txid,对于这些接口来说,多了一次 RPC 请求,降低了整体的元数据性能。 -针对以上 2 个问题,调研了 Google Percolator 事务模型以及现有的开源实现,并发现其能很好解决我们以上 2 个已知问题。 - -## 方案调研 - -### Google Percolator - -资料: - -[Large-scale Incremental Processing Using Distributed Transactions and Notifications](https://www.usenix.org/legacy/event/osdi10/tech/full_papers/Peng.pdf) - -[Google Percolator 的事务模型](https://github.com/ngaut/builddatabase/blob/master/percolator/README.md) - -### TiKV 中 percolator 事务实现优化 - -资料: - -[Optimized Percolator](https://tikv.org/deep-dive/distributed-transaction/optimized-percolator/) +针对以上 2 个问题,调研了 [Google Percolator](https://www.usenix.org/legacy/event/osdi10/tech/full_papers/Peng.pdf) 事务模型以及现有的开源实现([TiKV中 Percolator的优化](https://tikv.org/deep-dive/distributed-transaction/optimized-percolator/)),并发现其能很好解决我们以上 2 个已知问题。 ## 方案实现 -我们先阐述以原论文、没有任何优化的情况下如何在我们 CurveFS 中实现 pecolator 事务,然后再逐一将每一个优化点加入其中。 - -### 基础实现 - -基于我们之前的 rename 实现,我们可以比较好理解 percolator 的实现,percolator 主要对每一个 key 都有一把锁来解决写写冲突。percolator 论文中的 data 就是跟我们之前写入 txid 版本的 dentry 是一样的,而往 write 列写入时间戳则跟我们之前往 MDS 提交 txid 是一样的,也是提交版本号。 - -#### 整体实现 - -percalator 也是一个 2PC 的实现方案,分为 Prewrite 和 Commit 阶段: - -* 客户端首先会通过 Tso() 接口从 MDS 获取全局递增的时间戳,作为当前数据的版本号(start_ts),然后对所有涉及到的 key 进行 prewrite 操作,prewrite 包括对 key(会从所有的 key 中选出一个主 key) 加锁已经写入对应版本的数据。这一步与我们目前实现中的 prepare 是一样的,准备最终版本的数据 -* 当 prewrite 成功后,客户端会再次通过 Tso() 接口从 MDS 获取时间戳,作为提交时间戳 (commit_ts),然后首先对主键进行提交版本号,并进行解锁。如果主键成功后,再并行地对所有从键进行解锁。 - -异常处理: +### 整体实现 -percolator 中的异常处理是由下一个事务完成的: - -* 如果发现当前操作的 key 返回 WriteConflict 则表示写冲突了,需要重新 retry -* 而如果发现 key 被锁住,即 KeyIsLocked 状态,客户端需要调用 CheckTxnStatus 接口来获取上一个事务的状态,如果已经提交,则推动从键提交,如果是锁已经操作,则推动所有键 rollback,而如果前一个事务正在进行中,则需要等待。 +percalator 也是一个 2PC 的实现方案,分为 Prewrite 和 Commit 阶段。基于我们之前的 rename 实现,可以比较好理解 percolator 的实现,它主要对每一个 key 都有一把锁来解决写写冲突。percolator 论文中的 data 就是跟我们之前写入带 txid 版本的 dentry 是一样的,而往 write 列写入时间戳则类似我们之前往 MDS 提交 txid,也是提交版本号。 #### 数据结构 -考虑到性能,我们要保证 lock 和 write 这 2 张表永远保存在内存当中,因为这 2 张表中的内容比较小但是读写却很频繁。对于 RocksDB 来说,我们要给这 2 张表单独配置 column failmy,保证其独立性。 - -| table | key | value | 说明 | -|-----------------|---------|--------------|:----------| -| data | key:timestamp | | 实际的数据,这里保存多版本数据 | -| lock | key | struct lock { primary、timestamp、ttl } | 保护当前 key 的锁 | -| write | key:timestamp | struct write { timestamp、kind } | 记录已提交的版本号 | +在介绍整体流程之前,先约定必要的数据结构,percolator 事务模型中涉及到的三个表:data、lock、write,lock 和 write表内容比较小,但访问频繁,在 Rocksdb 中为其单独配置 column family,保证其独立性。 -**struct lock** +| Table name | Key | Value | 说明 | +| ---------- | --------------------------- | --------------------------------------------------- | -------------------------------- | +| data | dentryKey | struct DentryVec {dentrys} | 实际的dentry数据,保存多版本数据 | +| lock | dentryKey | struct TxLock {primaryKey, startTs, timestamp, ttl} | 包含当前key的锁 | +| write | dentryKey/commitTs或startTs | struct TxWrite {startTs, writeKind} | 记录已提交或已回滚的版本号 | -``` -struct Lock { - std::string primary; // 锁的主键相关信息 - uint64_t timestamp; // 加锁的时间戳 - uint64_t ttl; // 锁的 TTL。我们通过这个来判断锁有没有超时 +```protobuf +message DentryVec { + repeated Dentry dentrys = 1; // 记录多版本dentry信息,版本信息复用之前的txid字段 } -``` -**struct write** - -``` -struct Write { - uint64_t timestamp; // 最新版本数据的时间戳。我们通过这个时间戳可以在 data 表中找到相应版本的数据 - char kind; // 写入的类型,详见 WriteKind +message TxLock { + required string primaryKey = 1; // 事务的主键,rename中涉及两个dentry,默认以src dentry为primaryKey + required uint64 startTs = 2; // 事务开始序列号 + required uint64 timestamp = 3; // 事务开始物理时间 + optional uint32 index = 4; + optional int32 ttl = 5; } -``` -**事务状态码** +enum TxWriteKind { + Commit = 1; + Rollback = 2; +} -``` -enum class TxStatus { - WriteConflict, // 事务写写冲突。为解决该冲突,我们的处理原则是只保证一个事务成功 - KeyIsLocked, // 当前事务涉及的键已被锁住。 - OK, // 成功 -}; +message TxWrite { + required uint64 startTs = 1; + required TxWriteKind kind = 2; +} ``` -**写入类型** +data 表的清理:复用现有逻辑,在insert、delete 等 dentry 操作时会进行压缩操作,正常情况下最多存在两个版本的 dentry 数据。 -``` -enum class WriteKind { - Put, - Delete, - Rollback, -}; -``` - -以上的数据结构中,各数据结构的主要目的如下: +lock 表的清理:每个 key 最多只会有一条记录,事务提交或回滚都会清理对应记录。 -* timestamp:时间戳主要为了确定事务的顺序,为事务冲突提供判断依据。另一个作用是作为锁 TTL 的开始时间戳 -* Lock:主要是为了保证对某一个 key 来说,只能有一个写操作,以此来解决写写冲突 -* Lock.ttl:锁的 TTL 主要用来解决当客户端挂掉或 hang 住,锁仍然遗留在相应的 key 上而导致其他事务无法执行,或 -* Lock.primary:事务/锁的主键。在我们的实现中,我们可能涉及到多个 dentry,我们会选任意一个 -* Write:这个作用跟我们当前事务实现中的 txId 一样,用来保存当前 dentry 的版本号,对外提供统一的视图,通过版本号找到最新的 dentry。而在 Write 表中写入 timeStamp 就跟我们现在实现的往 MDS 提交 txId 一样。 +write 表的清理:key 的每次提交或回滚都会在 write 表中记录一条记录,如果不清理则会越来越多,清理规则是对应 writeKind==TxWriteKind::Commit的记录每个 key 只会保留最新的一条记录,对应 writeKind==TxWriteKind::Rollback 的记录都保留,便于后续判断事务状态,且发生回滚的事务本身极少,不会对容量造成负担。 -#### 相关函数 +#### 整体流程 -**一些约定** +1. 客户端首先会通过 Tso() 接口从 MDS 获取全局递增的事务序列号 startTs 和物理时间戳 timestamp。startTs 作为当前数据的版本号,timestamp用于验证 lock 的超时。 +2. 对涉及到的 dentry 进行 prewrite 操作,会任意选择一个 key 作为 primaryKey(默认选择 src dentry),先 prewrite primaryKey 再 prewrite 其他 key,原因见 write 表清理逻辑。 +3. 所有 key prewrite 完成后,客户端会再次通过 Tso() 接口从 MDS 获取 commitTs。 +4. 对 primaryKey 进行 commit 操作,成功后,对其他 key 并行 commit。 -* 对于每一个函数,我们会先阐述正常的处理流程,之后会阐述每个步骤出现异常、不符合预期时的异常处理流程 -* 对于 dentry 来说,我们以 dentry_key 作为实际存储 dentry 的键,这个键的编码格式如下:fsId:parentInodeId:name +异常处理:percolator 中的异常处理是延迟到下一个事务解决的 -**Tso** +1. prewrite 时从 write 表中获取该 key 最新的 ts,如果 ts >= startTs 则说明期间已有新的事务发生,返回 WriteConflict 错误。 +2. prewrite 时发现该 key 已经被 lock 住了,则返回 TxkeyLocked,客户端需要调用 CheckTxStatus 接口去主键获取上一个事务的状态,如果已经提交,则推动从键提交,如已回滚或锁已经超时,则推动所有键 rollback,而如果前一个事务正在进行中,则需要等待。 -客户端开启事务后,需要通过 Tso() 接口向 MDS 端获取一个全局递增的时间戳(该时间戳的作用类似于我们目前实现中的 txid)用于 prewrite,而在提交事务时同样需要通过 Tso() 接口获取一个时间戳用于 commit。 +#### 相关函数说明 -时间戳是一个 64 位整数,其由物理时间和逻辑时间两部分组成合成,其编码规则如下: +##### Tso -`TSO = 物理时间 + 序列号` +客户端开启事务后,需要通过 Tso 接口向 MDS 端获取一个全局递增的事务序列号 sn 和 一个物理时间 timestamp,sn 标识一次事务,同时表示该数据的版本,timestamp 记录于 lock 表中的 TxLock 中,便于在 client 异常时导致当前事务挂起,其他事务在 CheckTxStatus 时判断超时。 -**Prewrite** +```protobuf +message TsoRequest {} +message TsoResponse { + required FSStatusCode statusCode = 1; + optional uint64 sn = 2; + optional uint64 timestamp = 3; +} ``` -struct PrewriteRequest { - std::string primaryKey; // 事务/锁的主键 - uint64_t startTimestamp; // 开始时间戳。该时间戳从调用 Tso 接口从 MDS 端获取 - uint64_t lockTTL; // 锁的 TTL。用来判断当前 key 的锁是否超时 - ... // 涉及修改的 dentry 相关信息 -}; - -struct PrewriteResponse { - Status status; -}; -``` - -客户端通过 Tso() 接口获取到时间戳后,会调用 Prewrite() 接口依次对事务到涉及到所有 dentry 依次做 prewrite,MetaServer 端在接收到该请求后,会做如下处理: -* 获取当前请求的 dentry 在 write 表中最近一次写入的信息,如果该 dentry 最新一次写入的 Write.timestamp >= PrewriteRequest.startTimestamp,则表明在该事务开始后有其他事务修改了这个 key,返回 TXStatus::WriteConfict; -* 检查当前请求的 dentry 在 lock 表中是否有锁,如果有锁,则返回 status::KeyIsLocked; -* 若前面检查都成功,则调用 RocksDB 的事务接口,执行以下 2 个操作: - * 在 lock 表中对该 key 进行加锁。key="dentry", value=Lock{} - * 在 data 表中写入当前版本的 dentry。key="dentry:PrewriteRequest.startTimeStamp", value=mut_dentry -* 若以上事务成功,则返回 status::OK, 否则返回 status::Error +##### PrewriteRenameTx -注意:在客户端对所有涉及的 dentry 依次加锁的过程中,只要有一个 dentry 不是返回 status::OK,则表明 Prewrite 阶段失败。而当其中一个 dentry 收到 status::KeyIsLocked 的相应时,客户端需要调用 CheckTxnStatus() 接口来判断当前锁的状态,并根据锁的状态来执行相关的操作。而关于锁的状态,我们可以分 +首先 prewrite primaryKey,成功后再 prewrite 其他 key,如果rename 涉及的 dentry 位于同一个 partition 上,则可以合并成一次请求。metaserver 收到该请求进行如下处理: -**Commit** - -``` -struct CommitRequest { - uint64_t startTimestamp; // 该事务开始时间戳 - uint64_t commitTimestamp; // 该事务提交时间戳 +1. 在 write 表中获取该 dentryKey 最新的一次写入的 ts,如果 ts >= PrewriteRenameTxRequest.txLock.startts() 则返回错误码 MetaStatusCode::TX_WRITE_CONFLICT。 +2. 在 lock 表中检查该 dentryKey 是否存在,如果存在,则返回错误码MetaStatusCode::TX_KEY_LOCKED。 +3. 如果以上检查都通过,则通过 Rocksdb 事务接口完成如下两个操作: + 1. 在 lock 表中插入记录: key=dentryKey, value=TxLock + 2. 在 data 表中写入当前版本的 dentry -}; - -struct CommitResponse { +```protobuf +message PrewriteRenameTxRequest { + required uint32 poolId = 1; + required uint32 copysetId = 2; + required uint32 partitionId = 3; + repeated Dentry dentrys = 4; + required TxLock txLock = 5; +} -}; +message PrewriteRenameTxResponse { + required MetaStatusCode statusCode = 1; + repeated Dentry dentrys = 2; + optional TxLock txLock = 3; // 如果返回 MetaStatusCode::TX_KEY_LOCKED,则该字段为前一个事务的 txlock 信息 + optional uint64 appliedIndex = 4; +} ``` -客户端在 prewrite 阶段成功后,就会进入提交事务阶段。客户端首先通过 Tso() 接口获取一个时间戳作为事务提交时间戳 commit_ts,再去该事务主键所在 MetaServer 提交事务,如果主键提交事务成功,则代表整个事务已经成功(这个步骤类似于我们现在的 CommitTxId),之后会并行去所有次键提交事务。MetaServer 在接收到该请求后,会做如下处理: +##### CheckTxStatus -* 以 dentry_key 为键在 lock 表中查找当前 dentry 是都有锁,如果无锁则代表当前 dentry 已被提交,直接返回 Status::OK -* 如果有锁,则判断上锁的时间戳是否等于事务提交的时间戳,即 Lock.timestamp == CommitRequest.startTimestamp -* 若前面检查都成功,则调用 RocksDB 的事务接口,执行以下 2 个操作: - * 在 write 表中写入最新版本的 dentry 的时间戳,即更新该 dentry 的版本号。KEY=dentry_key:CommitRequest.commitTimestamp,VALUE=Write{ timestamp=CommitRequest.startTimestamp }。这一步跟我们目前的 CommitTxId 是一样的,主要为了更新版本号。 - * 在 lock 表中删除以 dentry_key 为键的锁 -* 失败见以下 +在 prewrite 过程如果返回 MetaStatusCode::TX_KEY_LOCKED,则表示事务冲突,客户端需要其检查前一个事务的状态,依据状态再去处理 locked key。metaserver 收到该请求进行如下处理,通过 Rocksdb 事务接口完成: -Commit 接口用于提交事务,客户端会先对事务中的主键进行提交,成功后并发对所有次键进行提交。如果提交事务失败,客户端会通过 CheckTxnStatus 接口获取锁的状态,再进行下一步动作。 +1. 在 lock 表中获取 primaryKey 对应的 TxLock。如果存在,如果 CheckTxStatusRequest.curTimestamp > txLock.startTs + txLock.ttl,则表明事务已超时,返回错误码 MetaStatusCode::TX_TIMEOUT,否则说明事务正在进行中,返回错误码 MetaStatusCode::TX_TX_INPROGRESS。 +2. 如果对应 TxLock 不存在,则在 write 表中用 key=CheckTxStatusRequest.startTs 进行查找,如果找到,并且 TxWrite.kind=TxWriteKind::rollback 则说明该事务已经回滚,返回错误码 MetaStatusCode::TX_ROLLBACKED;如果没找到,则说明事务已提交,返回错误码 MetaStatusCode::TX_COMMITTED。 -**Rollback** - -``` -struct RollbackRequest { - uint64_t StartTimestamp; -}; - -struct RollbackResponse { - TXStatus Status; -}; -``` - -* 如果当前 key 的 write 列已经写入 Rollback 的数据,则表明已经 rollback 过了,直接跳过 -* 而 write 列中如果不是 rollback,则直接 abort 当前事务 -* 若前面检查都成功,则调用 RocksDB 的事务接口,执行以下 3 个操作: - * 在 lock 表中删除以 dentry.key 为键的锁 - * 在 data 表中删除之前 prewrite 写入以 dentry.key+RollbackRequest.startTimestamp 为键的数据 - * 在 write 表中写入以下键值对 - * KEY: dentry.key + RollbackRequest.startTimestamp - * VALUE: Write{ timestamp=RollbackRequest.startTimestamp, kind=WriteKind::Rollback } - -**CheckTxnStatus** +```protobuf +message CheckTxStatusRequest { + required uint32 poolId = 1; + required uint32 copysetId = 2; + required uint32 partitionId = 3; + required string primaryKey = 4; + required uint64 startTs = 5; + required uint64 curTimestamp = 6; +} -``` -struct CheckTxnStatusRequest { - std::string primaryKey; - uint64_t lockTimestamp; - uint64_t currentTimestamp; -}; - -struct CheckTxnStatusResponse { -}; +message CheckTxStatusResponse { + required MetaStatusCode statusCode = 1; + optional uint64 appliedIndex = 2; +} ``` -当客户端在执行某事务时,发现与另一事务(我们称之为前事务)冲突时,即 prewrite 阶段接收到 **TxStatus::LockIsKey** 响应时,客户端需要去前事务的主键所在 MetaServer 获取前事务的状态,并根据前事务的状态来做相应的操作: +##### ResolveTxLock -* roll-forward:表明前事务已提交,我们也需要推动这个冲突的 key 去提交 -* roll-back:表明前事务失败、rollback、超时或 hang 住,我们需要推动这个冲突的 key 去回归 -* wait: 表明前事务正在进行中,我们可等待一段时间后再进行 retry。**特别需要注意**的是,重新 retry 开启的事务都需要重新通过 Tso() 获取时间戳并重头开始 +CheckTxStatus 完成后,需要根据返回的事务状态进行处理: -我们用一个 { primary, start_ts, commit_ts } 三元组来描述一个事务,以下的表格是 primary key 在事务中所有状态的列举,我们正是根据这个状态来得知事务处于什么状态: +MetaStatusCode::TX_COMMITTED:rollforward -| data@start_ts | lock? | lock is TTL? | write@start_ts | write@commit_ts | 说明 | 客户端需执行动作 | -|-----------------|---------|------|-------|-------|:----------|-------| -| | | | | timestamp = start_ts | 这个状态说明该事务已被成功提交 | roll-forward | -| | | | kind = rollback | | 这个状态说明事务已被 rollback | roll-back | -| | | | | |这个状态说明在此次事务中,该 primary key 没有被成功 prewrite | roll-back | -| <新版本数据> | Y | Y | | | 这个状态说明事务已超时。有可能是客户端挂了或 hang 住了 | roll-back | -| <新版本数据> | Y | N | | | 这个状态说明 primary key 已被成功 prewrite,事务正在进行当中 | wait | +MetaStatusCode::TX_TIMEOUT; MetaStatusCode::TX_ROLLBACKED : rollback - 之所以我们能根据主键的各个表中的值就能判断事务的状态,主要是因为我们操作的时候以锁为主要元素,不管是 prewrite、commit、rollback 都是原子操作,而且事务提交的成功是以主键是否成功提交为依据,所以该主键的状态就在由以上 5 个状态组成的状态机中转变。 +MetaStatusCode::TX_TX_INPROGRESS: retry -<补充状态机转换图> +metaserver 在收到该请求时进行如下处理,通过 Rocksdb 事务接口完成: -**ResolveLock** +1. 在 lock 表中查找该 key 是否仍然处于 locked 的状态,如果不存在,则直接返回成功;如果存在,判断 TxLock.startTs == ResolveTxLockRequest.startTs,如果不成立,则返回 MetaStatusCode::TX_MISMATCH。 +2. 如上对 lock 的检查通过后,如果 ResolveTxLockRequest.commitTs > 0,表示需要 rollforward:删除该 key 的 lock 记录;在 write 表中插入该版本的 dentry。如果 ResolveTxLockRequest.commitTs==0,表示需要 rollback:删除该 key 的 lock 记录;在 data 表中删除对应版本的 dentry; 在 write 表中写入回退的记录,key=dentryKey/startTs, value=TxWrite{startTs, TxWriteKind::Rollback}。 -``` -struct ResolveLockRequest { - uint64_t commitTimestamp; -}; +```protobuf +message ResolveTxLockRequest { + required uint32 poolId = 1; + required uint32 copysetId = 2; + required uint32 partitionId = 3; + required Dentry dentry = 4; + required uint64 startTs = 5; // 待处理事务标识 + required uint64 commitTs = 6; // commitTs > 0 表示rollward, 否则 rollback +} -struct CheckTxnStatusResponse { -}; +message ResolveTxLockResponse { + required MetaStatusCode statusCode = 1; + optional uint64 appliedIndex = 2; +} ``` -RevolveLock 接口比较简单,主要是根据当前的请求中是否有 commitTimestamp 字段来决定是 commit 事务,还是 rollback 事务。这个函数是一个接口的聚合。 - -**Get** - -Get 接口主要用来描述 GetDentry/DeleteDentry 等非 rename 事务怎么获取当前 dentry。如果当前键没有锁,则直接获取最近 write 列对应版本号的数据,如果有所,则通过 CheckTxnStatus 获取前一个事务的状态,并推动事务 commit 还是 rollback 后再进行 Get 等操作。 +##### CommitTx -#### FAQ +在没有异常情况下,prewrite 完成后会进行 commit 操作,先进行 primaryKey 的 commit,再同时 commit 其他key,只要 promaryKey commit 成功就代表了该事务成功。如果rename 涉及的 dentry 位于同一个 partition 上,则可以合并成一次请求,metaserver 收到该请求会进行如下处理: -1、如果事务 A prewrite 成功,但是此时事务 B 发现与事务 A 存在冲突,将 A 的主键解锁了。这时候事务 A 再去提交事务,能否成功? -<这种情况是否会出现?> +1. 在 lock 表中检查该 key 是否存在,如果不存在,则表示该事务已结束(commit 或 rollback),直接返回。 +2. 判断 txLock.startTs == CommitTxRequest.startTs ,如果不等则返回错误码 MetaStatusCode::TX_MISMATCH。 +3. 在 write 表中更新 dentry 版本,key=dentryKey/commitTs,value=TxWrite{startTs, TxWriteKind::Commit};删除 lock 表中对应 key 的记录。 -### 优化点 1:并发 Prewrite - -论文中的 prewrite 是序列进行的,而我们可以推动所有的键并发进行,提高性能 +```protobuf +message CommitTxRequest { + required uint32 poolId = 1; + required uint32 copysetId = 2; + required uint32 partitionId = 3; + repeated Dentry dentrys = 4; + required uint64 startTs = 5; + required uint64 commitTs = 6; +} -## 优化点 2:去除每次读取操作获取时间戳的流程 +message CommitTxResponse { + required MetaStatusCode statusCode = 1; + optional uint64 appliedIndex = 2; +} +``` -论文中的每一个获取也需要获取一个 Tso,这对于我们的实现来说会增加一个 RPC 时间,而其实这个 Tso 是多余的,我们只需要在无锁获取当前 key 的最新数据即可,而有锁的话则需要推动前一个事务。 +##### 其他 dentry 操作 -``` -Point Read Without Timestamp -Timestamps are critical to providing isolation for transactions. For every transaction, -we allocate a unique start_ts for it, and ensures transaction T can only see -the data committed before T’s start_ts. -But if transaction T does nothing but reads a single key, is it really necessary to allocate it a start_ts? -The answer is no. We can simply read the newest version directly, -because it’s equivalent to reading with start_ts which is exactly the instant when the key is read. - It’s even ok to read a locked key, because it’s equivalent to reading with -the start_ts allocated before the lock’s start_ts. -``` +CreateDentry、GetDentry、ListDentry、DeleteDentry 等非 rename 事务操作,在操作 dentry 时,如果对应 key 没有被 lock,则直接操作 write 表该 key 最新提交版本在 data 表中的数据即可;如果有锁,则需要通过 CheckTxStatus 来获取事务的状态,并进一步推动事务的完成,之后再处理对应 dentry。所以在这几种操作的 response 中增加 optional TxLock 字段,用于在有锁情况下,返回事务对应的 lock 信息。 diff --git a/src/client/mds_client.cpp b/src/client/mds_client.cpp index 835b18b566..5b857c229c 100644 --- a/src/client/mds_client.cpp +++ b/src/client/mds_client.cpp @@ -107,23 +107,15 @@ int RPCExcutorRetryPolicy::PreProcessBeforeRetry(int status, bool retryUnlimit, bool rpcTimeout = false; bool needChangeMDS = false; - // If retryUnlimit is set, sleep a long time to retry no matter what the - // error it is. - if (retryUnlimit) { - if (++(*normalRetryCount) > - retryOpt_.normalRetryTimesBeforeTriggerWait) { - bthread_usleep(retryOpt_.waitSleepMs * 1000); - } - - // 1. 访问存在的IP地址,但无人监听:ECONNREFUSED - // 2. 正常发送RPC情况下,对端进程挂掉了:EHOSTDOWN - // 3. 对端server调用了Stop:ELOGOFF - // 4. 对端链接已关闭:ECONNRESET - // 5. 在一个mds节点上rpc失败超过限定次数 - // 在这几种场景下,主动切换mds。 - } else if (status == -EHOSTDOWN || status == -ECONNRESET || - status == -ECONNREFUSED || status == -brpc::ELOGOFF || - *curMDSRetryCount >= retryOpt_.maxFailedTimesBeforeChangeAddr) { + // 1. 访问存在的IP地址,但无人监听:ECONNREFUSED + // 2. 正常发送RPC情况下,对端进程挂掉了:EHOSTDOWN + // 3. 对端server调用了Stop:ELOGOFF + // 4. 对端链接已关闭:ECONNRESET + // 5. 在一个mds节点上rpc失败超过限定次数 + // 在这几种场景下,主动切换mds。 + if (status == -EHOSTDOWN || status == -ECONNRESET || + status == -ECONNREFUSED || status == -brpc::ELOGOFF || + *curMDSRetryCount >= retryOpt_.maxFailedTimesBeforeChangeAddr) { needChangeMDS = true; // 在开启健康检查的情况下,在底层tcp连接失败时 @@ -140,6 +132,13 @@ int RPCExcutorRetryPolicy::PreProcessBeforeRetry(int status, bool retryUnlimit, *timeOutMS *= 2; *timeOutMS = std::min(*timeOutMS, retryOpt_.maxRPCTimeoutMS); *timeOutMS = std::max(*timeOutMS, retryOpt_.rpcTimeoutMs); + // If retryUnlimit is set, sleep a long time to retry no matter what the + // error it is. + } else if (retryUnlimit) { + if (++(*normalRetryCount) > + retryOpt_.normalRetryTimesBeforeTriggerWait) { + bthread_usleep(retryOpt_.waitSleepMs * 1000); + } } // 获取下一次需要重试的mds索引 diff --git a/src/common/encode.h b/src/common/encode.h index 0540ee0193..ab28d11321 100644 --- a/src/common/encode.h +++ b/src/common/encode.h @@ -54,6 +54,11 @@ inline void EncodeBigEndian_uint32(char* buf, uint32_t value) { buf[3] = value & 0xff; } +inline uint32_t DecodeBigEndian_uint32(const char* buf) { + return (uint32_t(buf[0]) << 24) | (uint32_t(buf[1]) << 16) | + (uint32_t(buf[2]) << 8) | uint32_t(buf[3]); +} + } // namespace common } // namespace curve diff --git a/src/common/timeutility.h b/src/common/timeutility.h index 1ba3483d34..4bdd91e6b4 100644 --- a/src/common/timeutility.h +++ b/src/common/timeutility.h @@ -57,6 +57,12 @@ class TimeUtility { return localtime(&now)->tm_hour; } + static uint64_t CLockRealTimeMs() { + struct timespec now; + clock_gettime(CLOCK_REALTIME, &now); + return now.tv_sec * 1000L + now.tv_nsec / 1000000; + } + // 时间戳转成标准时间输出在standard里面,时间戳单位为秒 static inline void TimeStampToStandard(time_t timeStamp, std::string* standard) {