全闪分布式存储之PureFlash-极短IO路径-极致性能-极简RDMA和SPDK引擎落盘-支持超融合-IO路径-源码流程及项目简介

2023-12-13 18:08:15 浏览数 (1)

PureFlash

极简IO路径, 原生RDMA(verbs)和SPDK引擎落盘加持, 能充分发挥硬件性能, 支持快照, 多副本等, 高可用的高性能分布式存储, 让我们一起见证全闪时代吧!

项目地址: https://github.com/cocalele/PureFlash

技术分享视频: https://www.bilibili.com/video/BV1s34y1F7Hr

1. PureFlash是什么

PureFlash是一个开源的ServerSAN实现,也就是通过大量的通用服务器,加上PureFlash的软件系统,构造出一套能满足企业各种业务需求的分布式SAN存储。

PureFlash的思想来自于全硬件加速闪存阵列S5, 因此虽然PureFlash本身是纯软件实现,但其存储协议对硬件加速是高度友好的。可以认为PureFlash的协议就是NVMe 协议加上云存储特性增强,包括快照、副本、shard、集群热升级等能力。

2. 为什么要做一个全新ServerSAN实现?

PureFlash是为全闪存时代而设计的存储系统。当前SSD盘的应用越来越广泛,大有全面取代HDD的趋势。SSD与HDD的显著区别就是性能差异,这也是用户体验最直接的差异,而且随着NVMe接口的普及,二者差异越来大,这种近百倍的量变差异足以带来架构设计上的质变。举个例子,原来HDD的性能很低,远远低于CPU、网络的性能能力,因此系统设计的准则是追求HDD的性能最大化,为达到这个目标可以以消耗CPU等资源为代价。而到了NVMe时代,性能关系已经完全颠倒了,盘不再是瓶颈,反而CPU、网络成为系统的瓶颈。那种消耗CPU以优化IO的方法只能适得其反。

因此我们需要一套全新的存储系统架构,以充分发挥SSD的能力,提高系统的效率。PureFlash的设计思想以简化IO stack, 数据通路与控制通路分离,快速路径优先为基本原则,确保高性能与高可靠性,提供云计算时代块存储核心能力。

3. 软件设计

当前的分布式存储系统几乎都有着非常深的软件栈,从客户端软件到最终服务端SSD盘,IO路径非常长。这个深厚的软件栈一方面消耗了大量的系统计算资源,另一方面也让SSD的性能优势荡然无存。PureFlash的设计贯彻了下面的几条原则:

  • “少就是多”, 去掉IO路径上复杂逻辑,使用独有BoB(Block over Bock)结构,将层级最小化
  • “资源为中心”, 围绕CPU资源,SSD资源规划软件结构、线程数量。而不是通常的根据软件代码逻辑需要进行规划
  • “控制/数据分离”, 控制部分使用java开发,数据路径使用C 开发,各取所长

此外PureFlash在网络模型上“以RDMA的模式使用TCP", 而不是通常的”把RDMA当成更快的TCP使用", RDMA一定要将one-sided API与two-sided API 根据业务需要正确的配置。这不但使得RDMA得到了正确的使用,而且让TCP使用效率也大大提高。

下面是我们这个系统的结构图:

代码语言:javascript复制
                                                             --------------- 
                                                            |               |
                                                        --->   MetaDB       |
                                                       |    |  (HA DB)      |
                              ------------------       |     --------------- 
                             |                   ------ 
                             | pfconductor      |            --------------- 
                         ---->  (Max 5 nodes)    ----------->               |
                        |     -------- ---------            | Zookeeper     |
                        |             |                     | (3 nodes)     |
                        |             |                      ------^-------- 
 -------------------    |             |                            |
|                    ---      --------v---------                   |
| pfbd  tcmu        |        |                  |                  |
| (User and          ------->  pfs               ------------------ 
| space client)     |        | (Max 1024 nodes) |
 -------------------          ------------------ 

3.1 pfs, PureFlash Store

这个模块是存储服务守护进程,提供所有的数据服务,包括:

  1. SSD盘空间管理(如: 64MB小块对象分配记录)
  2. 网络接口服务 (RDMA 和 TCP 协议)
  3. IO请求处理

