Skip to content

Commit

Permalink
curvefs: support space deallocate for curvebs volume as backend
Browse files Browse the repository at this point in the history
Signed-off-by: ilixiaocui <ilixiaocui@163.com>
  • Loading branch information
ilixiaocui committed Mar 22, 2023
1 parent c1bd0f1 commit be3376e
Show file tree
Hide file tree
Showing 22 changed files with 606 additions and 7 deletions.
7 changes: 7 additions & 0 deletions curvefs/proto/common.proto
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,13 @@ enum BitmapLocation {
AtEnd = 2;
}

message EmptyMsg {}

message BlockGroupID {
required uint64 fsId = 1;
required uint64 offset = 2;
}

// When creating fs, `volumeSize` and `extendAlignment` are fetched from the bs cluster
message Volume {
optional uint64 volumeSize = 1;
Expand Down
21 changes: 21 additions & 0 deletions curvefs/proto/metaserver.proto
Original file line number Diff line number Diff line change
Expand Up @@ -167,6 +167,24 @@ enum FsFileType {
TYPE_S3 = 4;
};

message DeallocatableBlockGroup {
required uint64 volumeOffset = 1;
required uint64 deallocatableSize = 2;
repeated uint64 inodeIdlist = 3;
repeated uint64 inodeIdlistAdd = 4;
}

message UpdateDeallocatableBlockGroupRequest {
required uint32 poolId = 1;
required uint32 copysetId = 2;
required uint64 fsId = 3;
repeated DeallocatableBlockGroup needUpdate = 4;
}

message UpdateDeallocatableBlockGroupResponse {
required MetaStatusCode statusCode = 1;
}

message VolumeExtent {
required uint64 fsOffset = 1;
required uint64 volumeOffset = 2;
Expand Down Expand Up @@ -508,4 +526,7 @@ service MetaServerService {
// volume extent interface
rpc GetVolumeExtent(GetVolumeExtentRequest) returns (GetVolumeExtentResponse);
rpc UpdateVolumeExtent(UpdateVolumeExtentRequest) returns (UpdateVolumeExtentResponse);

// block group with deallocatable inode list interface
rpc UpdateDeallocatableBlockGroup(UpdateDeallocatableBlockGroupRequest) returns (UpdateDeallocatableBlockGroupResponse);
}
9 changes: 9 additions & 0 deletions curvefs/src/client/rpcclient/metaserver_client.h
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ 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 S3ChunkInfoMap = google::protobuf::Map<uint64_t, S3ChunkInfoList>;
Expand Down Expand Up @@ -168,6 +169,10 @@ class MetaServerClient {
uint64_t inodeId,
bool streaming,
VolumeExtentList *extents) = 0;

virtual MetaStatusCode UpdateDeallocatableBlockGroup(
uint32_t fsId,
std::map<uint64_t, DeallocatableBlockGroup> *statistic) = 0;
};

class MetaServerClientImpl : public MetaServerClient {
Expand Down Expand Up @@ -274,6 +279,10 @@ class MetaServerClientImpl : public MetaServerClient {
bool streaming,
VolumeExtentList *extents) override;

MetaStatusCode UpdateDeallocatableBlockGroup(
uint32_t fsId,
std::map<uint64_t, DeallocatableBlockGroup> *statistic) override;

private:
MetaStatusCode UpdateInode(const UpdateInodeRequest &request,
bool internal = false);
Expand Down
4 changes: 4 additions & 0 deletions curvefs/src/mds/heartbeat/heartbeat_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,10 @@ class HeartbeatManager {

void UpdateMetaServerSpace(const MetaServerHeartbeatRequest &request);

// According to the free space of the BlockGroup in the heartbeat, update
// the statistics of the current free space
void UpdataBlockGroupSpaceStatis(const BlockGroupStatisticInfo &info);

private:
// Dependencies of heartbeat
std::shared_ptr<Topology> topology_;
Expand Down
10 changes: 9 additions & 1 deletion curvefs/src/metaserver/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,10 @@ cc_library(
["storage/*.cpp"],
) + glob(
["streaming/*.cpp"],
) + glob(
["space/*.cpp"],
) + glob(
["mds/*.cpp"],
),
hdrs = glob(
["*.h"],
Expand All @@ -38,6 +42,10 @@ cc_library(
["storage/*.h"],
) + glob(
["streaming/*.h"],
) + glob(
["space/*.h"],
) + glob(
["mds/*.h"],
),
copts = CURVE_DEFAULT_COPTS,
visibility = ["//visibility:public"],
Expand All @@ -61,7 +69,7 @@ cc_library(
"@com_google_absl//absl/container:btree",
"@com_google_absl//absl/memory",
"@com_google_absl//absl/utility",
"//external:gtest",
"//external:glog",
],
)

Expand Down
9 changes: 8 additions & 1 deletion curvefs/src/metaserver/copyset/meta_operator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -193,7 +193,8 @@ OPERATOR_ON_APPLY(CreateManageInode);
OPERATOR_ON_APPLY(CreatePartition);
OPERATOR_ON_APPLY(DeletePartition);
OPERATOR_ON_APPLY(PrepareRenameTx);
OPERATOR_ON_APPLY(UpdateVolumeExtent);;
OPERATOR_ON_APPLY(UpdateVolumeExtent);
OPERATOR_ON_APPLY(UpdateUpdateDeallocatableBlockGroup);

#undef OPERATOR_ON_APPLY

Expand Down Expand Up @@ -320,6 +321,7 @@ OPERATOR_ON_APPLY_FROM_LOG(CreatePartition);
OPERATOR_ON_APPLY_FROM_LOG(DeletePartition);
OPERATOR_ON_APPLY_FROM_LOG(PrepareRenameTx);
OPERATOR_ON_APPLY_FROM_LOG(UpdateVolumeExtent);
OPERATOR_ON_APPLY_FROM_LOG(UpdateDeallocatableBlockGroup);

#undef OPERATOR_ON_APPLY_FROM_LOG

Expand Down Expand Up @@ -377,6 +379,7 @@ OPERATOR_REDIRECT(DeletePartition);
OPERATOR_REDIRECT(PrepareRenameTx);
OPERATOR_REDIRECT(GetVolumeExtent);
OPERATOR_REDIRECT(UpdateVolumeExtent);
OPERATOR_REDIRECT(UpdateDeallocatableBlockGroup);

#undef OPERATOR_REDIRECT

Expand All @@ -403,6 +406,7 @@ OPERATOR_ON_FAILED(DeletePartition);
OPERATOR_ON_FAILED(PrepareRenameTx);
OPERATOR_ON_FAILED(GetVolumeExtent);
OPERATOR_ON_FAILED(UpdateVolumeExtent);
OPERATOR_ON_FAILED(UpdateDeallocatableBlockGroup);

#undef OPERATOR_ON_FAILED

Expand All @@ -428,6 +432,8 @@ OPERATOR_HASH_CODE(PrepareRenameTx);
OPERATOR_HASH_CODE(DeletePartition);
OPERATOR_HASH_CODE(GetVolumeExtent);
OPERATOR_HASH_CODE(UpdateVolumeExtent);
OPERATOR_HASH_CODE(UpdateDeallocatableBlockGroup);


#undef OPERATOR_HASH_CODE

Expand Down Expand Up @@ -465,6 +471,7 @@ OPERATOR_TYPE(CreatePartition);
OPERATOR_TYPE(DeletePartition);
OPERATOR_TYPE(GetVolumeExtent);
OPERATOR_TYPE(UpdateVolumeExtent);
OPERATOR_TYPE(UpdateDeallocatableBlockGroup);

#undef OPERATOR_TYPE

Expand Down
19 changes: 19 additions & 0 deletions curvefs/src/metaserver/copyset/meta_operator.h
Original file line number Diff line number Diff line change
Expand Up @@ -537,6 +537,25 @@ class UpdateVolumeExtentOperator : public MetaOperator {
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(uint64_t startTimeUs) override;

uint64_t HashCode() const override;

OperatorType GetOperatorType() const override;

private:
void Redirect() override;

void OnFailed(MetaStatusCode code) override;
};

} // namespace copyset
} // namespace metaserver
} // namespace curvefs
Expand Down
3 changes: 3 additions & 0 deletions curvefs/src/metaserver/heartbeat.h
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,9 @@ class Heartbeat {
bool GetMetaserverSpaceStatus(MetaServerSpaceStatus* status,
uint64_t ncopysets);

// Handle heartbeat and send recyclable BlockGroup requests
void DeallocateBolckGroup(const HeartbeatResponse &response);

private:
friend class HeartbeatTest;

Expand Down
1 change: 1 addition & 0 deletions curvefs/src/metaserver/inode_manager.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -352,6 +352,7 @@ MetaStatusCode InodeManager::UpdateInode(const UpdateInodeRequest& request) {
}

if (needAddTrash) {
// TODO: inode list need add to inode wait to delete list
trash_->Add(old.fsid(), old.inodeid(), old.dtime());
--(*type2InodeNum_)[old.type()];
}
Expand Down
46 changes: 42 additions & 4 deletions curvefs/src/metaserver/inode_storage.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
#include "src/common/concurrent/rw_lock.h"
#include "src/common/string_util.h"
#include "curvefs/proto/metaserver.pb.h"
#include "curvefs/proto/common.pb.h"
#include "curvefs/src/metaserver/storage/status.h"
#include "curvefs/src/metaserver/inode_storage.h"
#include "curvefs/src/metaserver/storage/converter.h"
Expand All @@ -39,6 +40,7 @@ namespace metaserver {
using ::curve::common::ReadLockGuard;
using ::curve::common::WriteLockGuard;
using ::curve::common::StringStartWith;
using ::curvefs::common::EmptyMsg;
using ::curvefs::metaserver::storage::Status;
using ::curvefs::metaserver::storage::KVStorage;
using ::curvefs::metaserver::storage::Key4S3ChunkInfoList;
Expand Down Expand Up @@ -181,18 +183,54 @@ MetaStatusCode InodeStorage::Delete(const Key4Inode& key) {
return MetaStatusCode::STORAGE_INTERNAL_ERROR;
}

MetaStatusCode InodeStorage::Update(const Inode& inode) {
MetaStatusCode InodeStorage::Update(const Inode &inode, bool inodeDeallocate) {
WriteLockGuard lg(rwLock_);
Key4Inode key(inode.fsid(), inode.inodeid());
std::string skey = conv_.SerializeToString(key);

Status s = kvStorage_->HSet(table4Inode_, skey, inode);
// only update inodes
if (!inodeDeallocate) {
Status s = kvStorage_->HSet(table4Inode_, skey, inode);
if (s.ok()) {
return MetaStatusCode::OK;
}
return MetaStatusCode::STORAGE_INTERNAL_ERROR;
}

// update inode and update deallocatable inode list
// std::string value;
EmptyMsg value;
auto txn = kvStorage_->BeginTransaction();
if (nullptr == txn) {
return MetaStatusCode::STORAGE_INTERNAL_ERROR;
}

std::string step = "update inode " + key.SerializeToString();

Status s = txn->HSet(table4Inode_, skey, inode);
if (s.ok()) {
return MetaStatusCode::OK;
s = txn->HSet(table4DeallocatableInode_, skey, value);
step = "add inode " + key.SerializeToString() +
" to inode deallocatable list";
}
return MetaStatusCode::STORAGE_INTERNAL_ERROR;

if (!s.ok()) {
LOG(ERROR) << "txn is failed in " << step;
if (!txn->Rollback().ok()) {
LOG(ERROR) << "rollback transaction failed, inode="
<< key.SerializeToString();
return MetaStatusCode::STORAGE_INTERNAL_ERROR;
}
} else if (!txn->Commit().ok()) {
LOG(ERROR) << "commit transaction failed, inode="
<< key.SerializeToString();
return MetaStatusCode::STORAGE_INTERNAL_ERROR;
}

return MetaStatusCode::OK;
}


std::shared_ptr<Iterator> InodeStorage::GetAllInode() {
ReadLockGuard lg(rwLock_);
std::string sprefix = conv_.SerializeToString(Prefix4AllInode());
Expand Down
11 changes: 10 additions & 1 deletion curvefs/src/metaserver/inode_storage.h
Original file line number Diff line number Diff line change
Expand Up @@ -98,9 +98,10 @@ class InodeStorage {
/**
* @brief update inode from storage
* @param[in] inode: the inode want to update
* @param[in] inodeDeallocate: Whether the inode needs to deallocate space
* @return If inode not exist, return NOT_FOUND; else replace and return OK
*/
MetaStatusCode Update(const Inode& inode);
MetaStatusCode Update(const Inode& inode, bool inodeDeallocate = false);

std::shared_ptr<Iterator> GetAllInode();

Expand Down Expand Up @@ -155,6 +156,13 @@ class InodeStorage {
uint64_t chunkIndex,
const S3ChunkInfoList* list2add);

// inode wait to deallocate
MetaStatusCode AddDeallocatableInode(uint32_t fsId, uint64_t inodeId);

// use the transaction to delete the Inodelist and update the
// statistics of each item in the blockgrouplist
MetaStatusCode UpdateBlockGroupStatistic();

private:
MetaStatusCode UpdateInodeS3MetaSize(Transaction txn, uint32_t fsId,
uint64_t inodeId, uint64_t size4add,
Expand All @@ -176,6 +184,7 @@ class InodeStorage {
RWLock rwLock_;
std::shared_ptr<KVStorage> kvStorage_;
std::string table4Inode_;
std::string table4DeallocatableInode_;
std::string table4S3ChunkInfo_;
std::string table4VolumeExtent_;
std::string table4InodeAuxInfo_;
Expand Down
50 changes: 50 additions & 0 deletions curvefs/src/metaserver/mds/fsinfo_manager.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/*
* 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
* Date: Wed Mar 22 10:39:52 CST 2023
* Author: lixiaocui
*/

#include "curvefs/src/metaserver/mds/fsinfo_manager.h"

namespace curvefs {
namespace metaserver {
bool FsInfoManager::GetFsInfo(uint32_t fsId, FsInfo *fsInfo) {
if (fsInfoMap_.find(fsId) == fsInfoMap_.end()) {
auto ret = mdsClient_->GetFsInfo(fsId, fsInfo);
if (ret != FSStatusCode::OK) {
if (FSStatusCode::NOT_FOUND == ret) {
LOG(ERROR) << "The fsName not exist, fsId = " << fsId;
return false;
} else {
LOG(ERROR)
<< "GetFsInfo failed, FSStatusCode = " << ret
<< ", FSStatusCode_Name = " << FSStatusCode_Name(ret)
<< ", fsId = " << fsId;
return false;
}
}
fsInfoMap_.insert({fsId, *fsInfo});
} else {
*fsInfo = fsInfoMap_.find(fsId)->second;
}

return true;
}
} // namespace metaserver
} // namespace curvefs
Loading

0 comments on commit be3376e

Please sign in to comment.