linux-block.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* remove QUEUE_FLAG_SCSI_PASSTHROUGH v3
@ 2021-10-21  6:06 Christoph Hellwig
  2021-10-21  6:06 ` [PATCH 1/7] block: add a ->get_unique_id method Christoph Hellwig
                   ` (7 more replies)
  0 siblings, 8 replies; 14+ messages in thread
From: Christoph Hellwig @ 2021-10-21  6:06 UTC (permalink / raw)
  To: Jens Axboe, Martin K. Petersen
  Cc: J. Bruce Fields, Chuck Lever, linux-block, linux-scsi,
	target-devel, linux-nfs

Hi all,

this series removes the QUEUE_FLAG_SCSI_PASSTHROUGH and thus the last
remaining SCSI passthrough concept from the block layer.

The changes to support pktcdvd are a bit ugly, but I can't think of
anything better (except for removing the driver entirely).
If we'd want to support packet writing today it would probably live
entirely inside the sr driver.

Changes since v2:
 - s/blk_uniqueue_id/blk_unique_id/g

Changes since v1:
 - use an extra local variable in sd_get_unique_id to make sure we
   always return the right length
 - add an enum and a comment to better document ->get_unique_id
 - spelling fixes

Diffstat:
 block/blk-core.c                   |    9 --
 block/blk-mq-debugfs.c             |    1 
 block/bsg-lib.c                    |   32 +++----
 drivers/block/Kconfig              |    2 
 drivers/block/pktcdvd.c            |    7 +
 drivers/scsi/scsi_bsg.c            |    4 
 drivers/scsi/scsi_error.c          |    2 
 drivers/scsi/scsi_ioctl.c          |    4 
 drivers/scsi/scsi_lib.c            |   27 ++++--
 drivers/scsi/scsi_scan.c           |    1 
 drivers/scsi/sd.c                  |   39 +++++++++
 drivers/scsi/sg.c                  |    4 
 drivers/scsi/sr.c                  |    2 
 drivers/scsi/st.c                  |    2 
 drivers/target/target_core_pscsi.c |    3 
 fs/nfsd/Kconfig                    |    1 
 fs/nfsd/blocklayout.c              |  158 +++++++++----------------------------
 fs/nfsd/nfs4layouts.c              |    5 -
 include/linux/blk-mq.h             |    5 -
 include/linux/blkdev.h             |   14 ++-
 include/scsi/scsi_cmnd.h           |    3 
 21 files changed, 148 insertions(+), 177 deletions(-)

^ permalink raw reply	[flat|nested] 14+ messages in thread

* [PATCH 1/7] block: add a ->get_unique_id method
  2021-10-21  6:06 remove QUEUE_FLAG_SCSI_PASSTHROUGH v3 Christoph Hellwig
@ 2021-10-21  6:06 ` Christoph Hellwig
  2021-10-21 13:34   ` Hannes Reinecke
  2021-10-21  6:06 ` [PATCH 2/7] sd: implement ->get_unique_id Christoph Hellwig
                   ` (6 subsequent siblings)
  7 siblings, 1 reply; 14+ messages in thread
From: Christoph Hellwig @ 2021-10-21  6:06 UTC (permalink / raw)
  To: Jens Axboe, Martin K. Petersen
  Cc: J. Bruce Fields, Chuck Lever, linux-block, linux-scsi,
	target-devel, linux-nfs

Add a method to query unique IDs from block devices.  It will be used to
remove code that deeply pokes into SCSI internals in the NFS server.
The implementation in the sd driver itself is also much nicer as it can
use the cached VPD page instead of always sending a command as the
current NFS code does.

For now the interface is kept very minimal but could be easily
extended when other users like a block-layer sysfs interface for
uniquue IDs shows up.

Signed-off-by: Christoph Hellwig <hch@lst.de>
---
 include/linux/blkdev.h | 11 +++++++++++
 1 file changed, 11 insertions(+)

diff --git a/include/linux/blkdev.h b/include/linux/blkdev.h
index fd9771a1da096..af61fb3e1502c 100644
--- a/include/linux/blkdev.h
+++ b/include/linux/blkdev.h
@@ -1177,6 +1177,14 @@ static inline void blk_ksm_unregister(struct request_queue *q) { }
 
 #endif /* CONFIG_BLK_INLINE_ENCRYPTION */
 