一个PureFlash集群最多可以支持1024个pfs存储节点。所有的pfs都对外提供服务,因此所有的节点都工作在active状态。

3.2 pfconductor

这个模块是集群控制模块。一个产品化的部署应该有至少2个pfconductor节点(最多5个)。主要功能包括:

1) 集群发现与状态维护,包括每个节点的活动与否,每个SSD的活动与否,容量

2) 响应用户的管理请求,创建volume, 快照,租户等

3) 集群运行控制,volume的打开/关闭,运行时故障处理 这个模块用Java编写,位于另外一个代码库: https://github.com/cocalele/pfconductor

3.3 Zookeeper

Zookeeper是集群中实现了Paxos协议的模块,解决网络分区问题。所有的pfconductor和pfs实例都注册到zookeeper, 这样活动的pfconductor就能发现整个集群中的其他成员。

3.4 MetaDB

MetaDB是用来保存集群元数据, 卷分配元数据(1级元数据, 卷映射到分片shard, 到节点, 到副本, 到具体的磁盘依赖关系),我们这里使用的MariaDB。生产部署时需要配合Galaera DB插件,确保拥有HA特性。

client端支持

client接口分两类:用户态和内核态。用户态以API形式给应用访问,这些API位于libpfbd中。

3.5.1 pfdd

pfdd是一个类似dd的工具,但是能访问PureFlash volume, https://github.com/cocalele/qemu/tree/pfbd

3.5.2 fio

支持pfbd的 fio,可以使用fio直接访问pureflash对其进行性能测试。代码库在:https://github.com/cocalele/fio.git

3.5.3 qemu

pfbd也已经集成到了qemu里面,可以直接对接给VM使用。代码库在:https://gitee.com/cocalele/qemu.git

3.5.4 内核态驱动

PureFlash提供了免费的内核态驱动,在物理机上可以直接将pfbd卷呈现成块设备,然后可以格式化成任意的文件系统,任何应用无需API适配就可以访问。

内核驱动非常适合容器PV和数据库场景使用。

3.5.5 nbd对接

支持将PureFlash volume以nbd的形式挂载到主机端, 代码库在: https://gitee.com/cocalele/pfs-nbd.git

编译后执行如下格式的命令挂载卷:

代码语言:javascript复制
    # pfsnbd  /dev/nbd3 test_v1 

3.5.6 iSCSI对接

支持将PureFlash volume作为LIO的后端设备,提供iSCSI接口。 代码库在:https://gitee.com/cocalele/tcmu-runner.git

网络端口

下面是pureflash使用到的网络端口,可以在出问题时检查服务是否正常。

49162 store node TCP port

49160 store node RDMA port

49180 conductor HTTP port

49181 store node HTTP port

尝试 PureFlash

最方便尝试PureFlash的方法是使用容器. 假定你已经有一个NVMe盘,比如, nvme1n1, 请确保这个盘上数据你已经不再需要. 然后按下面的步骤操作:

代码语言:javascript复制
# dd if=/dev/zero of=/dev/nvme1n1 bs=1M count=100 oflag=direct
# docker pull pureflash/pureflash:latest
# docker run -ti --rm  --env PFS_DISKS=/dev/nvme1n1 --ulimit core=-1 --privileged  -e TZ=Asia/Shanghai  --network host  pureflash/pureflash:latest
# pfcli list_store
 ---- --------------- -------- 
| Id | Management IP | Status |
 ---- --------------- -------- 
|  1 |     127.0.0.1 |     OK |
 ---- --------------- -------- 
 
# pfcli list_disk
 ---------- -------------------------------------- -------- 
| Store ID |                 uuid                 | Status |
 ---------- -------------------------------------- -------- 
|        1 | 9ae5b25f-a1b7-4b8d-9fd0-54b578578333 |     OK |
 ---------- -------------------------------------- -------- 

#let's create a volume
# pfcli create_volume -v test_v1 -s 128G --rep 1

#run fio test
# /opt/pureflash/fio -name=test -ioengine=pfbd -volume=test_v1 -iodepth=16  -rw=randwrite -size=128G -bs=4k -direct=1

