diff --git a/doc/Doxyfile b/doc/Doxyfile index 94f9daea7..4ccfd1526 100644 --- a/doc/Doxyfile +++ b/doc/Doxyfile @@ -811,6 +811,7 @@ INPUT += \ concurrency.md \ directory_structure.md \ event.md \ + ftl.md \ getting_started.md \ ioat.md \ iscsi.md \ diff --git a/doc/ftl.md b/doc/ftl.md new file mode 100644 index 000000000..80dae277b --- /dev/null +++ b/doc/ftl.md @@ -0,0 +1,134 @@ +# Flash Translation Layer {#ftl} + +The Flash Translation Layer library provides block device access on top of non-block SSDs +implementing Open Channel interface. It handles the logical to physical address mapping, responds to +the asynchronous media management events, and manages the defragmentation process. + +# Terminology {#ftl_terminology} + +## Logical to physical address map + + * Shorthand: L2P + +Contains the mapping of the logical addresses (LBA) to their on-disk physical location (PPA). The +LBAs are contiguous and in range from 0 to the number of surfaced blocks (the number of spare blocks +are calculated during device formation and are subtracted from the available address space). The +spare blocks account for chunks going offline throughout the lifespan of the device as well as +provide necessary buffer for data [defragmentation](#ftl_reloc). + +## Band {#ftl_band} + +Band describes a collection of chunks, each belonging to a different parallel unit. All writes to +the band follow the same pattern - a batch of logical blocks is written to one chunk, another batch +to the next one and so on. This ensures the parallelism of the write operations, as they can be +executed independently on a different chunks. Each band keeps track of the LBAs it consists of, as +well as their validity, as some of the data will be invalidated by subsequent writes to the same +logical address. The L2P mapping can be restored from the SSD by reading this information in order +from the oldest band to the youngest. + + +--------------+ +--------------+ +--------------+ + band 1 | chunk 1 +--------+ chk 1 +---- --- --- --- --- ---+ chk 1 | + +--------------+ +--------------+ +--------------+ + band 2 | chunk 2 +--------+ chk 2 +---- --- --- --- --- ---+ chk 2 | + +--------------+ +--------------+ +--------------+ + band 3 | chunk 3 +--------+ chk 3 +---- --- --- --- --- ---+ chk 3 | + +--------------+ +--------------+ +--------------+ + | ... | | ... | | ... | + +--------------+ +--------------+ +--------------+ + band m | chunk m +--------+ chk m +---- --- --- --- --- ---+ chk m | + +--------------+ +--------------+ +--------------+ + | ... | | ... | | ... | + +--------------+ +--------------+ +--------------+ + + parallel unit 1 pu 2 pu n + +The address map and valid map are, along with a several other things (e.g. UUID of the device it's +part of, number of surfaced LBAs, band's sequence number, etc.), parts of the band's metadata. The +metadata is split in two parts: + * the head part, containing information already known when opening the band (device's UUID, band's + sequence number, etc.), located at the beginning blocks of the band, + * the tail part, containing the address map and the valid map, located at the end of the band. + + + head metadata band's data tail metadata + +-------------------+-------------------------------+----------------------+ + |chk 1|...|chk n|...|...|chk 1|...| | ... |chk m-1 |chk m| + |lbk 1| |lbk 1| | |lbk x| | | |lblk y |lblk y| + +-------------------+-------------+-----------------+----------------------+ + + +Bands are being written sequentially (in a way that was described earlier). Before a band can be +written to, all of its chunks need to be erased. During that time, the band is considered to be in a +`PREP` state. After that is done, the band transitions to the `OPENING` state, in which head metadata +is being written. Then the band moves to the `OPEN` state and actual user data can be written to the +band. Once the whole available space is filled, tail metadata is written and the band transitions to +`CLOSING` state. When that finishes the band becomes `CLOSED`. + +## Ring write buffer {#ftl_rwb} + + * Shorthand: RWB + +Because the smallest write size the SSD may support can be a multiple of block size, in order to +support writes to a single block, the data needs to be buffered. The write buffer is the solution to +this problem. It consists of a number of pre-allocated buffers called batches, each of size allowing +for a single transfer to the SSD. A single batch is divided into block-sized buffer entries. + + write buffer + +-----------------------------------+ + |batch 1 | + | +-----------------------------+ | + | |rwb |rwb | ... |rwb | | + | |entry 1|entry 2| |entry n| | + | +-----------------------------+ | + +-----------------------------------+ + | ... | + +-----------------------------------+ + |batch m | + | +-----------------------------+ | + | |rwb |rwb | ... |rwb | | + | |entry 1|entry 2| |entry n| | + | +-----------------------------+ | + +-----------------------------------+ + +When a write is scheduled, it needs to acquire an entry for each of its blocks and copy the data +onto this buffer. Once all blocks are copied, the write can be signalled as completed to the user. +In the meantime, the `rwb` is polled for filled batches and, if one is found, it's sent to the SSD. +After that operation is completed the whole batch can be freed. For the whole time the data is in +the `rwb`, the L2P points at the buffer entry instead of a location on the SSD. This allows for +servicing read requests from the buffer. + +## Defragmentation and relocation {#ftl_reloc} + + * Shorthand: defrag, reloc + +Since a write to the same LBA invalidates its previous physical location, some of the blocks on a +band might contain old data that basically wastes space. As there is no way to overwrite an already +written block, this data will stay there until the whole chunk is reset. This might create a +situation in which all of the bands contain some valid data and no band can be erased, so no writes +can be executed anymore. Therefore a mechanism is needed to move valid data and invalidate whole +bands, so that they can be reused. + + band band + +-----------------------------------+ +-----------------------------------+ + | ** * * *** * *** * * | | | + |** * * * * * * *| +----> | | + |* *** * * * | | | + +-----------------------------------+ +-----------------------------------+ + +Valid blocks are marked with an asterisk '\*'. + +Another reason for data relocation might be an event from the SSD telling us that the data might +become corrupt if it's not relocated. This might happen due to its old age (if it was written a +long time ago) or due to read disturb (media characteristic, that causes corruption of neighbouring +blocks during a read operation). + +Module responsible for data relocation is called `reloc`. When a band is chosen for defragmentation +or an ANM (asynchronous NAND management) event is received, the appropriate blocks are marked as +required to be moved. The `reloc` module takes a band that has some of such blocks marked, checks +their validity and, if they're still valid, copies them. + +Choosing a band for defragmentation depends on several factors: its valid ratio (1) (proportion of +valid blocks to all user blocks), its age (2) (when was it written) and its write count / wear level +index of its chunks (3) (how many times the band was written to). The lower the ratio (1), the +higher its age (2) and the lower its write count (3), the higher the chance the band will be chosen +for defrag. diff --git a/doc/prog_guides.md b/doc/prog_guides.md index 7ccffca61..ae52df7de 100644 --- a/doc/prog_guides.md +++ b/doc/prog_guides.md @@ -5,3 +5,4 @@ - @subpage bdev_pg - @subpage bdev_module - @subpage nvmf_tgt_pg +- @subpage ftl diff --git a/include/spdk/ftl.h b/include/spdk/ftl.h new file mode 100644 index 000000000..bae4a8114 --- /dev/null +++ b/include/spdk/ftl.h @@ -0,0 +1,267 @@ +/*- + * BSD LICENSE + * + * Copyright (c) Intel Corporation. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * + * * Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * * Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in + * the documentation and/or other materials provided with the + * distribution. + * * Neither the name of Intel Corporation nor the names of its + * contributors may be used to endorse or promote products derived + * from this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + */ + +#ifndef SPDK_FTL_H +#define SPDK_FTL_H + +#include +#include +#include +#include +#include + +struct spdk_ftl_dev; + +/* Limit thresholds */ +enum { + SPDK_FTL_LIMIT_CRIT, + SPDK_FTL_LIMIT_HIGH, + SPDK_FTL_LIMIT_LOW, + SPDK_FTL_LIMIT_START, + SPDK_FTL_LIMIT_MAX +}; + +struct spdk_ftl_limit { + /* Threshold from which the limiting starts */ + size_t thld; + + /* Limit percentage */ + size_t limit; +}; + +struct spdk_ftl_conf { + /* Number of reserved addresses not exposed to the user */ + size_t lba_rsvd; + + /* Write buffer size */ + size_t rwb_size; + + /* Threshold for opening new band */ + size_t band_thld; + + /* Trace enabled flag */ + int trace; + + /* Trace file name */ + const char *trace_path; + + /* Maximum IO depth per band relocate */ + size_t max_reloc_qdepth; + + /* Maximum active band relocates */ + size_t max_active_relocs; + + /* IO pool size per user thread */ + size_t user_io_pool_size; + + struct { + /* Lowest percentage of invalid lbks for a band to be defragged */ + size_t invalid_thld; + + /* User writes limits */ + struct spdk_ftl_limit limits[SPDK_FTL_LIMIT_MAX]; + } defrag; +}; + +/* Range of parallel units (inclusive) */ +struct spdk_ftl_punit_range { + unsigned int begin; + unsigned int end; +}; + +enum spdk_ftl_mode { + /* Create new device */ + SPDK_FTL_MODE_CREATE = (1 << 0), +}; + +struct spdk_ftl_dev_init_opts { + /* NVMe controller */ + struct spdk_nvme_ctrlr *ctrlr; + /* Controller's transport ID */ + struct spdk_nvme_transport_id trid; + + /* Thread responsible for core tasks execution */ + struct spdk_thread *core_thread; + /* Thread responsible for read requests */ + struct spdk_thread *read_thread; + + /* Device's config */ + struct spdk_ftl_conf *conf; + /* Device's name */ + const char *name; + /* Parallel unit range */ + struct spdk_ftl_punit_range range; + /* Mode flags */ + unsigned int mode; + /* Device UUID (valid when restoring device from disk) */ + struct spdk_uuid uuid; +}; + +struct spdk_ftl_attrs { + /* Device's UUID */ + struct spdk_uuid uuid; + /* Parallel unit range */ + struct spdk_ftl_punit_range range; + /* Number of logical blocks */ + uint64_t lbk_cnt; + /* Logical block size */ + size_t lbk_size; +}; + +struct ftl_module_init_opts { + /* Thread on which to poll for ANM events */ + struct spdk_thread *anm_thread; +}; + +typedef void (*spdk_ftl_fn)(void *, int); +typedef void (*spdk_ftl_init_fn)(struct spdk_ftl_dev *, void *, int); + +/** + * Initialize the FTL module. + * + * \param opts module configuration + * \param cb callback function to call when the module is initialized + * \param cb_arg callback's argument + * + * \return 0 if successfully started initialization, negative values if + * resources could not be allocated. + */ +int spdk_ftl_module_init(const struct ftl_module_init_opts *opts, spdk_ftl_fn cb, void *cb_arg); + +/** + * Deinitialize the FTL module. All FTL devices have to be unregistered prior to + * calling this function. + * + * \param cb callback function to call when the deinitialization is completed + * \param cb_arg callback's argument + * + * \return 0 if successfully scheduled deinitialization, negative errno + * otherwise. + */ +int spdk_ftl_module_fini(spdk_ftl_fn cb, void *cb_arg); + +/** + * Initialize the FTL on given NVMe device and parallel unit range. + * + * Covers the following: + * - initialize and register NVMe ctrlr, + * - retrieve geometry and check if the device has proper configuration, + * - allocate buffers and resources, + * - initialize internal structures, + * - initialize internal thread(s), + * - restore or create L2P table. + * + * \param opts configuration for new device + * \param cb callback function to call when the device is created + * \param cb_arg callback's argument + * + * \return 0 if initialization was started successfully, negative errno otherwise. + */ +int spdk_ftl_dev_init(const struct spdk_ftl_dev_init_opts *opts, spdk_ftl_init_fn cb, void *cb_arg); + +/** + * Deinitialize and free given device. + * + * \param dev device + * \param cb callback function to call when the device is freed + * \param cb_arg callback's argument + * + * \return 0 if successfully scheduled free, negative errno otherwise. + */ +int spdk_ftl_dev_free(struct spdk_ftl_dev *dev, spdk_ftl_fn cb, void *cb_arg); + +/** + * Initialize FTL configuration structure with default values. + * + * \param conf FTL configuration to initialize + */ +void spdk_ftl_conf_init_defaults(struct spdk_ftl_conf *conf); + +/** + * Retrieve device’s attributes. + * + * \param dev device + * \param attr Attribute structure to fill + * + * \return 0 if successfully initialized, negated EINVAL otherwise. + */ +int spdk_ftl_dev_get_attrs(const struct spdk_ftl_dev *dev, struct spdk_ftl_attrs *attr); + +/** + * Submits a read to the specified device. + * + * \param dev Device + * \param ch I/O channel + * \param lba Starting LBA to read the data + * \param lba_cnt Number of sectors to read + * \param iov Single IO vector or pointer to IO vector table + * \param iov_cnt Number of IO vectors + * \param cb_fn Callback function to invoke when the I/O is completed + * \param cb_arg Argument to pass to the callback function + * + * \return 0 if successfully submitted, negated EINVAL otherwise. + */ +int spdk_ftl_read(struct spdk_ftl_dev *dev, struct spdk_io_channel *ch, uint64_t lba, + size_t lba_cnt, + struct iovec *iov, size_t iov_cnt, spdk_ftl_fn cb_fn, void *cb_arg); + +/** + * Submits a write to the specified device. + * + * \param dev Device + * \param ch I/O channel + * \param lba Starting LBA to write the data + * \param lba_cnt Number of sectors to write + * \param iov Single IO vector or pointer to IO vector table + * \param iov_cnt Number of IO vectors + * \param cb_fn Callback function to invoke when the I/O is completed + * \param cb_arg Argument to pass to the callback function + * + * \return 0 if successfully submitted, negative values otherwise. + */ +int spdk_ftl_write(struct spdk_ftl_dev *dev, struct spdk_io_channel *ch, uint64_t lba, + size_t lba_cnt, + struct iovec *iov, size_t iov_cnt, spdk_ftl_fn cb_fn, void *cb_arg); + +/** + * Submits a flush request to the specified device. + * + * \param dev device + * \param cb_fn Callback function to invoke when all prior IOs have been completed + * \param cb_arg Argument to pass to the callback function + * + * \return 0 if successfully submitted, negated EINVAL or ENOMEM otherwise. + */ +int spdk_ftl_flush(struct spdk_ftl_dev *dev, spdk_ftl_fn cb_fn, void *cb_arg); + +#endif /* SPDK_FTL_H */ diff --git a/lib/ftl/ftl_core.h b/lib/ftl/ftl_core.h new file mode 100644 index 000000000..26d8d0aff --- /dev/null +++ b/lib/ftl/ftl_core.h @@ -0,0 +1,434 @@ +/*- + * BSD LICENSE + * + * Copyright (c) Intel Corporation. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * + * * Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * * Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in + * the documentation and/or other materials provided with the + * distribution. + * * Neither the name of Intel Corporation nor the names of its + * contributors may be used to endorse or promote products derived + * from this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + */ + +#ifndef FTL_CORE_H +#define FTL_CORE_H + +#include "spdk/stdinc.h" +#include "spdk/nvme.h" +#include "spdk/nvme_ocssd.h" +#include "spdk/uuid.h" +#include "spdk/thread.h" +#include "spdk/util.h" +#include "spdk_internal/log.h" +#include "spdk/queue.h" +#include "spdk/ftl.h" + +#include "ftl_ppa.h" +#include "ftl_io.h" +#include "ftl_trace.h" + +struct spdk_ftl_dev; +struct ftl_band; +struct ftl_chunk; +struct ftl_io; +struct ftl_restore; +struct ftl_wptr; +struct ftl_flush; + +struct ftl_stats { + /* Number of writes scheduled directly by the user */ + uint64_t write_user; + + /* Total number of writes */ + uint64_t write_total; + + /* Traces */ + struct ftl_trace *trace; + + /* Number of limits applied */ + uint64_t limits[SPDK_FTL_LIMIT_MAX]; +}; + +struct ftl_punit { + struct spdk_ftl_dev *dev; + + struct ftl_ppa start_ppa; +}; + +struct ftl_thread { + /* Owner */ + struct spdk_ftl_dev *dev; + /* I/O queue pair */ + struct spdk_nvme_qpair *qpair; + + /* Thread on which the poller is running */ + struct spdk_thread *thread; + /* Thread id */ + pthread_t tid; + + /* Poller */ + struct spdk_poller *poller; + /* Poller's function */ + spdk_poller_fn poller_fn; + /* Poller's frequency */ + uint64_t period_us; +}; + +struct ftl_global_md { + /* Device instance */ + struct spdk_uuid uuid; + /* Size of the l2p table */ + uint64_t num_lbas; +}; + +struct spdk_ftl_dev { + /* Device instance */ + struct spdk_uuid uuid; + /* Device name */ + char *name; + /* Configuration */ + struct spdk_ftl_conf conf; + + /* Indicates the device is fully initialized */ + int initialized; + /* Indicates the device is about to be stopped */ + int halt; + + /* Init callback */ + spdk_ftl_init_fn init_cb; + /* Init callback's context */ + void *init_arg; + + /* Halt callback */ + spdk_ftl_fn halt_cb; + /* Halt callback's context */ + void *halt_arg; + /* Halt poller, checks if the device has been halted */ + struct spdk_poller *halt_poller; + + /* IO channel */ + struct spdk_io_channel *ioch; + + /* NVMe controller */ + struct spdk_nvme_ctrlr *ctrlr; + /* NVMe namespace */ + struct spdk_nvme_ns *ns; + /* NVMe transport ID */ + struct spdk_nvme_transport_id trid; + + /* LBA map memory pool */ + struct spdk_mempool *lba_pool; + + /* Statistics */ + struct ftl_stats stats; + + /* Parallel unit range */ + struct spdk_ftl_punit_range range; + /* Array of parallel units */ + struct ftl_punit *punits; + + /* Current sequence number */ + uint64_t seq; + + /* Array of bands */ + struct ftl_band *bands; + /* Band being curently defraged */ + struct ftl_band *df_band; + /* Number of operational bands */ + size_t num_bands; + /* Next write band */ + struct ftl_band *next_band; + /* Free band list */ + LIST_HEAD(, ftl_band) free_bands; + /* Closed bands list */ + LIST_HEAD(, ftl_band) shut_bands; + /* Number of free bands */ + size_t num_free; + + /* List of write pointers */ + LIST_HEAD(, ftl_wptr) wptr_list; + + /* Logical -> physical table */ + void *l2p; + /* Size of the l2p table */ + uint64_t num_lbas; + + /* PPA format */ + struct ftl_ppa_fmt ppaf; + /* PPA address size */ + size_t ppa_len; + /* Device's geometry */ + struct spdk_ocssd_geometry_data geo; + + /* Flush list */ + LIST_HEAD(, ftl_flush) flush_list; + + /* Device specific md buffer */ + struct ftl_global_md global_md; + + /* Metadata size */ + size_t md_size; + + /* Transfer unit size */ + size_t xfer_size; + /* Ring write buffer */ + struct ftl_rwb *rwb; + + /* Current user write limit */ + int limit; + + /* Inflight io operations */ + uint32_t num_inflight; + + /* Threads */ + struct ftl_thread core_thread; + struct ftl_thread read_thread; + + /* Devices' list */ + STAILQ_ENTRY(spdk_ftl_dev) stailq; +}; + +typedef void (*ftl_restore_fn)(struct spdk_ftl_dev *, struct ftl_restore *, int); + +void ftl_apply_limits(struct spdk_ftl_dev *dev); +int ftl_io_read(struct ftl_io *io); +int ftl_io_write(struct ftl_io *io); +int ftl_io_erase(struct ftl_io *io); +int ftl_io_flush(struct ftl_io *io); +int ftl_current_limit(const struct spdk_ftl_dev *dev); +int ftl_invalidate_addr(struct spdk_ftl_dev *dev, struct ftl_ppa ppa); +int ftl_task_core(void *ctx); +int ftl_task_read(void *ctx); +size_t ftl_tail_md_num_lbks(const struct spdk_ftl_dev *dev); +size_t ftl_tail_md_hdr_num_lbks(const struct spdk_ftl_dev *dev); +size_t ftl_vld_map_num_lbks(const struct spdk_ftl_dev *dev); +size_t ftl_lba_map_num_lbks(const struct spdk_ftl_dev *dev); +size_t ftl_head_md_num_lbks(const struct spdk_ftl_dev *dev); +int ftl_restore_md(struct spdk_ftl_dev *dev, ftl_restore_fn cb); +int ftl_restore_device(struct ftl_restore *restore, ftl_restore_fn cb); + +#define ftl_to_ppa(addr) \ + (struct ftl_ppa) { .ppa = (uint64_t)(addr) } + +#define ftl_to_ppa_packed(addr) \ + (struct ftl_ppa) { .pack.ppa = (uint32_t)(addr) } + +static inline struct spdk_thread * +ftl_get_core_thread(const struct spdk_ftl_dev *dev) +{ + return dev->core_thread.thread; +} + +static inline struct spdk_nvme_qpair * +ftl_get_write_qpair(const struct spdk_ftl_dev *dev) +{ + return dev->core_thread.qpair; +} + +static inline struct spdk_thread * +ftl_get_read_thread(const struct spdk_ftl_dev *dev) +{ + return dev->read_thread.thread; +} + +static inline struct spdk_nvme_qpair * +ftl_get_read_qpair(const struct spdk_ftl_dev *dev) +{ + return dev->read_thread.qpair; +} + +static inline int +ftl_ppa_packed(const struct spdk_ftl_dev *dev) +{ + return dev->ppa_len < 32; +} + +static inline int +ftl_ppa_invalid(struct ftl_ppa ppa) +{ + return ppa.ppa == ftl_to_ppa(FTL_PPA_INVALID).ppa; +} + +static inline int +ftl_ppa_cached(struct ftl_ppa ppa) +{ + return !ftl_ppa_invalid(ppa) && ppa.cached; +} + +static inline uint64_t +ftl_ppa_addr_pack(const struct spdk_ftl_dev *dev, struct ftl_ppa ppa) +{ + return (ppa.lbk << dev->ppaf.lbk_offset) | + (ppa.chk << dev->ppaf.chk_offset) | + (ppa.pu << dev->ppaf.pu_offset) | + (ppa.grp << dev->ppaf.grp_offset); +} + +static inline struct ftl_ppa +ftl_ppa_addr_unpack(const struct spdk_ftl_dev *dev, uint64_t ppa) +{ + struct ftl_ppa res = {}; + + res.lbk = (ppa >> dev->ppaf.lbk_offset) & dev->ppaf.lbk_mask; + res.chk = (ppa >> dev->ppaf.chk_offset) & dev->ppaf.chk_mask; + res.pu = (ppa >> dev->ppaf.pu_offset) & dev->ppaf.pu_mask; + res.grp = (ppa >> dev->ppaf.grp_offset) & dev->ppaf.grp_mask; + + return res; +} + +static inline struct ftl_ppa +ftl_ppa_to_packed(const struct spdk_ftl_dev *dev, struct ftl_ppa ppa) +{ + struct ftl_ppa p = {}; + + if (ftl_ppa_invalid(ppa)) { + p = ftl_to_ppa_packed(FTL_PPA_INVALID); + } else if (ftl_ppa_cached(ppa)) { + p.pack.cached = 1; + p.pack.offset = (uint32_t) ppa.offset; + } else { + p.pack.ppa = (uint32_t) ftl_ppa_addr_pack(dev, ppa); + } + + return p; +} + +static inline struct ftl_ppa +ftl_ppa_from_packed(const struct spdk_ftl_dev *dev, struct ftl_ppa p) +{ + struct ftl_ppa ppa = {}; + + if (p.pack.ppa == (uint32_t)FTL_PPA_INVALID) { + ppa = ftl_to_ppa(FTL_PPA_INVALID); + } else if (p.pack.cached) { + ppa.cached = 1; + ppa.offset = p.pack.offset; + } else { + ppa = ftl_ppa_addr_unpack(dev, p.pack.ppa); + } + + return ppa; +} + +static inline unsigned int +ftl_ppa_flatten_punit(const struct spdk_ftl_dev *dev, struct ftl_ppa ppa) +{ + return ppa.pu * dev->geo.num_grp + ppa.grp - dev->range.begin; +} + +static inline int +ftl_ppa_in_range(const struct spdk_ftl_dev *dev, struct ftl_ppa ppa) +{ + unsigned int punit = ftl_ppa_flatten_punit(dev, ppa) + dev->range.begin; + + if (punit >= dev->range.begin && punit <= dev->range.end) { + return 1; + } + + return 0; +} + +#define _ftl_l2p_set(l2p, off, val, bits) \ + __atomic_store_n(((uint##bits##_t *)(l2p)) + (off), val, __ATOMIC_SEQ_CST) + +#define _ftl_l2p_set32(l2p, off, val) \ + _ftl_l2p_set(l2p, off, val, 32) + +#define _ftl_l2p_set64(l2p, off, val) \ + _ftl_l2p_set(l2p, off, val, 64) + +#define _ftl_l2p_get(l2p, off, bits) \ + __atomic_load_n(((uint##bits##_t *)(l2p)) + (off), __ATOMIC_SEQ_CST) + +#define _ftl_l2p_get32(l2p, off) \ + _ftl_l2p_get(l2p, off, 32) + +#define _ftl_l2p_get64(l2p, off) \ + _ftl_l2p_get(l2p, off, 64) + +#define ftl_ppa_cmp(p1, p2) \ + ((p1).ppa == (p2).ppa) + +static inline void +ftl_l2p_set(struct spdk_ftl_dev *dev, uint64_t lba, struct ftl_ppa ppa) +{ + assert(dev->num_lbas > lba); + + if (ftl_ppa_packed(dev)) { + _ftl_l2p_set32(dev->l2p, lba, ftl_ppa_to_packed(dev, ppa).ppa); + } else { + _ftl_l2p_set64(dev->l2p, lba, ppa.ppa); + } +} + +static inline struct ftl_ppa +ftl_l2p_get(struct spdk_ftl_dev *dev, uint64_t lba) +{ + assert(dev->num_lbas > lba); + + if (ftl_ppa_packed(dev)) { + return ftl_ppa_from_packed(dev, ftl_to_ppa_packed( + _ftl_l2p_get32(dev->l2p, lba))); + } else { + return ftl_to_ppa(_ftl_l2p_get64(dev->l2p, lba)); + } +} +static inline size_t +ftl_dev_num_bands(const struct spdk_ftl_dev *dev) +{ + return dev->geo.num_chk; +} + +static inline size_t +ftl_dev_lbks_in_chunk(const struct spdk_ftl_dev *dev) +{ + return dev->geo.clba; +} + +static inline size_t +ftl_dev_num_punits(const struct spdk_ftl_dev *dev) +{ + return dev->range.end - dev->range.begin + 1; +} + +static inline uint64_t +ftl_num_band_lbks(const struct spdk_ftl_dev *dev) +{ + return ftl_dev_num_punits(dev) * ftl_dev_lbks_in_chunk(dev); +} + +static inline size_t +ftl_vld_map_size(const struct spdk_ftl_dev *dev) +{ + return (size_t)spdk_divide_round_up(ftl_num_band_lbks(dev), CHAR_BIT); +} + +static inline struct ftl_trace * +ftl_dev_trace(struct spdk_ftl_dev *dev) +{ + return dev->stats.trace; +} + +#endif /* FTL_CORE_H */ diff --git a/lib/ftl/ftl_io.h b/lib/ftl/ftl_io.h new file mode 100644 index 000000000..b5f0f9409 --- /dev/null +++ b/lib/ftl/ftl_io.h @@ -0,0 +1,266 @@ +/*- + * BSD LICENSE + * + * Copyright (c) Intel Corporation. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * + * * Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * * Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in + * the documentation and/or other materials provided with the + * distribution. + * * Neither the name of Intel Corporation nor the names of its + * contributors may be used to endorse or promote products derived + * from this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + */ + +#ifndef FTL_IO_H +#define FTL_IO_H + +#include +#include +#include +#include "ftl_ppa.h" +#include "ftl_trace.h" + +struct spdk_ftl_dev; +struct ftl_rwb_batch; +struct ftl_band; +struct ftl_io; +struct ftl_md; + +typedef int (*ftl_md_pack_fn)(struct spdk_ftl_dev *, struct ftl_md *, void *); + +/* IO flags */ +enum ftl_io_flags { + /* Indicates whether IO is already initialized */ + FTL_IO_INITIALIZED = (1 << 0), + /* Keep the IO when done with the request */ + FTL_IO_KEEP_ALIVE = (1 << 1), + /* Internal based IO (defrag, metadata etc.) */ + FTL_IO_INTERNAL = (1 << 2), + /* Indicates that the IO should not go through if there's */ + /* already another one scheduled to the same LBA */ + FTL_IO_WEAK = (1 << 3), + /* Indicates that the IO is used for padding */ + FTL_IO_PAD = (1 << 4), + /* The IO operates on metadata */ + FTL_IO_MD = (1 << 5), + /* Using PPA instead of LBA */ + FTL_IO_PPA_MODE = (1 << 6), + /* Indicates that IO contains noncontiguous LBAs */ + FTL_IO_VECTOR_LBA = (1 << 7), +}; + +enum ftl_io_type { + FTL_IO_READ, + FTL_IO_WRITE, + FTL_IO_ERASE, +}; + +struct ftl_io_init_opts { + struct spdk_ftl_dev *dev; + + /* IO descriptor */ + struct ftl_io *io; + + /* Size of IO descriptor */ + size_t size; + + /* IO flags */ + int flags; + + /* IO type */ + enum ftl_io_type type; + + /* Number of split requests */ + size_t iov_cnt; + + /* RWB entry */ + struct ftl_rwb_batch *rwb_batch; + + /* Band to which the IO is directed */ + struct ftl_band *band; + + /* Request size */ + size_t req_size; + + /* Data */ + void *data; + + /* Metadata */ + void *md; + + /* Callback */ + spdk_ftl_fn fn; +}; + +struct ftl_cb { + /* Callback function */ + spdk_ftl_fn fn; + + /* Callback's context */ + void *ctx; +}; + +struct ftl_io_channel { + /* IO pool element size */ + size_t elem_size; + + /* IO pool */ + struct spdk_mempool *io_pool; +}; + +/* General IO descriptor */ +struct ftl_io { + /* Device */ + struct spdk_ftl_dev *dev; + + /* IO channel */ + struct spdk_io_channel *ch; + + union { + /* LBA table */ + uint64_t *lbas; + + /* First LBA */ + uint64_t lba; + }; + + /* First PPA */ + struct ftl_ppa ppa; + + /* Number of processed lbks */ + size_t pos; + + /* Number of lbks */ + size_t lbk_cnt; + + union { + /* IO vector table */ + struct iovec *iovs; + + /* Single iovec */ + struct iovec iov; + }; + + /* Metadata */ + void *md; + + /* Number of IO vectors */ + size_t iov_cnt; + + /* Position within the iovec */ + size_t iov_pos; + + /* Offset within the iovec (in lbks) */ + size_t iov_off; + + /* RWB entry (valid only for RWB-based IO) */ + struct ftl_rwb_batch *rwb_batch; + + /* Band this IO is being written to */ + struct ftl_band *band; + + /* Request status */ + int status; + + /* Number of split requests */ + size_t req_cnt; + + /* Completion callback */ + struct ftl_cb cb; + + /* Flags */ + int flags; + + /* IO type */ + enum ftl_io_type type; + + /* Trace group id */ + ftl_trace_group_t trace; +}; + +/* Metadata IO */ +struct ftl_md_io { + /* Parent IO structure */ + struct ftl_io io; + + /* Destination metadata pointer */ + struct ftl_md *md; + + /* Metadata's buffer */ + void *buf; + + /* Serialization/deserialization callback */ + ftl_md_pack_fn pack_fn; + + /* User's callback */ + struct ftl_cb cb; +}; + +static inline bool +ftl_io_mode_ppa(const struct ftl_io *io) +{ + return io->flags & FTL_IO_PPA_MODE; +} + +static inline bool +ftl_io_mode_lba(const struct ftl_io *io) +{ + return !ftl_io_mode_ppa(io); +} + +static inline bool +ftl_io_done(const struct ftl_io *io) +{ + return io->req_cnt == 0; +} + +struct ftl_io *ftl_io_alloc(struct spdk_io_channel *ch); +void ftl_io_free(struct ftl_io *io); +struct ftl_io *ftl_io_init_internal(const struct ftl_io_init_opts *opts); +void ftl_io_reinit(struct ftl_io *io, spdk_ftl_fn cb, + void *ctx, int flags, int type); +void ftl_io_clear(struct ftl_io *io); +size_t ftl_io_inc_req(struct ftl_io *io); +size_t ftl_io_dec_req(struct ftl_io *io); +struct iovec *ftl_io_iovec(struct ftl_io *io); +uint64_t ftl_io_current_lba(struct ftl_io *io); +void ftl_io_update_iovec(struct ftl_io *io, size_t lbk_cnt); +size_t ftl_iovec_num_lbks(struct iovec *iov, size_t iov_cnt); +void *ftl_io_iovec_addr(struct ftl_io *io); +size_t ftl_io_iovec_len_left(struct ftl_io *io); +int ftl_io_init_iovec(struct ftl_io *io, void *buf, + size_t iov_cnt, size_t req_size); +void ftl_io_init(struct ftl_io *io, struct spdk_ftl_dev *dev, + spdk_ftl_fn cb, void *ctx, int flags, int type); +struct ftl_io *ftl_io_init_internal(const struct ftl_io_init_opts *opts); +struct ftl_io *ftl_io_rwb_init(struct spdk_ftl_dev *dev, struct ftl_band *band, + struct ftl_rwb_batch *entry, spdk_ftl_fn cb); +struct ftl_io *ftl_io_erase_init(struct ftl_band *band, size_t lbk_cnt, spdk_ftl_fn cb); +void ftl_io_user_init(struct spdk_ftl_dev *dev, struct ftl_io *io, uint64_t lba, size_t lbk_cnt, + struct iovec *iov, size_t iov_cnt, + spdk_ftl_fn fn, void *cb_arg, int type); +void *ftl_io_get_md(const struct ftl_io *io); +void ftl_io_complete(struct ftl_io *io); +void ftl_io_process_error(struct ftl_io *io, const struct spdk_nvme_cpl *status); + +#endif /* FTL_IO_H */ diff --git a/lib/ftl/ftl_ppa.h b/lib/ftl/ftl_ppa.h new file mode 100644 index 000000000..c4e11372f --- /dev/null +++ b/lib/ftl/ftl_ppa.h @@ -0,0 +1,102 @@ +/*- + * BSD LICENSE + * + * Copyright (c) Intel Corporation. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * + * * Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * * Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in + * the documentation and/or other materials provided with the + * distribution. + * * Neither the name of Intel Corporation nor the names of its + * contributors may be used to endorse or promote products derived + * from this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + */ + +#ifndef FTL_PPA_H +#define FTL_PPA_H + +#include + +/* Marks PPA as invalid */ +#define FTL_PPA_INVALID (-1) +/* Marks LBA as invalid */ +#define FTL_LBA_INVALID ((uint64_t)-1) +/* Smallest data unit size */ +#define FTL_BLOCK_SIZE 4096 + +/* This structure represents PPA address. It can have one of the following */ +/* formats: */ +/* - PPA describing the on-disk address */ +/* - offset inside the cache (indicated by the cached flag) */ +/* - packed version of the two formats above (can be only used when the */ +/* on-disk PPA address can be represented in less than 32 bits) */ +/* Packed format is used, when possible, to avoid wasting RAM on the L2P table. */ +struct ftl_ppa { + union { + struct { + uint64_t lbk : 32; + uint64_t chk : 16; + uint64_t pu : 8; + uint64_t grp : 7; + uint64_t rsvd : 1; + }; + + struct { + uint64_t offset : 63; + uint64_t cached : 1; + }; + + struct { + union { + struct { + uint32_t offset : 31; + uint32_t cached : 1; + }; + + uint32_t ppa; + }; + uint32_t rsvd; + } pack; + + uint64_t ppa; + }; +}; + +struct ftl_ppa_fmt { + /* Logical block */ + unsigned int lbk_offset; + unsigned int lbk_mask; + + /* Chunk */ + unsigned int chk_offset; + unsigned int chk_mask; + + /* Parallel unit (NAND die) */ + unsigned int pu_offset; + unsigned int pu_mask; + + /* Group */ + unsigned int grp_offset; + unsigned int grp_mask; +}; + +#endif /* FTL_PPA_H */ diff --git a/lib/ftl/ftl_trace.h b/lib/ftl/ftl_trace.h new file mode 100644 index 000000000..fee25cd08 --- /dev/null +++ b/lib/ftl/ftl_trace.h @@ -0,0 +1,111 @@ +/*- + * BSD LICENSE + * + * Copyright (c) Intel Corporation. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * + * * Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * * Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in + * the documentation and/or other materials provided with the + * distribution. + * * Neither the name of Intel Corporation nor the names of its + * contributors may be used to endorse or promote products derived + * from this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + */ + +#ifndef FTL_TRACE_H +#define FTL_TRACE_H + +#include "ftl_ppa.h" + +#define FTL_TRACE_INVALID_ID ((uint64_t) -1) + +typedef uint64_t ftl_trace_group_t; + +enum ftl_trace_source { + FTL_TRACE_SOURCE_INTERNAL = 0x20, + FTL_TRACE_SOURCE_USER, +}; + +enum ftl_trace_type { + FTL_TRACE_TYPE_READ, + FTL_TRACE_TYPE_MD_READ, + FTL_TRACE_TYPE_WRITE, + FTL_TRACE_TYPE_MD_WRITE, + FTL_TRACE_TYPE_ERASE, + FTL_TRACE_TYPE_OTHER, +}; + +enum ftl_trace_point { + FTL_TRACE_POINT_SCHEDULED, + FTL_TRACE_POINT_RWB_FILL, + FTL_TRACE_POINT_SUBMISSION, + FTL_TRACE_POINT_OTHER, +}; + +enum ftl_trace_completion { + FTL_TRACE_COMPLETION_INVALID = FTL_TRACE_POINT_OTHER + 1, + FTL_TRACE_COMPLETION_CACHE, + FTL_TRACE_COMPLETION_DISK, +}; + +struct ftl_event { + /* Id used for grouping multiple events of the same request */ + uint64_t id; +} __attribute__((packed)); + + +struct ftl_trace; +struct ftl_io; +struct ftl_rwb_entry; +struct ftl_band; + +#if defined(FTL_TRACE_ENABLED) + +#define ftl_trace(fn, trace, ...) \ + do { \ + if (trace) { \ + ftl_trace_##fn(trace, ## __VA_ARGS__); \ + } \ + } while (0) + +struct ftl_trace *ftl_trace_init(void); +void ftl_trace_free(struct ftl_trace *trace); +ftl_trace_group_t ftl_trace_alloc_group(struct ftl_trace *trace); +void ftl_trace_defrag_band(struct ftl_trace *trace, const struct ftl_band *band); +void ftl_trace_write_band(struct ftl_trace *trace, const struct ftl_band *band); +void ftl_trace_lba_io_init(struct ftl_trace *trace, const struct ftl_io *io); +void ftl_trace_rwb_fill(struct ftl_trace *trace, const struct ftl_io *io); +void ftl_trace_rwb_pop(struct ftl_trace *trace, const struct ftl_rwb_entry *entry); +void ftl_trace_submission(struct ftl_trace *trace, + const struct ftl_io *io, + struct ftl_ppa ppa, size_t ppa_cnt); +void ftl_trace_completion(struct ftl_trace *trace, + const struct ftl_io *io, + enum ftl_trace_completion type); +void ftl_trace_limits(struct ftl_trace *trace, const size_t *limits, size_t num_free); + +#else +#define ftl_trace(fn, trace, ...) +#define ftl_trace_alloc_group(trace) FTL_TRACE_INVALID_ID +#endif + +#endif /* FTL_TRACE_H */