+enum blk_unique_id {
+	/* these match the Designator Types specified in SPC */
+	BLK_UID_T10	= 1,
+	BLK_UID_EUI64	= 2,
+	BLK_UID_NAA	= 3,
+};
+
+#define NFL4_UFLG_MASK			0x0000003F
 
 struct block_device_operations {
 	void (*submit_bio)(struct bio *bio);
@@ -1195,6 +1203,9 @@ struct block_device_operations {
 	int (*report_zones)(struct gendisk *, sector_t sector,
 			unsigned int nr_zones, report_zones_cb cb, void *data);
 	char *(*devnode)(struct gendisk *disk, umode_t *mode);
+	/* returns the length of the identifier or a negative errno: */
+	int (*get_unique_id)(struct gendisk *disk, u8 id[16],
+			enum blk_unique_id id_type);
 	struct module *owner;
 	const struct pr_ops *pr_ops;
 
-- 
2.30.2


^ permalink raw reply related	[flat|nested] 14+ messages in thread

* [PATCH 2/7] sd: implement ->get_unique_id
  2021-10-21  6:06 remove QUEUE_FLAG_SCSI_PASSTHROUGH v3 Christoph Hellwig
  2021-10-21  6:06 ` [PATCH 1/7] block: add a ->get_unique_id method Christoph Hellwig
@ 2021-10-21  6:06 ` Christoph Hellwig
  2021-10-21 13:34   ` Hannes Reinecke
  2021-10-21  6:06 ` [PATCH 3/7] nfsd/blocklayout: use ->get_unique_id instead of sending SCSI commands Christoph Hellwig
                   ` (5 subsequent siblings)
  7 siblings, 1 reply; 14+ messages in thread
From: Christoph Hellwig @ 2021-10-21  6:06 UTC (permalink / raw)
  To: Jens Axboe, Martin K. Petersen
  Cc: J. Bruce Fields, Chuck Lever, linux-block, linux-scsi,
	target-devel, linux-nfs

Add the method to query for a uniqueue ID of a given type by looking
it up in the cached device identification VPD page.

Signed-off-by: Christoph Hellwig <hch@lst.de>
---
 drivers/scsi/sd.c | 39 +++++++++++++++++++++++++++++++++++++++
 1 file changed, 39 insertions(+)

diff --git a/drivers/scsi/sd.c b/drivers/scsi/sd.c
index d8f6add416c0a..9b386ca39a913 100644
--- a/drivers/scsi/sd.c
+++ b/drivers/scsi/sd.c
@@ -1757,6 +1757,44 @@ static void sd_rescan(struct device *dev)
 	sd_revalidate_disk(sdkp->disk);
 }
 
+static int sd_get_unique_id(struct gendisk *disk, u8 id[16],
+		enum blk_unique_id type)
+{
+	struct scsi_device *sdev = scsi_disk(disk)->device;
+	const struct scsi_vpd *vpd;
+	const unsigned char *d;
+	int ret = -ENXIO, len;
+
+	rcu_read_lock();
+	vpd = rcu_dereference(sdev->vpd_pg83);
+	if (!vpd)
+		goto out_unlock;
+
+	ret = -EINVAL;
+	for (d = vpd->data + 4; d < vpd->data + vpd->len; d += d[3] + 4) {
+		/* we only care about designators with LU association */
+		if (((d[1] >> 4) & 0x3) != 0x00)
+			continue;
+		if ((d[1] & 0xf) != type)
+			continue;
+
+		/*
+		 * Only exit early if a 16-byte descriptor was found.  Otherwise
+		 * keep looking as one with more entropy might still show up.
+		 */
+		len = d[3];
+		if (len != 8 && len != 12 && len != 16)
+			continue;
+		ret = len;
+		memcpy(id, d + 4, len);
+		if (len == 16)
+			break;
+	}
+out_unlock:
+	rcu_read_unlock();
+	return ret;
+}
+
 static char sd_pr_type(enum pr_type type)
 {
 	switch (type) {
@@ -1861,6 +1899,7 @@ static const struct block_device_operations sd_fops = {
 	.check_events		= sd_check_events,
 	.unlock_native_capacity	= sd_unlock_native_capacity,
 	.report_zones		= sd_zbc_report_zones,
+	.get_unique_id		= sd_get_unique_id,
 	.pr_ops			= &sd_pr_ops,
 };
 
-- 
2.30.2


^ permalink raw reply related	[flat|nested] 14+ messages in thread

* [PATCH 3/7] nfsd/blocklayout: use ->get_unique_id instead of sending SCSI commands
  2021-10-21  6:06 remove QUEUE_FLAG_SCSI_PASSTHROUGH v3 Christoph Hellwig
  2021-10-21  6:06 ` [PATCH 1/7] block: add a ->get_unique_id method Christoph Hellwig
  2021-10-21  6:06 ` [PATCH 2/7] sd: implement ->get_unique_id Christoph Hellwig
@ 2021-10-21  6:06 ` Christoph Hellwig
  2021-10-21  6:06 ` [PATCH 4/7] bsg-lib: initialize the bsg_job in bsg_transport_sg_io_fn Christoph Hellwig
                   ` (4 subsequent siblings)
  7 siblings, 0 replies; 14+ messages in thread
From: Christoph Hellwig @ 2021-10-21  6:06 UTC (permalink / raw)
  To: Jens Axboe, Martin K. Petersen
  Cc: J. Bruce Fields, Chuck Lever, linux-block, linux-scsi,
	target-devel, linux-nfs, J . Bruce Fields, Hannes Reinecke

Call the ->get_unique_id method to query the SCSI identifiers.  This can
use the cached VPD page in the sd driver instead of sending a command
on every LAYOUTGET.  It will also allow to support NVMe based volumes
if the draft for that ever takes off.

Signed-off-by: Christoph Hellwig <hch@lst.de>
Acked-by: J. Bruce Fields <bfields@redhat.com>
Reviewed-by: Hannes Reinecke <hare@suse.de>
---
 fs/nfsd/Kconfig       |   1 -
 fs/nfsd/blocklayout.c | 158 +++++++++++-------------------------------
 fs/nfsd/nfs4layouts.c |   5 +-
 3 files changed, 44 insertions(+), 120 deletions(-)

diff --git a/fs/nfsd/Kconfig b/fs/nfsd/Kconfig
index 6e9ea4ee0f737..3d1d17256a91c 100644
--- a/fs/nfsd/Kconfig
+++ b/fs/nfsd/Kconfig
@@ -109,7 +109,6 @@ config NFSD_SCSILAYOUT
 	depends on NFSD_V4 && BLOCK
 	select NFSD_PNFS
 	select EXPORTFS_BLOCK_OPS
-	select SCSI_COMMON
 	help
 	  This option enables support for the exporting pNFS SCSI layouts
 	  in the kernel's NFS server. The pNFS SCSI layout enables NFS
diff --git a/fs/nfsd/blocklayout.c b/fs/nfsd/blocklayout.c
index c99dee99a3c15..e5c0982a381de 100644
--- a/fs/nfsd/blocklayout.c
+++ b/fs/nfsd/blocklayout.c
@@ -9,9 +9,6 @@
 #include <linux/pr.h>
 
 #include <linux/nfsd/debug.h>
-#include <scsi/scsi_proto.h>
-#include <scsi/scsi_common.h>
-#include <scsi/scsi_request.h>
 
 #include "blocklayoutxdr.h"
 #include "pnfs.h"
@@ -211,109 +208,6 @@ const struct nfsd4_layout_ops bl_layout_ops = {
 #endif /* CONFIG_NFSD_BLOCKLAYOUT */
 
 #ifdef CONFIG_NFSD_SCSILAYOUT
-static int nfsd4_scsi_identify_device(struct block_device *bdev,
-		struct pnfs_block_volume *b)
-{
-	struct request_queue *q = bdev->bd_disk->queue;
-	struct request *rq;
-	struct scsi_request *req;
-	/*
-	 * The allocation length (passed in bytes 3 and 4 of the INQUIRY
-	 * command descriptor block) specifies the number of bytes that have
-	 * been allocated for the data-in buffer.
-	 * 252 is the highest one-byte value that is a multiple of 4.
-	 * 65532 is the highest two-byte value that is a multiple of 4.
-	 */
-	size_t bufflen = 252, maxlen = 65532, len, id_len;
-	u8 *buf, *d, type, assoc;
-	int retries = 1, error;
-
-	if (WARN_ON_ONCE(!blk_queue_scsi_passthrough(q)))
-		return -EINVAL;
-
-again:
-	buf = kzalloc(bufflen, GFP_KERNEL);
-	if (!buf)
-		return -ENOMEM;
-
-	rq = blk_get_request(q, REQ_OP_DRV_IN, 0);
-	if (IS_ERR(rq)) {
-		error = -ENOMEM;
-		goto out_free_buf;
-	}
-	req = scsi_req(rq);
-
-	error = blk_rq_map_kern(q, rq, buf, bufflen, GFP_KERNEL);
-	if (error)
-		goto out_put_request;
-
-	req->cmd[0] = INQUIRY;
-	req->cmd[1] = 1;
-	req->cmd[2] = 0x83;
-	req->cmd[3] = bufflen >> 8;
-	req->cmd[4] = bufflen & 0xff;
-	req->cmd_len = COMMAND_SIZE(INQUIRY);
-
-	blk_execute_rq(NULL, rq, 1);
-	if (req->result) {
-		pr_err("pNFS: INQUIRY 0x83 failed with: %x\n",
-			req->result);
-		error = -EIO;
-		goto out_put_request;
-	}
-
-	len = (buf[2] << 8) + buf[3] + 4;
-	if (len > bufflen) {
-		if (len <= maxlen && retries--) {
-			blk_put_request(rq);
-			kfree(buf);
-			bufflen = len;
-			goto again;
-		}
-		pr_err("pNFS: INQUIRY 0x83 response invalid (len = %zd)\n",
-			len);
-		goto out_put_request;
-	}
-
-	d = buf + 4;
-	for (d = buf + 4; d < buf + len; d += id_len + 4) {
-		id_len = d[3];
-		type = d[1] & 0xf;
-		assoc = (d[1] >> 4) & 0x3;
-
-		/*
-		 * We only care about a EUI-64 and NAA designator types
-		 * with LU association.
-		 */
-		if (assoc != 0x00)
-			continue;
-		if (type != 0x02 && type != 0x03)
-			continue;
-		if (id_len != 8 && id_len != 12 && id_len != 16)
-			continue;
-
-		b->scsi.code_set = PS_CODE_SET_BINARY;
-		b->scsi.designator_type = type == 0x02 ?
-			PS_DESIGNATOR_EUI64 : PS_DESIGNATOR_NAA;
-		b->scsi.designator_len = id_len;
-		memcpy(b->scsi.designator, d + 4, id_len);
-
-		/*
-		 * If we found a 8 or 12 byte descriptor continue on to
-		 * see if a 16 byte one is available.  If we find a
-		 * 16 byte descriptor we're done.
-		 */
-		if (id_len == 16)
-			break;
-	}
-
-out_put_request:
-	blk_put_request(rq);
-out_free_buf:
-	kfree(buf);
-	return error;
-}
-
 #define NFSD_MDS_PR_KEY		0x0100000000000000ULL
 
 /*
@@ -325,6 +219,31 @@ static u64 nfsd4_scsi_pr_key(struct nfs4_client *clp)
 	return ((u64)clp->cl_clientid.cl_boot << 32) | clp->cl_clientid.cl_id;
 }
 
+static const u8 designator_types[] = {
+	PS_DESIGNATOR_EUI64,
+	PS_DESIGNATOR_NAA,
+};
+
+static int
+nfsd4_block_get_unique_id(struct gendisk *disk, struct pnfs_block_volume *b)
+{
+	int ret, i;
+
+	for (i = 0; i < ARRAY_SIZE(designator_types); i++) {
+		u8 type = designator_types[i];
+
+		ret = disk->fops->get_unique_id(disk, b->scsi.designator, type);
+		if (ret > 0) {
+			b->scsi.code_set = PS_CODE_SET_BINARY;
+			b->scsi.designator_type = type;
+			b->scsi.designator_len = ret;
+			return 0;
+		}
+	}
+
+	return -EINVAL;
+}
+
 static int
 nfsd4_block_get_device_info_scsi(struct super_block *sb,
 		struct nfs4_client *clp,
@@ -333,7 +252,7 @@ nfsd4_block_get_device_info_scsi(struct super_block *sb,
 	struct pnfs_block_deviceaddr *dev;
 	struct pnfs_block_volume *b;
 	const struct pr_ops *ops;
-	int error;
+	int ret;
 
 	dev = kzalloc(sizeof(struct pnfs_block_deviceaddr) +
 		      sizeof(struct pnfs_block_volume), GFP_KERNEL);
@@ -347,33 +266,38 @@ nfsd4_block_get_device_info_scsi(struct super_block *sb,
 	b->type = PNFS_BLOCK_VOLUME_SCSI;
 	b->scsi.pr_key = nfsd4_scsi_pr_key(clp);
 
-	error = nfsd4_scsi_identify_device(sb->s_bdev, b);
-	if (error)
-		return error;
+	ret = nfsd4_block_get_unique_id(sb->s_bdev->bd_disk, b);
+	if (ret < 0)
+		goto out_free_dev;
 
+	ret = -EINVAL;
 	ops = sb->s_bdev->bd_disk->fops->pr_ops;
 	if (!ops) {
 		pr_err("pNFS: device %s does not support PRs.\n",
 			sb->s_id);
-		return -EINVAL;
+		goto out_free_dev;
 	}
 
-	error = ops->pr_register(sb->s_bdev, 0, NFSD_MDS_PR_KEY, true);
-	if (error) {
+	ret = ops->pr_register(sb->s_bdev, 0, NFSD_MDS_PR_KEY, true);
+	if (ret) {
 		pr_err("pNFS: failed to register key for device %s.\n",
 			sb->s_id);
-		return -EINVAL;
+		goto out_free_dev;
 	}
 
-	error = ops->pr_reserve(sb->s_bdev, NFSD_MDS_PR_KEY,
+	ret = ops->pr_reserve(sb->s_bdev, NFSD_MDS_PR_KEY,
 			PR_EXCLUSIVE_ACCESS_REG_ONLY, 0);
-	if (error) {
+	if (ret) {
 		pr_err("pNFS: failed to reserve device %s.\n",
 			sb->s_id);
-		return -EINVAL;
+		goto out_free_dev;
 	}
 
 	return 0;
+
+out_free_dev:
+	kfree(dev);
+	return ret;
 }
 
 static __be32
diff --git a/fs/nfsd/nfs4layouts.c b/fs/nfsd/nfs4layouts.c
index a97873f2d22b0..6d1b5bb051c56 100644
--- a/fs/nfsd/nfs4layouts.c
+++ b/fs/nfsd/nfs4layouts.c
@@ -145,8 +145,9 @@ void nfsd4_setup_layout_type(struct svc_export *exp)
 #ifdef CONFIG_NFSD_SCSILAYOUT
 	if (sb->s_export_op->map_blocks &&
 	    sb->s_export_op->commit_blocks &&
-	    sb->s_bdev && sb->s_bdev->bd_disk->fops->pr_ops &&
-		blk_queue_scsi_passthrough(sb->s_bdev->bd_disk->queue))
+	    sb->s_bdev &&
+	    sb->s_bdev->bd_disk->fops->pr_ops &&
+	    sb->s_bdev->bd_disk->fops->get_unique_id)
 		exp->ex_layout_types |= 1 << LAYOUT_SCSI;
 #endif
 }
-- 
2.30.2


^ permalink raw reply related	[flat|nested] 14+ messages in thread

* [PATCH 4/7] bsg-lib: initialize the bsg_job in bsg_transport_sg_io_fn
  2021-10-21  6:06 remove QUEUE_FLAG_SCSI_PASSTHROUGH v3 Christoph Hellwig
                   ` (2 preceding siblings ...)
  2021-10-21  6:06 ` [PATCH 3/7] nfsd/blocklayout: use ->get_unique_id instead of sending SCSI commands Christoph Hellwig
@ 2021-10-21  6:06 ` Christoph Hellwig
  2021-10-21  6:06 ` [PATCH 5/7] scsi: add a scsi_alloc_request helper Christoph Hellwig
                   ` (3 subsequent siblings)
  7 siblings, 0 replies; 14+ messages in thread
From: Christoph Hellwig @ 2021-10-21  6:06 UTC (permalink / raw)
  To: Jens Axboe, Martin K. Petersen
  Cc: J. Bruce Fields, Chuck Lever, linux-block, linux-scsi,
	target-devel, linux-nfs, Hannes Reinecke

Directly initialize the bsg_job structure instead of relying on the
->.initialize_rq_fn indirection.  This also removes the superflous
initialization of the second request used for BIDI requests.

Signed-off-by: Christoph Hellwig <hch@lst.de>
Reviewed-by: Hannes Reinecke <hare@suse.de>
---
 block/bsg-lib.c | 32 +++++++++++++-------------------
 1 file changed, 13 insertions(+), 19 deletions(-)

diff --git a/block/bsg-lib.c b/block/bsg-lib.c
index ccb98276c964a..10aa378702fab 100644
--- a/block/bsg-lib.c
+++ b/block/bsg-lib.c
@@ -31,6 +31,7 @@ static int bsg_transport_sg_io_fn(struct request_queue *q, struct sg_io_v4 *hdr,
 	struct bsg_job *job;
 	struct request *rq;
 	struct bio *bio;
+	void *reply;
 	int ret;
 
 	if (hdr->protocol != BSG_PROTOCOL_SCSI  ||
@@ -39,22 +40,28 @@ static int bsg_transport_sg_io_fn(struct request_queue *q, struct sg_io_v4 *hdr,
 	if (!capable(CAP_SYS_RAWIO))
 		return -EPERM;
 
-	rq = blk_get_request(q, hdr->dout_xfer_len ?
+	rq = blk_mq_alloc_request(q, hdr->dout_xfer_len ?
 			     REQ_OP_DRV_OUT : REQ_OP_DRV_IN, 0);
 	if (IS_ERR(rq))
 		return PTR_ERR(rq);
 	rq->timeout = timeout;
 
 	job = blk_mq_rq_to_pdu(rq);
+	reply = job->reply;
+	memset(job, 0, sizeof(*job));
+	job->reply = reply;
+	job->reply_len = SCSI_SENSE_BUFFERSIZE;
+	job->dd_data = job + 1;
+
 	job->request_len = hdr->request_len;
 	job->request = memdup_user(uptr64(hdr->request), hdr->request_len);
 	if (IS_ERR(job->request)) {
 		ret = PTR_ERR(job->request);
-		goto out_put_request;
+		goto out_free_rq;
 	}
 
 	if (hdr->dout_xfer_len && hdr->din_xfer_len) {
-		job->bidi_rq = blk_get_request(rq->q, REQ_OP_DRV_IN, 0);
+		job->bidi_rq = blk_mq_alloc_request(rq->q, REQ_OP_DRV_IN, 0);
 		if (IS_ERR(job->bidi_rq)) {
 			ret = PTR_ERR(job->bidi_rq);
 			goto out_free_job_request;
@@ -134,11 +141,11 @@ static int bsg_transport_sg_io_fn(struct request_queue *q, struct sg_io_v4 *hdr,
 		blk_rq_unmap_user(job->bidi_bio);
 out_free_bidi_rq:
 	if (job->bidi_rq)
-		blk_put_request(job->bidi_rq);
+		blk_mq_free_request(job->bidi_rq);
 out_free_job_request:
 	kfree(job->request);
-out_put_request:
-	blk_put_request(rq);
+out_free_rq:
+	blk_mq_free_request(rq);
 	return ret;
 }
 
@@ -302,18 +309,6 @@ static int bsg_init_rq(struct blk_mq_tag_set *set, struct request *req,
 	return 0;
 }
 
-/* called right before the request is given to the request_queue user */
-static void bsg_initialize_rq(struct request *req)
-{
-	struct bsg_job *job = blk_mq_rq_to_pdu(req);
-	void *reply = job->reply;
-
-	memset(job, 0, sizeof(*job));
-	job->reply = reply;
-	job->reply_len = SCSI_SENSE_BUFFERSIZE;
-	job->dd_data = job + 1;
-}
-
 static void bsg_exit_rq(struct blk_mq_tag_set *set, struct request *req,
 		       unsigned int hctx_idx)
 {
@@ -350,7 +345,6 @@ static const struct blk_mq_ops bsg_mq_ops = {
 	.queue_rq		= bsg_queue_rq,
 	.init_request		= bsg_init_rq,
 	.exit_request		= bsg_exit_rq,
-	.initialize_rq_fn	= bsg_initialize_rq,
 	.complete		= bsg_complete,
 	.timeout		= bsg_timeout,
 };
-- 
2.30.2


^ permalink raw reply related	[flat|nested] 14+ messages in thread

* [PATCH 5/7] scsi: add a scsi_alloc_request helper
  2021-10-21  6:06 remove QUEUE_FLAG_SCSI_PASSTHROUGH v3 Christoph Hellwig
                   ` (3 preceding siblings ...)
  2021-10-21  6:06 ` [PATCH 4/7] bsg-lib: initialize the bsg_job in bsg_transport_sg_io_fn Christoph Hellwig
@ 2021-10-21  6:06 ` Christoph Hellwig
  2021-10-21  6:06 ` [PATCH 6/7] block: remove the initialize_rq_fn blk_mq_ops method Christoph Hellwig
                   ` (2 subsequent siblings)
  7 siblings, 0 replies; 14+ messages in thread
From: Christoph Hellwig @ 2021-10-21  6:06 UTC (permalink / raw)
  To: Jens Axboe, Martin K. Petersen
  Cc: J. Bruce Fields, Chuck Lever, linux-block, linux-scsi,
	target-devel, linux-nfs, Hannes Reinecke

Add a new helper that calls blk_get_request and initializes the
scsi_request to avoid the indirect call through ->.initialize_rq_fn.

Note that this makes the pktcdvd driver depend on the SCSI core, but
given that only SCSI devices support SCSI passthrough requests that
is not a functional change.

Signed-off-by: Christoph Hellwig <hch@lst.de>
Reviewed-by: Hannes Reinecke <hare@suse.de>
---
 drivers/block/Kconfig              |  2 +-
 drivers/block/pktcdvd.c            |  2 +-
 drivers/scsi/scsi_bsg.c            |  4 ++--
 drivers/scsi/scsi_error.c          |  2 +-
 drivers/scsi/scsi_ioctl.c          |  4 ++--
 drivers/scsi/scsi_lib.c            | 19 +++++++++++++------
 drivers/scsi/sg.c                  |  4 ++--
 drivers/scsi/sr.c                  |  2 +-
 drivers/scsi/st.c                  |  2 +-
 drivers/target/target_core_pscsi.c |  3 +--
 include/scsi/scsi_cmnd.h           |  3 +++
 11 files changed, 28 insertions(+), 19 deletions(-)

diff --git a/drivers/block/Kconfig b/drivers/block/Kconfig
index ab3e37aa1830c..9151e8ffba1cf 100644
--- a/drivers/block/Kconfig
+++ b/drivers/block/Kconfig
@@ -304,8 +304,8 @@ config BLK_DEV_RAM_SIZE
 config CDROM_PKTCDVD
 	tristate "Packet writing on CD/DVD media (DEPRECATED)"
 	depends on !UML
+	depends on SCSI
 	select CDROM
-	select SCSI_COMMON
 	help
 	  Note: This driver is deprecated and will be removed from the
 	  kernel in the near future!
diff --git a/drivers/block/pktcdvd.c b/drivers/block/pktcdvd.c
index e48d4771d4c1a..d7d37131ab9dd 100644
--- a/drivers/block/pktcdvd.c
+++ b/drivers/block/pktcdvd.c
@@ -703,7 +703,7 @@ static int pkt_generic_packet(struct pktcdvd_device *pd, struct packet_command *
 	struct request *rq;
 	int ret = 0;
 
-	rq = blk_get_request(q, (cgc->data_direction == CGC_DATA_WRITE) ?
+	rq = scsi_alloc_request(q, (cgc->data_direction == CGC_DATA_WRITE) ?
 			     REQ_OP_DRV_OUT : REQ_OP_DRV_IN, 0);
 	if (IS_ERR(rq))
 		return PTR_ERR(rq);
diff --git a/drivers/scsi/scsi_bsg.c b/drivers/scsi/scsi_bsg.c
index 81c3853a2a800..551727a6f6941 100644
--- a/drivers/scsi/scsi_bsg.c
+++ b/drivers/scsi/scsi_bsg.c
@@ -25,8 +25,8 @@ static int scsi_bsg_sg_io_fn(struct request_queue *q, struct sg_io_v4 *hdr,
 		return -EOPNOTSUPP;
 	}
 
-	rq = blk_get_request(q, hdr->dout_xfer_len ?
-			     REQ_OP_DRV_OUT : REQ_OP_DRV_IN, 0);
+	rq = scsi_alloc_request(q, hdr->dout_xfer_len ?
+				REQ_OP_DRV_OUT : REQ_OP_DRV_IN, 0);
 	if (IS_ERR(rq))
 		return PTR_ERR(rq);
 	rq->timeout = timeout;
diff --git a/drivers/scsi/scsi_error.c b/drivers/scsi/scsi_error.c
index b6c86cce57bfa..71d027b94be40 100644
--- a/drivers/scsi/scsi_error.c
+++ b/drivers/scsi/scsi_error.c
@@ -1998,7 +1998,7 @@ static void scsi_eh_lock_door(struct scsi_device *sdev)
 	struct request *req;
 	struct scsi_request *rq;
 
-	req = blk_get_request(sdev->request_queue, REQ_OP_DRV_IN, 0);
+	req = scsi_alloc_request(sdev->request_queue, REQ_OP_DRV_IN, 0);
 	if (IS_ERR(req))
 		return;
 	rq = scsi_req(req);
diff --git a/drivers/scsi/scsi_ioctl.c b/drivers/scsi/scsi_ioctl.c
index 6ff2207bd45a0..0078975e3c07c 100644
--- a/drivers/scsi/scsi_ioctl.c
+++ b/drivers/scsi/scsi_ioctl.c
@@ -438,7 +438,7 @@ static int sg_io(struct scsi_device *sdev, struct gendisk *disk,
 		at_head = 1;
 
 	ret = -ENOMEM;
-	rq = blk_get_request(sdev->request_queue, writing ?
+	rq = scsi_alloc_request(sdev->request_queue, writing ?
 			     REQ_OP_DRV_OUT : REQ_OP_DRV_IN, 0);
 	if (IS_ERR(rq))
 		return PTR_ERR(rq);
@@ -561,7 +561,7 @@ static int sg_scsi_ioctl(struct request_queue *q, struct gendisk *disk,
 
 	}
 
-	rq = blk_get_request(q, in_len ? REQ_OP_DRV_OUT : REQ_OP_DRV_IN, 0);
+	rq = scsi_alloc_request(q, in_len ? REQ_OP_DRV_OUT : REQ_OP_DRV_IN, 0);
 	if (IS_ERR(rq)) {
 		err = PTR_ERR(rq);
 		goto error_free_buffer;
diff --git a/drivers/scsi/scsi_lib.c b/drivers/scsi/scsi_lib.c
index 30f7d0b4eb732..a0f801fc8943b 100644
--- a/drivers/scsi/scsi_lib.c
+++ b/drivers/scsi/scsi_lib.c
@@ -216,7 +216,7 @@ int __scsi_execute(struct scsi_device *sdev, const unsigned char *cmd,
 	struct scsi_request *rq;
 	int ret;
 
-	req = blk_get_request(sdev->request_queue,
+	req = scsi_alloc_request(sdev->request_queue,
 			data_direction == DMA_TO_DEVICE ?
 			REQ_OP_DRV_OUT : REQ_OP_DRV_IN,
 			rq_flags & RQF_PM ? BLK_MQ_REQ_PM : 0);
@@ -1079,9 +1079,6 @@ EXPORT_SYMBOL(scsi_alloc_sgtables);
  * This function initializes the members of struct scsi_cmnd that must be
  * initialized before request processing starts and that won't be
  * reinitialized if a SCSI command is requeued.
- *
- * Called from inside blk_get_request() for pass-through requests and from
- * inside scsi_init_command() for filesystem requests.
  */
 static void scsi_initialize_rq(struct request *rq)
 {
@@ -1098,6 +1095,18 @@ static void scsi_initialize_rq(struct request *rq)
 	cmd->retries = 0;
 }
 
+struct request *scsi_alloc_request(struct request_queue *q,
+		unsigned int op, blk_mq_req_flags_t flags)
+{
+	struct request *rq;
+
+	rq = blk_get_request(q, op, flags);
+	if (!IS_ERR(rq))
+		scsi_initialize_rq(rq);
+	return rq;
+}
+EXPORT_SYMBOL_GPL(scsi_alloc_request);
+
 /*
  * Only called when the request isn't completed by SCSI, and not freed by
  * SCSI
@@ -1864,7 +1873,6 @@ static const struct blk_mq_ops scsi_mq_ops_no_commit = {
 #endif
 	.init_request	= scsi_mq_init_request,
 	.exit_request	= scsi_mq_exit_request,
-	.initialize_rq_fn = scsi_initialize_rq,
 	.cleanup_rq	= scsi_cleanup_rq,
 	.busy		= scsi_mq_lld_busy,
 	.map_queues	= scsi_map_queues,
@@ -1894,7 +1902,6 @@ static const struct blk_mq_ops scsi_mq_ops = {
 #endif
 	.init_request	= scsi_mq_init_request,
 	.exit_request	= scsi_mq_exit_request,
-	.initialize_rq_fn = scsi_initialize_rq,
 	.cleanup_rq	= scsi_cleanup_rq,
 	.busy		= scsi_mq_lld_busy,
 	.map_queues	= scsi_map_queues,
diff --git a/drivers/scsi/sg.c b/drivers/scsi/sg.c
index 3c98f08dc25d9..85f57ac0b844e 100644
--- a/drivers/scsi/sg.c
+++ b/drivers/scsi/sg.c
@@ -1718,13 +1718,13 @@ sg_start_req(Sg_request *srp, unsigned char *cmd)
 	 *
 	 * With scsi-mq enabled, there are a fixed number of preallocated
 	 * requests equal in number to shost->can_queue.  If all of the
-	 * preallocated requests are already in use, then blk_get_request()
+	 * preallocated requests are already in use, then scsi_alloc_request()
 	 * will sleep until an active command completes, freeing up a request.
 	 * Although waiting in an asynchronous interface is less than ideal, we
 	 * do not want to use BLK_MQ_REQ_NOWAIT here because userspace might
 	 * not expect an EWOULDBLOCK from this condition.
 	 */
-	rq = blk_get_request(q, hp->dxfer_direction == SG_DXFER_TO_DEV ?
+	rq = scsi_alloc_request(q, hp->dxfer_direction == SG_DXFER_TO_DEV ?
 			REQ_OP_DRV_OUT : REQ_OP_DRV_IN, 0);
 	if (IS_ERR(rq)) {
 		kfree(long_cmdp);
diff --git a/drivers/scsi/sr.c b/drivers/scsi/sr.c
index 115f7ef7a5def..7c4d9a9647999 100644
--- a/drivers/scsi/sr.c
+++ b/drivers/scsi/sr.c
@@ -967,7 +967,7 @@ static int sr_read_cdda_bpc(struct cdrom_device_info *cdi, void __user *ubuf,
 	struct bio *bio;
 	int ret;
 
-	rq = blk_get_request(disk->queue, REQ_OP_DRV_IN, 0);
+	rq = scsi_alloc_request(disk->queue, REQ_OP_DRV_IN, 0);
 	if (IS_ERR(rq))
 		return PTR_ERR(rq);
 	req = scsi_req(rq);
diff --git a/drivers/scsi/st.c b/drivers/scsi/st.c
index 9933722acfd96..1275299f61597 100644
--- a/drivers/scsi/st.c
+++ b/drivers/scsi/st.c
@@ -543,7 +543,7 @@ static int st_scsi_execute(struct st_request *SRpnt, const unsigned char *cmd,
 	int err = 0;
 	struct scsi_tape *STp = SRpnt->stp;
 
-	req = blk_get_request(SRpnt->stp->device->request_queue,
+	req = scsi_alloc_request(SRpnt->stp->device->request_queue,
 			data_direction == DMA_TO_DEVICE ?
 			REQ_OP_DRV_OUT : REQ_OP_DRV_IN, 0);
 	if (IS_ERR(req))
diff --git a/drivers/target/target_core_pscsi.c b/drivers/target/target_core_pscsi.c
index 75ef52f008ff6..b5705a2bd7618 100644
--- a/drivers/target/target_core_pscsi.c
+++ b/drivers/target/target_core_pscsi.c
@@ -980,11 +980,10 @@ pscsi_execute_cmd(struct se_cmd *cmd)
 	memcpy(pt->pscsi_cdb, cmd->t_task_cdb,
 		scsi_command_size(cmd->t_task_cdb));
 
-	req = blk_get_request(pdv->pdv_sd->request_queue,
+	req = scsi_alloc_request(pdv->pdv_sd->request_queue,
 			cmd->data_direction == DMA_TO_DEVICE ?
 			REQ_OP_DRV_OUT : REQ_OP_DRV_IN, 0);
 	if (IS_ERR(req)) {
-		pr_err("PSCSI: blk_get_request() failed\n");
 		ret = TCM_LOGICAL_UNIT_COMMUNICATION_FAILURE;
 		goto fail;
 	}
diff --git a/include/scsi/scsi_cmnd.h b/include/scsi/scsi_cmnd.h
index eaf04c9a1dfcb..31078063afac2 100644
--- a/include/scsi/scsi_cmnd.h
+++ b/include/scsi/scsi_cmnd.h
@@ -396,4 +396,7 @@ static inline unsigned scsi_transfer_length(struct scsi_cmnd *scmd)
 extern void scsi_build_sense(struct scsi_cmnd *scmd, int desc,
 			     u8 key, u8 asc, u8 ascq);
 
+struct request *scsi_alloc_request(struct request_queue *q,
+		unsigned int op, blk_mq_req_flags_t flags);
+
 #endif /* _SCSI_SCSI_CMND_H */
-- 
2.30.2


^ permalink raw reply related	[flat|nested] 14+ messages in thread

* [PATCH 6/7] block: remove the initialize_rq_fn blk_mq_ops method
  2021-10-21  6:06 remove QUEUE_FLAG_SCSI_PASSTHROUGH v3 Christoph Hellwig
                   ` (4 preceding siblings ...)
  2021-10-21  6:06 ` [PATCH 5/7] scsi: add a scsi_alloc_request helper Christoph Hellwig
@ 2021-10-21  6:06 ` Christoph Hellwig
  2021-10-21  6:06 ` [PATCH 7/7] block: remove QUEUE_FLAG_SCSI_PASSTHROUGH Christoph Hellwig
  2021-10-22 14:34 ` remove QUEUE_FLAG_SCSI_PASSTHROUGH v3 Jens Axboe
  7 siblings, 0 replies; 14+ messages in thread
From: Christoph Hellwig @ 2021-10-21  6:06 UTC (permalink / raw)
  To: Jens Axboe, Martin K. Petersen
  Cc: J. Bruce Fields, Chuck Lever, linux-block, linux-scsi,
	target-devel, linux-nfs, Hannes Reinecke

Entirely unused now.

Signed-off-by: Christoph Hellwig <hch@lst.de>
Reviewed-by: Hannes Reinecke <hare@suse.de>
---
 block/blk-core.c       | 9 +--------
 include/linux/blk-mq.h | 5 -----
 2 files changed, 1 insertion(+), 13 deletions(-)

diff --git a/block/blk-core.c b/block/blk-core.c
index d0c2e11411d03..52a460d0aeb2a 100644
--- a/block/blk-core.c
+++ b/block/blk-core.c
@@ -606,16 +606,9 @@ EXPORT_SYMBOL(blk_get_queue);
 struct request *blk_get_request(struct request_queue *q, unsigned int op,
 				blk_mq_req_flags_t flags)
 {
-	struct request *req;
-
 	WARN_ON_ONCE(op & REQ_NOWAIT);
 	WARN_ON_ONCE(flags & ~(BLK_MQ_REQ_NOWAIT | BLK_MQ_REQ_PM));
-
-	req = blk_mq_alloc_request(q, op, flags);
-	if (!IS_ERR(req) && q->mq_ops->initialize_rq_fn)
-		q->mq_ops->initialize_rq_fn(req);
-
-	return req;
+	return blk_mq_alloc_request(q, op, flags);
 }
 EXPORT_SYMBOL(blk_get_request);
 
diff --git a/include/linux/blk-mq.h b/include/linux/blk-mq.h
index 656fe34bdb6cd..649be3f21d740 100644
--- a/include/linux/blk-mq.h
+++ b/include/linux/blk-mq.h
@@ -565,11 +565,6 @@ struct blk_mq_ops {
 	void (*exit_request)(struct blk_mq_tag_set *set, struct request *,
 			     unsigned int);
 
-	/**
-	 * @initialize_rq_fn: Called from inside blk_get_request().
-	 */
-	void (*initialize_rq_fn)(struct request *rq);
-
 	/**
 	 * @cleanup_rq: Called before freeing one request which isn't completed
 	 * yet, and usually for freeing the driver private data.
-- 
2.30.2


^ permalink raw reply related	[flat|nested] 14+ messages in thread

* [PATCH 7/7] block: remove QUEUE_FLAG_SCSI_PASSTHROUGH
  2021-10-21  6:06 remove QUEUE_FLAG_SCSI_PASSTHROUGH v3 Christoph Hellwig
                   ` (5 preceding siblings ...)
  2021-10-21  6:06 ` [PATCH 6/7] block: remove the initialize_rq_fn blk_mq_ops method Christoph Hellwig
@ 2021-10-21  6:06 ` Christoph Hellwig
  2021-10-22 14:34 ` remove QUEUE_FLAG_SCSI_PASSTHROUGH v3 Jens Axboe
  7 siblings, 0 replies; 14+ messages in thread
From: Christoph Hellwig @ 2021-10-21  6:06 UTC (permalink / raw)
  To: Jens Axboe, Martin K. Petersen
  Cc: J. Bruce Fields, Chuck Lever, linux-block, linux-scsi,
	target-devel, linux-nfs, Hannes Reinecke

Export scsi_device_from_queue for use with pktcdvd and use that instead
of the otherwise unused QUEUE_FLAG_SCSI_PASSTHROUGH queue flag.

Signed-off-by: Christoph Hellwig <hch@lst.de>
Reviewed-by: Hannes Reinecke <hare@suse.de>
---
 block/blk-mq-debugfs.c   | 1 -
 drivers/block/pktcdvd.c  | 5 ++++-
 drivers/scsi/scsi_lib.c  | 8 ++++++++
 drivers/scsi/scsi_scan.c | 1 -
 include/linux/blkdev.h   | 3 ---
 5 files changed, 12 insertions(+), 6 deletions(-)

diff --git a/block/blk-mq-debugfs.c b/block/blk-mq-debugfs.c
index 68ca5d21cda77..a317f05de466a 100644
--- a/block/blk-mq-debugfs.c
+++ b/block/blk-mq-debugfs.c
@@ -124,7 +124,6 @@ static const char *const blk_queue_flag_name[] = {
 	QUEUE_FLAG_NAME(STATS),
 	QUEUE_FLAG_NAME(POLL_STATS),
 	QUEUE_FLAG_NAME(REGISTERED),
-	QUEUE_FLAG_NAME(SCSI_PASSTHROUGH),
 	QUEUE_FLAG_NAME(QUIESCED),
 	QUEUE_FLAG_NAME(PCI_P2PDMA),
 	QUEUE_FLAG_NAME(ZONE_RESETALL),
diff --git a/drivers/block/pktcdvd.c b/drivers/block/pktcdvd.c
index d7d37131ab9dd..d7bcd12394b3c 100644
--- a/drivers/block/pktcdvd.c
+++ b/drivers/block/pktcdvd.c
@@ -2536,6 +2536,7 @@ static int pkt_new_dev(struct pktcdvd_device *pd, dev_t dev)
 	int i;
 	char b[BDEVNAME_SIZE];
 	struct block_device *bdev;
+	struct scsi_device *sdev;
 
 	if (pd->pkt_dev == dev) {
 		pkt_err(pd, "recursive setup not allowed\n");
@@ -2559,10 +2560,12 @@ static int pkt_new_dev(struct pktcdvd_device *pd, dev_t dev)
 	bdev = blkdev_get_by_dev(dev, FMODE_READ | FMODE_NDELAY, NULL);
 	if (IS_ERR(bdev))
 		return PTR_ERR(bdev);
-	if (!blk_queue_scsi_passthrough(bdev_get_queue(bdev))) {
+	sdev = scsi_device_from_queue(bdev->bd_disk->queue);
+	if (!sdev) {
 		blkdev_put(bdev, FMODE_READ | FMODE_NDELAY);
 		return -EINVAL;
 	}
+	put_device(&sdev->sdev_gendev);
 
 	/* This is safe, since we have a reference from open(). */
 	__module_get(THIS_MODULE);
diff --git a/drivers/scsi/scsi_lib.c b/drivers/scsi/scsi_lib.c
index a0f801fc8943b..9823b65d15368 100644
--- a/drivers/scsi/scsi_lib.c
+++ b/drivers/scsi/scsi_lib.c
@@ -1967,6 +1967,14 @@ struct scsi_device *scsi_device_from_queue(struct request_queue *q)
 
 	return sdev;
 }
+/*
+ * pktcdvd should have been integrated into the SCSI layers, but for historical
+ * reasons like the old IDE driver it isn't.  This export allows it to safely
+ * probe if a given device is a SCSI one and only attach to that.
+ */
+#ifdef CONFIG_CDROM_PKTCDVD_MODULE
+EXPORT_SYMBOL_GPL(scsi_device_from_queue);
+#endif
 
 /**
  * scsi_block_requests - Utility function used by low-level drivers to prevent
diff --git a/drivers/scsi/scsi_scan.c b/drivers/scsi/scsi_scan.c
index fe22191522a3b..2808c0cb57114 100644
--- a/drivers/scsi/scsi_scan.c
+++ b/drivers/scsi/scsi_scan.c
@@ -280,7 +280,6 @@ static struct scsi_device *scsi_alloc_sdev(struct scsi_target *starget,
 	sdev->request_queue = q;
 	q->queuedata = sdev;
 	__scsi_init_queue(sdev->host, q);
-	blk_queue_flag_set(QUEUE_FLAG_SCSI_PASSTHROUGH, q);
 	WARN_ON_ONCE(!blk_get_queue(q));
 
 	depth = sdev->host->cmd_per_lun ?: 1;
diff --git a/include/linux/blkdev.h b/include/linux/blkdev.h
index af61fb3e1502c..558aa7ab4c4c8 100644
--- a/include/linux/blkdev.h
+++ b/include/linux/blkdev.h
@@ -356,7 +356,6 @@ struct request_queue {
 #define QUEUE_FLAG_STATS	20	/* track IO start and completion times */
 #define QUEUE_FLAG_POLL_STATS	21	/* collecting stats for hybrid polling */
 #define QUEUE_FLAG_REGISTERED	22	/* queue has been registered to a disk */
-#define QUEUE_FLAG_SCSI_PASSTHROUGH 23	/* queue supports SCSI commands */
 #define QUEUE_FLAG_QUIESCED	24	/* queue has been quiesced */
 #define QUEUE_FLAG_PCI_P2PDMA	25	/* device supports PCI p2p requests */
 #define QUEUE_FLAG_ZONE_RESETALL 26	/* supports Zone Reset All */
@@ -390,8 +389,6 @@ bool blk_queue_flag_test_and_set(unsigned int flag, struct request_queue *q);
 #define blk_queue_secure_erase(q) \
 	(test_bit(QUEUE_FLAG_SECERASE, &(q)->queue_flags))
 #define blk_queue_dax(q)	test_bit(QUEUE_FLAG_DAX, &(q)->queue_flags)
-#define blk_queue_scsi_passthrough(q)	\
-	test_bit(QUEUE_FLAG_SCSI_PASSTHROUGH, &(q)->queue_flags)
 #define blk_queue_pci_p2pdma(q)	\
 	test_bit(QUEUE_FLAG_PCI_P2PDMA, &(q)->queue_flags)
 #ifdef CONFIG_BLK_RQ_ALLOC_TIME
-- 
2.30.2


^ permalink raw reply related	[flat|nested] 14+ messages in thread

* Re: [PATCH 1/7] block: add a ->get_unique_id method
  2021-10-21  6:06 ` [PATCH 1/7] block: add a ->get_unique_id method Christoph Hellwig
@ 2021-10-21 13:34   ` Hannes Reinecke
  0 siblings, 0 replies; 14+ messages in thread
From: Hannes Reinecke @ 2021-10-21 13:34 UTC (permalink / raw)
  To: Christoph Hellwig, Jens Axboe, Martin K. Petersen
  Cc: J. Bruce Fields, Chuck Lever, linux-block, linux-scsi,
	target-devel, linux-nfs

On 10/21/21 8:06 AM, Christoph Hellwig wrote:
> Add a method to query unique IDs from block devices.  It will be used to
> remove code that deeply pokes into SCSI internals in the NFS server.
> The implementation in the sd driver itself is also much nicer as it can
> use the cached VPD page instead of always sending a command as the
> current NFS code does.
> 
> For now the interface is kept very minimal but could be easily
> extended when other users like a block-layer sysfs interface for
> uniquue IDs shows up.
> 
> Signed-off-by: Christoph Hellwig <hch@lst.de>
> ---
>   include/linux/blkdev.h | 11 +++++++++++
>   1 file changed, 11 insertions(+)
> 
Reviewed-by: Hannes Reinecke <hare@suse.de>

Cheers,

Hannes
-- 
Dr. Hannes Reinecke                Kernel Storage Architect
hare@suse.de                              +49 911 74053 688
SUSE Software Solutions GmbH, Maxfeldstr. 5, 90409 Nürnberg
HRB 36809 (AG Nürnberg), Geschäftsführer: Felix Imendörffer

^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [PATCH 2/7] sd: implement ->get_unique_id
  2021-10-21  6:06 ` [PATCH 2/7] sd: implement ->get_unique_id Christoph Hellwig
@ 2021-10-21 13:34   ` Hannes Reinecke
  0 siblings, 0 replies; 14+ messages in thread
From: Hannes Reinecke @ 2021-10-21 13:34 UTC (permalink / raw)
  To: Christoph Hellwig, Jens Axboe, Martin K. Petersen
  Cc: J. Bruce Fields, Chuck Lever, linux-block, linux-scsi,
	target-devel, linux-nfs

On 10/21/21 8:06 AM, Christoph Hellwig wrote:
> Add the method to query for a uniqueue ID of a given type by looking
> it up in the cached device identification VPD page.
> 
> Signed-off-by: Christoph Hellwig <hch@lst.de>
> ---
>   drivers/scsi/sd.c | 39 +++++++++++++++++++++++++++++++++++++++
>   1 file changed, 39 insertions(+)
> 
Reviewed-by: Hannes Reinecke <hare@suse.de>

Cheers,

Hannes
-- 
Dr. Hannes Reinecke                Kernel Storage Architect
hare@suse.de                              +49 911 74053 688
SUSE Software Solutions GmbH, Maxfeldstr. 5, 90409 Nürnberg
HRB 36809 (AG Nürnberg), Geschäftsführer: Felix Imendörffer

^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: remove QUEUE_FLAG_SCSI_PASSTHROUGH v3
  2021-10-21  6:06 remove QUEUE_FLAG_SCSI_PASSTHROUGH v3 Christoph Hellwig
                   ` (6 preceding siblings ...)
  2021-10-21  6:06 ` [PATCH 7/7] block: remove QUEUE_FLAG_SCSI_PASSTHROUGH Christoph Hellwig
@ 2021-10-22 14:34 ` Jens Axboe
  7 siblings, 0 replies; 14+ messages in thread
From: Jens Axboe @ 2021-10-22 14:34 UTC (permalink / raw)
  To: Christoph Hellwig, Martin K. Petersen
  Cc: linux-nfs, linux-block, target-devel, J. Bruce Fields,
	linux-scsi, Chuck Lever

On Thu, 21 Oct 2021 08:06:00 +0200, Christoph Hellwig wrote:
> this series removes the QUEUE_FLAG_SCSI_PASSTHROUGH and thus the last
> remaining SCSI passthrough concept from the block layer.
> 
> The changes to support pktcdvd are a bit ugly, but I can't think of
> anything better (except for removing the driver entirely).
> If we'd want to support packet writing today it would probably live
> entirely inside the sr driver.
> 
> [...]

Applied, thanks!

[1/7] block: add a ->get_unique_id method
      commit: 9208d414975895f69e9aca49153060ddd31b18d0
[2/7] sd: implement ->get_unique_id
      commit: b83ce214af3885437ff223b3a0c8ec6072a84167
[3/7] nfsd/blocklayout: use ->get_unique_id instead of sending SCSI commands
      commit: 8c6aabd1c72bc241c55f5b71a86cea5ef28bceca
[4/7] bsg-lib: initialize the bsg_job in bsg_transport_sg_io_fn
      commit: 237ea1602fb4cd14cd31b745a56fd0639c58eea3
[5/7] scsi: add a scsi_alloc_request helper
      commit: 68ec3b819a5d600a4ede8b596761dccac9f39ebc
[6/7] block: remove the initialize_rq_fn blk_mq_ops method
      commit: 4abafdc4360d993104c2b2f85943938a0c6ad025
[7/7] block: remove QUEUE_FLAG_SCSI_PASSTHROUGH
      commit: 4845012eb5b4e56cadb5f484cb55dd4fd9d1df80

Best regards,
-- 
Jens Axboe



^ permalink raw reply	[flat|nested] 14+ messages in thread

* [PATCH 7/7] block: remove QUEUE_FLAG_SCSI_PASSTHROUGH
  2021-10-19  7:54 remove QUEUE_FLAG_SCSI_PASSTHROUGH v2 Christoph Hellwig
@ 2021-10-19  7:54 ` Christoph Hellwig
  0 siblings, 0 replies; 14+ messages in thread
From: Christoph Hellwig @ 2021-10-19  7:54 UTC (permalink / raw)
  To: Jens Axboe, Martin K. Petersen
  Cc: J. Bruce Fields, Chuck Lever, linux-block, linux-scsi,
	target-devel, linux-nfs, Hannes Reinecke

Export scsi_device_from_queue for use with pktcdvd and use that instead
of the otherwise unused QUEUE_FLAG_SCSI_PASSTHROUGH queue flag.

Signed-off-by: Christoph Hellwig <hch@lst.de>
Reviewed-by: Hannes Reinecke <hare@suse.de>
---
 block/blk-mq-debugfs.c   | 1 -
 drivers/block/pktcdvd.c  | 5 ++++-
 drivers/scsi/scsi_lib.c  | 8 ++++++++
 drivers/scsi/scsi_scan.c | 1 -
 include/linux/blkdev.h   | 3 ---
 5 files changed, 12 insertions(+), 6 deletions(-)

diff --git a/block/blk-mq-debugfs.c b/block/blk-mq-debugfs.c
index 68ca5d21cda77..a317f05de466a 100644
--- a/block/blk-mq-debugfs.c
+++ b/block/blk-mq-debugfs.c
@@ -124,7 +124,6 @@ static const char *const blk_queue_flag_name[] = {
 	QUEUE_FLAG_NAME(STATS),
 	QUEUE_FLAG_NAME(POLL_STATS),
 	QUEUE_FLAG_NAME(REGISTERED),
-	QUEUE_FLAG_NAME(SCSI_PASSTHROUGH),
 	QUEUE_FLAG_NAME(QUIESCED),
 	QUEUE_FLAG_NAME(PCI_P2PDMA),
 	QUEUE_FLAG_NAME(ZONE_RESETALL),
diff --git a/drivers/block/pktcdvd.c b/drivers/block/pktcdvd.c
index ea2262ec76d2c..cacf64eedad87 100644
--- a/drivers/block/pktcdvd.c
+++ b/drivers/block/pktcdvd.c
@@ -2536,6 +2536,7 @@ static int pkt_new_dev(struct pktcdvd_device *pd, dev_t dev)
 	int i;
 	char b[BDEVNAME_SIZE];
 	struct block_device *bdev;
+	struct scsi_device *sdev;
 
 	if (pd->pkt_dev == dev) {
 		pkt_err(pd, "recursive setup not allowed\n");
@@ -2559,10 +2560,12 @@ static int pkt_new_dev(struct pktcdvd_device *pd, dev_t dev)
 	bdev = blkdev_get_by_dev(dev, FMODE_READ | FMODE_NDELAY, NULL);
 	if (IS_ERR(bdev))
 		return PTR_ERR(bdev);
-	if (!blk_queue_scsi_passthrough(bdev_get_queue(bdev))) {
+	sdev = scsi_device_from_queue(bdev->bd_disk->queue);
+	if (!sdev) {
 		blkdev_put(bdev, FMODE_READ | FMODE_NDELAY);
 		return -EINVAL;
 	}
+	put_device(&sdev->sdev_gendev);
 
 	/* This is safe, since we have a reference from open(). */
 	__module_get(THIS_MODULE);
diff --git a/drivers/scsi/scsi_lib.c b/drivers/scsi/scsi_lib.c
index a0f801fc8943b..9823b65d15368 100644
--- a/drivers/scsi/scsi_lib.c
+++ b/drivers/scsi/scsi_lib.c
@@ -1967,6 +1967,14 @@ struct scsi_device *scsi_device_from_queue(struct request_queue *q)
 
 	return sdev;
 }
+/*
+ * pktcdvd should have been integrated into the SCSI layers, but for historical
+ * reasons like the old IDE driver it isn't.  This export allows it to safely
+ * probe if a given device is a SCSI one and only attach to that.
+ */
+#ifdef CONFIG_CDROM_PKTCDVD_MODULE
+EXPORT_SYMBOL_GPL(scsi_device_from_queue);
+#endif
 
 /**
  * scsi_block_requests - Utility function used by low-level drivers to prevent
diff --git a/drivers/scsi/scsi_scan.c b/drivers/scsi/scsi_scan.c
index fe22191522a3b..2808c0cb57114 100644
--- a/drivers/scsi/scsi_scan.c
+++ b/drivers/scsi/scsi_scan.c
@@ -280,7 +280,6 @@ static struct scsi_device *scsi_alloc_sdev(struct scsi_target *starget,
 	sdev->request_queue = q;
 	q->queuedata = sdev;
 	__scsi_init_queue(sdev->host, q);
-	blk_queue_flag_set(QUEUE_FLAG_SCSI_PASSTHROUGH, q);
 	WARN_ON_ONCE(!blk_get_queue(q));
 
 	depth = sdev->host->cmd_per_lun ?: 1;
diff --git a/include/linux/blkdev.h b/include/linux/blkdev.h
index 6e1c6fbdee0b5..0e811a9642d7f 100644
--- a/include/linux/blkdev.h
+++ b/include/linux/blkdev.h
@@ -356,7 +356,6 @@ struct request_queue {
 #define QUEUE_FLAG_STATS	20	/* track IO start and completion times */
 #define QUEUE_FLAG_POLL_STATS	21	/* collecting stats for hybrid polling */
 #define QUEUE_FLAG_REGISTERED	22	/* queue has been registered to a disk */
-#define QUEUE_FLAG_SCSI_PASSTHROUGH 23	/* queue supports SCSI commands */
 #define QUEUE_FLAG_QUIESCED	24	/* queue has been quiesced */
 #define QUEUE_FLAG_PCI_P2PDMA	25	/* device supports PCI p2p requests */
 #define QUEUE_FLAG_ZONE_RESETALL 26	/* supports Zone Reset All */
@@ -390,8 +389,6 @@ bool blk_queue_flag_test_and_set(unsigned int flag, struct request_queue *q);
 #define blk_queue_secure_erase(q) \
 	(test_bit(QUEUE_FLAG_SECERASE, &(q)->queue_flags))
 #define blk_queue_dax(q)	test_bit(QUEUE_FLAG_DAX, &(q)->queue_flags)
-#define blk_queue_scsi_passthrough(q)	\
-	test_bit(QUEUE_FLAG_SCSI_PASSTHROUGH, &(q)->queue_flags)
 #define blk_queue_pci_p2pdma(q)	\
 	test_bit(QUEUE_FLAG_PCI_P2PDMA, &(q)->queue_flags)
 #ifdef CONFIG_BLK_RQ_ALLOC_TIME
-- 
2.30.2


^ permalink raw reply related	[flat|nested] 14+ messages in thread

* Re: [PATCH 7/7] block: remove QUEUE_FLAG_SCSI_PASSTHROUGH
  2021-10-12 12:04 ` [PATCH 7/7] block: " Christoph Hellwig
@ 2021-10-14  7:38   ` Hannes Reinecke
  0 siblings, 0 replies; 14+ messages in thread
From: Hannes Reinecke @ 2021-10-14  7:38 UTC (permalink / raw)
  To: Christoph Hellwig, Jens Axboe, Martin K. Petersen
  Cc: J. Bruce Fields, Chuck Lever, linux-block, linux-scsi,
	target-devel, linux-nfs

On 10/12/21 2:04 PM, Christoph Hellwig wrote:
> Export scsi_device_from_queue for use with pktcdvd and use that instead
> of the otherwise unused QUEUE_FLAG_SCSI_PASSTHROUGH queue flag.
> 
> Signed-off-by: Christoph Hellwig <hch@lst.de>
> ---
>   block/blk-mq-debugfs.c   | 1 -
>   drivers/block/pktcdvd.c  | 5 ++++-
>   drivers/scsi/scsi_lib.c  | 8 ++++++++
>   drivers/scsi/scsi_scan.c | 1 -
>   include/linux/blkdev.h   | 3 ---
>   5 files changed, 12 insertions(+), 6 deletions(-)
> 
Reviewed-by: Hannes Reinecke <hare@suse.de>

Cheers,

Hannes
-- 
Dr. Hannes Reinecke                Kernel Storage Architect
hare@suse.de                              +49 911 74053 688
SUSE Software Solutions GmbH, Maxfeldstr. 5, 90409 Nürnberg
HRB 36809 (AG Nürnberg), Geschäftsführer: Felix Imendörffer

^ permalink raw reply	[flat|nested] 14+ messages in thread

* [PATCH 7/7] block: remove QUEUE_FLAG_SCSI_PASSTHROUGH
  2021-10-12 12:04 Christoph Hellwig
@ 2021-10-12 12:04 ` Christoph Hellwig
  2021-10-14  7:38   ` Hannes Reinecke
  0 siblings, 1 reply; 14+ messages in thread
From: Christoph Hellwig @ 2021-10-12 12:04 UTC (permalink / raw)
  To: Jens Axboe, Martin K. Petersen
  Cc: J. Bruce Fields, Chuck Lever, linux-block, linux-scsi,
	target-devel, linux-nfs

Export scsi_device_from_queue for use with pktcdvd and use that instead
of the otherwise unused QUEUE_FLAG_SCSI_PASSTHROUGH queue flag.

Signed-off-by: Christoph Hellwig <hch@lst.de>
---
 block/blk-mq-debugfs.c   | 1 -
 drivers/block/pktcdvd.c  | 5 ++++-
 drivers/scsi/scsi_lib.c  | 8 ++++++++
 drivers/scsi/scsi_scan.c | 1 -
 include/linux/blkdev.h   | 3 ---
 5 files changed, 12 insertions(+), 6 deletions(-)

diff --git a/block/blk-mq-debugfs.c b/block/blk-mq-debugfs.c
index 4000376330c90..8e716c20fca5d 100644
--- a/block/blk-mq-debugfs.c
+++ b/block/blk-mq-debugfs.c
@@ -124,7 +124,6 @@ static const char *const blk_queue_flag_name[] = {
 	QUEUE_FLAG_NAME(STATS),
 	QUEUE_FLAG_NAME(POLL_STATS),
 	QUEUE_FLAG_NAME(REGISTERED),
-	QUEUE_FLAG_NAME(SCSI_PASSTHROUGH),
 	QUEUE_FLAG_NAME(QUIESCED),
 	QUEUE_FLAG_NAME(PCI_P2PDMA),
 	QUEUE_FLAG_NAME(ZONE_RESETALL),
diff --git a/drivers/block/pktcdvd.c b/drivers/block/pktcdvd.c
index 7d3558aef58a5..e5826d09d9706 100644
--- a/drivers/block/pktcdvd.c
+++ b/drivers/block/pktcdvd.c
@@ -2537,6 +2537,7 @@ static int pkt_new_dev(struct pktcdvd_device *pd, dev_t dev)
 	int i;
 	char b[BDEVNAME_SIZE];
 	struct block_device *bdev;
+	struct scsi_device *sdev;
 
 	if (pd->pkt_dev == dev) {
 		pkt_err(pd, "recursive setup not allowed\n");
@@ -2560,10 +2561,12 @@ static int pkt_new_dev(struct pktcdvd_device *pd, dev_t dev)
 	bdev = blkdev_get_by_dev(dev, FMODE_READ | FMODE_NDELAY, NULL);
 	if (IS_ERR(bdev))
 		return PTR_ERR(bdev);
-	if (!blk_queue_scsi_passthrough(bdev_get_queue(bdev))) {
+	sdev = scsi_device_from_queue(bdev->bd_disk->queue);
+	if (!sdev) {
 		blkdev_put(bdev, FMODE_READ | FMODE_NDELAY);
 		return -EINVAL;
 	}
+	put_device(&sdev->sdev_gendev);
 
 	/* This is safe, since we have a reference from open(). */
 	__module_get(THIS_MODULE);
diff --git a/drivers/scsi/scsi_lib.c b/drivers/scsi/scsi_lib.c
index d787b7bd9cb06..e7dd427b927d6 100644
--- a/drivers/scsi/scsi_lib.c
+++ b/drivers/scsi/scsi_lib.c
@@ -1967,6 +1967,14 @@ struct scsi_device *scsi_device_from_queue(struct request_queue *q)
 
 	return sdev;
 }
+/*
+ * pktcdvd should have been integrated into the SCSI layers, but for historical
+ * reasons like the old IDE driver it isn't.  This export allows it to safely
+ * probe if a given device is a SCSI one and only attach to that.
+ */
+#ifdef CONFIG_CDROM_PKTCDVD_MODULE
+EXPORT_SYMBOL_GPL(scsi_device_from_queue);
+#endif
 
 /**
  * scsi_block_requests - Utility function used by low-level drivers to prevent
diff --git a/drivers/scsi/scsi_scan.c b/drivers/scsi/scsi_scan.c
index fe22191522a3b..2808c0cb57114 100644
--- a/drivers/scsi/scsi_scan.c
+++ b/drivers/scsi/scsi_scan.c
@@ -280,7 +280,6 @@ static struct scsi_device *scsi_alloc_sdev(struct scsi_target *starget,
 	sdev->request_queue = q;
 	q->queuedata = sdev;
 	__scsi_init_queue(sdev->host, q);
-	blk_queue_flag_set(QUEUE_FLAG_SCSI_PASSTHROUGH, q);
 	WARN_ON_ONCE(!blk_get_queue(q));
 
 	depth = sdev->host->cmd_per_lun ?: 1;
diff --git a/include/linux/blkdev.h b/include/linux/blkdev.h
index 81f94a7c54521..bf8d1996e4d4a 100644
--- a/include/linux/blkdev.h
+++ b/include/linux/blkdev.h
@@ -355,7 +355,6 @@ struct request_queue {
 #define QUEUE_FLAG_STATS	20	/* track IO start and completion times */
 #define QUEUE_FLAG_POLL_STATS	21	/* collecting stats for hybrid polling */
 #define QUEUE_FLAG_REGISTERED	22	/* queue has been registered to a disk */
-#define QUEUE_FLAG_SCSI_PASSTHROUGH 23	/* queue supports SCSI commands */
 #define QUEUE_FLAG_QUIESCED	24	/* queue has been quiesced */
 #define QUEUE_FLAG_PCI_P2PDMA	25	/* device supports PCI p2p requests */
 #define QUEUE_FLAG_ZONE_RESETALL 26	/* supports Zone Reset All */
@@ -389,8 +388,6 @@ bool blk_queue_flag_test_and_set(unsigned int flag, struct request_queue *q);
 #define blk_queue_secure_erase(q) \
 	(test_bit(QUEUE_FLAG_SECERASE, &(q)->queue_flags))
 #define blk_queue_dax(q)	test_bit(QUEUE_FLAG_DAX, &(q)->queue_flags)
-#define blk_queue_scsi_passthrough(q)	\
-	test_bit(QUEUE_FLAG_SCSI_PASSTHROUGH, &(q)->queue_flags)
 #define blk_queue_pci_p2pdma(q)	\
 	test_bit(QUEUE_FLAG_PCI_P2PDMA, &(q)->queue_flags)
 #ifdef CONFIG_BLK_RQ_ALLOC_TIME
-- 
2.30.2


^ permalink raw reply related	[flat|nested] 14+ messages in thread

end of thread, other threads:[~2021-10-22 14:34 UTC | newest]

Thread overview: 14+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2021-10-21  6:06 remove QUEUE_FLAG_SCSI_PASSTHROUGH v3 Christoph Hellwig
2021-10-21  6:06 ` [PATCH 1/7] block: add a ->get_unique_id method Christoph Hellwig
2021-10-21 13:34   ` Hannes Reinecke
2021-10-21  6:06 ` [PATCH 2/7] sd: implement ->get_unique_id Christoph Hellwig
2021-10-21 13:34   ` Hannes Reinecke
2021-10-21  6:06 ` [PATCH 3/7] nfsd/blocklayout: use ->get_unique_id instead of sending SCSI commands Christoph Hellwig
2021-10-21  6:06 ` [PATCH 4/7] bsg-lib: initialize the bsg_job in bsg_transport_sg_io_fn Christoph Hellwig
2021-10-21  6:06 ` [PATCH 5/7] scsi: add a scsi_alloc_request helper Christoph Hellwig
2021-10-21  6:06 ` [PATCH 6/7] block: remove the initialize_rq_fn blk_mq_ops method Christoph Hellwig
2021-10-21  6:06 ` [PATCH 7/7] block: remove QUEUE_FLAG_SCSI_PASSTHROUGH Christoph Hellwig
2021-10-22 14:34 ` remove QUEUE_FLAG_SCSI_PASSTHROUGH v3 Jens Axboe
  -- strict thread matches above, loose matches on Subject: below --
2021-10-19  7:54 remove QUEUE_FLAG_SCSI_PASSTHROUGH v2 Christoph Hellwig
2021-10-19  7:54 ` [PATCH 7/7] block: remove QUEUE_FLAG_SCSI_PASSTHROUGH Christoph Hellwig
2021-10-12 12:04 Christoph Hellwig
2021-10-12 12:04 ` [PATCH 7/7] block: " Christoph Hellwig
2021-10-14  7:38   ` Hannes Reinecke

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).