IO路径参考流程图

IO路径源码流程

代码语言:javascript复制
---------- IO路经,iopath, IO测试, pfdd, dd工具, IO路径, iopath, io路径, 写IO ----------
common/src/pf_pfdd.cpp -> int main(int argc, char* argv[])
string rw, bs_str, ifname, ofname, vol_name, cfg_file, snapshot_name;
void* buf = malloc(bs);
DeferCall _c([buf](){free (buf);}); -> 析构用作deffer语义
struct PfClientVolume* vol = pf_open_volume(vol_name.c_str(), cfg_file.c_str(), snapshot_name.c_str(), S5_LIB_VER) -> 开卷
    int PfClientVolume::do_open(bool reopen, bool is_aof)
        event_queue = runtime_ctx->vol_proc->event_queue
        ...
        init_app_ctx(cfg, 0, 0, 0) -> 初始化上下文
            ctx->init(cfg, io_depth, max_vol_cnt, 0 /* 0 for shared connection*/, io_timeout)
                vol_proc = new PfVolumeEventProc(this)
                vol_proc->start() -> void * thread_proc_eventq(void* arg)
                    pThis->process_event(t->type, t->arg_i, t->arg_p, t->arg_q)
                        int PfClientVolume::process_event(int event_type, int arg_i, void* arg_p)
for(int i=0;i<count;i  )
    pf_io_submit(vol, buf, bs, offset   i * bs, io_cbk, &arg, is_write) -> 写IO
        auto io = volume->runtime_ctx->iocb_pool.alloc() -> 泛型, 分配对象内存池
        io->ulp_handler = callback -> 设置上层业务回调/控制器
        struct PfMessageHead *cmd = io->cmd_bd->cmd_bd; -> 设置命令
        memcpy(io->data_bd->buf, buf, length) -> 第一次内存拷贝
        cmd->opcode = is_write ? S5_OP_WRITE : S5_OP_READ
        int rc = volume->event_queue->post_event( EVT_IO_REQ, 0, io, volume) -> 提交写事件,由其他线程处理事件 -> int PfClientVolume::process_event(int event_type, int arg_i, void* arg_p)
            current_queue->enqueue_nolock(S5Event{ type, arg_i, arg_p , arg_q})
            write(event_fd, &event_delta, sizeof(event_delta)) -> 通知事件线程处理
    sem_wait(&arg.sem)
    ssize_t rc = ::write(fd, buf, bs);


int PfClientVolume::process_event(int event_type, int arg_i, void* arg_p)
    case EVT_IO_REQ
        PfClientIocb* io = (PfClientIocb*)arg_p;
        if(shards[shard_index].is_local) -> 本地落盘(比如超融合场景)
            PfClientStore* local_store = get_local_store(shard_index)
            local_store->do_write(&io->io_subtasks[0]) -> 本地写 -> int PfClientStore::do_write(IoSubTask* io)
                ioengine->submit_io(io, entry->offset   offset_in_block(cmd->offset, in_obj_offset_mask), cmd->length) -> 提交本地写IO
            ...
        else -> 将IO发送到服务端
            struct PfConnection* conn = get_shard_conn(shard_index) -> 获取连接
            io_cmd->meta_ver = (uint16_t)meta_ver;
            BufferDescriptor* rbd = runtime_ctx->reply_pool.alloc()
            int rc = conn->post_recv(rbd);
            cmd_bd->cmd_bd->rkey = io->data_bd->mrs[((PfRdmaConnection*)conn)->dev_ctx->idx]->rkey -> RDMA 从连接中获取远程键
            rc = conn->post_send(cmd_bd) -> 提交IO命令
                buf->wr_op = RDMA_WR_SEND -> 设置IO操作为发送(即发送BULK描述信息)
                wr.opcode = IBV_WR_SEND
                wr.send_flags = IBV_SEND_SIGNALEDsge.addr = (uint64_t)buf->buf
                sge.lkey = buf->mrs[this->dev_ctx->idx]->lkey
                ibv_post_send(rdma_id->qp, &wr, &bad_wr) -> 提交工作请求 -> 触发服务端接收到IO描述: if(bd->wr_op == WrOpcode::RDMA_WR_RECV )



