[PATCH v2 5/5] nvme_fc: add controller reset support
Sagi Grimberg
sagi at grimberg.me
Thu Apr 20 05:56:20 PDT 2017
> From: James Smart <jsmart2021 at gmail.com>
>
> This patch actually does quite a few things. When looking to add
> controller reset support, the organization modeled after rdma was
> very fragmented. rdma duplicates the reset and teardown paths and does
> different things to the block layer on the two paths. The code to build
> up the controller is also duplicated between the initial creation and
> the reset/error recovery paths. So I decided to make this sane.
>
> I reorganized the controller creation and teardown so that there is a
> connect path and a disconnect path. Initial creation obviously uses
> the connect path. Controller teardown will use the disconnect path,
> followed last access code. Controller reset will use the disconnect
> path to stop operation, and then the connect path to re-establish
> the controller.
>
> Along the way, several things were fixed
> - aens were not properly set up. They are allocated differently from
> the per-request structure on the blk queues.
> - aens were oddly torn down. the prior patch corrected to abort, but
> we still need to dma unmap and free relative elements.
> - missed a few ref counting points: in aen completion and on i/o's
> that fail
> - controller initial create failure paths were still confused vs teardown
> before converting to ref counting vs after we convert to refcounting.
It would be very helpful to split this patch into several pieces:
1. fixes and cleanups
2. preps for teardown and establishment
3. delete support
4. reset support
5. reconnect support
As is, its very hard to review this patch...
>
> Signed-off-by: James Smart <james.smart at broadcom.com>
> ---
> drivers/nvme/host/fc.c | 1049 +++++++++++++++++++++++++++++++-----------------
> 1 file changed, 675 insertions(+), 374 deletions(-)
>
> diff --git a/drivers/nvme/host/fc.c b/drivers/nvme/host/fc.c
> index 759f9ef..b6d2ca8 100644
> --- a/drivers/nvme/host/fc.c
> +++ b/drivers/nvme/host/fc.c
> @@ -19,6 +19,7 @@
> #include <linux/parser.h>
> #include <uapi/scsi/fc/fc_fs.h>
> #include <uapi/scsi/fc/fc_els.h>
> +#include <linux/delay.h>
>
> #include "nvme.h"
> #include "fabrics.h"
> @@ -44,6 +45,8 @@ enum nvme_fc_queue_flags {
>
> #define NVMEFC_QUEUE_DELAY 3 /* ms units */
>
> +#define NVME_FC_MAX_CONNECT_ATTEMPTS 1
> +
Why 1?
> struct nvme_fc_queue {
> struct nvme_fc_ctrl *ctrl;
> struct device *dev;
> @@ -137,19 +140,17 @@ struct nvme_fc_rport {
> struct kref ref;
> } __aligned(sizeof(u64)); /* alignment for other things alloc'd with */
>
> -enum nvme_fcctrl_state {
> - FCCTRL_INIT = 0,
> - FCCTRL_ACTIVE = 1,
> +enum nvme_fcctrl_flags {
> + FCCTRL_TERMIO = (1 << 0),
> };
>
> struct nvme_fc_ctrl {
> spinlock_t lock;
> struct nvme_fc_queue *queues;
> - u32 queue_count;
> -
> struct device *dev;
> struct nvme_fc_lport *lport;
> struct nvme_fc_rport *rport;
> + u32 queue_count;
> u32 cnum;
>
> u64 association_id;
> @@ -162,8 +163,14 @@ struct nvme_fc_ctrl {
> struct blk_mq_tag_set tag_set;
>
> struct work_struct delete_work;
> + struct work_struct reset_work;
> + struct delayed_work connect_work;
> + int reconnect_delay;
> + int connect_attempts;
> +
> struct kref ref;
> - int state;
> + u32 flags;
> + u32 iocnt;
>
> struct nvme_fc_fcp_op aen_ops[NVME_FC_NR_AEN_COMMANDS];
>
> @@ -1204,7 +1211,10 @@ nvme_fc_abort_aen_ops(struct nvme_fc_ctrl *ctrl)
> continue;
>
> spin_lock_irqsave(&ctrl->lock, flags);
> - aen_op->flags |= FCOP_FLAGS_TERMIO;
> + if (ctrl->flags & FCCTRL_TERMIO) {
> + ctrl->iocnt++;
> + aen_op->flags |= FCOP_FLAGS_TERMIO;
> + }
> spin_unlock_irqrestore(&ctrl->lock, flags);
>
> ret = __nvme_fc_abort_op(ctrl, aen_op);
> @@ -1217,6 +1227,8 @@ nvme_fc_abort_aen_ops(struct nvme_fc_ctrl *ctrl)
>
> /* back out the flags/counters */
> spin_lock_irqsave(&ctrl->lock, flags);
> + if (ctrl->flags & FCCTRL_TERMIO)
> + ctrl->iocnt--;
> aen_op->flags &= ~FCOP_FLAGS_TERMIO;
> spin_unlock_irqrestore(&ctrl->lock, flags);
> return;
> @@ -1232,6 +1244,10 @@ __nvme_fc_fcpop_chk_teardowns(struct nvme_fc_ctrl *ctrl,
> bool complete_rq = false;
>
> spin_lock_irqsave(&ctrl->lock, flags);
> + if (unlikely(op->flags & FCOP_FLAGS_TERMIO)) {
> + if (ctrl->flags & FCCTRL_TERMIO)
> + ctrl->iocnt--;
> + }
> if (op->flags & FCOP_FLAGS_RELEASED)
> complete_rq = true;
> else
> @@ -1447,19 +1463,29 @@ nvme_fc_init_aen_ops(struct nvme_fc_ctrl *ctrl)
> struct nvme_fc_fcp_op *aen_op;
> struct nvme_fc_cmd_iu *cmdiu;
> struct nvme_command *sqe;
> + void *private;
> int i, ret;
>
> aen_op = ctrl->aen_ops;
> for (i = 0; i < NVME_FC_NR_AEN_COMMANDS; i++, aen_op++) {
> + private = kzalloc(ctrl->lport->ops->fcprqst_priv_sz,
> + GFP_KERNEL);
> + if (!private)
> + return -ENOMEM;
> +
> cmdiu = &aen_op->cmd_iu;
> sqe = &cmdiu->sqe;
> ret = __nvme_fc_init_request(ctrl, &ctrl->queues[0],
> aen_op, (struct request *)NULL,
> (AEN_CMDID_BASE + i));
> - if (ret)
> + if (ret) {
> + kfree(private);
> return ret;
> + }
>
> aen_op->flags = FCOP_FLAGS_AEN;
> + aen_op->fcp_req.first_sgl = NULL; /* no sg list */
> + aen_op->fcp_req.private = private;
>
> memset(sqe, 0, sizeof(*sqe));
> sqe->common.opcode = nvme_admin_async_event;
> @@ -1469,6 +1495,23 @@ nvme_fc_init_aen_ops(struct nvme_fc_ctrl *ctrl)
> return 0;
> }
>
> +static void
> +nvme_fc_term_aen_ops(struct nvme_fc_ctrl *ctrl)
> +{
> + struct nvme_fc_fcp_op *aen_op;
> + int i;
> +
> + aen_op = ctrl->aen_ops;
> + for (i = 0; i < NVME_FC_NR_AEN_COMMANDS; i++, aen_op++) {
> + if (!aen_op->fcp_req.private)
> + continue;
> +
> + __nvme_fc_exit_request(ctrl, aen_op);
> +
> + kfree(aen_op->fcp_req.private);
> + aen_op->fcp_req.private = NULL;
> + }
> +}
>
> static inline void
> __nvme_fc_init_hctx(struct blk_mq_hw_ctx *hctx, struct nvme_fc_ctrl *ctrl,
> @@ -1568,15 +1611,6 @@ __nvme_fc_delete_hw_queue(struct nvme_fc_ctrl *ctrl,
> }
>
> static void
> -nvme_fc_destroy_admin_queue(struct nvme_fc_ctrl *ctrl)
> -{
> - __nvme_fc_delete_hw_queue(ctrl, &ctrl->queues[0], 0);
> - blk_cleanup_queue(ctrl->ctrl.admin_q);
> - blk_mq_free_tag_set(&ctrl->admin_tag_set);
> - nvme_fc_free_queue(&ctrl->queues[0]);
> -}
> -
> -static void
> nvme_fc_free_io_queues(struct nvme_fc_ctrl *ctrl)
> {
> int i;
> @@ -1663,17 +1697,24 @@ nvme_fc_ctrl_free(struct kref *ref)
> container_of(ref, struct nvme_fc_ctrl, ref);
> unsigned long flags;
>
> - if (ctrl->state != FCCTRL_INIT) {
> - /* remove from rport list */
> - spin_lock_irqsave(&ctrl->rport->lock, flags);
> - list_del(&ctrl->ctrl_list);
> - spin_unlock_irqrestore(&ctrl->rport->lock, flags);
> + if (ctrl->ctrl.tagset) {
> + blk_cleanup_queue(ctrl->ctrl.connect_q);
> + blk_mq_free_tag_set(&ctrl->tag_set);
> }
>
> + /* remove from rport list */
> + spin_lock_irqsave(&ctrl->rport->lock, flags);
> + list_del(&ctrl->ctrl_list);
> + spin_unlock_irqrestore(&ctrl->rport->lock, flags);
> +
> + blk_cleanup_queue(ctrl->ctrl.admin_q);
> + blk_mq_free_tag_set(&ctrl->admin_tag_set);
> +
> + kfree(ctrl->queues);
> +
> put_device(ctrl->dev);
> nvme_fc_rport_put(ctrl->rport);
>
> - kfree(ctrl->queues);
> ida_simple_remove(&nvme_fc_ctrl_cnt, ctrl->cnum);
> nvmf_free_options(ctrl->ctrl.opts);
> kfree(ctrl);
> @@ -1696,32 +1737,35 @@ nvme_fc_ctrl_get(struct nvme_fc_ctrl *ctrl)
> * controller. Called after last nvme_put_ctrl() call
> */
> static void
> -nvme_fc_free_nvme_ctrl(struct nvme_ctrl *nctrl)
> +nvme_fc_nvme_ctrl_freed(struct nvme_ctrl *nctrl)
> {
> struct nvme_fc_ctrl *ctrl = to_fc_ctrl(nctrl);
>
> WARN_ON(nctrl != &ctrl->ctrl);
>
> - /*
> - * Tear down the association, which will generate link
> - * traffic to terminate connections
> - */
> -
> - if (ctrl->state != FCCTRL_INIT) {
> - /* send a Disconnect(association) LS to fc-nvme target */
> - nvme_fc_xmt_disconnect_assoc(ctrl);
> + nvme_fc_ctrl_put(ctrl);
> +}
>
> - if (ctrl->ctrl.tagset) {
> - blk_cleanup_queue(ctrl->ctrl.connect_q);
> - blk_mq_free_tag_set(&ctrl->tag_set);
> - nvme_fc_delete_hw_io_queues(ctrl);
> - nvme_fc_free_io_queues(ctrl);
> - }
> +static void
> +nvme_fc_error_recovery(struct nvme_fc_ctrl *ctrl, char *errmsg)
> +{
> + dev_warn(ctrl->ctrl.device,
> + "NVME-FC{%d}: transport association error detected: %s\n",
> + ctrl->cnum, errmsg);
> + dev_info(ctrl->ctrl.device,
> + "NVME-FC{%d}: resetting controller\n", ctrl->cnum);
>
> - nvme_fc_destroy_admin_queue(ctrl);
> + if (!nvme_change_ctrl_state(&ctrl->ctrl, NVME_CTRL_RECONNECTING)) {
> + dev_err(ctrl->ctrl.device,
> + "NVME-FC{%d}: error_recovery: Couldn't change state "
> + "to RECONNECTING\n", ctrl->cnum);
> + return;
> }
>
> - nvme_fc_ctrl_put(ctrl);
> + if (!queue_work(nvme_fc_wq, &ctrl->reset_work))
> + dev_err(ctrl->ctrl.device,
> + "NVME-FC{%d}: error_recovery: Failed to schedule "
> + "reset work\n", ctrl->cnum);
> }
Don't you want to stop the queues and fail inflight I/O?
>
> enum blk_eh_timer_return
> @@ -1740,11 +1784,13 @@ nvme_fc_timeout(struct request *rq, bool reserved)
> return BLK_EH_HANDLED;
>
> /*
> - * TODO: force a controller reset
> - * when that happens, queues will be torn down and outstanding
> - * ios will be terminated, and the above abort, on a single io
> - * will no longer be needed.
> + * we can't individually ABTS an io without affecting the queue,
> + * thus killing the queue, adn thus the association.
> + * So resolve by performing a controller reset, which will stop
> + * the host/io stack, terminate the association on the link,
> + * and recreate an association on the link.
> */
> + nvme_fc_error_recovery(ctrl, "io timeout error");
>
> return BLK_EH_HANDLED;
> }
> @@ -1838,6 +1884,13 @@ nvme_fc_start_fcp_op(struct nvme_fc_ctrl *ctrl, struct nvme_fc_queue *queue,
> u32 csn;
> int ret;
>
> + /*
> + * before attempting to send the io, check to see if we believe
> + * the target device is present
> + */
> + if (ctrl->rport->remoteport.port_state != FC_OBJSTATE_ONLINE)
> + return BLK_MQ_RQ_QUEUE_ERROR;
> +
> if (!nvme_fc_ctrl_get(ctrl))
> return BLK_MQ_RQ_QUEUE_ERROR;
>
> @@ -1885,8 +1938,6 @@ nvme_fc_start_fcp_op(struct nvme_fc_ctrl *ctrl, struct nvme_fc_queue *queue,
> if (!(op->flags & FCOP_FLAGS_AEN)) {
> ret = nvme_fc_map_data(ctrl, op->rq, op);
> if (ret < 0) {
> - dev_err(queue->ctrl->ctrl.device,
> - "Failed to map data (%d)\n", ret);
> nvme_cleanup_cmd(op->rq);
> nvme_fc_ctrl_put(ctrl);
> return (ret == -ENOMEM || ret == -EAGAIN) ?
> @@ -1907,9 +1958,6 @@ nvme_fc_start_fcp_op(struct nvme_fc_ctrl *ctrl, struct nvme_fc_queue *queue,
> queue->lldd_handle, &op->fcp_req);
>
> if (ret) {
> - dev_err(ctrl->dev,
> - "Send nvme command failed - lldd returned %d.\n", ret);
> -
> if (op->rq) { /* normal request */
> nvme_fc_unmap_data(ctrl, op->rq, op);
> nvme_cleanup_cmd(op->rq);
> @@ -1979,12 +2027,8 @@ nvme_fc_poll(struct blk_mq_hw_ctx *hctx, unsigned int tag)
> struct nvme_fc_fcp_op *op;
>
> req = blk_mq_tag_to_rq(nvme_fc_tagset(queue), tag);
> - if (!req) {
> - dev_err(queue->ctrl->ctrl.device,
> - "tag 0x%x on QNum %#x not found\n",
> - tag, queue->qnum);
> + if (!req)
> return 0;
> - }
>
> op = blk_mq_rq_to_pdu(req);
>
> @@ -2001,11 +2045,21 @@ nvme_fc_submit_async_event(struct nvme_ctrl *arg, int aer_idx)
> {
> struct nvme_fc_ctrl *ctrl = to_fc_ctrl(arg);
> struct nvme_fc_fcp_op *aen_op;
> + unsigned long flags;
> + bool terminating = false;
> int ret;
>
> if (aer_idx > NVME_FC_NR_AEN_COMMANDS)
> return;
>
> + spin_lock_irqsave(&ctrl->lock, flags);
> + if (ctrl->flags & FCCTRL_TERMIO)
> + terminating = true;
> + spin_unlock_irqrestore(&ctrl->lock, flags);
> +
> + if (terminating)
> + return;
> +
> aen_op = &ctrl->aen_ops[aer_idx];
>
> ret = nvme_fc_start_fcp_op(ctrl, aen_op->queue, aen_op, 0,
> @@ -2059,110 +2113,6 @@ nvme_fc_complete_rq(struct request *rq)
> __nvme_fc_final_op_cleanup(rq);
> }
>
> -static const struct blk_mq_ops nvme_fc_mq_ops = {
> - .queue_rq = nvme_fc_queue_rq,
> - .complete = nvme_fc_complete_rq,
> - .init_request = nvme_fc_init_request,
> - .exit_request = nvme_fc_exit_request,
> - .reinit_request = nvme_fc_reinit_request,
> - .init_hctx = nvme_fc_init_hctx,
> - .poll = nvme_fc_poll,
> - .timeout = nvme_fc_timeout,
> -};
> -
> -static const struct blk_mq_ops nvme_fc_admin_mq_ops = {
> - .queue_rq = nvme_fc_queue_rq,
> - .complete = nvme_fc_complete_rq,
> - .init_request = nvme_fc_init_admin_request,
> - .exit_request = nvme_fc_exit_request,
> - .reinit_request = nvme_fc_reinit_request,
> - .init_hctx = nvme_fc_init_admin_hctx,
> - .timeout = nvme_fc_timeout,
> -};
> -
> -static int
> -nvme_fc_configure_admin_queue(struct nvme_fc_ctrl *ctrl)
> -{
> - u32 segs;
> - int error;
> -
> - nvme_fc_init_queue(ctrl, 0, NVME_FC_AQ_BLKMQ_DEPTH);
> -
> - error = nvme_fc_connect_admin_queue(ctrl, &ctrl->queues[0],
> - NVME_FC_AQ_BLKMQ_DEPTH,
> - (NVME_FC_AQ_BLKMQ_DEPTH / 4));
> - if (error)
> - return error;
> -
> - memset(&ctrl->admin_tag_set, 0, sizeof(ctrl->admin_tag_set));
> - ctrl->admin_tag_set.ops = &nvme_fc_admin_mq_ops;
> - ctrl->admin_tag_set.queue_depth = NVME_FC_AQ_BLKMQ_DEPTH;
> - ctrl->admin_tag_set.reserved_tags = 2; /* fabric connect + Keep-Alive */
> - ctrl->admin_tag_set.numa_node = NUMA_NO_NODE;
> - ctrl->admin_tag_set.cmd_size = sizeof(struct nvme_fc_fcp_op) +
> - (SG_CHUNK_SIZE *
> - sizeof(struct scatterlist)) +
> - ctrl->lport->ops->fcprqst_priv_sz;
> - ctrl->admin_tag_set.driver_data = ctrl;
> - ctrl->admin_tag_set.nr_hw_queues = 1;
> - ctrl->admin_tag_set.timeout = ADMIN_TIMEOUT;
> -
> - error = blk_mq_alloc_tag_set(&ctrl->admin_tag_set);
> - if (error)
> - goto out_free_queue;
> -
> - ctrl->ctrl.admin_q = blk_mq_init_queue(&ctrl->admin_tag_set);
> - if (IS_ERR(ctrl->ctrl.admin_q)) {
> - error = PTR_ERR(ctrl->ctrl.admin_q);
> - goto out_free_tagset;
> - }
> -
> - error = __nvme_fc_create_hw_queue(ctrl, &ctrl->queues[0], 0,
> - NVME_FC_AQ_BLKMQ_DEPTH);
> - if (error)
> - goto out_cleanup_queue;
> -
> - error = nvmf_connect_admin_queue(&ctrl->ctrl);
> - if (error)
> - goto out_delete_hw_queue;
> -
> - error = nvmf_reg_read64(&ctrl->ctrl, NVME_REG_CAP, &ctrl->cap);
> - if (error) {
> - dev_err(ctrl->ctrl.device,
> - "prop_get NVME_REG_CAP failed\n");
> - goto out_delete_hw_queue;
> - }
> -
> - ctrl->ctrl.sqsize =
> - min_t(int, NVME_CAP_MQES(ctrl->cap) + 1, ctrl->ctrl.sqsize);
> -
> - error = nvme_enable_ctrl(&ctrl->ctrl, ctrl->cap);
> - if (error)
> - goto out_delete_hw_queue;
> -
> - segs = min_t(u32, NVME_FC_MAX_SEGMENTS,
> - ctrl->lport->ops->max_sgl_segments);
> - ctrl->ctrl.max_hw_sectors = (segs - 1) << (PAGE_SHIFT - 9);
> -
> - error = nvme_init_identify(&ctrl->ctrl);
> - if (error)
> - goto out_delete_hw_queue;
> -
> - nvme_start_keep_alive(&ctrl->ctrl);
> -
> - return 0;
> -
> -out_delete_hw_queue:
> - __nvme_fc_delete_hw_queue(ctrl, &ctrl->queues[0], 0);
> -out_cleanup_queue:
> - blk_cleanup_queue(ctrl->ctrl.admin_q);
> -out_free_tagset:
> - blk_mq_free_tag_set(&ctrl->admin_tag_set);
> -out_free_queue:
> - nvme_fc_free_queue(&ctrl->queues[0]);
> - return error;
> -}
> -
> /*
> * This routine is used by the transport when it needs to find active
> * io on a queue that is to be terminated. The transport uses
> @@ -2189,7 +2139,10 @@ nvme_fc_terminate_exchange(struct request *req, void *data, bool reserved)
> return;
>
> spin_lock_irqsave(&ctrl->lock, flags);
> - op->flags |= FCOP_FLAGS_TERMIO;
> + if (ctrl->flags & FCCTRL_TERMIO) {
> + ctrl->iocnt++;
> + op->flags |= FCOP_FLAGS_TERMIO;
> + }
> spin_unlock_irqrestore(&ctrl->lock, flags);
>
> status = __nvme_fc_abort_op(ctrl, op);
> @@ -2202,144 +2155,101 @@ nvme_fc_terminate_exchange(struct request *req, void *data, bool reserved)
>
> /* back out the flags/counters */
> spin_lock_irqsave(&ctrl->lock, flags);
> + if (ctrl->flags & FCCTRL_TERMIO)
> + ctrl->iocnt--;
> op->flags &= ~FCOP_FLAGS_TERMIO;
> spin_unlock_irqrestore(&ctrl->lock, flags);
> return;
> }
> }
>
> -/*
> - * This routine stops operation of the controller. Admin and IO queues
> - * are stopped, outstanding ios on them terminated, and the nvme ctrl
> - * is shutdown.
> - */
> -static void
> -nvme_fc_shutdown_ctrl(struct nvme_fc_ctrl *ctrl)
> -{
> - /*
> - * If io queues are present, stop them and terminate all outstanding
> - * ios on them. As FC allocates FC exchange for each io, the
> - * transport must contact the LLDD to terminate the exchange,
> - * thus releasing the FC exchange. We use blk_mq_tagset_busy_itr()
> - * to tell us what io's are busy and invoke a transport routine
> - * to kill them with the LLDD. After terminating the exchange
> - * the LLDD will call the transport's normal io done path, but it
> - * will have an aborted status. The done path will return the
> - * io requests back to the block layer as part of normal completions
> - * (but with error status).
> - */
> - if (ctrl->queue_count > 1) {
> - nvme_stop_queues(&ctrl->ctrl);
> - blk_mq_tagset_busy_iter(&ctrl->tag_set,
> - nvme_fc_terminate_exchange, &ctrl->ctrl);
> - }
> -
> - if (ctrl->ctrl.state == NVME_CTRL_LIVE)
> - nvme_shutdown_ctrl(&ctrl->ctrl);
> -
> - /*
> - * now clean up the admin queue. Same thing as above.
> - * use blk_mq_tagset_busy_itr() and the transport routine to
> - * terminate the exchanges.
> - */
> - blk_mq_stop_hw_queues(ctrl->ctrl.admin_q);
> - blk_mq_tagset_busy_iter(&ctrl->admin_tag_set,
> - nvme_fc_terminate_exchange, &ctrl->ctrl);
>
> - /* kill the aens as they are a separate path */
> - nvme_fc_abort_aen_ops(ctrl);
> -}
> +static const struct blk_mq_ops nvme_fc_mq_ops = {
> + .queue_rq = nvme_fc_queue_rq,
> + .complete = nvme_fc_complete_rq,
> + .init_request = nvme_fc_init_request,
> + .exit_request = nvme_fc_exit_request,
> + .reinit_request = nvme_fc_reinit_request,
> + .init_hctx = nvme_fc_init_hctx,
> + .poll = nvme_fc_poll,
> + .timeout = nvme_fc_timeout,
> +};
>
> -/*
> - * Called to teardown an association.
> - * May be called with association fully in place or partially in place.
> - */
> -static void
> -__nvme_fc_remove_ctrl(struct nvme_fc_ctrl *ctrl)
> +static int
> +nvme_fc_create_io_queues(struct nvme_fc_ctrl *ctrl)
> {
> - nvme_stop_keep_alive(&ctrl->ctrl);
> + struct nvmf_ctrl_options *opts = ctrl->ctrl.opts;
> + int ret;
>
> - /* stop and terminate ios on admin and io queues */
> - nvme_fc_shutdown_ctrl(ctrl);
> + ret = nvme_set_queue_count(&ctrl->ctrl, &opts->nr_io_queues);
> + if (ret) {
> + dev_info(ctrl->ctrl.device,
> + "set_queue_count failed: %d\n", ret);
> + return ret;
> + }
>
> - /*
> - * tear down the controller
> - * This will result in the last reference on the nvme ctrl to
> - * expire, calling the transport nvme_fc_free_nvme_ctrl() callback.
> - * From there, the transport will tear down it's logical queues and
> - * association.
> - */
> - nvme_uninit_ctrl(&ctrl->ctrl);
> + ctrl->queue_count = opts->nr_io_queues + 1;
> + if (!opts->nr_io_queues)
> + return 0;
>
> - nvme_put_ctrl(&ctrl->ctrl);
> -}
> + dev_info(ctrl->ctrl.device, "creating %d I/O queues.\n",
> + opts->nr_io_queues);
>
> -static void
> -nvme_fc_del_ctrl_work(struct work_struct *work)
> -{
> - struct nvme_fc_ctrl *ctrl =
> - container_of(work, struct nvme_fc_ctrl, delete_work);
> + nvme_fc_init_io_queues(ctrl);
>
> - __nvme_fc_remove_ctrl(ctrl);
> -}
> + memset(&ctrl->tag_set, 0, sizeof(ctrl->tag_set));
> + ctrl->tag_set.ops = &nvme_fc_mq_ops;
> + ctrl->tag_set.queue_depth = ctrl->ctrl.opts->queue_size;
> + ctrl->tag_set.reserved_tags = 1; /* fabric connect */
> + ctrl->tag_set.numa_node = NUMA_NO_NODE;
> + ctrl->tag_set.flags = BLK_MQ_F_SHOULD_MERGE;
> + ctrl->tag_set.cmd_size = sizeof(struct nvme_fc_fcp_op) +
> + (SG_CHUNK_SIZE *
> + sizeof(struct scatterlist)) +
> + ctrl->lport->ops->fcprqst_priv_sz;
> + ctrl->tag_set.driver_data = ctrl;
> + ctrl->tag_set.nr_hw_queues = ctrl->queue_count - 1;
> + ctrl->tag_set.timeout = NVME_IO_TIMEOUT;
>
> -static int
> -__nvme_fc_del_ctrl(struct nvme_fc_ctrl *ctrl)
> -{
> - if (!nvme_change_ctrl_state(&ctrl->ctrl, NVME_CTRL_DELETING))
> - return -EBUSY;
> + ret = blk_mq_alloc_tag_set(&ctrl->tag_set);
> + if (ret)
> + return ret;
>
> - if (!queue_work(nvme_fc_wq, &ctrl->delete_work))
> - return -EBUSY;
> + ctrl->ctrl.tagset = &ctrl->tag_set;
>
> - return 0;
> -}
> + ctrl->ctrl.connect_q = blk_mq_init_queue(&ctrl->tag_set);
> + if (IS_ERR(ctrl->ctrl.connect_q)) {
> + ret = PTR_ERR(ctrl->ctrl.connect_q);
> + goto out_free_tag_set;
> + }
>
> -/*
> - * Request from nvme core layer to delete the controller
> - */
> -static int
> -nvme_fc_del_nvme_ctrl(struct nvme_ctrl *nctrl)
> -{
> - struct nvme_fc_ctrl *ctrl = to_fc_ctrl(nctrl);
> - struct nvme_fc_rport *rport = ctrl->rport;
> - unsigned long flags;
> - int ret;
> -
> - spin_lock_irqsave(&rport->lock, flags);
> - ret = __nvme_fc_del_ctrl(ctrl);
> - spin_unlock_irqrestore(&rport->lock, flags);
> + ret = nvme_fc_create_hw_io_queues(ctrl, ctrl->ctrl.opts->queue_size);
> if (ret)
> - return ret;
> + goto out_cleanup_blk_queue;
>
> - flush_work(&ctrl->delete_work);
> + ret = nvme_fc_connect_io_queues(ctrl, ctrl->ctrl.opts->queue_size);
> + if (ret)
> + goto out_delete_hw_queues;
>
> return 0;
> -}
>
> -static int
> -nvme_fc_reset_nvme_ctrl(struct nvme_ctrl *nctrl)
> -{
> - return -EIO;
> -}
> +out_delete_hw_queues:
> + nvme_fc_delete_hw_io_queues(ctrl);
> +out_cleanup_blk_queue:
> + nvme_stop_keep_alive(&ctrl->ctrl);
> + blk_cleanup_queue(ctrl->ctrl.connect_q);
> +out_free_tag_set:
> + blk_mq_free_tag_set(&ctrl->tag_set);
> + nvme_fc_free_io_queues(ctrl);
>
> -static const struct nvme_ctrl_ops nvme_fc_ctrl_ops = {
> - .name = "fc",
> - .module = THIS_MODULE,
> - .is_fabrics = true,
> - .reg_read32 = nvmf_reg_read32,
> - .reg_read64 = nvmf_reg_read64,
> - .reg_write32 = nvmf_reg_write32,
> - .reset_ctrl = nvme_fc_reset_nvme_ctrl,
> - .free_ctrl = nvme_fc_free_nvme_ctrl,
> - .submit_async_event = nvme_fc_submit_async_event,
> - .delete_ctrl = nvme_fc_del_nvme_ctrl,
> - .get_subsysnqn = nvmf_get_subsysnqn,
> - .get_address = nvmf_get_address,
> -};
> + /* force put free routine to ignore io queues */
> + ctrl->ctrl.tagset = NULL;
> +
> + return ret;
> +}
>
> static int
> -nvme_fc_create_io_queues(struct nvme_fc_ctrl *ctrl)
> +nvme_fc_reinit_io_queues(struct nvme_fc_ctrl *ctrl)
> {
> struct nvmf_ctrl_options *opts = ctrl->ctrl.opts;
> int ret;
> @@ -2351,44 +2261,22 @@ nvme_fc_create_io_queues(struct nvme_fc_ctrl *ctrl)
> return ret;
> }
>
> - ctrl->queue_count = opts->nr_io_queues + 1;
> - if (!opts->nr_io_queues)
> + /* check for io queues existing */
> + if (ctrl->queue_count == 1)
> return 0;
>
> - dev_info(ctrl->ctrl.device, "creating %d I/O queues.\n",
> + dev_info(ctrl->ctrl.device, "Recreating %d I/O queues.\n",
> opts->nr_io_queues);
>
> nvme_fc_init_io_queues(ctrl);
>
> - memset(&ctrl->tag_set, 0, sizeof(ctrl->tag_set));
> - ctrl->tag_set.ops = &nvme_fc_mq_ops;
> - ctrl->tag_set.queue_depth = ctrl->ctrl.opts->queue_size;
> - ctrl->tag_set.reserved_tags = 1; /* fabric connect */
> - ctrl->tag_set.numa_node = NUMA_NO_NODE;
> - ctrl->tag_set.flags = BLK_MQ_F_SHOULD_MERGE;
> - ctrl->tag_set.cmd_size = sizeof(struct nvme_fc_fcp_op) +
> - (SG_CHUNK_SIZE *
> - sizeof(struct scatterlist)) +
> - ctrl->lport->ops->fcprqst_priv_sz;
> - ctrl->tag_set.driver_data = ctrl;
> - ctrl->tag_set.nr_hw_queues = ctrl->queue_count - 1;
> - ctrl->tag_set.timeout = NVME_IO_TIMEOUT;
> -
> - ret = blk_mq_alloc_tag_set(&ctrl->tag_set);
> + ret = blk_mq_reinit_tagset(&ctrl->tag_set);
> if (ret)
> - return ret;
> -
> - ctrl->ctrl.tagset = &ctrl->tag_set;
> -
> - ctrl->ctrl.connect_q = blk_mq_init_queue(&ctrl->tag_set);
> - if (IS_ERR(ctrl->ctrl.connect_q)) {
> - ret = PTR_ERR(ctrl->ctrl.connect_q);
> - goto out_free_tag_set;
> - }
> + goto out_free_io_queues;
>
> ret = nvme_fc_create_hw_io_queues(ctrl, ctrl->ctrl.opts->queue_size);
> if (ret)
> - goto out_cleanup_blk_queue;
> + goto out_free_io_queues;
>
> ret = nvme_fc_connect_io_queues(ctrl, ctrl->ctrl.opts->queue_size);
> if (ret)
> @@ -2398,28 +2286,440 @@ nvme_fc_create_io_queues(struct nvme_fc_ctrl *ctrl)
>
> out_delete_hw_queues:
> nvme_fc_delete_hw_io_queues(ctrl);
> -out_cleanup_blk_queue:
> - nvme_stop_keep_alive(&ctrl->ctrl);
> - blk_cleanup_queue(ctrl->ctrl.connect_q);
> -out_free_tag_set:
> - blk_mq_free_tag_set(&ctrl->tag_set);
> +out_free_io_queues:
> nvme_fc_free_io_queues(ctrl);
> + return ret;
> +}
>
> - /* force put free routine to ignore io queues */
> - ctrl->ctrl.tagset = NULL;
> +/*
> + * This routine restarts the controller on the host side, and
> + * on the link side, recreates the controller association.
> + */
> +static int
> +nvme_fc_create_association(struct nvme_fc_ctrl *ctrl)
> +{
> + struct nvmf_ctrl_options *opts = ctrl->ctrl.opts;
> + u32 segs;
> + int ret;
> + bool changed;
> +
> + ctrl->connect_attempts++;
> +
> + /*
> + * Create the admin queue
> + */
> +
> + nvme_fc_init_queue(ctrl, 0, NVME_FC_AQ_BLKMQ_DEPTH);
> +
> + ret = __nvme_fc_create_hw_queue(ctrl, &ctrl->queues[0], 0,
> + NVME_FC_AQ_BLKMQ_DEPTH);
> + if (ret)
> + goto out_free_queue;
> +
> + ret = nvme_fc_connect_admin_queue(ctrl, &ctrl->queues[0],
> + NVME_FC_AQ_BLKMQ_DEPTH,
> + (NVME_FC_AQ_BLKMQ_DEPTH / 4));
> + if (ret)
> + goto out_delete_hw_queue;
> +
> + if (ctrl->ctrl.state != NVME_CTRL_NEW)
> + blk_mq_start_stopped_hw_queues(ctrl->ctrl.admin_q, true);
> +
> + ret = nvmf_connect_admin_queue(&ctrl->ctrl);
> + if (ret)
> + goto out_disconnect_admin_queue;
> +
> + /*
> + * Check controller capabilities
> + *
> + * todo:- add code to check if ctrl attributes changed from
> + * prior connection values
> + */
> +
> + ret = nvmf_reg_read64(&ctrl->ctrl, NVME_REG_CAP, &ctrl->cap);
> + if (ret) {
> + dev_err(ctrl->ctrl.device,
> + "prop_get NVME_REG_CAP failed\n");
> + goto out_disconnect_admin_queue;
> + }
> +
> + ctrl->ctrl.sqsize =
> + min_t(int, NVME_CAP_MQES(ctrl->cap) + 1, ctrl->ctrl.sqsize);
> +
> + ret = nvme_enable_ctrl(&ctrl->ctrl, ctrl->cap);
> + if (ret)
> + goto out_disconnect_admin_queue;
> +
> + segs = min_t(u32, NVME_FC_MAX_SEGMENTS,
> + ctrl->lport->ops->max_sgl_segments);
> + ctrl->ctrl.max_hw_sectors = (segs - 1) << (PAGE_SHIFT - 9);
> +
> + ret = nvme_init_identify(&ctrl->ctrl);
> + if (ret)
> + goto out_disconnect_admin_queue;
> +
> + /* sanity checks */
> +
> + /* FC-NVME does not have other data in the capsule */
> + if (ctrl->ctrl.icdoff) {
> + dev_err(ctrl->ctrl.device, "icdoff %d is not supported!\n",
> + ctrl->ctrl.icdoff);
> + goto out_disconnect_admin_queue;
> + }
> +
> + nvme_start_keep_alive(&ctrl->ctrl);
> +
> + /* FC-NVME supports normal SGL Data Block Descriptors */
> +
> + if (opts->queue_size > ctrl->ctrl.maxcmd) {
> + /* warn if maxcmd is lower than queue_size */
> + dev_warn(ctrl->ctrl.device,
> + "queue_size %zu > ctrl maxcmd %u, reducing "
> + "to queue_size\n",
> + opts->queue_size, ctrl->ctrl.maxcmd);
> + opts->queue_size = ctrl->ctrl.maxcmd;
> + }
> +
> + ret = nvme_fc_init_aen_ops(ctrl);
> + if (ret)
> + goto out_term_aen_ops;
> +
> + /*
> + * Create the io queues
> + */
> +
> + if (ctrl->queue_count > 1) {
> + if (ctrl->ctrl.state == NVME_CTRL_NEW)
> + ret = nvme_fc_create_io_queues(ctrl);
> + else
> + ret = nvme_fc_reinit_io_queues(ctrl);
> + if (ret)
> + goto out_term_aen_ops;
> + }
> +
> + changed = nvme_change_ctrl_state(&ctrl->ctrl, NVME_CTRL_LIVE);
> + WARN_ON_ONCE(!changed);
> +
> + ctrl->connect_attempts = 0;
> +
> + kref_get(&ctrl->ctrl.kref);
> +
> + if (ctrl->queue_count > 1) {
> + nvme_start_queues(&ctrl->ctrl);
> + nvme_queue_scan(&ctrl->ctrl);
> + nvme_queue_async_events(&ctrl->ctrl);
> + }
> +
> + return 0; /* Success */
> +
> +out_term_aen_ops:
> + nvme_fc_term_aen_ops(ctrl);
> + nvme_stop_keep_alive(&ctrl->ctrl);
> +out_disconnect_admin_queue:
> + /* send a Disconnect(association) LS to fc-nvme target */
> + nvme_fc_xmt_disconnect_assoc(ctrl);
> +out_delete_hw_queue:
> + __nvme_fc_delete_hw_queue(ctrl, &ctrl->queues[0], 0);
> +out_free_queue:
> + nvme_fc_free_queue(&ctrl->queues[0]);
>
> return ret;
> }
>
> +/*
> + * This routine stops operation of the controller on the host side.
> + * On the host os stack side: Admin and IO queues are stopped,
> + * outstanding ios on them terminated via FC ABTS.
> + * On the link side: the association is terminated.
> + */
> +static void
> +nvme_fc_delete_association(struct nvme_fc_ctrl *ctrl)
> +{
> + unsigned long flags;
> +
> + nvme_stop_keep_alive(&ctrl->ctrl);
> +
> + spin_lock_irqsave(&ctrl->lock, flags);
> + ctrl->flags |= FCCTRL_TERMIO;
> + ctrl->iocnt = 0;
> + spin_unlock_irqrestore(&ctrl->lock, flags);
> +
> + /*
> + * If io queues are present, stop them and terminate all outstanding
> + * ios on them. As FC allocates FC exchange for each io, the
> + * transport must contact the LLDD to terminate the exchange,
> + * thus releasing the FC exchange. We use blk_mq_tagset_busy_itr()
> + * to tell us what io's are busy and invoke a transport routine
> + * to kill them with the LLDD. After terminating the exchange
> + * the LLDD will call the transport's normal io done path, but it
> + * will have an aborted status. The done path will return the
> + * io requests back to the block layer as part of normal completions
> + * (but with error status).
> + */
> + if (ctrl->queue_count > 1) {
> + nvme_stop_queues(&ctrl->ctrl);
> + blk_mq_tagset_busy_iter(&ctrl->tag_set,
> + nvme_fc_terminate_exchange, &ctrl->ctrl);
> + }
> +
> + /*
> + * Other transports, which don't have link-level contexts bound
> + * to sqe's, would try to gracefully shutdown the controller by
> + * writing the registers for shutdown and polling (call
> + * nvme_shutdown_ctrl()). Given a bunch of i/o was potentially
> + * just aborted and we will wait on those contexts, and given
> + * there was no indication of how live the controlelr is on the
> + * link, don't send more io to create more contexts for the
> + * shutdown. Let the controller fail via keepalive failure if
> + * its still present.
> + */
> +
> + /*
> + * clean up the admin queue. Same thing as above.
> + * use blk_mq_tagset_busy_itr() and the transport routine to
> + * terminate the exchanges.
> + */
> + blk_mq_stop_hw_queues(ctrl->ctrl.admin_q);
> + blk_mq_tagset_busy_iter(&ctrl->admin_tag_set,
> + nvme_fc_terminate_exchange, &ctrl->ctrl);
> +
> + /* kill the aens as they are a separate path */
> + nvme_fc_abort_aen_ops(ctrl);
> +
> + /* wait for all io that had to be aborted */
> + spin_lock_irqsave(&ctrl->lock, flags);
> + while (ctrl->iocnt) {
> + spin_unlock_irqrestore(&ctrl->lock, flags);
> + msleep(1000);
> + spin_lock_irqsave(&ctrl->lock, flags);
> + }
struct completion for this?
> + ctrl->flags &= ~FCCTRL_TERMIO;
> + spin_unlock_irqrestore(&ctrl->lock, flags);
> +
> + nvme_fc_term_aen_ops(ctrl);
> +
> + /*
> + * send a Disconnect(association) LS to fc-nvme target
> + * Note: could have been sent at top of process, but
> + * cleaner on link traffic if after the aborts complete.
> + * Note: if association doesn't exist, association_id will be 0
> + */
> + if (ctrl->association_id)
> + nvme_fc_xmt_disconnect_assoc(ctrl);
> +
> + if (ctrl->ctrl.tagset) {
> + nvme_fc_delete_hw_io_queues(ctrl);
> + nvme_fc_free_io_queues(ctrl);
> + }
> +
> + __nvme_fc_delete_hw_queue(ctrl, &ctrl->queues[0], 0);
> + nvme_fc_free_queue(&ctrl->queues[0]);
> +}
> +
> +static void
> +nvme_fc_delete_ctrl_work(struct work_struct *work)
> +{
> + struct nvme_fc_ctrl *ctrl =
> + container_of(work, struct nvme_fc_ctrl, delete_work);
> +
> + cancel_work_sync(&ctrl->reset_work);
> + cancel_delayed_work_sync(&ctrl->connect_work);
> +
> + /*
> + * kill the association on the link side. this will block
> + * waiting for io to terminate
> + */
> + nvme_fc_delete_association(ctrl);
> +
> + /*
> + * tear down the controller
> + * This will result in the last reference on the nvme ctrl to
> + * expire, calling the transport nvme_fc_nvme_ctrl_freed() callback.
> + * From there, the transport will tear down it's logical queues and
> + * association.
> + */
Aren't you taking an extra ref in nvme_fc_del_nvme_ctrl?
> + nvme_uninit_ctrl(&ctrl->ctrl);
> +
> + nvme_put_ctrl(&ctrl->ctrl);
> +}
> +
> +static int
> +__nvme_fc_del_ctrl(struct nvme_fc_ctrl *ctrl)
> +{
> + if (!nvme_change_ctrl_state(&ctrl->ctrl, NVME_CTRL_DELETING))
> + return -EBUSY;
> +
> + if (!queue_work(nvme_fc_wq, &ctrl->delete_work))
> + return -EBUSY;
> +
> + return 0;
> +}
> +
> +/*
> + * Request from nvme core layer to delete the controller
> + */
> +static int
> +nvme_fc_del_nvme_ctrl(struct nvme_ctrl *nctrl)
> +{
> + struct nvme_fc_ctrl *ctrl = to_fc_ctrl(nctrl);
> + int ret;
> +
> + if (!kref_get_unless_zero(&ctrl->ctrl.kref))
> + return -EBUSY;
> +
> + ret = __nvme_fc_del_ctrl(ctrl);
> +
> + if (!ret)
> + flush_workqueue(nvme_fc_wq);
> +
> + nvme_put_ctrl(&ctrl->ctrl);
> +
> + return ret;
> +}
> +
> +static void
> +nvme_fc_reset_ctrl_work(struct work_struct *work)
> +{
> + struct nvme_fc_ctrl *ctrl =
> + container_of(work, struct nvme_fc_ctrl, reset_work);
> + int ret;
> +
> + /* will block will waiting for io to terminate */
> + nvme_fc_delete_association(ctrl);
> +
> + ret = nvme_fc_create_association(ctrl);
> + if (ret) {
> + dev_warn(ctrl->ctrl.device,
> + "NVME-FC{%d}: reset: Reconnect attempt failed (%d)\n",
> + ctrl->cnum, ret);
> + if (ctrl->connect_attempts >= NVME_FC_MAX_CONNECT_ATTEMPTS) {
> + dev_warn(ctrl->ctrl.device,
> + "NVME-FC{%d}: Max reconnect attempts (%d) "
> + "reached. Removing controller\n",
> + ctrl->cnum, ctrl->connect_attempts);
> +
> + if (!nvme_change_ctrl_state(&ctrl->ctrl,
> + NVME_CTRL_DELETING)) {
> + dev_err(ctrl->ctrl.device,
> + "NVME-FC{%d}: failed to change state "
> + "to DELETING\n", ctrl->cnum);
> + return;
> + }
> +
> + WARN_ON(!queue_work(nvme_fc_wq, &ctrl->delete_work));
> + return;
> + }
> +
> + dev_warn(ctrl->ctrl.device,
> + "NVME-FC{%d}: Reconnect attempt in %d seconds.\n",
> + ctrl->cnum, ctrl->reconnect_delay);
> + queue_delayed_work(nvme_fc_wq, &ctrl->connect_work,
> + ctrl->reconnect_delay * HZ);
> + } else
> + dev_info(ctrl->ctrl.device,
> + "NVME-FC{%d}: controller reset complete\n", ctrl->cnum);
> +}
> +
> +/*
> + * called by the nvme core layer, for sysfs interface that requests
> + * a reset of the nvme controller
> + */
> +static int
> +nvme_fc_reset_nvme_ctrl(struct nvme_ctrl *nctrl)
> +{
> + struct nvme_fc_ctrl *ctrl = to_fc_ctrl(nctrl);
> +
> + dev_warn(ctrl->ctrl.device,
> + "NVME-FC{%d}: admin requested controller reset\n", ctrl->cnum);
Isn't this warn to chatty?
More information about the Linux-nvme
mailing list