服务端启动事件处理线程:
rc = app_context.rdma_server->init(RDMA_PORT_BASE)
    int PfRdmaServer::init(int port)
        static void *rdma_server_event_proc(void* arg)
            int PfRdmaServer::on_connect_request
                conn->dev_ctx = build_context(id->verbs)
                    init_rdmd_cq_poller(&rdma_dev_ctx->prdc_poller_ctx[pindex], pindex, rdma_dev_ctx, rdma_context) -> 初始化RDMA轮询完成队列的线程
                        poller->prp_cq = ibv_create_cq(rdma_ctx, 512, NULL, poller->prp_comp_channel, 0)
                        ibv_req_notify_cq(poller->prp_cq, 0)
                        poller->poller.init("rdma_cq_poller", 1024)
                            epfd = epoll_create(max_fd_count)
                        int rc = poller->poller.add_fd(poller->prp_comp_channel->fd, EPOLLIN, on_rdma_cq_event, poller)
                            ...
                            conn->on_work_complete(msg, (WcStatus)wc[i].status, conn, NULL) -> 
                rc = rdma_create_qp(id, conn->dev_ctx->pd, &qp_attr)
                conn->connection_info = get_rdma_desc(id, false)
                conn->on_work_complete = server_on_rdma_network_done
                rc = rdma_accept(id, &cm_params)

服务端接收数据:
static int server_on_rdma_network_done(BufferDescriptor* bd, WcStatus complete_status, PfConnection* _conn, void* cbk_data)
if(bd->wr_op == WrOpcode::RDMA_WR_RECV )
    conn->post_read(iocb->data_bd, bd->cmd_bd->buf_addr, bd->cmd_bd->rkey) -> 收到客户端RDMA的SEND请求后, 将客户端数据读到服务端
        buf->wr_op = RDMA_WR_READ;
        wr.opcode = IBV_WR_RDMA_READ;
        wr.send_flags = IBV_SEND_SIGNALED;
        ibv_post_send -> 读完后, 在网络完成回调(server_on_rdma_network_done)中触发:
        else if (bd->wr_op == WrOpcode::RDMA_WR_READ) -> 读取客户端数据成功
            PfServerIocb *iocb = bd->server_iocb
            if (spdk_engine_used()) -> 如果是SPDK引擎, 加锁提交IO事件(分发事件)
                ((PfSpdkQueue *)(conn->dispatcher->event_queue))->post_event_locked(EVT_IO_REQ, 0, iocb) -> int PfDispatcher::process_event
            else
                conn->dispatcher->event_queue->post_event(EVT_IO_REQ, 0, iocb);

        
分发器处理事件
int PfDispatcher::process_event
case EVT_IO_REQ:
    rc = dispatch_io((PfServerIocb*)arg_p) -> 分发事件 -> int PfDispatcher::dispatch_io(PfServerIocb *iocb)
        uint32_t shard_index = (uint32_t)OFFSET_TO_SHARD_INDEX(cmd->offset); -> 计算分片
        PfShard * s = vol->shards[shard_index]
        switch(cmd->opcode)
            case S5_OP_WRITE: -> 写IO
                stat.wr_cnt  ;
                return dispatch_write(iocb, vol, s) -> int PfDispatcher::dispatch_write
                    PfMessageHead* cmd = iocb->cmd_bd->cmd_bd
                    iocb->setup_subtask(s, cmd->opcode) -> 根据副本数,设置子任务
                    for (int i = 0; i < vol->rep_count; i  )
                        rc = s->replicas[i]->submit_io(&iocb->io_subtasks[i]) -> 遍历副本以及提交到副本IO -> disk->event_queue->post_event(EVT_IO_REQ, 0, subtask) -> int PfReplicator::process_event

副本器处理事件:
int PfReplicator::process_event
    case EVT_IO_REQ:
        return begin_replicate_io((IoSubTask*)arg_p) -> 开始执行副本IO
            PfConnection* c = (PfConnection*)conn_pool->get_conn((int)t->store_id); -> 获取连接
            if(!c->get_throttle()) -> 限流
            PfClientIocb* io = iocb_pool.alloc()
            memcpy(io->cmd_bd->cmd_bd, t->parent_iocb->cmd_bd->cmd_bd, sizeof(PfMessageHead))
            t->opcode = PfOpCode::S5_OP_REPLICATE_WRITE -> 设置操作码为副本写
            BufferDescriptor* rbd = mem_pool.reply_pool.alloc()
            rc = c->post_recv(rbd) -> 接收IO描述(IO头部信息) -> server_on_rdma_network_done
                buf->wr_op = RDMA_WR_RECV
                ibv_post_recv(rdma_id->qp, &wr, &bad_wr)
            io->reply_bd = rbd
            rc = c->post_send(io->cmd_bd) -> 通过连接提交IO到其他副本(所在的服务器)
                buf->wr_op = RDMA_WR_SEND
                wr.opcode = IBV_WR_SEND
                ibv_post_send(rdma_id->qp, &wr, &bad_wr) -> server_on_rdma_network_done
                    else if(bd->wr_op == WrOpcode::RDMA_WR_SEND) -> 本次IO完成, 继续接收下一个IO
                        iocb->re_init();
                        conn->post_recv(iocb->cmd_bd);

其他副本节点收到主副本节点的副本IO:
int PfFlashStore::process_event
    case EVT_IO_REQ:
        case PfOpCode::S5_OP_REPLICATE_WRITE:
            do_write((IoSubTask*)arg_p); -> 将IO写入后端全闪存储 -> int PfFlashStore::do_write(IoSubTask* io)
                PfMessageHead* cmd = io->parent_iocb->cmd_bd->cmd_bd
                BufferDescriptor* data_bd = io->parent_iocb->data_bd
                lmt_key key = {VOLUME_ID(io->rep_id), (int64_t)vol_offset_to_block_slba(cmd->offset, head.objsize_order), 0, 0} -> 每64MB一个key, 卷ID   这笔卷IO基于起始逻辑块的偏移作为键
                auto block_pos = obj_lmt.find(key)
                if (unlikely(block_pos == obj_lmt.end())) -> 没找到块则新分配,并记录重做日志
                    int obj_id = free_obj_queue.dequeue() -> 获取小对象ID
                    entry = lmt_entry_pool.alloc()
                    *entry = lmt_entry { offset: obj_id_to_offset(obj_id),   // -> { return (obj_id << head.objsize_order)   head.meta_size; } 用对象ID得到对象在磁盘上的偏移
                        snap_seq : cmd->snap_seq,
                        status : EntryStatus::NORMAL,
                        prev_snap : NULL,
                        waiting_io : NULL
                    };
                    obj_lmt[key] = entry;
                    int rc = redolog->log_allocation(&key, entry, free_obj_queue.head); -> 将小块的分配记录到重做日志
                        if (store->ioengine->sync_write(entry_buff, LBA_LENGTH, current_offset) == -1) -> 同步下刷元数据
                            uint64_t PfspdkEngine::sync_write(void* buffer, uint64_t buf_size, uint64_t offset) -> 将buf,长度,偏移通过SPDK接口写入裸盘
                        return store->meta_data_compaction_trigger(COMPACT_TODO, true) -> 当前偏移大于等于起始偏移   头部的重做日志长度时(256MB), 触发元数据合并
                            last_state = to_run_compact.load()

                else -> 之前存在块记录
                    ...
                    ioengine->submit_io(io, entry->offset   offset_in_block(cmd->offset, in_obj_offset_mask), cmd->length) -> 将IO提交到引擎,准备落盘
                    int PfspdkEngine::submit_io -> SPDK引擎
                        if (spdk_nvme_bytes_to_blocks(media_offset, &lba, media_len, &lba_cnt) != 0) -> 将偏移和长度转成块
                        if (IS_READ_OP(io->opcode)) -> 读IO
                            spdk_nvme_ns_cmd_read_with_md(ns->ns, qpair[1], data_bd->buf, NULL, lba, (uint32_t)lba_cnt, spdk_io_complete, io, 0, 0, 0)
                        esle -> 写IO
                            spdk_nvme_ns_cmd_write_with_md(ns->ns, qpair[1], data_bd->buf, NULL, lba, (uint32_t)lba_cnt, spdk_io_complete, io, 0, 0, 0); -> 带命名空间的写入(SPDK引擎继承自通用IO引擎,增加NS, 控制器, 块大小, 块数量等)
                            void PfspdkEngine::spdk_io_complete(void* ctx, const struct spdk_nvme_cpl* cpl) -> 写IO完成
                                io->ops->complete(io, PfMessageStatus::MSG_STATUS_SUCCESS) -> 在初始化内存池(int PfDispatcher::init_mempools)的时候静态注册: static struct TaskCompleteOps _server_task_complete_ops={ server_complete , server_complete_with_metaver };
                    or int PfAioEngine::submit_io -> AIO引擎
                        io_prep_pread(&io->aio_cb, fd, data_bd->buf, media_len, media_offset)
                        or
                        io_prep_pwrite(&io->aio_cb, fd, data_bd->buf, media_len, media_offset)
                        return submit_batch() -> 批量提交

static void server_complete(SubTask* t, PfMessageStatus comp_status)
    ((PfServerIocb*)t->parent_iocb)->conn->dispatcher->event_queue->post_event(EVT_IO_COMPLETE, 0, t) -> 给分发器提交完成事件 -> int PfDispatcher::process_event
        case EVT_IO_COMPLETE:
            rc = dispatch_complete((SubTask*)arg_p);
                if(iocb->task_mask == 0) -> 副本IO任务完成后,回复客户端IO完成
                    reply_io_to_client(iocb)
                        if (io_elapse_time > 2000) -> 慢IO
                        rc = iocb->conn->post_send(iocb->reply_bd); -> 回复客户端, 触发客户端RDMA处理事件: client_on_rdma_network_done

static int client_on_rdma_network_done
    if (bd->wr_op == RDMA_WR_RECV)
        PfClientIocb* iocb = conn->client_ctx->pick_iocb(bd->reply_bd->command_id, bd->reply_bd->command_seq) -> 客户端获取服务端的RDMA的IO回复
            PfClientIocb* io = &iocb_pool.data[cid]
        iocb->reply_bd = bd
        iocb->volume->event_queue->post_event(EVT_IO_COMPLETE, complete_status, bd, iocb->volume) -> 给卷事件线程提交一个IO完成事件 -> int PfClientVolume::process_event(int event_type, int arg_i, void* arg_p)
            client_do_complete(arg_i, (BufferDescriptor*)arg_p) -> 客户端完成IO
                ulp_io_handler h = io->ulp_handler
                if (wr_bd->wr_op == TCP_WR_RECV || wr_bd->wr_op == RDMA_WR_RECV) -> 客户端接收到数据
                    PfMessageHead* io_cmd = io->cmd_bd->cmd_bd
                    runtime_ctx->reply_pool.free(io->reply_bd)
                    if(io->cmd_bd->cmd_bd->opcode == S5_OP_READ) -> 读IO完成
                        iov_from_buf(io->user_iov, io->user_iov_cnt, io->data_bd->buf, io->data_bd->data_len) -> 拷贝向量IO或普通IO
                        or memcpy(io->user_buf, io->data_bd->buf, io->data_bd->data_len);
                    runtime_ctx->free_iocb(io) -> 释放资源
                    h(arg, s) -> 执行业务回调(ULP) -> void io_cbk(void* cbk_arg, int complete_status)
                        sem_post(&w->sem) -> 通过信号量通知其他线程
---------- IO路经,iopath, IO测试, pfdd, dd工具, IO路径, iopath, io路径, 写IO END ----------

磁盘头小块对象分配元数据(即head(4KB), 这里配置盘头10GB用于存储block元数据)

元数据结构及分布(内存元数据 = 磁盘初始元数据 log记录的增量修改)

trim_proc线程定期清理未使用的小对象分配日志

元数据也会按规则做聚合, 聚合后释放不用的log

代码语言:c复制
初始化磁盘头, 元数据, 重要函数:
int PfFlashStore::initialize_store_head()
    ...
    uuid_generate(head.uuid)
    head.objsize=DEFAULT_OBJ_SIZE -> 默认小块分配对象大小为64MB
    head.free_list_position_first = OFFSET_FREE_LIST_FIRST
    ...
    memcpy(buf, &head, sizeof(head)) -> 拷贝元数据头
    ioengine->sync_write(buf, LBA_LENGTH, 0) -> 将元数据落盘

int PfFlashStore::initialize_store_head()
{
	memset(&head, 0, sizeof(head));
	head.magic = 0x3553424e; //magic number, NBS5 894648910
	head.version= S5_VERSION; //S5 version 196608
	uuid_generate(head.uuid);
	uuid_unparse(head.uuid, uuid_str);
	S5LOG_INFO("generate disk uuid:%s", uuid_str);
	head.key_size=sizeof(lmt_key); // 32B
	head.entry_size=sizeof(lmt_entry);  //48B
	head.objsize=DEFAULT_OBJ_SIZE; // 64MB
	head.objsize_order=DEFAULT_OBJ_SIZE_ORDER; //objsize = 2 ^ objsize_order  26, 小对象掩码(2的26次方)
	head.tray_capacity = ioengine->get_device_cap();    //磁盘大小: 20GB(和配置相关)
	head.meta_size = app_context.meta_size; // 元数据: 比如10GB
	head.free_list_position_first = OFFSET_FREE_LIST_FIRST; // 空闲块起始位置偏移: 4096
	head.free_list_size_first = (128LL << 20) - 4096; // 128MB
	head.free_list_position_second = OFFSET_FREE_LIST_SECOND;
	head.free_list_size_second = (128LL << 20); // 128MB
	head.trim_list_position_first = OFFSET_TRIM_LIST_FIRST;  
	head.trim_list_position_second = OFFSET_TRIM_LIST_SECOND;
	head.trim_list_size = (128LL << 20);
	head.lmt_position_first = OFFSET_LMT_MAP_FIRST;
	head.lmt_position_second = OFFSET_LMT_MAP_SECOND;
	head.lmt_size = (2LL << 30); //2GB的lmt大小
	head.redolog_position_first = OFFSET_REDO_LOG_FIRST;
	head.redolog_position_second = OFFSET_REDO_LOG_SECOND;
	head.redolog_size = REDO_LOG_SIZE;
	head.current_metadata = FIRST_METADATA_ZONE;
	head.current_redolog = FIRST_REDOLOG_ZONE; // 比如2
	head.redolog_phase = 0;
	time_t time_now = time(0);
	strftime(head.create_time, sizeof(head.create_time), "%Y%m%d %H:%M:%S", localtime(&time_now));
    ...

struct HeadPage {
    uint32_t magic;
    uint32_t version;
    unsigned char uuid[16];
    uint32_t key_size;
    uint32_t entry_size;
    uint64_t objsize;
    uint64_t tray_capacity;
    uint64_t meta_size;
    uint32_t objsize_order; //objsize = 2 ^ objsize_order
    uint32_t rsv1; //to make alignment at 8 byte
    uint64_t free_list_position_first;
    uint64_t free_list_position_second;
    uint64_t free_list_size_first;
    uint64_t free_list_size_second;
    uint64_t trim_list_position_first;
    uint64_t trim_list_position_second;
    uint64_t trim_list_size;
    uint64_t lmt_position_first;
    uint64_t lmt_position_second;
    uint64_t lmt_size;
    uint64_t redolog_position_first;
    uint64_t redolog_position_second;
    uint64_t redolog_size;
    /**update after save metadata**/
    int64_t  redolog_phase;
    uint8_t  current_metadata;
    uint8_t  current_redolog;
    char md5_first[MD5_RESULT_LEN];
    char md5_second[MD5_RESULT_LEN];
    /***/
    char create_time[32];
};

晓兵(ssbandjl)

博客: https://cloud.tencent.com/developer/user/5060293/articles | https://logread.cn | https://blog.csdn.net/ssbandjl

欢迎对高性能分布式存储PureFlash, SPDK, RDMA, 等高性能技术感兴趣的朋友加入PureFlash技术交流(群)

0 人点赞