rust-for-linux.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-13 11:05 [RFC PATCH 0/5] Rust block device driver API and null block driver Andreas Hindborg
@ 2024-03-13 11:05 ` Andreas Hindborg
  2024-03-13 23:55   ` Boqun Feng
  0 siblings, 1 reply; 26+ messages in thread
From: Andreas Hindborg @ 2024-03-13 11:05 UTC (permalink / raw)
  To: Jens Axboe, Christoph Hellwig, Keith Busch, Damien Le Moal,
	Bart Van Assche, Hannes Reinecke, linux-block@vger.kernel.org
  Cc: Andreas Hindborg, Wedson Almeida Filho, Niklas Cassel, Greg KH,
	Matthew Wilcox, Miguel Ojeda, Alex Gaynor, Boqun Feng, Gary Guo,
	Björn Roy Baron, Benno Lossin, Alice Ryhl,
	Chaitanya Kulkarni, Luis Chamberlain, Yexuan Yang,
	Sergio González Collado, Joel Granados,
	Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

From: Andreas Hindborg <a.hindborg@samsung.com>

Add initial abstractions for working with blk-mq.

This patch is a maintained, refactored subset of code originally published by
Wedson Almeida Filho <wedsonaf@gmail.com> [1].

[1] https://github.com/wedsonaf/linux/tree/f2cfd2fe0e2ca4e90994f96afe268bbd4382a891/rust/kernel/blk/mq.rs

Cc: Wedson Almeida Filho <wedsonaf@gmail.com>
Signed-off-by: Andreas Hindborg <a.hindborg@samsung.com>
---
 block/blk-mq.c                     |   3 +-
 include/linux/blk-mq.h             |   1 +
 rust/bindings/bindings_helper.h    |   2 +
 rust/helpers.c                     |  45 ++++
 rust/kernel/block.rs               |   5 +
 rust/kernel/block/mq.rs            | 131 +++++++++++
 rust/kernel/block/mq/gen_disk.rs   | 174 +++++++++++++++
 rust/kernel/block/mq/operations.rs | 346 +++++++++++++++++++++++++++++
 rust/kernel/block/mq/raw_writer.rs |  60 +++++
 rust/kernel/block/mq/request.rs    | 182 +++++++++++++++
 rust/kernel/block/mq/tag_set.rs    | 117 ++++++++++
 rust/kernel/error.rs               |   5 +
 rust/kernel/lib.rs                 |   1 +
 13 files changed, 1071 insertions(+), 1 deletion(-)
 create mode 100644 rust/kernel/block.rs
 create mode 100644 rust/kernel/block/mq.rs
 create mode 100644 rust/kernel/block/mq/gen_disk.rs
 create mode 100644 rust/kernel/block/mq/operations.rs
 create mode 100644 rust/kernel/block/mq/raw_writer.rs
 create mode 100644 rust/kernel/block/mq/request.rs
 create mode 100644 rust/kernel/block/mq/tag_set.rs

diff --git a/block/blk-mq.c b/block/blk-mq.c
index 2dc01551e27c..a531f664bee7 100644
--- a/block/blk-mq.c
+++ b/block/blk-mq.c
@@ -702,7 +702,7 @@ static void blk_mq_finish_request(struct request *rq)
 	}
 }
 
-static void __blk_mq_free_request(struct request *rq)
+void __blk_mq_free_request(struct request *rq)
 {
 	struct request_queue *q = rq->q;
 	struct blk_mq_ctx *ctx = rq->mq_ctx;
@@ -722,6 +722,7 @@ static void __blk_mq_free_request(struct request *rq)
 	blk_mq_sched_restart(hctx);
 	blk_queue_exit(q);
 }
+EXPORT_SYMBOL_GPL(__blk_mq_free_request);
 
 void blk_mq_free_request(struct request *rq)
 {
diff --git a/include/linux/blk-mq.h b/include/linux/blk-mq.h
index 7a8150a5f051..842bb88e6e78 100644
--- a/include/linux/blk-mq.h
+++ b/include/linux/blk-mq.h
@@ -703,6 +703,7 @@ int blk_mq_alloc_sq_tag_set(struct blk_mq_tag_set *set,
 		unsigned int set_flags);
 void blk_mq_free_tag_set(struct blk_mq_tag_set *set);
 
+void __blk_mq_free_request(struct request *rq);
 void blk_mq_free_request(struct request *rq);
 int blk_rq_poll(struct request *rq, struct io_comp_batch *iob,
 		unsigned int poll_flags);
diff --git a/rust/bindings/bindings_helper.h b/rust/bindings/bindings_helper.h
index f8e54d398c19..df18acb229d9 100644
--- a/rust/bindings/bindings_helper.h
+++ b/rust/bindings/bindings_helper.h
@@ -7,6 +7,8 @@
  */
 
 #include <kunit/test.h>
+#include <linux/blk_types.h>
+#include <linux/blk-mq.h>
 #include <linux/errname.h>
 #include <linux/ethtool.h>
 #include <linux/mdio.h>
diff --git a/rust/helpers.c b/rust/helpers.c
index 66411845536e..017fa90366e6 100644
--- a/rust/helpers.c
+++ b/rust/helpers.c
@@ -21,6 +21,9 @@
  */
 
 #include <kunit/test-bug.h>
+#include <linux/bio.h>
+#include <linux/blk-mq.h>
+#include <linux/blkdev.h>
 #include <linux/bug.h>
 #include <linux/build_bug.h>
 #include <linux/cacheflush.h>
@@ -242,6 +245,30 @@ void *rust_helper_kmap_local_folio(struct folio *folio, size_t offset)
 }
 EXPORT_SYMBOL_GPL(rust_helper_kmap_local_folio);
 
+struct bio_vec rust_helper_req_bvec(struct request *rq)
+{
+	return req_bvec(rq);
+}
+EXPORT_SYMBOL_GPL(rust_helper_req_bvec);
+
+void *rust_helper_blk_mq_rq_to_pdu(struct request *rq)
+{
+	return blk_mq_rq_to_pdu(rq);
+}
+EXPORT_SYMBOL_GPL(rust_helper_blk_mq_rq_to_pdu);
+
+struct request *rust_helper_blk_mq_rq_from_pdu(void* pdu) {
+  return blk_mq_rq_from_pdu(pdu);
+}
+EXPORT_SYMBOL_GPL(rust_helper_blk_mq_rq_from_pdu);
+
+void rust_helper_bio_advance_iter_single(const struct bio *bio,
+                                         struct bvec_iter *iter,
+                                         unsigned int bytes) {
+  bio_advance_iter_single(bio, iter, bytes);
+}
+EXPORT_SYMBOL_GPL(rust_helper_bio_advance_iter_single);
+
 void *rust_helper_kmap(struct page *page)
 {
 	return kmap(page);
@@ -306,6 +333,24 @@ int rust_helper_xa_err(void *entry)
 }
 EXPORT_SYMBOL_GPL(rust_helper_xa_err);
 
+bool rust_helper_req_ref_inc_not_zero(struct request *req)
+{
+	return atomic_inc_not_zero(&req->ref);
+}
+EXPORT_SYMBOL_GPL(rust_helper_req_ref_inc_not_zero);
+
+bool rust_helper_req_ref_put_and_test(struct request *req)
+{
+	return atomic_dec_and_test(&req->ref);
+}
+EXPORT_SYMBOL_GPL(rust_helper_req_ref_put_and_test);
+
+void rust_helper_blk_mq_free_request_internal(struct request *req)
+{
+	__blk_mq_free_request(req);
+}
+EXPORT_SYMBOL_GPL(rust_helper_blk_mq_free_request_internal);
+
 /*
  * `bindgen` binds the C `size_t` type as the Rust `usize` type, so we can
  * use it in contexts where Rust expects a `usize` like slice (array) indices.
diff --git a/rust/kernel/block.rs b/rust/kernel/block.rs
new file mode 100644
index 000000000000..4c93317a568a
--- /dev/null
+++ b/rust/kernel/block.rs
@@ -0,0 +1,5 @@
+// SPDX-License-Identifier: GPL-2.0
+
+//! Types for working with the block layer
+
+pub mod mq;
diff --git a/rust/kernel/block/mq.rs b/rust/kernel/block/mq.rs
new file mode 100644
index 000000000000..08de1cc114ff
--- /dev/null
+++ b/rust/kernel/block/mq.rs
@@ -0,0 +1,131 @@
+// SPDX-License-Identifier: GPL-2.0
+
+//! This module provides types for implementing block drivers that interface the
+//! blk-mq subsystem.
+//!
+//! To implement a block device driver, a Rust module must do the following:
+//!
+//! - Implement [`Operations`] for a type `T`
+//! - Create a [`TagSet<T>`]
+//! - Create a [`GenDisk<T>`], passing in the `TagSet` reference
+//! - Add the disk to the system by calling [`GenDisk::add`]
+//!
+//! The types available in this module that have direct C counterparts are:
+//!
+//! - The `TagSet` type that abstracts the C type `struct tag_set`.
+//! - The `GenDisk` type that abstracts the C type `struct gendisk`.
+//! - The `Request` type that abstracts the C type `struct request`.
+//!
+//! Many of the C types that this module abstracts allow a driver to carry
+//! private data, either embedded in the stuct directly, or as a C `void*`. In
+//! these abstractions, this data is typed. The types of the data is defined by
+//! associated types in `Operations`, see [`Operations::RequestData`] for an
+//! example.
+//!
+//! The kernel will interface with the block evice driver by calling the method
+//! implementations of the `Operations` trait.
+//!
+//! IO requests are passed to the driver as [`Request`] references. The
+//! `Request` type is a wrapper around the C `struct request`. The driver must
+//! mark start of request processing by calling [`Request::start`] and end of
+//! processing by calling one of the [`Request::end`], methods. Failure to do so
+//! can lead to IO failures.
+//!
+//! The `TagSet` is responsible for creating and maintaining a mapping between
+//! `Request`s and integer ids as well as carrying a pointer to the vtable
+//! generated by `Operations`. This mapping is useful for associating
+//! completions from hardware with the correct `Request` instance. The `TagSet`
+//! determines the maximum queue depth by setting the number of `Request`
+//! instances available to the driver, and it determines the number of queues to
+//! instantiate for the driver. If possible, a driver should allocate one queue
+//! per core, to keep queue data local to a core.
+//!
+//! One `TagSet` instance can be shared between multiple `GenDisk` instances.
+//! This can be useful when implementing drivers where one piece of hardware
+//! with one set of IO resources are represented to the user as multiple disks.
+//!
+//! One significant difference between block device drivers implemented with
+//! these Rust abstractions and drivers implemented in C, is that the Rust
+//! drivers have to own a reference count on the `Request` type when the IO is
+//! in flight. This is to ensure that the C `struct request` instances backing
+//! the Rust `Request` instances are live while the Rust driver holds a
+//! reference to the `Request`. In addition, the conversion of an ineger tag to
+//! a `Request` via the `TagSet` would not be sound without this bookkeeping.
+//!
+//! # ⚠ Note
+//!
+//! For Rust block device drivers, the point in time where a request
+//! is freed and made available for recycling is usualy at the point in time
+//! when the last `ARef<Request>` is dropped. For C drivers, this event usually
+//! occurs when `bindings::blk_mq_end_request` is called.
+//!
+//! # Example
+//!
+//! ```rust
+//! use kernel::{
+//!     block::mq::*,
+//!     new_mutex,
+//!     prelude::*,
+//!     sync::{Arc, Mutex},
+//!     types::{ARef, ForeignOwnable},
+//! };
+//!
+//! struct MyBlkDevice;
+//!
+//! #[vtable]
+//! impl Operations for MyBlkDevice {
+//!     type RequestData = ();
+//!     type RequestDataInit = impl PinInit<()>;
+//!     type QueueData = ();
+//!     type HwData = ();
+//!     type TagSetData = ();
+//!
+//!     fn new_request_data(
+//!         _tagset_data: <Self::TagSetData as ForeignOwnable>::Borrowed<'_>,
+//!     ) -> Self::RequestDataInit {
+//!         kernel::init::zeroed()
+//!     }
+//!
+//!     fn queue_rq(_hw_data: (), _queue_data: (), rq: ARef<Request<Self>>, _is_last: bool) -> Result {
+//!         rq.start();
+//!         rq.end_ok();
+//!         Ok(())
+//!     }
+//!
+//!     fn commit_rqs(
+//!         _hw_data: <Self::HwData as ForeignOwnable>::Borrowed<'_>,
+//!         _queue_data: <Self::QueueData as ForeignOwnable>::Borrowed<'_>,
+//!     ) {
+//!     }
+//!
+//!     fn complete(rq: &Request<Self>) {
+//!         rq.end_ok();
+//!     }
+//!
+//!     fn init_hctx(
+//!         _tagset_data: <Self::TagSetData as ForeignOwnable>::Borrowed<'_>,
+//!         _hctx_idx: u32,
+//!     ) -> Result<Self::HwData> {
+//!         Ok(())
+//!     }
+//! }
+//!
+//! let tagset: Arc<TagSet<MyBlkDevice>> = Arc::pin_init(TagSet::try_new(1, (), 256, 1))?;
+//! let mut disk = GenDisk::try_new(tagset, ())?;
+//! disk.set_name(format_args!("myblk"))?;
+//! disk.set_capacity_sectors(4096);
+//! disk.add()?;
+//!
+//! # Ok::<(), kernel::error::Error>(())
+//! ```
+
+mod gen_disk;
+mod operations;
+mod raw_writer;
+mod request;
+mod tag_set;
+
+pub use gen_disk::GenDisk;
+pub use operations::Operations;
+pub use request::{Request, RequestDataRef};
+pub use tag_set::TagSet;
diff --git a/rust/kernel/block/mq/gen_disk.rs b/rust/kernel/block/mq/gen_disk.rs
new file mode 100644
index 000000000000..b7845fc9e39f
--- /dev/null
+++ b/rust/kernel/block/mq/gen_disk.rs
@@ -0,0 +1,174 @@
+// SPDX-License-Identifier: GPL-2.0
+
+//! Generic disk abstraction.
+//!
+//! C header: [`include/linux/blkdev.h`](srctree/include/linux/blkdev.h)
+//! C header: [`include/linux/blk_mq.h`](srctree/include/linux/blk_mq.h)
+
+use crate::block::mq::{raw_writer::RawWriter, Operations, TagSet};
+use crate::{
+    bindings, error::from_err_ptr, error::Result, sync::Arc, types::ForeignOwnable,
+    types::ScopeGuard,
+};
+use core::fmt::{self, Write};
+
+/// A generic block device
+///
+/// # Invariants
+///
+///  - `gendisk` must always point to an initialized and valid `struct gendisk`.
+pub struct GenDisk<T: Operations> {
+    _tagset: Arc<TagSet<T>>,
+    gendisk: *mut bindings::gendisk,
+}
+
+// SAFETY: `GenDisk` is an owned pointer to a `struct gendisk` and an `Arc` to a
+// `TagSet` It is safe to send this to other threads as long as T is Send.
+unsafe impl<T: Operations + Send> Send for GenDisk<T> {}
+
+impl<T: Operations> GenDisk<T> {
+    /// Try to create a new `GenDisk`
+    pub fn try_new(tagset: Arc<TagSet<T>>, queue_data: T::QueueData) -> Result<Self> {
+        let data = queue_data.into_foreign();
+        let recover_data = ScopeGuard::new(|| {
+            // SAFETY: T::QueueData was created by the call to `into_foreign()` above
+            unsafe { T::QueueData::from_foreign(data) };
+        });
+
+        let lock_class_key = crate::sync::LockClassKey::new();
+
+        // SAFETY: `tagset.raw_tag_set()` points to a valid and initialized tag set
+        let gendisk = from_err_ptr(unsafe {
+            bindings::__blk_mq_alloc_disk(
+                tagset.raw_tag_set(),
+                data.cast_mut(),
+                lock_class_key.as_ptr(),
+            )
+        })?;
+
+        const TABLE: bindings::block_device_operations = bindings::block_device_operations {
+            submit_bio: None,
+            open: None,
+            release: None,
+            ioctl: None,
+            compat_ioctl: None,
+            check_events: None,
+            unlock_native_capacity: None,
+            getgeo: None,
+            set_read_only: None,
+            swap_slot_free_notify: None,
+            report_zones: None,
+            devnode: None,
+            alternative_gpt_sector: None,
+            get_unique_id: None,
+            // TODO: Set to THIS_MODULE. Waiting for const_refs_to_static feature to be merged
+            // https://github.com/rust-lang/rust/issues/119618
+            owner: core::ptr::null_mut(),
+            pr_ops: core::ptr::null_mut(),
+            free_disk: None,
+            poll_bio: None,
+        };
+
+        // SAFETY: gendisk is a valid pointer as we initialized it above
+        unsafe { (*gendisk).fops = &TABLE };
+
+        recover_data.dismiss();
+        Ok(Self {
+            _tagset: tagset,
+            gendisk,
+        })
+    }
+
+    /// Set the name of the device
+    pub fn set_name(&mut self, args: fmt::Arguments<'_>) -> Result {
+        let mut raw_writer = RawWriter::from_array(
+            // SAFETY: By type invariant `self.gendisk` points to a valid and initialized instance
+            unsafe { &mut (*self.gendisk).disk_name },
+        );
+        raw_writer.write_fmt(args)?;
+        raw_writer.write_char('\0')?;
+        Ok(())
+    }
+
+    /// Register the device with the kernel. When this function return, the
+    /// device is accessible from VFS. The kernel may issue reads to the device
+    /// during registration to discover partition infomation.
+    pub fn add(&self) -> Result {
+        crate::error::to_result(
+            // SAFETY: By type invariant, `self.gendisk` points to a valid and
+            // initialized instance of `struct gendisk`
+            unsafe {
+                bindings::device_add_disk(
+                    core::ptr::null_mut(),
+                    self.gendisk,
+                    core::ptr::null_mut(),
+                )
+            },
+        )
+    }
+
+    /// Call to tell the block layer the capacity of the device in sectors (512B)
+    pub fn set_capacity_sectors(&self, sectors: u64) {
+        // SAFETY: By type invariant, `self.gendisk` points to a valid and
+        // initialized instance of `struct gendisk`
+        unsafe { bindings::set_capacity(self.gendisk, sectors) };
+    }
+
+    /// Set the logical block size of the device.
+    ///
+    /// This is the smallest unit the storage device can address. It is
+    /// typically 512 bytes.
+    pub fn set_queue_logical_block_size(&self, size: u32) {
+        // SAFETY: By type invariant, `self.gendisk` points to a valid and
+        // initialized instance of `struct gendisk`
+        unsafe { bindings::blk_queue_logical_block_size((*self.gendisk).queue, size) };
+    }
+
+    /// Set the physical block size of the device.
+    ///
+    /// This is the smallest unit a physical storage device can write
+    /// atomically. It is usually the same as the logical block size but may be
+    /// bigger. One example is SATA drives with 4KB sectors that expose a
+    /// 512-byte logical block size to the operating system.
+    pub fn set_queue_physical_block_size(&self, size: u32) {
+        // SAFETY: By type invariant, `self.gendisk` points to a valid and
+        // initialized instance of `struct gendisk`
+        unsafe { bindings::blk_queue_physical_block_size((*self.gendisk).queue, size) };
+    }
+
+    /// Set the rotational media attribute for the device
+    pub fn set_rotational(&self, rotational: bool) {
+        if !rotational {
+            // SAFETY: By type invariant, `self.gendisk` points to a valid and
+            // initialized instance of `struct gendisk`
+            unsafe {
+                bindings::blk_queue_flag_set(bindings::QUEUE_FLAG_NONROT, (*self.gendisk).queue)
+            };
+        } else {
+            // SAFETY: By type invariant, `self.gendisk` points to a valid and
+            // initialized instance of `struct gendisk`
+            unsafe {
+                bindings::blk_queue_flag_clear(bindings::QUEUE_FLAG_NONROT, (*self.gendisk).queue)
+            };
+        }
+    }
+}
+
+impl<T: Operations> Drop for GenDisk<T> {
+    fn drop(&mut self) {
+        // SAFETY: By type invariant of `Self`, `self.gendisk` points to a valid
+        // and initialized instance of `struct gendisk`. As such, `queuedata`
+        // was initialized by the initializer returned by `try_new` with a call
+        // to `ForeignOwnable::into_foreign`.
+        let queue_data = unsafe { (*(*self.gendisk).queue).queuedata };
+
+        // SAFETY: By type invariant, `self.gendisk` points to a valid and
+        // initialized instance of `struct gendisk`
+        unsafe { bindings::del_gendisk(self.gendisk) };
+
+        // SAFETY: `queue.queuedata` was created by `GenDisk::try_new()` with a
+        // call to `ForeignOwnable::into_pointer()` to create `queuedata`.
+        // `ForeignOwnable::from_foreign()` is only called here.
+        let _queue_data = unsafe { T::QueueData::from_foreign(queue_data) };
+    }
+}
diff --git a/rust/kernel/block/mq/operations.rs b/rust/kernel/block/mq/operations.rs
new file mode 100644
index 000000000000..53c6ad663208
--- /dev/null
+++ b/rust/kernel/block/mq/operations.rs
@@ -0,0 +1,346 @@
+// SPDX-License-Identifier: GPL-2.0
+
+//! This module provides an interface for blk-mq drivers to implement.
+//!
+//! C header: [`include/linux/blk-mq.h`](srctree/include/linux/blk-mq.h)
+
+use crate::{
+    bindings,
+    block::mq::Request,
+    error::{from_result, Result},
+    init::PinInit,
+    types::{ARef, ForeignOwnable},
+};
+use core::{marker::PhantomData, ptr::NonNull};
+
+use super::TagSet;
+
+/// Implement this trait to interface blk-mq as block devices
+#[macros::vtable]
+pub trait Operations: Sized {
+    /// Data associated with a request. This data is located next to the request
+    /// structure.
+    ///
+    /// To be able to handle accessing this data from interrupt context, this
+    /// data must be `Sync`.
+    type RequestData: Sized + Sync;
+
+    /// Initializer for `Self::RequestDta`. Used to initialize private data area
+    /// when requst structure is allocated.
+    type RequestDataInit: PinInit<Self::RequestData>;
+
+    /// Data associated with the `struct request_queue` that is allocated for
+    /// the `GenDisk` associated with this `Operations` implementation.
+    type QueueData: ForeignOwnable;
+
+    /// Data associated with a dispatch queue. This is stored as a pointer in
+    /// the C `struct blk_mq_hw_ctx` that represents a hardware queue.
+    type HwData: ForeignOwnable;
+
+    /// Data associated with a `TagSet`. This is stored as a pointer in `struct
+    /// blk_mq_tag_set`.
+    type TagSetData: ForeignOwnable;
+
+    /// Called by the kernel to get an initializer for a `Pin<&mut RequestData>`.
+    fn new_request_data(
+        //rq: ARef<Request<Self>>,
+        tagset_data: <Self::TagSetData as ForeignOwnable>::Borrowed<'_>,
+    ) -> Self::RequestDataInit;
+
+    /// Called by the kernel to queue a request with the driver. If `is_last` is
+    /// `false`, the driver is allowed to defer commiting the request.
+    fn queue_rq(
+        hw_data: <Self::HwData as ForeignOwnable>::Borrowed<'_>,
+        queue_data: <Self::QueueData as ForeignOwnable>::Borrowed<'_>,
+        rq: ARef<Request<Self>>,
+        is_last: bool,
+    ) -> Result;
+
+    /// Called by the kernel to indicate that queued requests should be submitted
+    fn commit_rqs(
+        hw_data: <Self::HwData as ForeignOwnable>::Borrowed<'_>,
+        queue_data: <Self::QueueData as ForeignOwnable>::Borrowed<'_>,
+    );
+
+    /// Called by the kernel when the request is completed
+    fn complete(_rq: &Request<Self>);
+
+    /// Called by the kernel to allocate and initialize a driver specific hardware context data
+    fn init_hctx(
+        tagset_data: <Self::TagSetData as ForeignOwnable>::Borrowed<'_>,
+        hctx_idx: u32,
+    ) -> Result<Self::HwData>;
+
+    /// Called by the kernel to poll the device for completed requests. Only
+    /// used for poll queues.
+    fn poll(_hw_data: <Self::HwData as ForeignOwnable>::Borrowed<'_>) -> bool {
+        crate::build_error(crate::error::VTABLE_DEFAULT_ERROR)
+    }
+
+    /// Called by the kernel to map submission queues to CPU cores.
+    fn map_queues(_tag_set: &TagSet<Self>) {
+        crate::build_error(crate::error::VTABLE_DEFAULT_ERROR)
+    }
+
+    // There is no need for exit_request() because `drop` will be called.
+}
+
+/// A vtable for blk-mq to interact with a block device driver.
+///
+/// A `bindings::blk_mq_opa` vtable is constructed from pointers to the `extern
+/// "C"` functions of this struct, exposed through the `OperationsVTable::VTABLE`.
+///
+/// For general documentation of these methods, see the kernel source
+/// documentation related to `struct blk_mq_operations` in
+/// [`include/linux/blk-mq.h`].
+///
+/// [`include/linux/blk-mq.h`]: srctree/include/linux/blk-mq.h
+pub(crate) struct OperationsVTable<T: Operations>(PhantomData<T>);
+
+impl<T: Operations> OperationsVTable<T> {
+    // # Safety
+    //
+    // - The caller of this function must ensure that `hctx` and `bd` are valid
+    //   and initialized. The pointees must outlive this function.
+    // - `hctx->driver_data` must be a pointer created by a call to
+    //   `Self::init_hctx_callback()` and the pointee must outlive this
+    //   function.
+    // - This function must not be called with a `hctx` for which
+    //   `Self::exit_hctx_callback()` has been called.
+    // - (*bd).rq must point to a valid `bindings:request` with a positive refcount in the `ref` field.
+    unsafe extern "C" fn queue_rq_callback(
+        hctx: *mut bindings::blk_mq_hw_ctx,
+        bd: *const bindings::blk_mq_queue_data,
+    ) -> bindings::blk_status_t {
+        // SAFETY: `bd` is valid as required by the safety requirement for this
+        // function.
+        let request_ptr = unsafe { (*bd).rq };
+
+        // SAFETY: By C API contract, the pointee of `request_ptr` is valid and has a refcount of 1
+        #[cfg_attr(not(CONFIG_DEBUG_MISC), allow(unused_variables))]
+        let updated = unsafe { bindings::req_ref_inc_not_zero(request_ptr) };
+
+        #[cfg(CONFIG_DEBUG_MISC)]
+        if !updated {
+            crate::pr_err!("Request ref was zero at queue time\n");
+        }
+
+        let rq =
+            // SAFETY: We own a refcount that we took above. We pass that to
+            // `ARef`.
+            unsafe { ARef::from_raw(NonNull::new_unchecked(request_ptr.cast::<Request<T>>())) };
+
+        // SAFETY: The safety requirement for this function ensure that `hctx`
+        // is valid and that `driver_data` was produced by a call to
+        // `into_foreign` in `Self::init_hctx_callback`.
+        let hw_data = unsafe { T::HwData::borrow((*hctx).driver_data) };
+
+        // SAFETY: `hctx` is valid as required by this function.
+        let queue_data = unsafe { (*(*hctx).queue).queuedata };
+
+        // SAFETY: `queue.queuedata` was created by `GenDisk::try_new()` with a
+        // call to `ForeignOwnable::into_pointer()` to create `queuedata`.
+        // `ForeignOwnable::from_foreign()` is only called when the tagset is
+        // dropped, which happens after we are dropped.
+        let queue_data = unsafe { T::QueueData::borrow(queue_data) };
+
+        let ret = T::queue_rq(
+            hw_data,
+            queue_data,
+            rq,
+            // SAFETY: `bd` is valid as required by the safety requirement for this function.
+            unsafe { (*bd).last },
+        );
+        if let Err(e) = ret {
+            e.to_blk_status()
+        } else {
+            bindings::BLK_STS_OK as _
+        }
+    }
+
+    /// # Safety
+    ///
+    /// This function may only be called by blk-mq C infrastructure. The caller
+    /// must ensure that `hctx` is valid.
+    unsafe extern "C" fn commit_rqs_callback(hctx: *mut bindings::blk_mq_hw_ctx) {
+        // SAFETY: `driver_data` was installed by us in `init_hctx_callback` as
+        // the result of a call to `into_foreign`.
+        let hw_data = unsafe { T::HwData::borrow((*hctx).driver_data) };
+
+        // SAFETY: `hctx` is valid as required by this function.
+        let queue_data = unsafe { (*(*hctx).queue).queuedata };
+
+        // SAFETY: `queue.queuedata` was created by `GenDisk::try_new()` with a
+        // call to `ForeignOwnable::into_pointer()` to create `queuedata`.
+        // `ForeignOwnable::from_foreign()` is only called when the tagset is
+        // dropped, which happens after we are dropped.
+        let queue_data = unsafe { T::QueueData::borrow(queue_data) };
+        T::commit_rqs(hw_data, queue_data)
+    }
+
+    /// # Safety
+    ///
+    /// This function may only be called by blk-mq C infrastructure. `rq` must
+    /// point to a valid request that has been marked as completed. The pointee
+    /// of `rq` must be valid for write for the duration of this function.
+    unsafe extern "C" fn complete_callback(rq: *mut bindings::request) {
+        // SAFETY: By function safety requirement `rq`is valid for write for the
+        // lifetime of the returned `Request`.
+        T::complete(unsafe { Request::from_ptr_mut(rq) });
+    }
+
+    /// # Safety
+    ///
+    /// This function may only be called by blk-mq C infrastructure. `hctx` must
+    /// be a pointer to a valid and aligned `struct blk_mq_hw_ctx` that was
+    /// previously initialized by a call to `init_hctx_callback`.
+    unsafe extern "C" fn poll_callback(
+        hctx: *mut bindings::blk_mq_hw_ctx,
+        _iob: *mut bindings::io_comp_batch,
+    ) -> core::ffi::c_int {
+        // SAFETY: By function safety requirement, `hctx` was initialized by
+        // `init_hctx_callback` and thus `driver_data` came from a call to
+        // `into_foreign`.
+        let hw_data = unsafe { T::HwData::borrow((*hctx).driver_data) };
+        T::poll(hw_data).into()
+    }
+
+    /// # Safety
+    ///
+    /// This function may only be called by blk-mq C infrastructure.
+    /// `tagset_data` must be initialized by the initializer returned by
+    /// `TagSet::try_new` as part of tag set initialization. `hctx` must be a
+    /// pointer to a valid `blk_mq_hw_ctx` where the `driver_data` field was not
+    /// yet initialized. This function may only be called onece before
+    /// `exit_hctx_callback` is called for the same context.
+    unsafe extern "C" fn init_hctx_callback(
+        hctx: *mut bindings::blk_mq_hw_ctx,
+        tagset_data: *mut core::ffi::c_void,
+        hctx_idx: core::ffi::c_uint,
+    ) -> core::ffi::c_int {
+        from_result(|| {
+            // SAFETY: By the safety requirements of this function,
+            // `tagset_data` came from a call to `into_foreign` when the
+            // `TagSet` was initialized.
+            let tagset_data = unsafe { T::TagSetData::borrow(tagset_data) };
+            let data = T::init_hctx(tagset_data, hctx_idx)?;
+
+            // SAFETY: by the safety requirments of this function, `hctx` is
+            // valid for write
+            unsafe { (*hctx).driver_data = data.into_foreign() as _ };
+            Ok(0)
+        })
+    }
+
+    /// # Safety
+    ///
+    /// This function may only be called by blk-mq C infrastructure. `hctx` must
+    /// be a valid pointer that was previously initialized by a call to
+    /// `init_hctx_callback`. This function may be called only once after
+    /// `init_hctx_callback` was called.
+    unsafe extern "C" fn exit_hctx_callback(
+        hctx: *mut bindings::blk_mq_hw_ctx,
+        _hctx_idx: core::ffi::c_uint,
+    ) {
+        // SAFETY: By the safety requirements of this function, `hctx` is valid for read.
+        let ptr = unsafe { (*hctx).driver_data };
+
+        // SAFETY: By the safety requirements of this function, `ptr` came from
+        // a call to `into_foreign` in `init_hctx_callback`
+        unsafe { T::HwData::from_foreign(ptr) };
+    }
+
+    /// # Safety
+    ///
+    /// This function may only be called by blk-mq C infrastructure. `set` must point to an initialized `TagSet<T>`.
+    unsafe extern "C" fn init_request_callback(
+        set: *mut bindings::blk_mq_tag_set,
+        rq: *mut bindings::request,
+        _hctx_idx: core::ffi::c_uint,
+        _numa_node: core::ffi::c_uint,
+    ) -> core::ffi::c_int {
+        from_result(|| {
+            // SAFETY: The tagset invariants guarantee that all requests are allocated with extra memory
+            // for the request data.
+            let pdu = unsafe { bindings::blk_mq_rq_to_pdu(rq) }.cast::<T::RequestData>();
+
+            // SAFETY: Because `set` is a `TagSet<T>`, `driver_data` comes from
+            // a call to `into_foregn` by the initializer returned by
+            // `TagSet::try_new`.
+            let tagset_data = unsafe { T::TagSetData::borrow((*set).driver_data) };
+
+            let initializer = T::new_request_data(tagset_data);
+
+            // SAFETY: `pdu` is a valid pointer as established above. We do not
+            // touch `pdu` if `__pinned_init` returns an error. We promise ot to
+            // move the pointee of `pdu`.
+            unsafe { initializer.__pinned_init(pdu)? };
+
+            Ok(0)
+        })
+    }
+
+    /// # Safety
+    ///
+    /// This function may only be called by blk-mq C infrastructure. `rq` must
+    /// point to a request that was initialized by a call to
+    /// `Self::init_request_callback`.
+    unsafe extern "C" fn exit_request_callback(
+        _set: *mut bindings::blk_mq_tag_set,
+        rq: *mut bindings::request,
+        _hctx_idx: core::ffi::c_uint,
+    ) {
+        // SAFETY: The tagset invariants guarantee that all requests are allocated with extra memory
+        // for the request data.
+        let pdu = unsafe { bindings::blk_mq_rq_to_pdu(rq) }.cast::<T::RequestData>();
+
+        // SAFETY: `pdu` is valid for read and write and is properly initialised.
+        unsafe { core::ptr::drop_in_place(pdu) };
+    }
+
+    /// # Safety
+    ///
+    /// This function may only be called by blk-mq C infrastructure. `tag_set`
+    /// must be a pointer to a valid and initialized `TagSet<T>`. The pointee
+    /// must be valid for use as a reference at least the duration of this call.
+    unsafe extern "C" fn map_queues_callback(tag_set: *mut bindings::blk_mq_tag_set) {
+        // SAFETY: The safety requirements of this function satiesfies the
+        // requirements of `TagSet::from_ptr`.
+        let tag_set = unsafe { TagSet::from_ptr(tag_set) };
+        T::map_queues(tag_set);
+    }
+
+    const VTABLE: bindings::blk_mq_ops = bindings::blk_mq_ops {
+        queue_rq: Some(Self::queue_rq_callback),
+        queue_rqs: None,
+        commit_rqs: Some(Self::commit_rqs_callback),
+        get_budget: None,
+        put_budget: None,
+        set_rq_budget_token: None,
+        get_rq_budget_token: None,
+        timeout: None,
+        poll: if T::HAS_POLL {
+            Some(Self::poll_callback)
+        } else {
+            None
+        },
+        complete: Some(Self::complete_callback),
+        init_hctx: Some(Self::init_hctx_callback),
+        exit_hctx: Some(Self::exit_hctx_callback),
+        init_request: Some(Self::init_request_callback),
+        exit_request: Some(Self::exit_request_callback),
+        cleanup_rq: None,
+        busy: None,
+        map_queues: if T::HAS_MAP_QUEUES {
+            Some(Self::map_queues_callback)
+        } else {
+            None
+        },
+        #[cfg(CONFIG_BLK_DEBUG_FS)]
+        show_rq: None,
+    };
+
+    pub(crate) const fn build() -> &'static bindings::blk_mq_ops {
+        &Self::VTABLE
+    }
+}
diff --git a/rust/kernel/block/mq/raw_writer.rs b/rust/kernel/block/mq/raw_writer.rs
new file mode 100644
index 000000000000..f7857740af29
--- /dev/null
+++ b/rust/kernel/block/mq/raw_writer.rs
@@ -0,0 +1,60 @@
+use core::{
+    fmt::{self, Write},
+    marker::PhantomData,
+};
+
+/// A mutable reference to a byte buffer where a string can be written into
+///
+/// # Invariants
+///
+/// * `ptr` is not aliased and valid for read and write for `len` bytes
+///
+pub(crate) struct RawWriter<'a> {
+    ptr: *mut u8,
+    len: usize,
+    _p: PhantomData<&'a ()>,
+}
+
+impl<'a> RawWriter<'a> {
+    /// Create a new `RawWriter` instance.
+    ///
+    /// # Safety
+    ///
+    /// * `ptr` must be valid for read and write for `len` consecutive `u8` elements
+    /// * `ptr` must not be aliased
+    unsafe fn new(ptr: *mut u8, len: usize) -> RawWriter<'a> {
+        Self {
+            ptr,
+            len,
+            _p: PhantomData,
+        }
+    }
+
+    pub(crate) fn from_array<const N: usize>(a: &'a mut [core::ffi::c_char; N]) -> RawWriter<'a> {
+        // SAFETY: the buffer of `a` is valid for read and write for at least `N` bytes
+        unsafe { Self::new(a.as_mut_ptr().cast::<u8>(), N) }
+    }
+}
+
+impl Write for RawWriter<'_> {
+    fn write_str(&mut self, s: &str) -> fmt::Result {
+        let bytes = s.as_bytes();
+        let len = bytes.len();
+        if len > self.len {
+            return Err(fmt::Error);
+        }
+
+        // SAFETY:
+        // * `bytes` is valid for reads of `bytes.len()` size because we hold a shared reference to `s`
+        // * By type invariant `self.ptr` is valid for writes for at lest `self.len` bytes
+        // * The regions are not overlapping as `ptr` is not aliased
+        unsafe { core::ptr::copy_nonoverlapping(&bytes[0], self.ptr, len) };
+
+        // SAFETY: By type invariant of `Self`, `ptr` is in bounds of an
+        // allocation. Also by type invariant, the pointer resulting from this
+        // addition is also in bounds.
+        self.ptr = unsafe { self.ptr.add(len) };
+        self.len -= len;
+        Ok(())
+    }
+}
diff --git a/rust/kernel/block/mq/request.rs b/rust/kernel/block/mq/request.rs
new file mode 100644
index 000000000000..b4dacac5e091
--- /dev/null
+++ b/rust/kernel/block/mq/request.rs
@@ -0,0 +1,182 @@
+// SPDX-License-Identifier: GPL-2.0
+
+//! This module provides a wrapper for the C `struct request` type.
+//!
+//! C header: [`include/linux/blk-mq.h`](srctree/include/linux/blk-mq.h)
+
+use crate::{
+    bindings,
+    block::mq::Operations,
+    error::{Error, Result},
+    types::{ARef, AlwaysRefCounted, Opaque},
+};
+use core::{ffi::c_void, marker::PhantomData, ops::Deref};
+
+/// A wrapper around a blk-mq `struct request`. This represents an IO request.
+///
+/// # Invariants
+///
+/// * `self.0` is a valid `struct request` created by the C portion of the kernel
+/// * `self` is reference counted. a call to `req_ref_inc_not_zero` keeps the
+///    instance alive at least until a matching call to `req_ref_put_and_test`
+///
+#[repr(transparent)]
+pub struct Request<T: Operations>(Opaque<bindings::request>, PhantomData<T>);
+
+impl<T: Operations> Request<T> {
+    /// Create a `&mut Request` from a `bindings::request` pointer
+    ///
+    /// # Safety
+    ///
+    /// * `ptr` must be aligned and point to a valid `bindings::request` instance
+    /// * Caller must ensure that the pointee of `ptr` is live and owned
+    ///   exclusively by caller for at least `'a`
+    ///
+    pub(crate) unsafe fn from_ptr_mut<'a>(ptr: *mut bindings::request) -> &'a mut Self {
+        // SAFETY:
+        // * The cast is valid as `Self` is transparent.
+        // * By safety requirements of this function, the reference will be
+        //   valid for 'a.
+        unsafe { &mut *(ptr.cast::<Self>()) }
+    }
+
+    /// Get the command identifier for the request
+    pub fn command(&self) -> u32 {
+        // SAFETY: By C API contract and type invariant, `cmd_flags` is valid for read
+        unsafe { (*self.0.get()).cmd_flags & ((1 << bindings::REQ_OP_BITS) - 1) }
+    }
+
+    /// Call this to indicate to the kernel that the request has been issued by the driver
+    pub fn start(&self) {
+        // SAFETY: By type invariant, `self.0` is a valid `struct request`. By
+        // existence of `&mut self` we have exclusive access.
+        unsafe { bindings::blk_mq_start_request(self.0.get()) };
+    }
+
+    /// Call this to indicate to the kernel that the request has been completed without errors
+    pub fn end_ok(&self) {
+        // SAFETY: By type invariant, `self.0` is a valid `struct request`. By
+        // existence of `&mut self` we have exclusive access.
+        unsafe { bindings::blk_mq_end_request(self.0.get(), bindings::BLK_STS_OK as _) };
+    }
+
+    /// Call this to indicate to the kernel that the request completed with an error
+    pub fn end_err(&self, err: Error) {
+        // SAFETY: By type invariant, `self.0` is a valid `struct request`. By
+        // existence of `&mut self` we have exclusive access.
+        unsafe { bindings::blk_mq_end_request(self.0.get(), err.to_blk_status()) };
+    }
+
+    /// Call this to indicate that the request completed with the status indicated by `status`
+    pub fn end(&self, status: Result) {
+        if let Err(e) = status {
+            self.end_err(e);
+        } else {
+            self.end_ok();
+        }
+    }
+
+    /// Call this to schedule defered completion of the request
+    pub fn complete(&self) {
+        // SAFETY: By type invariant, `self.0` is a valid `struct request`
+        if !unsafe { bindings::blk_mq_complete_request_remote(self.0.get()) } {
+            T::complete(self);
+        }
+    }
+
+    /// Get the target sector for the request
+    #[inline(always)]
+    pub fn sector(&self) -> usize {
+        // SAFETY: By type invariant of `Self`, `self.0` is valid and live.
+        unsafe { (*self.0.get()).__sector as usize }
+    }
+
+    /// Returns an owned reference to the per-request data associated with this
+    /// request
+    pub fn owned_data_ref(request: ARef<Self>) -> RequestDataRef<T> {
+        RequestDataRef::new(request)
+    }
+
+    /// Returns a reference to the oer-request data associated with this request
+    pub fn data_ref(&self) -> &T::RequestData {
+        let request_ptr = self.0.get().cast::<bindings::request>();
+
+        // SAFETY: `request_ptr` is a valid `struct request` because `ARef` is
+        // `repr(transparent)`
+        let p: *mut c_void = unsafe { bindings::blk_mq_rq_to_pdu(request_ptr) };
+
+        let p = p.cast::<T::RequestData>();
+
+        // SAFETY: By C API contract, `p` is initialized by a call to
+        // `OperationsVTable::init_request_callback()`. By existence of `&self`
+        // it must be valid for use as a shared reference.
+        unsafe { &*p }
+    }
+}
+
+// SAFETY: It is impossible to obtain an owned or mutable `Request`, so we can
+// mark it `Send`.
+unsafe impl<T: Operations> Send for Request<T> {}
+
+// SAFETY: `Request` references can be shared across threads.
+unsafe impl<T: Operations> Sync for Request<T> {}
+
+/// An owned reference to a `Request<T>`
+#[repr(transparent)]
+pub struct RequestDataRef<T: Operations> {
+    request: ARef<Request<T>>,
+}
+
+impl<T> RequestDataRef<T>
+where
+    T: Operations,
+{
+    /// Create a new instance.
+    fn new(request: ARef<Request<T>>) -> Self {
+        Self { request }
+    }
+
+    /// Get a reference to the underlying request
+    pub fn request(&self) -> &Request<T> {
+        &self.request
+    }
+}
+
+impl<T> Deref for RequestDataRef<T>
+where
+    T: Operations,
+{
+    type Target = T::RequestData;
+
+    fn deref(&self) -> &Self::Target {
+        self.request.data_ref()
+    }
+}
+
+// SAFETY: All instances of `Request<T>` are reference counted. This
+// implementation of `AlwaysRefCounted` ensure that increments to the ref count
+// keeps the object alive in memory at least until a matching reference count
+// decrement is executed.
+unsafe impl<T: Operations> AlwaysRefCounted for Request<T> {
+    fn inc_ref(&self) {
+        // SAFETY: By type invariant `self.0` is a valid `struct reqeust`
+        #[cfg_attr(not(CONFIG_DEBUG_MISC), allow(unused_variables))]
+        let updated = unsafe { bindings::req_ref_inc_not_zero(self.0.get()) };
+        #[cfg(CONFIG_DEBUG_MISC)]
+        if !updated {
+            crate::pr_err!("Request refcount zero on clone");
+        }
+    }
+
+    unsafe fn dec_ref(obj: core::ptr::NonNull<Self>) {
+        // SAFETY: By type invariant `self.0` is a valid `struct reqeust`
+        let zero = unsafe { bindings::req_ref_put_and_test(obj.as_ref().0.get()) };
+        if zero {
+            // SAFETY: By type invariant of `self` we have the last reference to
+            // `obj` and it is safe to free it.
+            unsafe {
+                bindings::blk_mq_free_request_internal(obj.as_ptr().cast::<bindings::request>())
+            };
+        }
+    }
+}
diff --git a/rust/kernel/block/mq/tag_set.rs b/rust/kernel/block/mq/tag_set.rs
new file mode 100644
index 000000000000..7f463b7e288b
--- /dev/null
+++ b/rust/kernel/block/mq/tag_set.rs
@@ -0,0 +1,117 @@
+// SPDX-License-Identifier: GPL-2.0
+
+//! This module provides the `TagSet` struct to wrap the C `struct blk_mq_tag_set`.
+//!
+//! C header: [`include/linux/blk-mq.h`](srctree/include/linux/blk-mq.h)
+
+use core::pin::Pin;
+
+use crate::{
+    bindings,
+    block::mq::{operations::OperationsVTable, Operations},
+    error::{self, Error, Result},
+    prelude::PinInit,
+    try_pin_init,
+    types::{ForeignOwnable, Opaque},
+};
+use core::{convert::TryInto, marker::PhantomData};
+use macros::{pin_data, pinned_drop};
+
+/// A wrapper for the C `struct blk_mq_tag_set`.
+///
+/// `struct blk_mq_tag_set` contains a `struct list_head` and so must be pinned.
+#[pin_data(PinnedDrop)]
+#[repr(transparent)]
+pub struct TagSet<T: Operations> {
+    #[pin]
+    inner: Opaque<bindings::blk_mq_tag_set>,
+    _p: PhantomData<T>,
+}
+
+impl<T: Operations> TagSet<T> {
+    /// Try to create a new tag set
+    pub fn try_new(
+        nr_hw_queues: u32,
+        tagset_data: T::TagSetData,
+        num_tags: u32,
+        num_maps: u32,
+    ) -> impl PinInit<Self, error::Error> {
+        try_pin_init!( TagSet {
+            inner <- Opaque::try_ffi_init(move |place: *mut bindings::blk_mq_tag_set| -> Result<()> {
+
+                // SAFETY: try_ffi_init promises that `place` is writable, and
+                // zeroes is a valid bit pattern for this structure.
+                unsafe { core::ptr::write_bytes(place, 0, 1) };
+
+                /// For a raw pointer to a struct, write a struct field without
+                /// creating a reference to the field
+                macro_rules! write_ptr_field {
+                    ($target:ident, $field:ident, $value:expr) => {
+                        ::core::ptr::write(::core::ptr::addr_of_mut!((*$target).$field), $value)
+                    };
+                }
+
+                // SAFETY: try_ffi_init promises that `place` is writable
+                unsafe {
+                    write_ptr_field!(place, ops, OperationsVTable::<T>::build());
+                    write_ptr_field!(place, nr_hw_queues , nr_hw_queues);
+                    write_ptr_field!(place, timeout , 0); // 0 means default which is 30 * HZ in C
+                    write_ptr_field!(place, numa_node , bindings::NUMA_NO_NODE);
+                    write_ptr_field!(place, queue_depth , num_tags);
+                    write_ptr_field!(place, cmd_size , core::mem::size_of::<T::RequestData>().try_into()?);
+                    write_ptr_field!(place, flags , bindings::BLK_MQ_F_SHOULD_MERGE);
+                    write_ptr_field!(place, driver_data , tagset_data.into_foreign() as _);
+                    write_ptr_field!(place, nr_maps , num_maps);
+                }
+
+                // SAFETY: Relevant fields of `place` are initialised above
+                let ret = unsafe { bindings::blk_mq_alloc_tag_set(place) };
+                if ret < 0 {
+                    // SAFETY: We created `driver_data` above with `into_foreign`
+                    unsafe { T::TagSetData::from_foreign((*place).driver_data) };
+                    return Err(Error::from_errno(ret));
+                }
+
+                Ok(())
+            }),
+            _p: PhantomData,
+        })
+    }
+
+    /// Return the pointer to the wrapped `struct blk_mq_tag_set`
+    pub(crate) fn raw_tag_set(&self) -> *mut bindings::blk_mq_tag_set {
+        self.inner.get()
+    }
+
+    /// Create a `TagSet<T>` from a raw pointer.
+    ///
+    /// # Safety
+    ///
+    /// `ptr` must be a pointer to a valid and initialized `TagSet<T>`. There
+    /// may be no other mutable references to the tag set. The pointee must be
+    /// live and valid at least for the duration of the returned lifetime `'a`.
+    pub(crate) unsafe fn from_ptr<'a>(ptr: *mut bindings::blk_mq_tag_set) -> &'a Self {
+        // SAFETY: By the safety requirements of this function, `ptr` is valid
+        // for use as a reference for the duration of `'a`.
+        unsafe { &*(ptr.cast::<Self>()) }
+    }
+}
+
+#[pinned_drop]
+impl<T: Operations> PinnedDrop for TagSet<T> {
+    fn drop(self: Pin<&mut Self>) {
+        // SAFETY: We are not moving self below
+        let this = unsafe { Pin::into_inner_unchecked(self) };
+
+        // SAFETY: `this.inner.get()` points to a valid `blk_mq_tag_set` and
+        // thus is safe to dereference.
+        let tagset_data = unsafe { (*this.inner.get()).driver_data };
+
+        // SAFETY: `inner` is valid and has been properly initialised during construction.
+        unsafe { bindings::blk_mq_free_tag_set(this.inner.get()) };
+
+        // SAFETY: `tagset_data` was created by a call to
+        // `ForeignOwnable::into_foreign` in `TagSet::try_new()`
+        unsafe { T::TagSetData::from_foreign(tagset_data) };
+    }
+}
diff --git a/rust/kernel/error.rs b/rust/kernel/error.rs
index 4f0c1edd63b7..c947fd631416 100644
--- a/rust/kernel/error.rs
+++ b/rust/kernel/error.rs
@@ -130,6 +130,11 @@ pub fn to_errno(self) -> core::ffi::c_int {
         self.0
     }
 
+    pub(crate) fn to_blk_status(self) -> bindings::blk_status_t {
+        // SAFETY: `self.0` is a valid error due to its invariant.
+        unsafe { bindings::errno_to_blk_status(self.0) }
+    }
+
     /// Returns the error encoded as a pointer.
     #[allow(dead_code)]
     pub(crate) fn to_ptr<T>(self) -> *mut T {
diff --git a/rust/kernel/lib.rs b/rust/kernel/lib.rs
index 638a68af341a..9f02a8b352e0 100644
--- a/rust/kernel/lib.rs
+++ b/rust/kernel/lib.rs
@@ -34,6 +34,7 @@
 #[cfg(not(test))]
 #[cfg(not(testlib))]
 mod allocator;
+pub mod block;
 mod build_assert;
 mod cache_aligned;
 pub mod error;
-- 
2.44.0


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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-13 11:05 ` [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module Andreas Hindborg
@ 2024-03-13 23:55   ` Boqun Feng
  2024-03-14  8:58     ` Andreas Hindborg
  0 siblings, 1 reply; 26+ messages in thread
From: Boqun Feng @ 2024-03-13 23:55 UTC (permalink / raw)
  To: Andreas Hindborg
  Cc: Jens Axboe, Christoph Hellwig, Keith Busch, Damien Le Moal,
	Bart Van Assche, Hannes Reinecke, linux-block@vger.kernel.org,
	Andreas Hindborg, Wedson Almeida Filho, Niklas Cassel, Greg KH,
	Matthew Wilcox, Miguel Ojeda, Alex Gaynor, Gary Guo,
	Björn Roy Baron, Benno Lossin, Alice Ryhl,
	Chaitanya Kulkarni, Luis Chamberlain, Yexuan Yang,
	Sergio González Collado, Joel Granados,
	Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

On Wed, Mar 13, 2024 at 12:05:08PM +0100, Andreas Hindborg wrote:
> From: Andreas Hindborg <a.hindborg@samsung.com>
> 
> Add initial abstractions for working with blk-mq.
> 
> This patch is a maintained, refactored subset of code originally published by
> Wedson Almeida Filho <wedsonaf@gmail.com> [1].
> 
> [1] https://github.com/wedsonaf/linux/tree/f2cfd2fe0e2ca4e90994f96afe268bbd4382a891/rust/kernel/blk/mq.rs
> 
> Cc: Wedson Almeida Filho <wedsonaf@gmail.com>
> Signed-off-by: Andreas Hindborg <a.hindborg@samsung.com>
> ---
>  block/blk-mq.c                     |   3 +-
>  include/linux/blk-mq.h             |   1 +
>  rust/bindings/bindings_helper.h    |   2 +
>  rust/helpers.c                     |  45 ++++
>  rust/kernel/block.rs               |   5 +
>  rust/kernel/block/mq.rs            | 131 +++++++++++
>  rust/kernel/block/mq/gen_disk.rs   | 174 +++++++++++++++
>  rust/kernel/block/mq/operations.rs | 346 +++++++++++++++++++++++++++++
>  rust/kernel/block/mq/raw_writer.rs |  60 +++++
>  rust/kernel/block/mq/request.rs    | 182 +++++++++++++++
>  rust/kernel/block/mq/tag_set.rs    | 117 ++++++++++
>  rust/kernel/error.rs               |   5 +
>  rust/kernel/lib.rs                 |   1 +
>  13 files changed, 1071 insertions(+), 1 deletion(-)
>  create mode 100644 rust/kernel/block.rs
>  create mode 100644 rust/kernel/block/mq.rs
>  create mode 100644 rust/kernel/block/mq/gen_disk.rs
>  create mode 100644 rust/kernel/block/mq/operations.rs
>  create mode 100644 rust/kernel/block/mq/raw_writer.rs
>  create mode 100644 rust/kernel/block/mq/request.rs
>  create mode 100644 rust/kernel/block/mq/tag_set.rs
> 
> diff --git a/block/blk-mq.c b/block/blk-mq.c
> index 2dc01551e27c..a531f664bee7 100644
> --- a/block/blk-mq.c
> +++ b/block/blk-mq.c
> @@ -702,7 +702,7 @@ static void blk_mq_finish_request(struct request *rq)
>  	}
>  }
>  
> -static void __blk_mq_free_request(struct request *rq)
> +void __blk_mq_free_request(struct request *rq)
>  {
>  	struct request_queue *q = rq->q;
>  	struct blk_mq_ctx *ctx = rq->mq_ctx;
> @@ -722,6 +722,7 @@ static void __blk_mq_free_request(struct request *rq)
>  	blk_mq_sched_restart(hctx);
>  	blk_queue_exit(q);
>  }
> +EXPORT_SYMBOL_GPL(__blk_mq_free_request);
>  

Note that for an EXPORT_SYMBOL_GPL() symbol, you can just add the
corresponding header file in rust/bindings/bindings_helper.h:

+#include <linux/blk-mq.h>

and you will be able to call it from Rust via:

	bindings::__blk_mq_free_request()

in other words, rust_helper_blk_mq_free_request_internal() is probably
not necessary.

Regards,
Boqun

>  void blk_mq_free_request(struct request *rq)
>  {
> diff --git a/include/linux/blk-mq.h b/include/linux/blk-mq.h
> index 7a8150a5f051..842bb88e6e78 100644
> --- a/include/linux/blk-mq.h
> +++ b/include/linux/blk-mq.h
> @@ -703,6 +703,7 @@ int blk_mq_alloc_sq_tag_set(struct blk_mq_tag_set *set,
>  		unsigned int set_flags);
>  void blk_mq_free_tag_set(struct blk_mq_tag_set *set);
>  
> +void __blk_mq_free_request(struct request *rq);
>  void blk_mq_free_request(struct request *rq);
>  int blk_rq_poll(struct request *rq, struct io_comp_batch *iob,
>  		unsigned int poll_flags);
> diff --git a/rust/bindings/bindings_helper.h b/rust/bindings/bindings_helper.h
> index f8e54d398c19..df18acb229d9 100644
> --- a/rust/bindings/bindings_helper.h
> +++ b/rust/bindings/bindings_helper.h
> @@ -7,6 +7,8 @@
>   */
>  
>  #include <kunit/test.h>
> +#include <linux/blk_types.h>
> +#include <linux/blk-mq.h>
>  #include <linux/errname.h>
>  #include <linux/ethtool.h>
>  #include <linux/mdio.h>
> diff --git a/rust/helpers.c b/rust/helpers.c
> index 66411845536e..017fa90366e6 100644
> --- a/rust/helpers.c
> +++ b/rust/helpers.c
> @@ -21,6 +21,9 @@
>   */
>  
[...]
> +
> +void rust_helper_blk_mq_free_request_internal(struct request *req)
> +{
> +	__blk_mq_free_request(req);
> +}
> +EXPORT_SYMBOL_GPL(rust_helper_blk_mq_free_request_internal);
> +
[...]

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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-13 23:55   ` Boqun Feng
@ 2024-03-14  8:58     ` Andreas Hindborg
  2024-03-14 18:55       ` Miguel Ojeda
  0 siblings, 1 reply; 26+ messages in thread
From: Andreas Hindborg @ 2024-03-14  8:58 UTC (permalink / raw)
  To: Boqun Feng
  Cc: Jens Axboe, Christoph Hellwig, Keith Busch, Damien Le Moal,
	Bart Van Assche, Hannes Reinecke, linux-block@vger.kernel.org,
	Andreas Hindborg, Wedson Almeida Filho, Niklas Cassel, Greg KH,
	Matthew Wilcox, Miguel Ojeda, Alex Gaynor, Gary Guo,
	Björn Roy Baron, Benno Lossin, Alice Ryhl,
	Chaitanya Kulkarni, Luis Chamberlain, Yexuan Yang,
	Sergio González Collado, Joel Granados,
	Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

Boqun Feng <boqun.feng@gmail.com> writes:

> On Wed, Mar 13, 2024 at 12:05:08PM +0100, Andreas Hindborg wrote:
>> From: Andreas Hindborg <a.hindborg@samsung.com>
>> 
>> Add initial abstractions for working with blk-mq.
>> 
>> This patch is a maintained, refactored subset of code originally published by
>> Wedson Almeida Filho <wedsonaf@gmail.com> [1].
>> 
>> [1] https://github.com/wedsonaf/linux/tree/f2cfd2fe0e2ca4e90994f96afe268bbd4382a891/rust/kernel/blk/mq.rs
>> 
>> Cc: Wedson Almeida Filho <wedsonaf@gmail.com>
>> Signed-off-by: Andreas Hindborg <a.hindborg@samsung.com>
>> ---
>>  block/blk-mq.c                     |   3 +-
>>  include/linux/blk-mq.h             |   1 +
>>  rust/bindings/bindings_helper.h    |   2 +
>>  rust/helpers.c                     |  45 ++++
>>  rust/kernel/block.rs               |   5 +
>>  rust/kernel/block/mq.rs            | 131 +++++++++++
>>  rust/kernel/block/mq/gen_disk.rs   | 174 +++++++++++++++
>>  rust/kernel/block/mq/operations.rs | 346 +++++++++++++++++++++++++++++
>>  rust/kernel/block/mq/raw_writer.rs |  60 +++++
>>  rust/kernel/block/mq/request.rs    | 182 +++++++++++++++
>>  rust/kernel/block/mq/tag_set.rs    | 117 ++++++++++
>>  rust/kernel/error.rs               |   5 +
>>  rust/kernel/lib.rs                 |   1 +
>>  13 files changed, 1071 insertions(+), 1 deletion(-)
>>  create mode 100644 rust/kernel/block.rs
>>  create mode 100644 rust/kernel/block/mq.rs
>>  create mode 100644 rust/kernel/block/mq/gen_disk.rs
>>  create mode 100644 rust/kernel/block/mq/operations.rs
>>  create mode 100644 rust/kernel/block/mq/raw_writer.rs
>>  create mode 100644 rust/kernel/block/mq/request.rs
>>  create mode 100644 rust/kernel/block/mq/tag_set.rs
>> 
>> diff --git a/block/blk-mq.c b/block/blk-mq.c
>> index 2dc01551e27c..a531f664bee7 100644
>> --- a/block/blk-mq.c
>> +++ b/block/blk-mq.c
>> @@ -702,7 +702,7 @@ static void blk_mq_finish_request(struct request *rq)
>>  	}
>>  }
>>  
>> -static void __blk_mq_free_request(struct request *rq)
>> +void __blk_mq_free_request(struct request *rq)
>>  {
>>  	struct request_queue *q = rq->q;
>>  	struct blk_mq_ctx *ctx = rq->mq_ctx;
>> @@ -722,6 +722,7 @@ static void __blk_mq_free_request(struct request *rq)
>>  	blk_mq_sched_restart(hctx);
>>  	blk_queue_exit(q);
>>  }
>> +EXPORT_SYMBOL_GPL(__blk_mq_free_request);
>>  
>
> Note that for an EXPORT_SYMBOL_GPL() symbol, you can just add the
> corresponding header file in rust/bindings/bindings_helper.h:
>
> +#include <linux/blk-mq.h>
>
> and you will be able to call it from Rust via:
>
> 	bindings::__blk_mq_free_request()
>
> in other words, rust_helper_blk_mq_free_request_internal() is probably
> not necessary.

Yes, good point. Another option suggested by Miguel is that
`__blk_mq_free_request` need not be exported at all. We can make it
non-static and then call it from
`rust_helper_blk_mq_free_request_internal()`. Then only the latter will
be in the kernel image symbol table, which might be better in terms of
not exposing `__blk_mq_free_request()` directly.

BR Andreas

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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-14  8:58     ` Andreas Hindborg
@ 2024-03-14 18:55       ` Miguel Ojeda
  2024-03-14 19:22         ` Andreas Hindborg
  0 siblings, 1 reply; 26+ messages in thread
From: Miguel Ojeda @ 2024-03-14 18:55 UTC (permalink / raw)
  To: Andreas Hindborg
  Cc: Boqun Feng, Jens Axboe, Christoph Hellwig, Keith Busch,
	Damien Le Moal, Bart Van Assche, Hannes Reinecke,
	linux-block@vger.kernel.org, Andreas Hindborg,
	Wedson Almeida Filho, Niklas Cassel, Greg KH, Matthew Wilcox,
	Miguel Ojeda, Alex Gaynor, Gary Guo, Björn Roy Baron,
	Benno Lossin, Alice Ryhl, Chaitanya Kulkarni, Luis Chamberlain,
	Yexuan Yang, Sergio González Collado, Joel Granados,
	Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

On Thu, Mar 14, 2024 at 9:58 AM Andreas Hindborg <nmi@metaspace.dk> wrote:
>
> Yes, good point. Another option suggested by Miguel is that
> `__blk_mq_free_request` need not be exported at all. We can make it
> non-static and then call it from
> `rust_helper_blk_mq_free_request_internal()`. Then only the latter will
> be in the kernel image symbol table, which might be better in terms of
> not exposing `__blk_mq_free_request()` directly.

The helper is not needed, i.e. what I meant was to make it non-static
and add it to `include/linux/blk-mq.h`.

Cheers,
Miguel

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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-14 18:55       ` Miguel Ojeda
@ 2024-03-14 19:22         ` Andreas Hindborg
  2024-03-14 19:41           ` Andreas Hindborg
  2024-03-14 19:41           ` Miguel Ojeda
  0 siblings, 2 replies; 26+ messages in thread
From: Andreas Hindborg @ 2024-03-14 19:22 UTC (permalink / raw)
  To: Miguel Ojeda
  Cc: Boqun Feng, Jens Axboe, Christoph Hellwig, Keith Busch,
	Damien Le Moal, Bart Van Assche, Hannes Reinecke,
	linux-block@vger.kernel.org, Andreas Hindborg,
	Wedson Almeida Filho, Niklas Cassel, Greg KH, Matthew Wilcox,
	Miguel Ojeda, Alex Gaynor, Gary Guo, Björn Roy Baron,
	Benno Lossin, Alice Ryhl, Chaitanya Kulkarni, Luis Chamberlain,
	Yexuan Yang, Sergio González Collado, Joel Granados,
	Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

Miguel Ojeda <miguel.ojeda.sandonis@gmail.com> writes:

> On Thu, Mar 14, 2024 at 9:58 AM Andreas Hindborg <nmi@metaspace.dk> wrote:
>>
>> Yes, good point. Another option suggested by Miguel is that
>> `__blk_mq_free_request` need not be exported at all. We can make it
>> non-static and then call it from
>> `rust_helper_blk_mq_free_request_internal()`. Then only the latter will
>> be in the kernel image symbol table, which might be better in terms of
>> not exposing `__blk_mq_free_request()` directly.
>
> The helper is not needed, i.e. what I meant was to make it non-static
> and add it to `include/linux/blk-mq.h`.

The way the current code compiles, <kernel::block::mq::Request as
kernel::types::AlwaysRefCounted>::dec_ref` is inlined into the `rnull`
module. A relocation for `rust_helper_blk_mq_free_request_internal`
appears in `rnull_mod.ko`. I didn't test it yet, but if
`__blk_mq_free_request` (or the helper) is not exported, I don't think
this would be possible?

BR Andreas

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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-14 19:22         ` Andreas Hindborg
@ 2024-03-14 19:41           ` Andreas Hindborg
  2024-03-14 19:41           ` Miguel Ojeda
  1 sibling, 0 replies; 26+ messages in thread
From: Andreas Hindborg @ 2024-03-14 19:41 UTC (permalink / raw)
  To: Miguel Ojeda
  Cc: Boqun Feng, Jens Axboe, Christoph Hellwig, Keith Busch,
	Damien Le Moal, Bart Van Assche, Hannes Reinecke,
	linux-block@vger.kernel.org, Andreas Hindborg,
	Wedson Almeida Filho, Niklas Cassel, Greg KH, Matthew Wilcox,
	Miguel Ojeda, Alex Gaynor, Gary Guo, Björn Roy Baron,
	Benno Lossin, Alice Ryhl, Chaitanya Kulkarni, Luis Chamberlain,
	Yexuan Yang, Sergio González Collado, Joel Granados,
	Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

Andreas Hindborg <nmi@metaspace.dk> writes:

> Miguel Ojeda <miguel.ojeda.sandonis@gmail.com> writes:
>
>> On Thu, Mar 14, 2024 at 9:58 AM Andreas Hindborg <nmi@metaspace.dk> wrote:
>>>
>>> Yes, good point. Another option suggested by Miguel is that
>>> `__blk_mq_free_request` need not be exported at all. We can make it
>>> non-static and then call it from
>>> `rust_helper_blk_mq_free_request_internal()`. Then only the latter will
>>> be in the kernel image symbol table, which might be better in terms of
>>> not exposing `__blk_mq_free_request()` directly.
>>
>> The helper is not needed, i.e. what I meant was to make it non-static
>> and add it to `include/linux/blk-mq.h`.
>
> The way the current code compiles, <kernel::block::mq::Request as
> kernel::types::AlwaysRefCounted>::dec_ref` is inlined into the `rnull`
> module. A relocation for `rust_helper_blk_mq_free_request_internal`
> appears in `rnull_mod.ko`. I didn't test it yet, but if
> `__blk_mq_free_request` (or the helper) is not exported, I don't think
> this would be possible?
>

I just tested what you suggested Miguel, and I get a link error for
`__blk_mq_free_request` being undefined when the module is linked. This
is even though the code that calls this function lives in the `kernel`
crate, because it is inlined.

BR Andreas

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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-14 19:22         ` Andreas Hindborg
  2024-03-14 19:41           ` Andreas Hindborg
@ 2024-03-14 19:41           ` Miguel Ojeda
  2024-03-14 20:56             ` Miguel Ojeda
  2024-03-15  7:52             ` Andreas Hindborg
  1 sibling, 2 replies; 26+ messages in thread
From: Miguel Ojeda @ 2024-03-14 19:41 UTC (permalink / raw)
  To: Andreas Hindborg
  Cc: Boqun Feng, Jens Axboe, Christoph Hellwig, Keith Busch,
	Damien Le Moal, Bart Van Assche, Hannes Reinecke,
	linux-block@vger.kernel.org, Andreas Hindborg,
	Wedson Almeida Filho, Niklas Cassel, Greg KH, Matthew Wilcox,
	Miguel Ojeda, Alex Gaynor, Gary Guo, Björn Roy Baron,
	Benno Lossin, Alice Ryhl, Chaitanya Kulkarni, Luis Chamberlain,
	Yexuan Yang, Sergio González Collado, Joel Granados,
	Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

On Thu, Mar 14, 2024 at 8:23 PM Andreas Hindborg <nmi@metaspace.dk> wrote:
>
> The way the current code compiles, <kernel::block::mq::Request as
> kernel::types::AlwaysRefCounted>::dec_ref` is inlined into the `rnull`
> module. A relocation for `rust_helper_blk_mq_free_request_internal`
> appears in `rnull_mod.ko`. I didn't test it yet, but if
> `__blk_mq_free_request` (or the helper) is not exported, I don't think
> this would be possible?

Yeah, something needs to be exported since there is a generic
involved, but even if you want to go the route of exporting only a
different symbol, you would still want to put it in the C header so
that you don't get the C missing declaration warning and so that we
don't have to write the declaration manually in the helper.

In any case, if we really want to avoid exporting the original symbol
(perhaps so that "only Rust" can use it -- or someone trying hard to
bypass things on purpose), then we could still avoid the helper and
instead write a non-generic `kernel`-private Rust function instead.

Cheers,
Miguel

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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-14 19:41           ` Miguel Ojeda
@ 2024-03-14 20:56             ` Miguel Ojeda
  2024-03-15  7:52             ` Andreas Hindborg
  1 sibling, 0 replies; 26+ messages in thread
From: Miguel Ojeda @ 2024-03-14 20:56 UTC (permalink / raw)
  To: Andreas Hindborg
  Cc: Boqun Feng, Jens Axboe, Christoph Hellwig, Keith Busch,
	Damien Le Moal, Bart Van Assche, Hannes Reinecke,
	linux-block@vger.kernel.org, Andreas Hindborg,
	Wedson Almeida Filho, Niklas Cassel, Greg KH, Matthew Wilcox,
	Miguel Ojeda, Alex Gaynor, Gary Guo, Björn Roy Baron,
	Benno Lossin, Alice Ryhl, Chaitanya Kulkarni, Luis Chamberlain,
	Yexuan Yang, Sergio González Collado, Joel Granados,
	Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

On Thu, Mar 14, 2024 at 8:41 PM Miguel Ojeda
<miguel.ojeda.sandonis@gmail.com> wrote:
>
> In any case, if we really want to avoid exporting the original symbol
> (perhaps so that "only Rust" can use it -- or someone trying hard to
> bypass things on purpose), then we could still avoid the helper and
> instead write a non-generic `kernel`-private Rust function instead.

The advantages would be that we get the export done automatically and
that we could write in Rust, e.g. with restricted visibility.

But we could need `#[inline(never)]` (or ideally `#[used]` if it could
go on functions) or `#[no_mangle]` (but we would lose the mangling).

Anyway, the simplest is to export the original symbol, but we could
consider to provide support one way or another for this kind of
"helpers" (i.e. leaving the current helpers as those for macros and
inline functions), so that it is clear what symbols are only exported
for Rust use (and not other C code).

Cheers,
Miguel

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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-14 19:41           ` Miguel Ojeda
  2024-03-14 20:56             ` Miguel Ojeda
@ 2024-03-15  7:52             ` Andreas Hindborg
  2024-03-15 12:17               ` Ming Lei
  1 sibling, 1 reply; 26+ messages in thread
From: Andreas Hindborg @ 2024-03-15  7:52 UTC (permalink / raw)
  To: Miguel Ojeda, Jens Axboe, Ming Lei, Keith Busch
  Cc: Boqun Feng, Christoph Hellwig, Damien Le Moal, Bart Van Assche,
	Hannes Reinecke, linux-block@vger.kernel.org, Andreas Hindborg,
	Wedson Almeida Filho, Niklas Cassel, Greg KH, Matthew Wilcox,
	Miguel Ojeda, Alex Gaynor, Gary Guo, Björn Roy Baron,
	Benno Lossin, Alice Ryhl, Chaitanya Kulkarni, Luis Chamberlain,
	Yexuan Yang, Sergio González Collado, Joel Granados,
	Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

Miguel Ojeda <miguel.ojeda.sandonis@gmail.com> writes:

> On Thu, Mar 14, 2024 at 8:23 PM Andreas Hindborg <nmi@metaspace.dk> wrote:
>>
>> The way the current code compiles, <kernel::block::mq::Request as
>> kernel::types::AlwaysRefCounted>::dec_ref` is inlined into the `rnull`
>> module. A relocation for `rust_helper_blk_mq_free_request_internal`
>> appears in `rnull_mod.ko`. I didn't test it yet, but if
>> `__blk_mq_free_request` (or the helper) is not exported, I don't think
>> this would be possible?
>
> Yeah, something needs to be exported since there is a generic
> involved, but even if you want to go the route of exporting only a
> different symbol, you would still want to put it in the C header so
> that you don't get the C missing declaration warning and so that we
> don't have to write the declaration manually in the helper.

That is what I did:

@@ -703,6 +703,7 @@ int blk_mq_alloc_sq_tag_set(struct blk_mq_tag_set *set,
 		unsigned int set_flags);
 void blk_mq_free_tag_set(struct blk_mq_tag_set *set);
 
+void __blk_mq_free_request(struct request *rq);
 void blk_mq_free_request(struct request *rq);
 int blk_rq_poll(struct request *rq, struct io_comp_batch *iob,
 		unsigned int poll_flags);

> In any case, if we really want to avoid exporting the original symbol
> (perhaps so that "only Rust" can use it -- or someone trying hard to
> bypass things on purpose), then we could still avoid the helper and
> instead write a non-generic `kernel`-private Rust function instead.

Let's see what the block layer experts say first. Perhaps it is OK to
expose this symbol like this or maybe it can be made more generic
somehow.

Jens, Ming, Keith, do you have any comments?

Best regards,
Andreas

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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-15  7:52             ` Andreas Hindborg
@ 2024-03-15 12:17               ` Ming Lei
  2024-03-15 12:46                 ` Andreas Hindborg
  0 siblings, 1 reply; 26+ messages in thread
From: Ming Lei @ 2024-03-15 12:17 UTC (permalink / raw)
  To: Andreas Hindborg
  Cc: Miguel Ojeda, Jens Axboe, Keith Busch, Boqun Feng,
	Christoph Hellwig, Damien Le Moal, Bart Van Assche,
	Hannes Reinecke, linux-block@vger.kernel.org, Andreas Hindborg,
	Wedson Almeida Filho, Niklas Cassel, Greg KH, Matthew Wilcox,
	Miguel Ojeda, Alex Gaynor, Gary Guo, Björn Roy Baron,
	Benno Lossin, Alice Ryhl, Chaitanya Kulkarni, Luis Chamberlain,
	Yexuan Yang, Sergio González Collado, Joel Granados,
	Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

On Fri, Mar 15, 2024 at 08:52:46AM +0100, Andreas Hindborg wrote:
> Miguel Ojeda <miguel.ojeda.sandonis@gmail.com> writes:
> 
> > On Thu, Mar 14, 2024 at 8:23 PM Andreas Hindborg <nmi@metaspace.dk> wrote:
> >>
> >> The way the current code compiles, <kernel::block::mq::Request as
> >> kernel::types::AlwaysRefCounted>::dec_ref` is inlined into the `rnull`
> >> module. A relocation for `rust_helper_blk_mq_free_request_internal`
> >> appears in `rnull_mod.ko`. I didn't test it yet, but if
> >> `__blk_mq_free_request` (or the helper) is not exported, I don't think
> >> this would be possible?
> >
> > Yeah, something needs to be exported since there is a generic
> > involved, but even if you want to go the route of exporting only a
> > different symbol, you would still want to put it in the C header so
> > that you don't get the C missing declaration warning and so that we
> > don't have to write the declaration manually in the helper.
> 
> That is what I did:
> 
> @@ -703,6 +703,7 @@ int blk_mq_alloc_sq_tag_set(struct blk_mq_tag_set *set,
>  		unsigned int set_flags);
>  void blk_mq_free_tag_set(struct blk_mq_tag_set *set);
>  
> +void __blk_mq_free_request(struct request *rq);
>  void blk_mq_free_request(struct request *rq);
>  int blk_rq_poll(struct request *rq, struct io_comp_batch *iob,
>  		unsigned int poll_flags);

Can you explain in detail why one block layer internal helper is
called into rnull driver directly? It never happens in C driver code.


Thanks,
Ming


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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-15 12:17               ` Ming Lei
@ 2024-03-15 12:46                 ` Andreas Hindborg
  2024-03-15 15:24                   ` Ming Lei
  0 siblings, 1 reply; 26+ messages in thread
From: Andreas Hindborg @ 2024-03-15 12:46 UTC (permalink / raw)
  To: Ming Lei
  Cc: Miguel Ojeda, Jens Axboe, Keith Busch, Boqun Feng,
	Christoph Hellwig, Damien Le Moal, Bart Van Assche,
	Hannes Reinecke, linux-block@vger.kernel.org, Andreas Hindborg,
	Wedson Almeida Filho, Niklas Cassel, Greg KH, Matthew Wilcox,
	Miguel Ojeda, Alex Gaynor, Gary Guo, Björn Roy Baron,
	Benno Lossin, Alice Ryhl, Chaitanya Kulkarni, Luis Chamberlain,
	Yexuan Yang, Sergio González Collado, Joel Granados,
	Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

Ming Lei <ming.lei@redhat.com> writes:
> On Fri, Mar 15, 2024 at 08:52:46AM +0100, Andreas Hindborg wrote:
>> Miguel Ojeda <miguel.ojeda.sandonis@gmail.com> writes:
>> 
>> > On Thu, Mar 14, 2024 at 8:23 PM Andreas Hindborg <nmi@metaspace.dk> wrote:
>> >>
>> >> The way the current code compiles, <kernel::block::mq::Request as
>> >> kernel::types::AlwaysRefCounted>::dec_ref` is inlined into the `rnull`
>> >> module. A relocation for `rust_helper_blk_mq_free_request_internal`
>> >> appears in `rnull_mod.ko`. I didn't test it yet, but if
>> >> `__blk_mq_free_request` (or the helper) is not exported, I don't think
>> >> this would be possible?
>> >
>> > Yeah, something needs to be exported since there is a generic
>> > involved, but even if you want to go the route of exporting only a
>> > different symbol, you would still want to put it in the C header so
>> > that you don't get the C missing declaration warning and so that we
>> > don't have to write the declaration manually in the helper.
>> 
>> That is what I did:
>> 
>> @@ -703,6 +703,7 @@ int blk_mq_alloc_sq_tag_set(struct blk_mq_tag_set *set,
>>  		unsigned int set_flags);
>>  void blk_mq_free_tag_set(struct blk_mq_tag_set *set);
>>  
>> +void __blk_mq_free_request(struct request *rq);
>>  void blk_mq_free_request(struct request *rq);
>>  int blk_rq_poll(struct request *rq, struct io_comp_batch *iob,
>>  		unsigned int poll_flags);
>
> Can you explain in detail why one block layer internal helper is
> called into rnull driver directly? It never happens in C driver code.

It is not the rust null block driver that calls this symbol directly. It
is called by the Rust block device driver API. But because of inlining,
the symbol is referenced from the loadable object.

The reason we have to call this symbol directly is to ensure proper
lifetime of the `struct request`. For example in C, when a driver
converts a tag to a request, the developer makes sure to only ask for
requests which are outstanding in the driver. In Rust, for the API to be
sound, we must ensure that the developer cannot write safe code that
obtains a reference to a request that is not owned by the driver.

A similar issue exists in the null block driver when timer completions
are enabled. If the request is cancelled and the timer fires after the
request has been recycled, there is a problem because the timer holds a
reference to the request private data area.

To that end, I use the `atomic_t ref` field of the C `struct request`
and implement the `AlwaysRefCounted` Rust trait for the request type.
This is a smart pointer that owns a reference to the pointee. In this
way, the request is not freed and recycled until the smart pointer is
dropped. But if the smart pointer holds the last reference when it is
dropped, it must be able to free the request, and hence it has to call
`__blk_mq_free_request`.

We could tag the function with `#[inline(never)]`, but that would impact
performance.

Best regards,
Andreas

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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-15 12:46                 ` Andreas Hindborg
@ 2024-03-15 15:24                   ` Ming Lei
  2024-03-15 17:49                     ` Andreas Hindborg
  0 siblings, 1 reply; 26+ messages in thread
From: Ming Lei @ 2024-03-15 15:24 UTC (permalink / raw)
  To: Andreas Hindborg
  Cc: Miguel Ojeda, Jens Axboe, Keith Busch, Boqun Feng,
	Christoph Hellwig, Damien Le Moal, Bart Van Assche,
	Hannes Reinecke, linux-block@vger.kernel.org, Andreas Hindborg,
	Wedson Almeida Filho, Niklas Cassel, Greg KH, Matthew Wilcox,
	Miguel Ojeda, Alex Gaynor, Gary Guo, Björn Roy Baron,
	Benno Lossin, Alice Ryhl, Chaitanya Kulkarni, Luis Chamberlain,
	Yexuan Yang, Sergio González Collado, Joel Granados,
	Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

On Fri, Mar 15, 2024 at 01:46:30PM +0100, Andreas Hindborg wrote:
> Ming Lei <ming.lei@redhat.com> writes:
> > On Fri, Mar 15, 2024 at 08:52:46AM +0100, Andreas Hindborg wrote:
> >> Miguel Ojeda <miguel.ojeda.sandonis@gmail.com> writes:
> >> 
> >> > On Thu, Mar 14, 2024 at 8:23 PM Andreas Hindborg <nmi@metaspace.dk> wrote:
> >> >>
> >> >> The way the current code compiles, <kernel::block::mq::Request as
> >> >> kernel::types::AlwaysRefCounted>::dec_ref` is inlined into the `rnull`
> >> >> module. A relocation for `rust_helper_blk_mq_free_request_internal`
> >> >> appears in `rnull_mod.ko`. I didn't test it yet, but if
> >> >> `__blk_mq_free_request` (or the helper) is not exported, I don't think
> >> >> this would be possible?
> >> >
> >> > Yeah, something needs to be exported since there is a generic
> >> > involved, but even if you want to go the route of exporting only a
> >> > different symbol, you would still want to put it in the C header so
> >> > that you don't get the C missing declaration warning and so that we
> >> > don't have to write the declaration manually in the helper.
> >> 
> >> That is what I did:
> >> 
> >> @@ -703,6 +703,7 @@ int blk_mq_alloc_sq_tag_set(struct blk_mq_tag_set *set,
> >>  		unsigned int set_flags);
> >>  void blk_mq_free_tag_set(struct blk_mq_tag_set *set);
> >>  
> >> +void __blk_mq_free_request(struct request *rq);
> >>  void blk_mq_free_request(struct request *rq);
> >>  int blk_rq_poll(struct request *rq, struct io_comp_batch *iob,
> >>  		unsigned int poll_flags);
> >
> > Can you explain in detail why one block layer internal helper is
> > called into rnull driver directly? It never happens in C driver code.
> 
> It is not the rust null block driver that calls this symbol directly. It
> is called by the Rust block device driver API. But because of inlining,
> the symbol is referenced from the loadable object.

What is the exact Rust block device driver API? The key point is that how
the body of one exported kernel C API(EXPORT_SYMBOL) becomes inlined
with Rust driver.

> 
> The reason we have to call this symbol directly is to ensure proper
> lifetime of the `struct request`. For example in C, when a driver

Sounds Rust API still calls into __blk_mq_free_request() directly, right?

If that is the case, the usecase need to be justified, and you need
to write one standalone patch with the exact story for exporting
__blk_mq_free_request().

> converts a tag to a request, the developer makes sure to only ask for
> requests which are outstanding in the driver. In Rust, for the API to be
> sound, we must ensure that the developer cannot write safe code that
> obtains a reference to a request that is not owned by the driver.
> 
> A similar issue exists in the null block driver when timer completions
> are enabled. If the request is cancelled and the timer fires after the
> request has been recycled, there is a problem because the timer holds a
> reference to the request private data area.
> 
> To that end, I use the `atomic_t ref` field of the C `struct request`
> and implement the `AlwaysRefCounted` Rust trait for the request type.
> This is a smart pointer that owns a reference to the pointee. In this
> way, the request is not freed and recycled until the smart pointer is
> dropped. But if the smart pointer holds the last reference when it is
> dropped, it must be able to free the request, and hence it has to call
> `__blk_mq_free_request`.

For callbacks(queue_rq, timeout, complete) implemented by driver, block
layer core guaranteed that the passed request reference is live.

So driver needn't to worry about request lifetime, same with Rust
driver, I think smart pointer isn't necessary for using request in
Rust driver.



Thanks, 
Ming


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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-15 15:24                   ` Ming Lei
@ 2024-03-15 17:49                     ` Andreas Hindborg
  2024-03-16 14:48                       ` Ming Lei
  0 siblings, 1 reply; 26+ messages in thread
From: Andreas Hindborg @ 2024-03-15 17:49 UTC (permalink / raw)
  To: Ming Lei
  Cc: Miguel Ojeda, Jens Axboe, Keith Busch, Boqun Feng,
	Christoph Hellwig, Damien Le Moal, Bart Van Assche,
	Hannes Reinecke, linux-block@vger.kernel.org, Andreas Hindborg,
	Wedson Almeida Filho, Niklas Cassel, Greg KH, Matthew Wilcox,
	Miguel Ojeda, Alex Gaynor, Gary Guo, Björn Roy Baron,
	Benno Lossin, Alice Ryhl, Chaitanya Kulkarni, Luis Chamberlain,
	Yexuan Yang, Sergio González Collado, Joel Granados,
	Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

Ming Lei <ming.lei@redhat.com> writes:

> On Fri, Mar 15, 2024 at 01:46:30PM +0100, Andreas Hindborg wrote:
>> Ming Lei <ming.lei@redhat.com> writes:
>> > On Fri, Mar 15, 2024 at 08:52:46AM +0100, Andreas Hindborg wrote:
>> >> Miguel Ojeda <miguel.ojeda.sandonis@gmail.com> writes:
>> >> 
>> >> > On Thu, Mar 14, 2024 at 8:23 PM Andreas Hindborg <nmi@metaspace.dk> wrote:
>> >> >>
>> >> >> The way the current code compiles, <kernel::block::mq::Request as
>> >> >> kernel::types::AlwaysRefCounted>::dec_ref` is inlined into the `rnull`
>> >> >> module. A relocation for `rust_helper_blk_mq_free_request_internal`
>> >> >> appears in `rnull_mod.ko`. I didn't test it yet, but if
>> >> >> `__blk_mq_free_request` (or the helper) is not exported, I don't think
>> >> >> this would be possible?
>> >> >
>> >> > Yeah, something needs to be exported since there is a generic
>> >> > involved, but even if you want to go the route of exporting only a
>> >> > different symbol, you would still want to put it in the C header so
>> >> > that you don't get the C missing declaration warning and so that we
>> >> > don't have to write the declaration manually in the helper.
>> >> 
>> >> That is what I did:
>> >> 
>> >> @@ -703,6 +703,7 @@ int blk_mq_alloc_sq_tag_set(struct blk_mq_tag_set *set,
>> >>  		unsigned int set_flags);
>> >>  void blk_mq_free_tag_set(struct blk_mq_tag_set *set);
>> >>  
>> >> +void __blk_mq_free_request(struct request *rq);
>> >>  void blk_mq_free_request(struct request *rq);
>> >>  int blk_rq_poll(struct request *rq, struct io_comp_batch *iob,
>> >>  		unsigned int poll_flags);
>> >
>> > Can you explain in detail why one block layer internal helper is
>> > called into rnull driver directly? It never happens in C driver code.
>> 
>> It is not the rust null block driver that calls this symbol directly. It
>> is called by the Rust block device driver API. But because of inlining,
>> the symbol is referenced from the loadable object.
>
> What is the exact Rust block device driver API? The key point is that how
> the body of one exported kernel C API(EXPORT_SYMBOL) becomes inlined
> with Rust driver.

This happens when `ARef<Request<_>>` is dropped. The drop method
(destructor) of this smart pointer decrements the refcount and
potentially calls `__blk_mq_free_request`.

>> 
>> The reason we have to call this symbol directly is to ensure proper
>> lifetime of the `struct request`. For example in C, when a driver
>
> Sounds Rust API still calls into __blk_mq_free_request() directly, right?

Yes, the Rust block device driver API will call this request if an
`ARef<Request<_>>` is dropped and the refcount goes to 0.

> If that is the case, the usecase need to be justified, and you need
> to write one standalone patch with the exact story for exporting
> __blk_mq_free_request().

Ok, I can do that.

>
>> converts a tag to a request, the developer makes sure to only ask for
>> requests which are outstanding in the driver. In Rust, for the API to be
>> sound, we must ensure that the developer cannot write safe code that
>> obtains a reference to a request that is not owned by the driver.
>> 
>> A similar issue exists in the null block driver when timer completions
>> are enabled. If the request is cancelled and the timer fires after the
>> request has been recycled, there is a problem because the timer holds a
>> reference to the request private data area.
>> 
>> To that end, I use the `atomic_t ref` field of the C `struct request`
>> and implement the `AlwaysRefCounted` Rust trait for the request type.
>> This is a smart pointer that owns a reference to the pointee. In this
>> way, the request is not freed and recycled until the smart pointer is
>> dropped. But if the smart pointer holds the last reference when it is
>> dropped, it must be able to free the request, and hence it has to call
>> `__blk_mq_free_request`.
>
> For callbacks(queue_rq, timeout, complete) implemented by driver, block
> layer core guaranteed that the passed request reference is live.
>
> So driver needn't to worry about request lifetime, same with Rust
> driver, I think smart pointer isn't necessary for using request in
> Rust driver.

Using the C API, there is nothing preventing a driver from using the
request after the lifetime ends. With Rust, we have to make it
impossible. Without the refcount and associated call to
`__blk_mq_free_request`, it would be possible to write Rust code that
access the request after the lifetime ends. This is not sound, and it is
something we need to avoid in the Rust abstractions.

One concrete way to do write unsound code with a Rust API where lifetime
is not tracked with refcount, is if the null block timer completion
callback fires after the request is completed. Perhaps the driver
cancels the request but forgets to cancel the timer. When the timer
fires, it will access the request via the context pointer, but the
request will be invalid. In C we have to write the driver code so this
cannot happen. In Rust, the API must prevent this from happening. So any
driver written in the safe subset of Rust using this API can never
trigger this behavior.

By using the refcount, we ensure that the request is alive until all
users who hold a reference to it are dropped.

Another concrete example is when a driver calls `blk_mq_tag_to_rq` with
an invalid tag. This can return a reference to an invalid tag, if the
driver is not implemented correctly. By using `req_ref_inc_not_zero` we
can assert that the request is live before we create a Rust reference to
it, and even if the driver code has bugs, it can never access an invalid
request, and thus it can be memory safe.

We move the responsibility of correctness, in relation to memory safety,
from the driver implementation to the API implementation.

Best regards,
Andreas

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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-15 17:49                     ` Andreas Hindborg
@ 2024-03-16 14:48                       ` Ming Lei
  2024-03-16 17:27                         ` Andreas Hindborg
  0 siblings, 1 reply; 26+ messages in thread
From: Ming Lei @ 2024-03-16 14:48 UTC (permalink / raw)
  To: Andreas Hindborg
  Cc: Miguel Ojeda, Jens Axboe, Keith Busch, Boqun Feng,
	Christoph Hellwig, Damien Le Moal, Bart Van Assche,
	Hannes Reinecke, linux-block@vger.kernel.org, Andreas Hindborg,
	Wedson Almeida Filho, Niklas Cassel, Greg KH, Matthew Wilcox,
	Miguel Ojeda, Alex Gaynor, Gary Guo, Björn Roy Baron,
	Benno Lossin, Alice Ryhl, Chaitanya Kulkarni, Luis Chamberlain,
	Yexuan Yang, Sergio González Collado, Joel Granados,
	Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com, ming.lei

On Fri, Mar 15, 2024 at 06:49:39PM +0100, Andreas Hindborg wrote:
> Ming Lei <ming.lei@redhat.com> writes:
> 
> > On Fri, Mar 15, 2024 at 01:46:30PM +0100, Andreas Hindborg wrote:
> >> Ming Lei <ming.lei@redhat.com> writes:
> >> > On Fri, Mar 15, 2024 at 08:52:46AM +0100, Andreas Hindborg wrote:
> >> >> Miguel Ojeda <miguel.ojeda.sandonis@gmail.com> writes:
> >> >> 
> >> >> > On Thu, Mar 14, 2024 at 8:23 PM Andreas Hindborg <nmi@metaspace.dk> wrote:
> >> >> >>
> >> >> >> The way the current code compiles, <kernel::block::mq::Request as
> >> >> >> kernel::types::AlwaysRefCounted>::dec_ref` is inlined into the `rnull`
> >> >> >> module. A relocation for `rust_helper_blk_mq_free_request_internal`
> >> >> >> appears in `rnull_mod.ko`. I didn't test it yet, but if
> >> >> >> `__blk_mq_free_request` (or the helper) is not exported, I don't think
> >> >> >> this would be possible?
> >> >> >
> >> >> > Yeah, something needs to be exported since there is a generic
> >> >> > involved, but even if you want to go the route of exporting only a
> >> >> > different symbol, you would still want to put it in the C header so
> >> >> > that you don't get the C missing declaration warning and so that we
> >> >> > don't have to write the declaration manually in the helper.
> >> >> 
> >> >> That is what I did:
> >> >> 
> >> >> @@ -703,6 +703,7 @@ int blk_mq_alloc_sq_tag_set(struct blk_mq_tag_set *set,
> >> >>  		unsigned int set_flags);
> >> >>  void blk_mq_free_tag_set(struct blk_mq_tag_set *set);
> >> >>  
> >> >> +void __blk_mq_free_request(struct request *rq);
> >> >>  void blk_mq_free_request(struct request *rq);
> >> >>  int blk_rq_poll(struct request *rq, struct io_comp_batch *iob,
> >> >>  		unsigned int poll_flags);
> >> >
> >> > Can you explain in detail why one block layer internal helper is
> >> > called into rnull driver directly? It never happens in C driver code.
> >> 
> >> It is not the rust null block driver that calls this symbol directly. It
> >> is called by the Rust block device driver API. But because of inlining,
> >> the symbol is referenced from the loadable object.
> >
> > What is the exact Rust block device driver API? The key point is that how
> > the body of one exported kernel C API(EXPORT_SYMBOL) becomes inlined
> > with Rust driver.
> 
> This happens when `ARef<Request<_>>` is dropped. The drop method
> (destructor) of this smart pointer decrements the refcount and
> potentially calls `__blk_mq_free_request`.
> 
> >> 
> >> The reason we have to call this symbol directly is to ensure proper
> >> lifetime of the `struct request`. For example in C, when a driver
> >
> > Sounds Rust API still calls into __blk_mq_free_request() directly, right?
> 
> Yes, the Rust block device driver API will call this request if an
> `ARef<Request<_>>` is dropped and the refcount goes to 0.
> 
> > If that is the case, the usecase need to be justified, and you need
> > to write one standalone patch with the exact story for exporting
> > __blk_mq_free_request().
> 
> Ok, I can do that.
> 
> >
> >> converts a tag to a request, the developer makes sure to only ask for
> >> requests which are outstanding in the driver. In Rust, for the API to be
> >> sound, we must ensure that the developer cannot write safe code that
> >> obtains a reference to a request that is not owned by the driver.
> >> 
> >> A similar issue exists in the null block driver when timer completions
> >> are enabled. If the request is cancelled and the timer fires after the
> >> request has been recycled, there is a problem because the timer holds a
> >> reference to the request private data area.
> >> 
> >> To that end, I use the `atomic_t ref` field of the C `struct request`
> >> and implement the `AlwaysRefCounted` Rust trait for the request type.
> >> This is a smart pointer that owns a reference to the pointee. In this
> >> way, the request is not freed and recycled until the smart pointer is
> >> dropped. But if the smart pointer holds the last reference when it is
> >> dropped, it must be able to free the request, and hence it has to call
> >> `__blk_mq_free_request`.
> >
> > For callbacks(queue_rq, timeout, complete) implemented by driver, block
> > layer core guaranteed that the passed request reference is live.
> >
> > So driver needn't to worry about request lifetime, same with Rust
> > driver, I think smart pointer isn't necessary for using request in
> > Rust driver.
> 
> Using the C API, there is nothing preventing a driver from using the
> request after the lifetime ends.

Yes, it is true for C, so will Rust-for-linux need to add refcount for
most exported kernel C structure? such as by implementing AlwaysRefCounted
traits?

> With Rust, we have to make it
> impossible.Without the refcount and associated call to
> `__blk_mq_free_request`, it would be possible to write Rust code that
> access the request after the lifetime ends. This is not sound, and it is
> something we need to avoid in the Rust abstractions.
> 
> One concrete way to do write unsound code with a Rust API where lifetime
> is not tracked with refcount, is if the null block timer completion
> callback fires after the request is completed. Perhaps the driver
> cancels the request but forgets to cancel the timer. When the timer
> fires, it will access the request via the context pointer, but the
> request will be invalid.

The issue is less serious for blk-mq request, which is pre-allocated,
and one freed request just means it can be re-allocated for other IO
in same queue, and the pointed memory won't be really freed.

Also as I mentioned, inside driver's ->timeout(), the request is
guaranteed to be live by block layer core(won't be re-allocated to other IO),
the passed-in request is referenced already, please see bt_iter() which
is called from blk_mq_timeout_work(). Here, block layer core just
borrows request, then passed the reference to ->timeout(), when
request is owned by driver actually.

I understand Rust block driver still need to implement ->queue_rq(),
->timeout(), ..., just like C driver, but maybe I am wrong? Or Rust block driver
will re-implement part of block layer core code? such as, get one extra
reference of request no matter block core has done that.

> In C we have to write the driver code so this
> cannot happen. In Rust, the API must prevent this from happening. So any
> driver written in the safe subset of Rust using this API can never
> trigger this behavior.
> 
> By using the refcount, we ensure that the request is alive until all
> users who hold a reference to it are dropped.

block layer has provided such guarantee if Rust driver follows current
block driver model.

> 
> Another concrete example is when a driver calls `blk_mq_tag_to_rq` with
> an invalid tag. This can return a reference to an invalid tag, if the
> driver is not implemented correctly. By using `req_ref_inc_not_zero` we
> can assert that the request is live before we create a Rust reference to
> it, and even if the driver code has bugs, it can never access an invalid
> request, and thus it can be memory safe.
> 
> We move the responsibility of correctness, in relation to memory safety,
> from the driver implementation to the API implementation.

After queue_rq(req) is called, request ownership is actually transferred to
driver like Rust's move, then driver is free to call blk_mq_tag_to_rq(), and
finally return request to block core after the request is completed by driver.

The biggest question should be how Rust block driver will be designed &
implemented? Will rust block driver follow current C driver's model, such
as implementing ->queue_rq(), ->timeout(), ->complete()...?



thanks,
Ming


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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-16 14:48                       ` Ming Lei
@ 2024-03-16 17:27                         ` Andreas Hindborg
  0 siblings, 0 replies; 26+ messages in thread
From: Andreas Hindborg @ 2024-03-16 17:27 UTC (permalink / raw)
  To: Ming Lei
  Cc: Miguel Ojeda, Jens Axboe, Keith Busch, Boqun Feng,
	Christoph Hellwig, Damien Le Moal, Bart Van Assche,
	Hannes Reinecke, linux-block@vger.kernel.org, Andreas Hindborg,
	Wedson Almeida Filho, Niklas Cassel, Greg KH, Matthew Wilcox,
	Miguel Ojeda, Alex Gaynor, Gary Guo, Björn Roy Baron,
	Benno Lossin, Alice Ryhl, Chaitanya Kulkarni, Luis Chamberlain,
	Yexuan Yang, Sergio González Collado, Joel Granados,
	Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

Ming Lei <ming.lei@redhat.com> writes:

> On Fri, Mar 15, 2024 at 06:49:39PM +0100, Andreas Hindborg wrote:
>> Ming Lei <ming.lei@redhat.com> writes:
>> 
>> > On Fri, Mar 15, 2024 at 01:46:30PM +0100, Andreas Hindborg wrote:
>> >> Ming Lei <ming.lei@redhat.com> writes:
>> >> > On Fri, Mar 15, 2024 at 08:52:46AM +0100, Andreas Hindborg wrote:
>> >> >> Miguel Ojeda <miguel.ojeda.sandonis@gmail.com> writes:
>> >> >> 
>> >> >> > On Thu, Mar 14, 2024 at 8:23 PM Andreas Hindborg <nmi@metaspace.dk> wrote:
>> >> >> >>
>> >> >> >> The way the current code compiles, <kernel::block::mq::Request as
>> >> >> >> kernel::types::AlwaysRefCounted>::dec_ref` is inlined into the `rnull`
>> >> >> >> module. A relocation for `rust_helper_blk_mq_free_request_internal`
>> >> >> >> appears in `rnull_mod.ko`. I didn't test it yet, but if
>> >> >> >> `__blk_mq_free_request` (or the helper) is not exported, I don't think
>> >> >> >> this would be possible?
>> >> >> >
>> >> >> > Yeah, something needs to be exported since there is a generic
>> >> >> > involved, but even if you want to go the route of exporting only a
>> >> >> > different symbol, you would still want to put it in the C header so
>> >> >> > that you don't get the C missing declaration warning and so that we
>> >> >> > don't have to write the declaration manually in the helper.
>> >> >> 
>> >> >> That is what I did:
>> >> >> 
>> >> >> @@ -703,6 +703,7 @@ int blk_mq_alloc_sq_tag_set(struct blk_mq_tag_set *set,
>> >> >>  		unsigned int set_flags);
>> >> >>  void blk_mq_free_tag_set(struct blk_mq_tag_set *set);
>> >> >>  
>> >> >> +void __blk_mq_free_request(struct request *rq);
>> >> >>  void blk_mq_free_request(struct request *rq);
>> >> >>  int blk_rq_poll(struct request *rq, struct io_comp_batch *iob,
>> >> >>  		unsigned int poll_flags);
>> >> >
>> >> > Can you explain in detail why one block layer internal helper is
>> >> > called into rnull driver directly? It never happens in C driver code.
>> >> 
>> >> It is not the rust null block driver that calls this symbol directly. It
>> >> is called by the Rust block device driver API. But because of inlining,
>> >> the symbol is referenced from the loadable object.
>> >
>> > What is the exact Rust block device driver API? The key point is that how
>> > the body of one exported kernel C API(EXPORT_SYMBOL) becomes inlined
>> > with Rust driver.
>> 
>> This happens when `ARef<Request<_>>` is dropped. The drop method
>> (destructor) of this smart pointer decrements the refcount and
>> potentially calls `__blk_mq_free_request`.
>> 
>> >> 
>> >> The reason we have to call this symbol directly is to ensure proper
>> >> lifetime of the `struct request`. For example in C, when a driver
>> >
>> > Sounds Rust API still calls into __blk_mq_free_request() directly, right?
>> 
>> Yes, the Rust block device driver API will call this request if an
>> `ARef<Request<_>>` is dropped and the refcount goes to 0.
>> 
>> > If that is the case, the usecase need to be justified, and you need
>> > to write one standalone patch with the exact story for exporting
>> > __blk_mq_free_request().
>> 
>> Ok, I can do that.
>> 
>> >
>> >> converts a tag to a request, the developer makes sure to only ask for
>> >> requests which are outstanding in the driver. In Rust, for the API to be
>> >> sound, we must ensure that the developer cannot write safe code that
>> >> obtains a reference to a request that is not owned by the driver.
>> >> 
>> >> A similar issue exists in the null block driver when timer completions
>> >> are enabled. If the request is cancelled and the timer fires after the
>> >> request has been recycled, there is a problem because the timer holds a
>> >> reference to the request private data area.
>> >> 
>> >> To that end, I use the `atomic_t ref` field of the C `struct request`
>> >> and implement the `AlwaysRefCounted` Rust trait for the request type.
>> >> This is a smart pointer that owns a reference to the pointee. In this
>> >> way, the request is not freed and recycled until the smart pointer is
>> >> dropped. But if the smart pointer holds the last reference when it is
>> >> dropped, it must be able to free the request, and hence it has to call
>> >> `__blk_mq_free_request`.
>> >
>> > For callbacks(queue_rq, timeout, complete) implemented by driver, block
>> > layer core guaranteed that the passed request reference is live.
>> >
>> > So driver needn't to worry about request lifetime, same with Rust
>> > driver, I think smart pointer isn't necessary for using request in
>> > Rust driver.
>> 
>> Using the C API, there is nothing preventing a driver from using the
>> request after the lifetime ends.
>
> Yes, it is true for C, so will Rust-for-linux need to add refcount for
> most exported kernel C structure? such as by implementing AlwaysRefCounted
> traits?

Not for all types and not all the time. For instance the Rust block
device driver API does not always use refcounting for `struct request`.
Only when it cannot determine the lifetime of a request reference at
compile time.

>
>> With Rust, we have to make it
>> impossible.Without the refcount and associated call to
>> `__blk_mq_free_request`, it would be possible to write Rust code that
>> access the request after the lifetime ends. This is not sound, and it is
>> something we need to avoid in the Rust abstractions.
>> 
>> One concrete way to do write unsound code with a Rust API where lifetime
>> is not tracked with refcount, is if the null block timer completion
>> callback fires after the request is completed. Perhaps the driver
>> cancels the request but forgets to cancel the timer. When the timer
>> fires, it will access the request via the context pointer, but the
>> request will be invalid.
>
> The issue is less serious for blk-mq request, which is pre-allocated,
> and one freed request just means it can be re-allocated for other IO
> in same queue, and the pointed memory won't be really freed.

The issue here is not so much use after free as it is aliasing of
mutable and shared references. This is illegal in Rust, and programs
that allow this may exhibit undefined behavior.

> Also as I mentioned, inside driver's ->timeout(), the request is
> guaranteed to be live by block layer core(won't be re-allocated to other IO),
> the passed-in request is referenced already, please see bt_iter() which
> is called from blk_mq_timeout_work(). Here, block layer core just
> borrows request, then passed the reference to ->timeout(), when
> request is owned by driver actually.

I understand. I am not referring to `blk_mq_opw.timeout`. The null block
driver has a feature where requests complete after a delay. This is
implemented via the `hrtimer` subsystem.

> I understand Rust block driver still need to implement ->queue_rq(),
> ->timeout(), ..., just like C driver, but maybe I am wrong? Or Rust block driver
> will re-implement part of block layer core code? such as, get one extra
> reference of request no matter block core has done that.

The Rust block driver API implements the `blk_mq_ops` vtable on behalf
of the driver. There is a little bit of glue code inserted between the C
symbols called by the kernel and the Rust functions that the block device
driver provides.

>> In C we have to write the driver code so this
>> cannot happen. In Rust, the API must prevent this from happening. So any
>> driver written in the safe subset of Rust using this API can never
>> trigger this behavior.
>> 
>> By using the refcount, we ensure that the request is alive until all
>> users who hold a reference to it are dropped.
>
> block layer has provided such guarantee if Rust driver follows current
> block driver model.

I understand that the driver has exclusive ownership of the request
between `queue_rq()` and `complete()`. What we want to guarantee is that
the author of a block device driver is not able to access a request
after block layer has regained ownership of the request (after complete).
The way we achieve this is by preventing block layer from calling
`__blk_mq_free_request` if the Rust driver did not destroy all
references to the request.

I understand that this is not necessary for correct operation of a
block device driver. However, it is necessary to uphold invariants of
the Rust language for references.

>> 
>> Another concrete example is when a driver calls `blk_mq_tag_to_rq` with
>> an invalid tag. This can return a reference to an invalid tag, if the
>> driver is not implemented correctly. By using `req_ref_inc_not_zero` we
>> can assert that the request is live before we create a Rust reference to
>> it, and even if the driver code has bugs, it can never access an invalid
>> request, and thus it can be memory safe.
>> 
>> We move the responsibility of correctness, in relation to memory safety,
>> from the driver implementation to the API implementation.
>
> After queue_rq(req) is called, request ownership is actually transferred to
> driver like Rust's move, then driver is free to call blk_mq_tag_to_rq(), and
> finally return request to block core after the request is completed by driver.

As I said, I understand and appreciate this design. But there is a
possibility for a buggy driver to not obey. The Rust abstractions must
prevent this buggy code from compiling at all. In Rust, it must be
impossible to call `blk_mq_tag_to_rq()` for a tag that is not owned by
the driver. `blk_mq_tag_to_rq` takes an integer, so the driver developer
can call this function with an invalid tag like
`blk_mq_tag_to_rq(tagset, 5)`. If 5 is not owned by the driver, this
will return a request pointer that will alias with mutable access. When
this pointer is converted to a Rust reference, this will give undefined
behavior. For Rust, we must prevent code like this from compiling in the
first place.

> The biggest question should be how Rust block driver will be designed &
> implemented? Will rust block driver follow current C driver's model, such
> as implementing ->queue_rq(), ->timeout(), ->complete()...?

The Rust driver API does follow the design of the C driver model. It
does implement the `blk_mq_ops` vtable. Rust block drivers must
implement a set of Rust functions that very closely follow this vtable.
But when it is not possible to statically determine the lifetime of an
object, to the point where the compiler will refuse to compile the code
that violates the lifetime durations of the object, we must revert to
reference counting.

I understand that a correctly implemented block device driver will not
violate lifetime of the request structure. Incorrect driver code might
violate the lifetime though. With Rust, we must prevent incorrectly
implemented drivers from compiling at all. We need the refcount for
that. By only allowing access to the request through the refcounted
pointer in certain cases, we prevent the safe driver code from
misbehaving.

It might seem cumbersome to do this dance at first. After all, we can
just write the device drivers correctly without memory safety bugs. But
the reward for all this work is that if a device driver is implemented
in the safe subset of Rust, it _cannot_ exhibit memory safety related
bugs. We don't have to review a device driver implemented in safe Rust
for memory safety issues at all.

Best regards,
Andreas

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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
@ 2024-03-22 23:40 Benno Lossin
  2024-03-23  6:32 ` Andreas Hindborg
  0 siblings, 1 reply; 26+ messages in thread
From: Benno Lossin @ 2024-03-22 23:40 UTC (permalink / raw)
  To: Andreas Hindborg, Jens Axboe, Christoph Hellwig, Keith Busch,
	Damien Le Moal, Bart Van Assche, Hannes Reinecke,
	linux-block@vger.kernel.org
  Cc: Andreas Hindborg, Wedson Almeida Filho, Niklas Cassel, Greg KH,
	Matthew Wilcox, Miguel Ojeda, Alex Gaynor, Boqun Feng, Gary Guo,
	Björn Roy Baron, Alice Ryhl, Chaitanya Kulkarni,
	Luis Chamberlain, Yexuan Yang, Sergio González Collado,
	Joel Granados, Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

On 3/13/24 12:05, Andreas Hindborg wrote:
> From: Andreas Hindborg <a.hindborg@samsung.com>
> 
> Add initial abstractions for working with blk-mq.
> 
> This patch is a maintained, refactored subset of code originally published by
> Wedson Almeida Filho <wedsonaf@gmail.com> [1].
> 
> [1] https://github.com/wedsonaf/linux/tree/f2cfd2fe0e2ca4e90994f96afe268bbd4382a891/rust/kernel/blk/mq.rs
> 
> Cc: Wedson Almeida Filho <wedsonaf@gmail.com>
> Signed-off-by: Andreas Hindborg <a.hindborg@samsung.com>
> ---
>   block/blk-mq.c                     |   3 +-
>   include/linux/blk-mq.h             |   1 +
>   rust/bindings/bindings_helper.h    |   2 +
>   rust/helpers.c                     |  45 ++++
>   rust/kernel/block.rs               |   5 +
>   rust/kernel/block/mq.rs            | 131 +++++++++++
>   rust/kernel/block/mq/gen_disk.rs   | 174 +++++++++++++++
>   rust/kernel/block/mq/operations.rs | 346 +++++++++++++++++++++++++++++
>   rust/kernel/block/mq/raw_writer.rs |  60 +++++
>   rust/kernel/block/mq/request.rs    | 182 +++++++++++++++
>   rust/kernel/block/mq/tag_set.rs    | 117 ++++++++++
>   rust/kernel/error.rs               |   5 +
>   rust/kernel/lib.rs                 |   1 +
>   13 files changed, 1071 insertions(+), 1 deletion(-)

Do you think that it's possible to split this into smaller
patches? It would make review a lot easier.

>   create mode 100644 rust/kernel/block.rs
>   create mode 100644 rust/kernel/block/mq.rs
>   create mode 100644 rust/kernel/block/mq/gen_disk.rs
>   create mode 100644 rust/kernel/block/mq/operations.rs
>   create mode 100644 rust/kernel/block/mq/raw_writer.rs
>   create mode 100644 rust/kernel/block/mq/request.rs
>   create mode 100644 rust/kernel/block/mq/tag_set.rs

[...]

> diff --git a/rust/kernel/block.rs b/rust/kernel/block.rs
> new file mode 100644
> index 000000000000..4c93317a568a
> --- /dev/null
> +++ b/rust/kernel/block.rs
> @@ -0,0 +1,5 @@
> +// SPDX-License-Identifier: GPL-2.0
> +
> +//! Types for working with the block layer

Missing '.'.

> +
> +pub mod mq;
> diff --git a/rust/kernel/block/mq.rs b/rust/kernel/block/mq.rs
> new file mode 100644
> index 000000000000..08de1cc114ff
> --- /dev/null
> +++ b/rust/kernel/block/mq.rs
> @@ -0,0 +1,131 @@
> +// SPDX-License-Identifier: GPL-2.0
> +
> +//! This module provides types for implementing block drivers that interface the
> +//! blk-mq subsystem.
> +//!
> +//! To implement a block device driver, a Rust module must do the following:
> +//!
> +//! - Implement [`Operations`] for a type `T`

I think it would be nicer to use `Driver` (or `MyBlkDevice`) instead of
`T`.

> +//! - Create a [`TagSet<T>`]
> +//! - Create a [`GenDisk<T>`], passing in the `TagSet` reference
> +//! - Add the disk to the system by calling [`GenDisk::add`]
> +//!
> +//! The types available in this module that have direct C counterparts are:
> +//!
> +//! - The `TagSet` type that abstracts the C type `struct tag_set`.
> +//! - The `GenDisk` type that abstracts the C type `struct gendisk`.
> +//! - The `Request` type that abstracts the C type `struct request`.
> +//!
> +//! Many of the C types that this module abstracts allow a driver to carry
> +//! private data, either embedded in the stuct directly, or as a C `void*`. In
> +//! these abstractions, this data is typed. The types of the data is defined by
> +//! associated types in `Operations`, see [`Operations::RequestData`] for an
> +//! example.
> +//!
> +//! The kernel will interface with the block evice driver by calling the method

Typo: "block evice driver"

> +//! implementations of the `Operations` trait.
> +//!
> +//! IO requests are passed to the driver as [`Request`] references. The
> +//! `Request` type is a wrapper around the C `struct request`. The driver must
> +//! mark start of request processing by calling [`Request::start`] and end of
> +//! processing by calling one of the [`Request::end`], methods. Failure to do so
> +//! can lead to IO failures.

I am unfamiliar with this, what are "IO failures"?
Do you think that it might be better to change the API to use a
callback? So instead of calling start and end, you would do

     request.handle(|req| {
         // do the stuff that would be done between start and end
     });

I took a quick look at the rnull driver and there you are calling
`Request::end_ok` from a different function. So my suggestion might not
be possible, since you really need the freedom.

Do you think that a guard approach might work better? ie `start` returns
a guard that when dropped will call `end` and you need the guard to
operate on the request.

> +//!
> +//! The `TagSet` is responsible for creating and maintaining a mapping between
> +//! `Request`s and integer ids as well as carrying a pointer to the vtable
> +//! generated by `Operations`. This mapping is useful for associating
> +//! completions from hardware with the correct `Request` instance. The `TagSet`
> +//! determines the maximum queue depth by setting the number of `Request`
> +//! instances available to the driver, and it determines the number of queues to
> +//! instantiate for the driver. If possible, a driver should allocate one queue
> +//! per core, to keep queue data local to a core.
> +//!
> +//! One `TagSet` instance can be shared between multiple `GenDisk` instances.
> +//! This can be useful when implementing drivers where one piece of hardware
> +//! with one set of IO resources are represented to the user as multiple disks.
> +//!
> +//! One significant difference between block device drivers implemented with
> +//! these Rust abstractions and drivers implemented in C, is that the Rust
> +//! drivers have to own a reference count on the `Request` type when the IO is
> +//! in flight. This is to ensure that the C `struct request` instances backing
> +//! the Rust `Request` instances are live while the Rust driver holds a
> +//! reference to the `Request`. In addition, the conversion of an ineger tag to

Typo: "of an ineger tag"

> +//! a `Request` via the `TagSet` would not be sound without this bookkeeping.
> +//!
> +//! # ⚠ Note
> +//!
> +//! For Rust block device drivers, the point in time where a request
> +//! is freed and made available for recycling is usualy at the point in time
> +//! when the last `ARef<Request>` is dropped. For C drivers, this event usually
> +//! occurs when `bindings::blk_mq_end_request` is called.
> +//!
> +//! # Example
> +//!
> +//! ```rust
> +//! use kernel::{
> +//!     block::mq::*,
> +//!     new_mutex,
> +//!     prelude::*,
> +//!     sync::{Arc, Mutex},
> +//!     types::{ARef, ForeignOwnable},
> +//! };
> +//!
> +//! struct MyBlkDevice;
> +//!
> +//! #[vtable]
> +//! impl Operations for MyBlkDevice {
> +//!     type RequestData = ();
> +//!     type RequestDataInit = impl PinInit<()>;
> +//!     type QueueData = ();
> +//!     type HwData = ();
> +//!     type TagSetData = ();
> +//!
> +//!     fn new_request_data(
> +//!         _tagset_data: <Self::TagSetData as ForeignOwnable>::Borrowed<'_>,
> +//!     ) -> Self::RequestDataInit {
> +//!         kernel::init::zeroed()
> +//!     }
> +//!
> +//!     fn queue_rq(_hw_data: (), _queue_data: (), rq: ARef<Request<Self>>, _is_last: bool) -> Result {
> +//!         rq.start();
> +//!         rq.end_ok();
> +//!         Ok(())
> +//!     }
> +//!
> +//!     fn commit_rqs(
> +//!         _hw_data: <Self::HwData as ForeignOwnable>::Borrowed<'_>,
> +//!         _queue_data: <Self::QueueData as ForeignOwnable>::Borrowed<'_>,
> +//!     ) {
> +//!     }
> +//!
> +//!     fn complete(rq: &Request<Self>) {
> +//!         rq.end_ok();
> +//!     }
> +//!
> +//!     fn init_hctx(
> +//!         _tagset_data: <Self::TagSetData as ForeignOwnable>::Borrowed<'_>,
> +//!         _hctx_idx: u32,
> +//!     ) -> Result<Self::HwData> {
> +//!         Ok(())
> +//!     }
> +//! }
> +//!
> +//! let tagset: Arc<TagSet<MyBlkDevice>> = Arc::pin_init(TagSet::try_new(1, (), 256, 1))?;
> +//! let mut disk = GenDisk::try_new(tagset, ())?;
> +//! disk.set_name(format_args!("myblk"))?;
> +//! disk.set_capacity_sectors(4096);
> +//! disk.add()?;
> +//!
> +//! # Ok::<(), kernel::error::Error>(())
> +//! ```

This piece of documentation is **really** valuable, thanks a lot for
taking the time to write it.

> +
> +mod gen_disk;
> +mod operations;
> +mod raw_writer;
> +mod request;
> +mod tag_set;
> +
> +pub use gen_disk::GenDisk;
> +pub use operations::Operations;
> +pub use request::{Request, RequestDataRef};
> +pub use tag_set::TagSet;
> diff --git a/rust/kernel/block/mq/gen_disk.rs b/rust/kernel/block/mq/gen_disk.rs
> new file mode 100644
> index 000000000000..b7845fc9e39f
> --- /dev/null
> +++ b/rust/kernel/block/mq/gen_disk.rs
> @@ -0,0 +1,174 @@
> +// SPDX-License-Identifier: GPL-2.0
> +
> +//! Generic disk abstraction.
> +//!
> +//! C header: [`include/linux/blkdev.h`](srctree/include/linux/blkdev.h)
> +//! C header: [`include/linux/blk_mq.h`](srctree/include/linux/blk_mq.h)
> +
> +use crate::block::mq::{raw_writer::RawWriter, Operations, TagSet};
> +use crate::{
> +    bindings, error::from_err_ptr, error::Result, sync::Arc, types::ForeignOwnable,
> +    types::ScopeGuard,
> +};
> +use core::fmt::{self, Write};
> +
> +/// A generic block device
> +///
> +/// # Invariants
> +///
> +///  - `gendisk` must always point to an initialized and valid `struct gendisk`.
> +pub struct GenDisk<T: Operations> {
> +    _tagset: Arc<TagSet<T>>,
> +    gendisk: *mut bindings::gendisk,
> +}
> +
> +// SAFETY: `GenDisk` is an owned pointer to a `struct gendisk` and an `Arc` to a
> +// `TagSet` It is safe to send this to other threads as long as T is Send.
> +unsafe impl<T: Operations + Send> Send for GenDisk<T> {}
> +
> +impl<T: Operations> GenDisk<T> {
> +    /// Try to create a new `GenDisk`
> +    pub fn try_new(tagset: Arc<TagSet<T>>, queue_data: T::QueueData) -> Result<Self> {
> +        let data = queue_data.into_foreign();
> +        let recover_data = ScopeGuard::new(|| {
> +            // SAFETY: T::QueueData was created by the call to `into_foreign()` above
> +            unsafe { T::QueueData::from_foreign(data) };
> +        });
> +
> +        let lock_class_key = crate::sync::LockClassKey::new();
> +
> +        // SAFETY: `tagset.raw_tag_set()` points to a valid and initialized tag set
> +        let gendisk = from_err_ptr(unsafe {
> +            bindings::__blk_mq_alloc_disk(
> +                tagset.raw_tag_set(),
> +                data.cast_mut(),
> +                lock_class_key.as_ptr(),
> +            )
> +        })?;
> +
> +        const TABLE: bindings::block_device_operations = bindings::block_device_operations {
> +            submit_bio: None,
> +            open: None,
> +            release: None,
> +            ioctl: None,
> +            compat_ioctl: None,
> +            check_events: None,
> +            unlock_native_capacity: None,
> +            getgeo: None,
> +            set_read_only: None,
> +            swap_slot_free_notify: None,
> +            report_zones: None,
> +            devnode: None,
> +            alternative_gpt_sector: None,
> +            get_unique_id: None,
> +            // TODO: Set to THIS_MODULE. Waiting for const_refs_to_static feature to be merged
> +            // https://github.com/rust-lang/rust/issues/119618
> +            owner: core::ptr::null_mut(),
> +            pr_ops: core::ptr::null_mut(),
> +            free_disk: None,
> +            poll_bio: None,
> +        };
> +
> +        // SAFETY: gendisk is a valid pointer as we initialized it above
> +        unsafe { (*gendisk).fops = &TABLE };
> +
> +        recover_data.dismiss();
> +        Ok(Self {
> +            _tagset: tagset,
> +            gendisk,

Missing INVARIANT comment.

> +        })
> +    }
> +
> +    /// Set the name of the device

Missing '.'.

> +    pub fn set_name(&mut self, args: fmt::Arguments<'_>) -> Result {
> +        let mut raw_writer = RawWriter::from_array(
> +            // SAFETY: By type invariant `self.gendisk` points to a valid and initialized instance
> +            unsafe { &mut (*self.gendisk).disk_name },

To create a `&mut` reference, you need exclusive access, it should be
sufficient to add to the invariant that `gendisk` is owned/unique.

> +        );
> +        raw_writer.write_fmt(args)?;
> +        raw_writer.write_char('\0')?;
> +        Ok(())
> +    }

[...]

> +impl<T: Operations> Drop for GenDisk<T> {
> +    fn drop(&mut self) {
> +        // SAFETY: By type invariant of `Self`, `self.gendisk` points to a valid
> +        // and initialized instance of `struct gendisk`. As such, `queuedata`
> +        // was initialized by the initializer returned by `try_new` with a call
> +        // to `ForeignOwnable::into_foreign`.

This should also be an invariant of `GenDisk`.

> +        let queue_data = unsafe { (*(*self.gendisk).queue).queuedata };
> +
> +        // SAFETY: By type invariant, `self.gendisk` points to a valid and
> +        // initialized instance of `struct gendisk`
> +        unsafe { bindings::del_gendisk(self.gendisk) };
> +
> +        // SAFETY: `queue.queuedata` was created by `GenDisk::try_new()` with a
> +        // call to `ForeignOwnable::into_pointer()` to create `queuedata`.
> +        // `ForeignOwnable::from_foreign()` is only called here.
> +        let _queue_data = unsafe { T::QueueData::from_foreign(queue_data) };
> +    }
> +}
> diff --git a/rust/kernel/block/mq/operations.rs b/rust/kernel/block/mq/operations.rs
> new file mode 100644
> index 000000000000..53c6ad663208
> --- /dev/null
> +++ b/rust/kernel/block/mq/operations.rs
> @@ -0,0 +1,346 @@
> +// SPDX-License-Identifier: GPL-2.0
> +
> +//! This module provides an interface for blk-mq drivers to implement.
> +//!
> +//! C header: [`include/linux/blk-mq.h`](srctree/include/linux/blk-mq.h)
> +
> +use crate::{
> +    bindings,
> +    block::mq::Request,
> +    error::{from_result, Result},
> +    init::PinInit,
> +    types::{ARef, ForeignOwnable},
> +};
> +use core::{marker::PhantomData, ptr::NonNull};
> +
> +use super::TagSet;
> +
> +/// Implement this trait to interface blk-mq as block devices
> +#[macros::vtable]
> +pub trait Operations: Sized {
> +    /// Data associated with a request. This data is located next to the request
> +    /// structure.
> +    ///
> +    /// To be able to handle accessing this data from interrupt context, this
> +    /// data must be `Sync`.
> +    type RequestData: Sized + Sync;
> +
> +    /// Initializer for `Self::RequestDta`. Used to initialize private data area
> +    /// when requst structure is allocated.
> +    type RequestDataInit: PinInit<Self::RequestData>;

Just to let you know, this dance with the associated types is not needed
any longer. RPITIT (return position impl trait in trait) has been
stabilized in 1.75 and you should be able to just write this:

     fn new_request_data(
         //rq: ARef<Request<Self>>,
         tagset_data: <Self::TagSetData as ForeignOwnable>::Borrowed<'_>,
     ) -> impl PinInit<Self::RequestData>;


> +
> +    /// Data associated with the `struct request_queue` that is allocated for
> +    /// the `GenDisk` associated with this `Operations` implementation.
> +    type QueueData: ForeignOwnable;
> +
> +    /// Data associated with a dispatch queue. This is stored as a pointer in
> +    /// the C `struct blk_mq_hw_ctx` that represents a hardware queue.
> +    type HwData: ForeignOwnable;
> +
> +    /// Data associated with a `TagSet`. This is stored as a pointer in `struct
> +    /// blk_mq_tag_set`.
> +    type TagSetData: ForeignOwnable;
> +
> +    /// Called by the kernel to get an initializer for a `Pin<&mut RequestData>`.
> +    fn new_request_data(
> +        //rq: ARef<Request<Self>>,
> +        tagset_data: <Self::TagSetData as ForeignOwnable>::Borrowed<'_>,

Since you need to use this pattern a lot, it might be a good idea to
introduce a type alias to help with this:

     type ForeignBorrowed<'a, T> = <T as ForeignOwnable>::Borrowed<'a>;

What do the others think?

The function would then become (with the RPITIT improvement as well):

     fn new_request_data(
         //rq: ARef<Request<Self>>,
         tagset_data: ForeignBorrowed<'_, Self::TagSetData>,
     ) -> impl PinInit<Self::RequestData>;


> +    ) -> Self::RequestDataInit;
> +
> +    /// Called by the kernel to queue a request with the driver. If `is_last` is
> +    /// `false`, the driver is allowed to defer commiting the request.
> +    fn queue_rq(
> +        hw_data: <Self::HwData as ForeignOwnable>::Borrowed<'_>,
> +        queue_data: <Self::QueueData as ForeignOwnable>::Borrowed<'_>,
> +        rq: ARef<Request<Self>>,
> +        is_last: bool,
> +    ) -> Result;
> +
> +    /// Called by the kernel to indicate that queued requests should be submitted
> +    fn commit_rqs(
> +        hw_data: <Self::HwData as ForeignOwnable>::Borrowed<'_>,
> +        queue_data: <Self::QueueData as ForeignOwnable>::Borrowed<'_>,
> +    );
> +
> +    /// Called by the kernel when the request is completed
> +    fn complete(_rq: &Request<Self>);
> +
> +    /// Called by the kernel to allocate and initialize a driver specific hardware context data
> +    fn init_hctx(
> +        tagset_data: <Self::TagSetData as ForeignOwnable>::Borrowed<'_>,
> +        hctx_idx: u32,
> +    ) -> Result<Self::HwData>;
> +
> +    /// Called by the kernel to poll the device for completed requests. Only
> +    /// used for poll queues.
> +    fn poll(_hw_data: <Self::HwData as ForeignOwnable>::Borrowed<'_>) -> bool {
> +        crate::build_error(crate::error::VTABLE_DEFAULT_ERROR)
> +    }
> +
> +    /// Called by the kernel to map submission queues to CPU cores.
> +    fn map_queues(_tag_set: &TagSet<Self>) {
> +        crate::build_error(crate::error::VTABLE_DEFAULT_ERROR)
> +    }
> +
> +    // There is no need for exit_request() because `drop` will be called.

I think it would be a good idea to mention this in the documentation of
the trait.

> +}

[...]

> diff --git a/rust/kernel/block/mq/raw_writer.rs b/rust/kernel/block/mq/raw_writer.rs
> new file mode 100644
> index 000000000000..f7857740af29
> --- /dev/null
> +++ b/rust/kernel/block/mq/raw_writer.rs
> @@ -0,0 +1,60 @@
> +use core::{
> +    fmt::{self, Write},
> +    marker::PhantomData,
> +};
> +
> +/// A mutable reference to a byte buffer where a string can be written into
> +///
> +/// # Invariants
> +///
> +/// * `ptr` is not aliased and valid for read and write for `len` bytes

You probably also want to add "for the duration of `'a`".

> +///
> +pub(crate) struct RawWriter<'a> {
> +    ptr: *mut u8,
> +    len: usize,
> +    _p: PhantomData<&'a ()>,
> +}
> +
> +impl<'a> RawWriter<'a> {
> +    /// Create a new `RawWriter` instance.
> +    ///
> +    /// # Safety
> +    ///
> +    /// * `ptr` must be valid for read and write for `len` consecutive `u8` elements
> +    /// * `ptr` must not be aliased
> +    unsafe fn new(ptr: *mut u8, len: usize) -> RawWriter<'a> {
> +        Self {
> +            ptr,
> +            len,
> +            _p: PhantomData,
> +        }
> +    }

Since this function is not used (except in the function below), what is
the reason for using a raw pointer?
I looked in your other patches, but did not find another user, so could
this be improved by using mutable references?

> +
> +    pub(crate) fn from_array<const N: usize>(a: &'a mut [core::ffi::c_char; N]) -> RawWriter<'a> {
> +        // SAFETY: the buffer of `a` is valid for read and write for at least `N` bytes
> +        unsafe { Self::new(a.as_mut_ptr().cast::<u8>(), N) }
> +    }
> +}
> +
> +impl Write for RawWriter<'_> {
> +    fn write_str(&mut self, s: &str) -> fmt::Result {
> +        let bytes = s.as_bytes();
> +        let len = bytes.len();
> +        if len > self.len {
> +            return Err(fmt::Error);
> +        }
> +
> +        // SAFETY:
> +        // * `bytes` is valid for reads of `bytes.len()` size because we hold a shared reference to `s`
> +        // * By type invariant `self.ptr` is valid for writes for at lest `self.len` bytes
> +        // * The regions are not overlapping as `ptr` is not aliased
> +        unsafe { core::ptr::copy_nonoverlapping(&bytes[0], self.ptr, len) };
> +
> +        // SAFETY: By type invariant of `Self`, `ptr` is in bounds of an
> +        // allocation. Also by type invariant, the pointer resulting from this
> +        // addition is also in bounds.
> +        self.ptr = unsafe { self.ptr.add(len) };
> +        self.len -= len;
> +        Ok(())
> +    }
> +}
> diff --git a/rust/kernel/block/mq/request.rs b/rust/kernel/block/mq/request.rs
> new file mode 100644
> index 000000000000..b4dacac5e091
> --- /dev/null
> +++ b/rust/kernel/block/mq/request.rs
> @@ -0,0 +1,182 @@
> +// SPDX-License-Identifier: GPL-2.0
> +
> +//! This module provides a wrapper for the C `struct request` type.
> +//!
> +//! C header: [`include/linux/blk-mq.h`](srctree/include/linux/blk-mq.h)
> +
> +use crate::{
> +    bindings,
> +    block::mq::Operations,
> +    error::{Error, Result},
> +    types::{ARef, AlwaysRefCounted, Opaque},
> +};
> +use core::{ffi::c_void, marker::PhantomData, ops::Deref};
> +
> +/// A wrapper around a blk-mq `struct request`. This represents an IO request.
> +///
> +/// # Invariants
> +///
> +/// * `self.0` is a valid `struct request` created by the C portion of the kernel
> +/// * `self` is reference counted. a call to `req_ref_inc_not_zero` keeps the
> +///    instance alive at least until a matching call to `req_ref_put_and_test`
> +///
> +#[repr(transparent)]
> +pub struct Request<T: Operations>(Opaque<bindings::request>, PhantomData<T>);
> +
> +impl<T: Operations> Request<T> {
> +    /// Create a `&mut Request` from a `bindings::request` pointer
> +    ///
> +    /// # Safety
> +    ///
> +    /// * `ptr` must be aligned and point to a valid `bindings::request` instance
> +    /// * Caller must ensure that the pointee of `ptr` is live and owned
> +    ///   exclusively by caller for at least `'a`
> +    ///
> +    pub(crate) unsafe fn from_ptr_mut<'a>(ptr: *mut bindings::request) -> &'a mut Self {
> +        // SAFETY:
> +        // * The cast is valid as `Self` is transparent.
> +        // * By safety requirements of this function, the reference will be
> +        //   valid for 'a.
> +        unsafe { &mut *(ptr.cast::<Self>()) }
> +    }
> +
> +    /// Get the command identifier for the request
> +    pub fn command(&self) -> u32 {
> +        // SAFETY: By C API contract and type invariant, `cmd_flags` is valid for read
> +        unsafe { (*self.0.get()).cmd_flags & ((1 << bindings::REQ_OP_BITS) - 1) }
> +    }
> +
> +    /// Call this to indicate to the kernel that the request has been issued by the driver

I am a bit confused, is this not supposed to signal that the processing
of the request is going to start now? cf C documentation:

     /**
      * blk_mq_start_request - Start processing a request
      * @rq: Pointer to request to be started
      *
      * Function used by device drivers to notify the block layer that a request
      * is going to be processed now, so blk layer can do proper initializations
      * such as starting the timeout timer.
      */

> +    pub fn start(&self) {
> +        // SAFETY: By type invariant, `self.0` is a valid `struct request`. By
> +        // existence of `&mut self` we have exclusive access.
> +        unsafe { bindings::blk_mq_start_request(self.0.get()) };
> +    }
> +
> +    /// Call this to indicate to the kernel that the request has been completed without errors

I dislike the "Call this", it feels redundant, what about "Signal the
block layer that the request has been completed without errors.".

Also with the other functions below.

> +    pub fn end_ok(&self) {
> +        // SAFETY: By type invariant, `self.0` is a valid `struct request`. By
> +        // existence of `&mut self` we have exclusive access.
> +        unsafe { bindings::blk_mq_end_request(self.0.get(), bindings::BLK_STS_OK as _) };
> +    }
> +
> +    /// Call this to indicate to the kernel that the request completed with an error
> +    pub fn end_err(&self, err: Error) {
> +        // SAFETY: By type invariant, `self.0` is a valid `struct request`. By
> +        // existence of `&mut self` we have exclusive access.
> +        unsafe { bindings::blk_mq_end_request(self.0.get(), err.to_blk_status()) };
> +    }
> +
> +    /// Call this to indicate that the request completed with the status indicated by `status`
> +    pub fn end(&self, status: Result) {
> +        if let Err(e) = status {
> +            self.end_err(e);
> +        } else {
> +            self.end_ok();
> +        }
> +    }
> +
> +    /// Call this to schedule defered completion of the request
> +    pub fn complete(&self) {
> +        // SAFETY: By type invariant, `self.0` is a valid `struct request`
> +        if !unsafe { bindings::blk_mq_complete_request_remote(self.0.get()) } {
> +            T::complete(self);
> +        }
> +    }
> +
> +    /// Get the target sector for the request
> +    #[inline(always)]
> +    pub fn sector(&self) -> usize {
> +        // SAFETY: By type invariant of `Self`, `self.0` is valid and live.
> +        unsafe { (*self.0.get()).__sector as usize }
> +    }
> +
> +    /// Returns an owned reference to the per-request data associated with this
> +    /// request
> +    pub fn owned_data_ref(request: ARef<Self>) -> RequestDataRef<T> {
> +        RequestDataRef::new(request)
> +    }
> +
> +    /// Returns a reference to the oer-request data associated with this request

Typo: "the oer-request"

> +    pub fn data_ref(&self) -> &T::RequestData {
> +        let request_ptr = self.0.get().cast::<bindings::request>();
> +
> +        // SAFETY: `request_ptr` is a valid `struct request` because `ARef` is
> +        // `repr(transparent)`
> +        let p: *mut c_void = unsafe { bindings::blk_mq_rq_to_pdu(request_ptr) };
> +
> +        let p = p.cast::<T::RequestData>();
> +
> +        // SAFETY: By C API contract, `p` is initialized by a call to
> +        // `OperationsVTable::init_request_callback()`. By existence of `&self`
> +        // it must be valid for use as a shared reference.
> +        unsafe { &*p }
> +    }
> +}
> +
> +// SAFETY: It is impossible to obtain an owned or mutable `Request`, so we can

What do you mean by "mutable `Request`"? There is the function to obtain
a `&mut Request`.
Also this should probably be an invariant if you use it as a
justification here.

> +// mark it `Send`.
> +unsafe impl<T: Operations> Send for Request<T> {}
> +
> +// SAFETY: `Request` references can be shared across threads.

Does not explain why that is the case.

> +unsafe impl<T: Operations> Sync for Request<T> {}
> +
> +/// An owned reference to a `Request<T>`
> +#[repr(transparent)]
> +pub struct RequestDataRef<T: Operations> {
> +    request: ARef<Request<T>>,
> +}

Is this extra type really needed? I have not yet taken a look at patch 3,
which uses this. But would it hurt if you implemented the traits
there directly on `ARef<Request<T>>`?

> +
> +impl<T> RequestDataRef<T>
> +where
> +    T: Operations,
> +{
> +    /// Create a new instance.
> +    fn new(request: ARef<Request<T>>) -> Self {
> +        Self { request }
> +    }
> +
> +    /// Get a reference to the underlying request
> +    pub fn request(&self) -> &Request<T> {
> +        &self.request
> +    }
> +}

I really like how you improved the safety comments and documentation. It
was a lot easier to wrap my head around this time (might also me getting
more experienced, but I think it helped a lot).

-- 
Cheers,
Benno


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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-22 23:40 [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module Benno Lossin
@ 2024-03-23  6:32 ` Andreas Hindborg
  2024-04-02 23:09   ` Benno Lossin
  0 siblings, 1 reply; 26+ messages in thread
From: Andreas Hindborg @ 2024-03-23  6:32 UTC (permalink / raw)
  To: Benno Lossin
  Cc: Jens Axboe, Christoph Hellwig, Keith Busch, Damien Le Moal,
	Bart Van Assche, Hannes Reinecke, linux-block@vger.kernel.org,
	Andreas Hindborg, Wedson Almeida Filho, Niklas Cassel, Greg KH,
	Matthew Wilcox, Miguel Ojeda, Alex Gaynor, Boqun Feng, Gary Guo,
	Björn Roy Baron, Alice Ryhl, Chaitanya Kulkarni,
	Luis Chamberlain, Yexuan Yang, Sergio González Collado,
	Joel Granados, Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

Benno Lossin <benno.lossin@proton.me> writes:

> On 3/13/24 12:05, Andreas Hindborg wrote:
>> From: Andreas Hindborg <a.hindborg@samsung.com>
>> 
>> Add initial abstractions for working with blk-mq.
>> 
>> This patch is a maintained, refactored subset of code originally published by
>> Wedson Almeida Filho <wedsonaf@gmail.com> [1].
>> 
>> [1] https://github.com/wedsonaf/linux/tree/f2cfd2fe0e2ca4e90994f96afe268bbd4382a891/rust/kernel/blk/mq.rs
>> 
>> Cc: Wedson Almeida Filho <wedsonaf@gmail.com>
>> Signed-off-by: Andreas Hindborg <a.hindborg@samsung.com>
>> ---
>>   block/blk-mq.c                     |   3 +-
>>   include/linux/blk-mq.h             |   1 +
>>   rust/bindings/bindings_helper.h    |   2 +
>>   rust/helpers.c                     |  45 ++++
>>   rust/kernel/block.rs               |   5 +
>>   rust/kernel/block/mq.rs            | 131 +++++++++++
>>   rust/kernel/block/mq/gen_disk.rs   | 174 +++++++++++++++
>>   rust/kernel/block/mq/operations.rs | 346 +++++++++++++++++++++++++++++
>>   rust/kernel/block/mq/raw_writer.rs |  60 +++++
>>   rust/kernel/block/mq/request.rs    | 182 +++++++++++++++
>>   rust/kernel/block/mq/tag_set.rs    | 117 ++++++++++
>>   rust/kernel/error.rs               |   5 +
>>   rust/kernel/lib.rs                 |   1 +
>>   13 files changed, 1071 insertions(+), 1 deletion(-)
>
> Do you think that it's possible to split this into smaller
> patches? It would make review a lot easier.

Probably, I'll look into that.

>
>>   create mode 100644 rust/kernel/block.rs
>>   create mode 100644 rust/kernel/block/mq.rs
>>   create mode 100644 rust/kernel/block/mq/gen_disk.rs
>>   create mode 100644 rust/kernel/block/mq/operations.rs
>>   create mode 100644 rust/kernel/block/mq/raw_writer.rs
>>   create mode 100644 rust/kernel/block/mq/request.rs
>>   create mode 100644 rust/kernel/block/mq/tag_set.rs
>
> [...]
>
>> diff --git a/rust/kernel/block.rs b/rust/kernel/block.rs
>> new file mode 100644
>> index 000000000000..4c93317a568a
>> --- /dev/null
>> +++ b/rust/kernel/block.rs
>> @@ -0,0 +1,5 @@
>> +// SPDX-License-Identifier: GPL-2.0
>> +
>> +//! Types for working with the block layer
>
> Missing '.'.

👍

>
>> +
>> +pub mod mq;
>> diff --git a/rust/kernel/block/mq.rs b/rust/kernel/block/mq.rs
>> new file mode 100644
>> index 000000000000..08de1cc114ff
>> --- /dev/null
>> +++ b/rust/kernel/block/mq.rs
>> @@ -0,0 +1,131 @@
>> +// SPDX-License-Identifier: GPL-2.0
>> +
>> +//! This module provides types for implementing block drivers that interface the
>> +//! blk-mq subsystem.
>> +//!
>> +//! To implement a block device driver, a Rust module must do the following:
>> +//!
>> +//! - Implement [`Operations`] for a type `T`
>
> I think it would be nicer to use `Driver` (or `MyBlkDevice`) instead of
> `T`.

I think I like `T` better, because it signals placeholder more
effectively.

>
>> +//! - Create a [`TagSet<T>`]
>> +//! - Create a [`GenDisk<T>`], passing in the `TagSet` reference
>> +//! - Add the disk to the system by calling [`GenDisk::add`]
>> +//!
>> +//! The types available in this module that have direct C counterparts are:
>> +//!
>> +//! - The `TagSet` type that abstracts the C type `struct tag_set`.
>> +//! - The `GenDisk` type that abstracts the C type `struct gendisk`.
>> +//! - The `Request` type that abstracts the C type `struct request`.
>> +//!
>> +//! Many of the C types that this module abstracts allow a driver to carry
>> +//! private data, either embedded in the stuct directly, or as a C `void*`. In
>> +//! these abstractions, this data is typed. The types of the data is defined by
>> +//! associated types in `Operations`, see [`Operations::RequestData`] for an
>> +//! example.
>> +//!
>> +//! The kernel will interface with the block evice driver by calling the method
>
> Typo: "block evice driver"

Thanks.

>
>> +//! implementations of the `Operations` trait.
>> +//!
>> +//! IO requests are passed to the driver as [`Request`] references. The
>> +//! `Request` type is a wrapper around the C `struct request`. The driver must
>> +//! mark start of request processing by calling [`Request::start`] and end of
>> +//! processing by calling one of the [`Request::end`], methods. Failure to do so
>> +//! can lead to IO failures.
>
> I am unfamiliar with this, what are "IO failures"?
> Do you think that it might be better to change the API to use a
> callback? So instead of calling start and end, you would do
>
>      request.handle(|req| {
>          // do the stuff that would be done between start and end
>      });
>
> I took a quick look at the rnull driver and there you are calling
> `Request::end_ok` from a different function. So my suggestion might not
> be possible, since you really need the freedom.
>
> Do you think that a guard approach might work better? ie `start` returns
> a guard that when dropped will call `end` and you need the guard to
> operate on the request.

I don't think that would fit, since the driver might not complete the
request immediately. We might be able to call `start` on behalf of the
driver.

At any rate, since the request is reference counted now, we can
automatically fail a request when the last reference is dropped and it
was not marked successfully completed. I would need to measure the
performance implications of such a feature.

The comment needs update too. Failure to complete requests will lead to
either deadlock or timeout errors, depending on configuration.

>
>> +//!
>> +//! The `TagSet` is responsible for creating and maintaining a mapping between
>> +//! `Request`s and integer ids as well as carrying a pointer to the vtable
>> +//! generated by `Operations`. This mapping is useful for associating
>> +//! completions from hardware with the correct `Request` instance. The `TagSet`
>> +//! determines the maximum queue depth by setting the number of `Request`
>> +//! instances available to the driver, and it determines the number of queues to
>> +//! instantiate for the driver. If possible, a driver should allocate one queue
>> +//! per core, to keep queue data local to a core.
>> +//!
>> +//! One `TagSet` instance can be shared between multiple `GenDisk` instances.
>> +//! This can be useful when implementing drivers where one piece of hardware
>> +//! with one set of IO resources are represented to the user as multiple disks.
>> +//!
>> +//! One significant difference between block device drivers implemented with
>> +//! these Rust abstractions and drivers implemented in C, is that the Rust
>> +//! drivers have to own a reference count on the `Request` type when the IO is
>> +//! in flight. This is to ensure that the C `struct request` instances backing
>> +//! the Rust `Request` instances are live while the Rust driver holds a
>> +//! reference to the `Request`. In addition, the conversion of an ineger tag to
>
> Typo: "of an ineger tag"

Thanks. Looks like I need to properly configure my editor to spell check
in comments.

>
>> +//! a `Request` via the `TagSet` would not be sound without this bookkeeping.
>> +//!
>> +//! # ⚠ Note
>> +//!
>> +//! For Rust block device drivers, the point in time where a request
>> +//! is freed and made available for recycling is usualy at the point in time
>> +//! when the last `ARef<Request>` is dropped. For C drivers, this event usually
>> +//! occurs when `bindings::blk_mq_end_request` is called.
>> +//!
>> +//! # Example
>> +//!
>> +//! ```rust
>> +//! use kernel::{
>> +//!     block::mq::*,
>> +//!     new_mutex,
>> +//!     prelude::*,
>> +//!     sync::{Arc, Mutex},
>> +//!     types::{ARef, ForeignOwnable},
>> +//! };
>> +//!
>> +//! struct MyBlkDevice;
>> +//!
>> +//! #[vtable]
>> +//! impl Operations for MyBlkDevice {
>> +//!     type RequestData = ();
>> +//!     type RequestDataInit = impl PinInit<()>;
>> +//!     type QueueData = ();
>> +//!     type HwData = ();
>> +//!     type TagSetData = ();
>> +//!
>> +//!     fn new_request_data(
>> +//!         _tagset_data: <Self::TagSetData as ForeignOwnable>::Borrowed<'_>,
>> +//!     ) -> Self::RequestDataInit {
>> +//!         kernel::init::zeroed()
>> +//!     }
>> +//!
>> +//!     fn queue_rq(_hw_data: (), _queue_data: (), rq: ARef<Request<Self>>, _is_last: bool) -> Result {
>> +//!         rq.start();
>> +//!         rq.end_ok();
>> +//!         Ok(())
>> +//!     }
>> +//!
>> +//!     fn commit_rqs(
>> +//!         _hw_data: <Self::HwData as ForeignOwnable>::Borrowed<'_>,
>> +//!         _queue_data: <Self::QueueData as ForeignOwnable>::Borrowed<'_>,
>> +//!     ) {
>> +//!     }
>> +//!
>> +//!     fn complete(rq: &Request<Self>) {
>> +//!         rq.end_ok();
>> +//!     }
>> +//!
>> +//!     fn init_hctx(
>> +//!         _tagset_data: <Self::TagSetData as ForeignOwnable>::Borrowed<'_>,
>> +//!         _hctx_idx: u32,
>> +//!     ) -> Result<Self::HwData> {
>> +//!         Ok(())
>> +//!     }
>> +//! }
>> +//!
>> +//! let tagset: Arc<TagSet<MyBlkDevice>> = Arc::pin_init(TagSet::try_new(1, (), 256, 1))?;
>> +//! let mut disk = GenDisk::try_new(tagset, ())?;
>> +//! disk.set_name(format_args!("myblk"))?;
>> +//! disk.set_capacity_sectors(4096);
>> +//! disk.add()?;
>> +//!
>> +//! # Ok::<(), kernel::error::Error>(())
>> +//! ```
>
> This piece of documentation is **really** valuable, thanks a lot for
> taking the time to write it.

Great!

>
>> +
>> +mod gen_disk;
>> +mod operations;
>> +mod raw_writer;
>> +mod request;
>> +mod tag_set;
>> +
>> +pub use gen_disk::GenDisk;
>> +pub use operations::Operations;
>> +pub use request::{Request, RequestDataRef};
>> +pub use tag_set::TagSet;
>> diff --git a/rust/kernel/block/mq/gen_disk.rs b/rust/kernel/block/mq/gen_disk.rs
>> new file mode 100644
>> index 000000000000..b7845fc9e39f
>> --- /dev/null
>> +++ b/rust/kernel/block/mq/gen_disk.rs
>> @@ -0,0 +1,174 @@
>> +// SPDX-License-Identifier: GPL-2.0
>> +
>> +//! Generic disk abstraction.
>> +//!
>> +//! C header: [`include/linux/blkdev.h`](srctree/include/linux/blkdev.h)
>> +//! C header: [`include/linux/blk_mq.h`](srctree/include/linux/blk_mq.h)
>> +
>> +use crate::block::mq::{raw_writer::RawWriter, Operations, TagSet};
>> +use crate::{
>> +    bindings, error::from_err_ptr, error::Result, sync::Arc, types::ForeignOwnable,
>> +    types::ScopeGuard,
>> +};
>> +use core::fmt::{self, Write};
>> +
>> +/// A generic block device
>> +///
>> +/// # Invariants
>> +///
>> +///  - `gendisk` must always point to an initialized and valid `struct gendisk`.
>> +pub struct GenDisk<T: Operations> {
>> +    _tagset: Arc<TagSet<T>>,
>> +    gendisk: *mut bindings::gendisk,
>> +}
>> +
>> +// SAFETY: `GenDisk` is an owned pointer to a `struct gendisk` and an `Arc` to a
>> +// `TagSet` It is safe to send this to other threads as long as T is Send.
>> +unsafe impl<T: Operations + Send> Send for GenDisk<T> {}
>> +
>> +impl<T: Operations> GenDisk<T> {
>> +    /// Try to create a new `GenDisk`
>> +    pub fn try_new(tagset: Arc<TagSet<T>>, queue_data: T::QueueData) -> Result<Self> {
>> +        let data = queue_data.into_foreign();
>> +        let recover_data = ScopeGuard::new(|| {
>> +            // SAFETY: T::QueueData was created by the call to `into_foreign()` above
>> +            unsafe { T::QueueData::from_foreign(data) };
>> +        });
>> +
>> +        let lock_class_key = crate::sync::LockClassKey::new();
>> +
>> +        // SAFETY: `tagset.raw_tag_set()` points to a valid and initialized tag set
>> +        let gendisk = from_err_ptr(unsafe {
>> +            bindings::__blk_mq_alloc_disk(
>> +                tagset.raw_tag_set(),
>> +                data.cast_mut(),
>> +                lock_class_key.as_ptr(),
>> +            )
>> +        })?;
>> +
>> +        const TABLE: bindings::block_device_operations = bindings::block_device_operations {
>> +            submit_bio: None,
>> +            open: None,
>> +            release: None,
>> +            ioctl: None,
>> +            compat_ioctl: None,
>> +            check_events: None,
>> +            unlock_native_capacity: None,
>> +            getgeo: None,
>> +            set_read_only: None,
>> +            swap_slot_free_notify: None,
>> +            report_zones: None,
>> +            devnode: None,
>> +            alternative_gpt_sector: None,
>> +            get_unique_id: None,
>> +            // TODO: Set to THIS_MODULE. Waiting for const_refs_to_static feature to be merged
>> +            // https://github.com/rust-lang/rust/issues/119618
>> +            owner: core::ptr::null_mut(),
>> +            pr_ops: core::ptr::null_mut(),
>> +            free_disk: None,
>> +            poll_bio: None,
>> +        };
>> +
>> +        // SAFETY: gendisk is a valid pointer as we initialized it above
>> +        unsafe { (*gendisk).fops = &TABLE };
>> +
>> +        recover_data.dismiss();
>> +        Ok(Self {
>> +            _tagset: tagset,
>> +            gendisk,
>
> Missing INVARIANT comment.

Will fix.

>
>> +        })
>> +    }
>> +
>> +    /// Set the name of the device
>
> Missing '.'.

Thanks.

>
>> +    pub fn set_name(&mut self, args: fmt::Arguments<'_>) -> Result {
>> +        let mut raw_writer = RawWriter::from_array(
>> +            // SAFETY: By type invariant `self.gendisk` points to a valid and initialized instance
>> +            unsafe { &mut (*self.gendisk).disk_name },
>
> To create a `&mut` reference, you need exclusive access, it should be
> sufficient to add to the invariant that `gendisk` is owned/unique.

Hmm, we don't actually _always_ have unique ownership of this string
buffer. I will change the API to only allow configuration of the
instance before it is hooked in. Thanks for spotting this.

>
>> +        );
>> +        raw_writer.write_fmt(args)?;
>> +        raw_writer.write_char('\0')?;
>> +        Ok(())
>> +    }
>
> [...]
>
>> +impl<T: Operations> Drop for GenDisk<T> {
>> +    fn drop(&mut self) {
>> +        // SAFETY: By type invariant of `Self`, `self.gendisk` points to a valid
>> +        // and initialized instance of `struct gendisk`. As such, `queuedata`
>> +        // was initialized by the initializer returned by `try_new` with a call
>> +        // to `ForeignOwnable::into_foreign`.
>
> This should also be an invariant of `GenDisk`.

Ok.

>
>> +        let queue_data = unsafe { (*(*self.gendisk).queue).queuedata };
>> +
>> +        // SAFETY: By type invariant, `self.gendisk` points to a valid and
>> +        // initialized instance of `struct gendisk`
>> +        unsafe { bindings::del_gendisk(self.gendisk) };
>> +
>> +        // SAFETY: `queue.queuedata` was created by `GenDisk::try_new()` with a
>> +        // call to `ForeignOwnable::into_pointer()` to create `queuedata`.
>> +        // `ForeignOwnable::from_foreign()` is only called here.
>> +        let _queue_data = unsafe { T::QueueData::from_foreign(queue_data) };
>> +    }
>> +}
>> diff --git a/rust/kernel/block/mq/operations.rs b/rust/kernel/block/mq/operations.rs
>> new file mode 100644
>> index 000000000000..53c6ad663208
>> --- /dev/null
>> +++ b/rust/kernel/block/mq/operations.rs
>> @@ -0,0 +1,346 @@
>> +// SPDX-License-Identifier: GPL-2.0
>> +
>> +//! This module provides an interface for blk-mq drivers to implement.
>> +//!
>> +//! C header: [`include/linux/blk-mq.h`](srctree/include/linux/blk-mq.h)
>> +
>> +use crate::{
>> +    bindings,
>> +    block::mq::Request,
>> +    error::{from_result, Result},
>> +    init::PinInit,
>> +    types::{ARef, ForeignOwnable},
>> +};
>> +use core::{marker::PhantomData, ptr::NonNull};
>> +
>> +use super::TagSet;
>> +
>> +/// Implement this trait to interface blk-mq as block devices
>> +#[macros::vtable]
>> +pub trait Operations: Sized {
>> +    /// Data associated with a request. This data is located next to the request
>> +    /// structure.
>> +    ///
>> +    /// To be able to handle accessing this data from interrupt context, this
>> +    /// data must be `Sync`.
>> +    type RequestData: Sized + Sync;
>> +
>> +    /// Initializer for `Self::RequestDta`. Used to initialize private data area
>> +    /// when requst structure is allocated.
>> +    type RequestDataInit: PinInit<Self::RequestData>;
>
> Just to let you know, this dance with the associated types is not needed
> any longer. RPITIT (return position impl trait in trait) has been
> stabilized in 1.75 and you should be able to just write this:
>
>      fn new_request_data(
>          //rq: ARef<Request<Self>>,
>          tagset_data: <Self::TagSetData as ForeignOwnable>::Borrowed<'_>,
>      ) -> impl PinInit<Self::RequestData>;
>

Thanks 👍

>
>> +
>> +    /// Data associated with the `struct request_queue` that is allocated for
>> +    /// the `GenDisk` associated with this `Operations` implementation.
>> +    type QueueData: ForeignOwnable;
>> +
>> +    /// Data associated with a dispatch queue. This is stored as a pointer in
>> +    /// the C `struct blk_mq_hw_ctx` that represents a hardware queue.
>> +    type HwData: ForeignOwnable;
>> +
>> +    /// Data associated with a `TagSet`. This is stored as a pointer in `struct
>> +    /// blk_mq_tag_set`.
>> +    type TagSetData: ForeignOwnable;
>> +
>> +    /// Called by the kernel to get an initializer for a `Pin<&mut RequestData>`.
>> +    fn new_request_data(
>> +        //rq: ARef<Request<Self>>,
>> +        tagset_data: <Self::TagSetData as ForeignOwnable>::Borrowed<'_>,
>
> Since you need to use this pattern a lot, it might be a good idea to
> introduce a type alias to help with this:
>
>      type ForeignBorrowed<'a, T> = <T as ForeignOwnable>::Borrowed<'a>;
>
> What do the others think?
>
> The function would then become (with the RPITIT improvement as well):
>
>      fn new_request_data(
>          //rq: ARef<Request<Self>>,
>          tagset_data: ForeignBorrowed<'_, Self::TagSetData>,
>      ) -> impl PinInit<Self::RequestData>;

A bit more concise, I think it is better. I'll go ahead and kill that
commented out argument that sneaked into the patch as well :)

>
>
>> +    ) -> Self::RequestDataInit;
>> +
>> +    /// Called by the kernel to queue a request with the driver. If `is_last` is
>> +    /// `false`, the driver is allowed to defer commiting the request.
>> +    fn queue_rq(
>> +        hw_data: <Self::HwData as ForeignOwnable>::Borrowed<'_>,
>> +        queue_data: <Self::QueueData as ForeignOwnable>::Borrowed<'_>,
>> +        rq: ARef<Request<Self>>,
>> +        is_last: bool,
>> +    ) -> Result;
>> +
>> +    /// Called by the kernel to indicate that queued requests should be submitted
>> +    fn commit_rqs(
>> +        hw_data: <Self::HwData as ForeignOwnable>::Borrowed<'_>,
>> +        queue_data: <Self::QueueData as ForeignOwnable>::Borrowed<'_>,
>> +    );
>> +
>> +    /// Called by the kernel when the request is completed
>> +    fn complete(_rq: &Request<Self>);
>> +
>> +    /// Called by the kernel to allocate and initialize a driver specific hardware context data
>> +    fn init_hctx(
>> +        tagset_data: <Self::TagSetData as ForeignOwnable>::Borrowed<'_>,
>> +        hctx_idx: u32,
>> +    ) -> Result<Self::HwData>;
>> +
>> +    /// Called by the kernel to poll the device for completed requests. Only
>> +    /// used for poll queues.
>> +    fn poll(_hw_data: <Self::HwData as ForeignOwnable>::Borrowed<'_>) -> bool {
>> +        crate::build_error(crate::error::VTABLE_DEFAULT_ERROR)
>> +    }
>> +
>> +    /// Called by the kernel to map submission queues to CPU cores.
>> +    fn map_queues(_tag_set: &TagSet<Self>) {
>> +        crate::build_error(crate::error::VTABLE_DEFAULT_ERROR)
>> +    }
>> +
>> +    // There is no need for exit_request() because `drop` will be called.
>
> I think it would be a good idea to mention this in the documentation of
> the trait.

Yes.

>
>> +}
>
> [...]
>
>> diff --git a/rust/kernel/block/mq/raw_writer.rs b/rust/kernel/block/mq/raw_writer.rs
>> new file mode 100644
>> index 000000000000..f7857740af29
>> --- /dev/null
>> +++ b/rust/kernel/block/mq/raw_writer.rs
>> @@ -0,0 +1,60 @@
>> +use core::{
>> +    fmt::{self, Write},
>> +    marker::PhantomData,
>> +};
>> +
>> +/// A mutable reference to a byte buffer where a string can be written into
>> +///
>> +/// # Invariants
>> +///
>> +/// * `ptr` is not aliased and valid for read and write for `len` bytes
>
> You probably also want to add "for the duration of `'a`".

👍

>
>> +///
>> +pub(crate) struct RawWriter<'a> {
>> +    ptr: *mut u8,
>> +    len: usize,
>> +    _p: PhantomData<&'a ()>,
>> +}
>> +
>> +impl<'a> RawWriter<'a> {
>> +    /// Create a new `RawWriter` instance.
>> +    ///
>> +    /// # Safety
>> +    ///
>> +    /// * `ptr` must be valid for read and write for `len` consecutive `u8` elements
>> +    /// * `ptr` must not be aliased
>> +    unsafe fn new(ptr: *mut u8, len: usize) -> RawWriter<'a> {
>> +        Self {
>> +            ptr,
>> +            len,
>> +            _p: PhantomData,
>> +        }
>> +    }
>
> Since this function is not used (except in the function below), what is
> the reason for using a raw pointer?
> I looked in your other patches, but did not find another user, so could
> this be improved by using mutable references?

I am not sure. But I think the code could benefit from getting passed a
mutable slice instead, move the safety comment to the call site. I will
try that.

>
>> +
>> +    pub(crate) fn from_array<const N: usize>(a: &'a mut [core::ffi::c_char; N]) -> RawWriter<'a> {
>> +        // SAFETY: the buffer of `a` is valid for read and write for at least `N` bytes
>> +        unsafe { Self::new(a.as_mut_ptr().cast::<u8>(), N) }
>> +    }
>> +}
>> +
>> +impl Write for RawWriter<'_> {
>> +    fn write_str(&mut self, s: &str) -> fmt::Result {
>> +        let bytes = s.as_bytes();
>> +        let len = bytes.len();
>> +        if len > self.len {
>> +            return Err(fmt::Error);
>> +        }
>> +
>> +        // SAFETY:
>> +        // * `bytes` is valid for reads of `bytes.len()` size because we hold a shared reference to `s`
>> +        // * By type invariant `self.ptr` is valid for writes for at lest `self.len` bytes
>> +        // * The regions are not overlapping as `ptr` is not aliased
>> +        unsafe { core::ptr::copy_nonoverlapping(&bytes[0], self.ptr, len) };
>> +
>> +        // SAFETY: By type invariant of `Self`, `ptr` is in bounds of an
>> +        // allocation. Also by type invariant, the pointer resulting from this
>> +        // addition is also in bounds.
>> +        self.ptr = unsafe { self.ptr.add(len) };
>> +        self.len -= len;
>> +        Ok(())
>> +    }
>> +}
>> diff --git a/rust/kernel/block/mq/request.rs b/rust/kernel/block/mq/request.rs
>> new file mode 100644
>> index 000000000000..b4dacac5e091
>> --- /dev/null
>> +++ b/rust/kernel/block/mq/request.rs
>> @@ -0,0 +1,182 @@
>> +// SPDX-License-Identifier: GPL-2.0
>> +
>> +//! This module provides a wrapper for the C `struct request` type.
>> +//!
>> +//! C header: [`include/linux/blk-mq.h`](srctree/include/linux/blk-mq.h)
>> +
>> +use crate::{
>> +    bindings,
>> +    block::mq::Operations,
>> +    error::{Error, Result},
>> +    types::{ARef, AlwaysRefCounted, Opaque},
>> +};
>> +use core::{ffi::c_void, marker::PhantomData, ops::Deref};
>> +
>> +/// A wrapper around a blk-mq `struct request`. This represents an IO request.
>> +///
>> +/// # Invariants
>> +///
>> +/// * `self.0` is a valid `struct request` created by the C portion of the kernel
>> +/// * `self` is reference counted. a call to `req_ref_inc_not_zero` keeps the
>> +///    instance alive at least until a matching call to `req_ref_put_and_test`
>> +///
>> +#[repr(transparent)]
>> +pub struct Request<T: Operations>(Opaque<bindings::request>, PhantomData<T>);
>> +
>> +impl<T: Operations> Request<T> {
>> +    /// Create a `&mut Request` from a `bindings::request` pointer
>> +    ///
>> +    /// # Safety
>> +    ///
>> +    /// * `ptr` must be aligned and point to a valid `bindings::request` instance
>> +    /// * Caller must ensure that the pointee of `ptr` is live and owned
>> +    ///   exclusively by caller for at least `'a`
>> +    ///
>> +    pub(crate) unsafe fn from_ptr_mut<'a>(ptr: *mut bindings::request) -> &'a mut Self {
>> +        // SAFETY:
>> +        // * The cast is valid as `Self` is transparent.
>> +        // * By safety requirements of this function, the reference will be
>> +        //   valid for 'a.
>> +        unsafe { &mut *(ptr.cast::<Self>()) }
>> +    }
>> +
>> +    /// Get the command identifier for the request
>> +    pub fn command(&self) -> u32 {
>> +        // SAFETY: By C API contract and type invariant, `cmd_flags` is valid for read
>> +        unsafe { (*self.0.get()).cmd_flags & ((1 << bindings::REQ_OP_BITS) - 1) }
>> +    }
>> +
>> +    /// Call this to indicate to the kernel that the request has been issued by the driver
>
> I am a bit confused, is this not supposed to signal that the processing
> of the request is going to start now? cf C documentation:
>
>      /**
>       * blk_mq_start_request - Start processing a request
>       * @rq: Pointer to request to be started
>       *
>       * Function used by device drivers to notify the block layer that a request
>       * is going to be processed now, so blk layer can do proper initializations
>       * such as starting the timeout timer.
>       */

Yes, it is to indicate that the request is now considered in-flight by
the driver. I'll change the wording a bit.

>
>> +    pub fn start(&self) {
>> +        // SAFETY: By type invariant, `self.0` is a valid `struct request`. By
>> +        // existence of `&mut self` we have exclusive access.
>> +        unsafe { bindings::blk_mq_start_request(self.0.get()) };
>> +    }
>> +
>> +    /// Call this to indicate to the kernel that the request has been completed without errors
>
> I dislike the "Call this", it feels redundant, what about "Signal the
> block layer that the request has been completed without errors.".
>
> Also with the other functions below.

I agree.

>
>> +    pub fn end_ok(&self) {
>> +        // SAFETY: By type invariant, `self.0` is a valid `struct request`. By
>> +        // existence of `&mut self` we have exclusive access.
>> +        unsafe { bindings::blk_mq_end_request(self.0.get(), bindings::BLK_STS_OK as _) };
>> +    }
>> +
>> +    /// Call this to indicate to the kernel that the request completed with an error
>> +    pub fn end_err(&self, err: Error) {
>> +        // SAFETY: By type invariant, `self.0` is a valid `struct request`. By
>> +        // existence of `&mut self` we have exclusive access.
>> +        unsafe { bindings::blk_mq_end_request(self.0.get(), err.to_blk_status()) };
>> +    }
>> +
>> +    /// Call this to indicate that the request completed with the status indicated by `status`
>> +    pub fn end(&self, status: Result) {
>> +        if let Err(e) = status {
>> +            self.end_err(e);
>> +        } else {
>> +            self.end_ok();
>> +        }
>> +    }
>> +
>> +    /// Call this to schedule defered completion of the request
>> +    pub fn complete(&self) {
>> +        // SAFETY: By type invariant, `self.0` is a valid `struct request`
>> +        if !unsafe { bindings::blk_mq_complete_request_remote(self.0.get()) } {
>> +            T::complete(self);
>> +        }
>> +    }
>> +
>> +    /// Get the target sector for the request
>> +    #[inline(always)]
>> +    pub fn sector(&self) -> usize {
>> +        // SAFETY: By type invariant of `Self`, `self.0` is valid and live.
>> +        unsafe { (*self.0.get()).__sector as usize }
>> +    }
>> +
>> +    /// Returns an owned reference to the per-request data associated with this
>> +    /// request
>> +    pub fn owned_data_ref(request: ARef<Self>) -> RequestDataRef<T> {
>> +        RequestDataRef::new(request)
>> +    }
>> +
>> +    /// Returns a reference to the oer-request data associated with this request
>
> Typo: "the oer-request"

Thanks.

>
>> +    pub fn data_ref(&self) -> &T::RequestData {
>> +        let request_ptr = self.0.get().cast::<bindings::request>();
>> +
>> +        // SAFETY: `request_ptr` is a valid `struct request` because `ARef` is
>> +        // `repr(transparent)`
>> +        let p: *mut c_void = unsafe { bindings::blk_mq_rq_to_pdu(request_ptr) };
>> +
>> +        let p = p.cast::<T::RequestData>();
>> +
>> +        // SAFETY: By C API contract, `p` is initialized by a call to
>> +        // `OperationsVTable::init_request_callback()`. By existence of `&self`
>> +        // it must be valid for use as a shared reference.
>> +        unsafe { &*p }
>> +    }
>> +}
>> +
>> +// SAFETY: It is impossible to obtain an owned or mutable `Request`, so we can
>
> What do you mean by "mutable `Request`"? There is the function to obtain
> a `&mut Request`.

The idea behind this comment is that it is not possible to have an owned
`Request` instance. You can only ever have something that will deref
(shared) to `Request`. Construction of the `Request` type is not
possible in safe driver code. At least that is the intention.

The `from_ptr_mut` is unsafe, and could be downgraded to
`from_ptr`, since `Operations::complete` takes a shared reference
anyway. Bottom line is that user code does not handle `&mut Request`.

> Also this should probably be an invariant if you use it as a
> justification here.

Ok.

>
>> +// mark it `Send`.
>> +unsafe impl<T: Operations> Send for Request<T> {}
>> +
>> +// SAFETY: `Request` references can be shared across threads.
>
> Does not explain why that is the case.

Will add.

>
>> +unsafe impl<T: Operations> Sync for Request<T> {}
>> +
>> +/// An owned reference to a `Request<T>`
>> +#[repr(transparent)]
>> +pub struct RequestDataRef<T: Operations> {
>> +    request: ARef<Request<T>>,
>> +}
>
> Is this extra type really needed? I have not yet taken a look at patch 3,
> which uses this. But would it hurt if you implemented the traits
> there directly on `ARef<Request<T>>`?

Yes, thanks :) Way better with just `ARef<Request<T>>`.

>
>> +
>> +impl<T> RequestDataRef<T>
>> +where
>> +    T: Operations,
>> +{
>> +    /// Create a new instance.
>> +    fn new(request: ARef<Request<T>>) -> Self {
>> +        Self { request }
>> +    }
>> +
>> +    /// Get a reference to the underlying request
>> +    pub fn request(&self) -> &Request<T> {
>> +        &self.request
>> +    }
>> +}
>
> I really like how you improved the safety comments and documentation. It
> was a lot easier to wrap my head around this time (might also me getting
> more experienced, but I think it helped a lot).

That's great!

Best regards,
Andreas


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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-03-23  6:32 ` Andreas Hindborg
@ 2024-04-02 23:09   ` Benno Lossin
  2024-04-03  8:46     ` Andreas Hindborg
  0 siblings, 1 reply; 26+ messages in thread
From: Benno Lossin @ 2024-04-02 23:09 UTC (permalink / raw)
  To: Andreas Hindborg
  Cc: Jens Axboe, Christoph Hellwig, Keith Busch, Damien Le Moal,
	Bart Van Assche, Hannes Reinecke, linux-block@vger.kernel.org,
	Andreas Hindborg, Wedson Almeida Filho, Niklas Cassel, Greg KH,
	Matthew Wilcox, Miguel Ojeda, Alex Gaynor, Boqun Feng, Gary Guo,
	Björn Roy Baron, Alice Ryhl, Chaitanya Kulkarni,
	Luis Chamberlain, Yexuan Yang, Sergio González Collado,
	Joel Granados, Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com

On 23.03.24 07:32, Andreas Hindborg wrote:
> Benno Lossin <benno.lossin@proton.me> writes:
>> On 3/13/24 12:05, Andreas Hindborg wrote:
>>> +//! implementations of the `Operations` trait.
>>> +//!
>>> +//! IO requests are passed to the driver as [`Request`] references. The
>>> +//! `Request` type is a wrapper around the C `struct request`. The driver must
>>> +//! mark start of request processing by calling [`Request::start`] and end of
>>> +//! processing by calling one of the [`Request::end`], methods. Failure to do so
>>> +//! can lead to IO failures.
>>
>> I am unfamiliar with this, what are "IO failures"?
>> Do you think that it might be better to change the API to use a
>> callback? So instead of calling start and end, you would do
>>
>>       request.handle(|req| {
>>           // do the stuff that would be done between start and end
>>       });
>>
>> I took a quick look at the rnull driver and there you are calling
>> `Request::end_ok` from a different function. So my suggestion might not
>> be possible, since you really need the freedom.
>>
>> Do you think that a guard approach might work better? ie `start` returns
>> a guard that when dropped will call `end` and you need the guard to
>> operate on the request.
> 
> I don't think that would fit, since the driver might not complete the
> request immediately. We might be able to call `start` on behalf of the
> driver.
> 
> At any rate, since the request is reference counted now, we can
> automatically fail a request when the last reference is dropped and it
> was not marked successfully completed. I would need to measure the
> performance implications of such a feature.

Are there cases where you still need access to the request after you
have called `end`? If no, I think it would be better for the request to
be consumed by the `end` function.
This is a bit difficult with `ARef`, since the user can just clone it
though... Do you think that it might be necessary to clone requests?

Also what happens if I call `end_ok` and then `end_err` or vice versa?

>>> +    pub fn data_ref(&self) -> &T::RequestData {
>>> +        let request_ptr = self.0.get().cast::<bindings::request>();
>>> +
>>> +        // SAFETY: `request_ptr` is a valid `struct request` because `ARef` is
>>> +        // `repr(transparent)`
>>> +        let p: *mut c_void = unsafe { bindings::blk_mq_rq_to_pdu(request_ptr) };
>>> +
>>> +        let p = p.cast::<T::RequestData>();
>>> +
>>> +        // SAFETY: By C API contract, `p` is initialized by a call to
>>> +        // `OperationsVTable::init_request_callback()`. By existence of `&self`
>>> +        // it must be valid for use as a shared reference.
>>> +        unsafe { &*p }
>>> +    }
>>> +}
>>> +
>>> +// SAFETY: It is impossible to obtain an owned or mutable `Request`, so we can
>>
>> What do you mean by "mutable `Request`"? There is the function to obtain
>> a `&mut Request`.
> 
> The idea behind this comment is that it is not possible to have an owned
> `Request` instance. You can only ever have something that will deref
> (shared) to `Request`. Construction of the `Request` type is not
> possible in safe driver code. At least that is the intention.
> 
> The `from_ptr_mut` is unsafe, and could be downgraded to
> `from_ptr`, since `Operations::complete` takes a shared reference
> anyway. Bottom line is that user code does not handle `&mut Request`.

Ah I see what you mean. But the user is able to have an `ARef<Request>`.
Which you own, if it is the only refcount currently held on that
request. When you drop it, you will run the destructor of the request.

A more suitable safety comment would be "SAFETY: A `struct request` may
be destroyed from any thread.".

-- 
Cheers,
Benno


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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-04-02 23:09   ` Benno Lossin
@ 2024-04-03  8:46     ` Andreas Hindborg
  2024-04-03 19:37       ` Benno Lossin
  0 siblings, 1 reply; 26+ messages in thread
From: Andreas Hindborg @ 2024-04-03  8:46 UTC (permalink / raw)
  To: Benno Lossin
  Cc: Jens Axboe, Christoph Hellwig, Keith Busch, Damien Le Moal,
	Bart Van Assche, Hannes Reinecke, linux-block@vger.kernel.org,
	Andreas Hindborg, Wedson Almeida Filho, Niklas Cassel, Greg KH,
	Matthew Wilcox, Miguel Ojeda, Alex Gaynor, Boqun Feng, Gary Guo,
	Björn Roy Baron, Alice Ryhl, Chaitanya Kulkarni,
	Luis Chamberlain, Yexuan Yang, Sergio González Collado,
	Joel Granados, Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com, Ming Lei

Benno Lossin <benno.lossin@proton.me> writes:

> On 23.03.24 07:32, Andreas Hindborg wrote:
>> Benno Lossin <benno.lossin@proton.me> writes:
>>> On 3/13/24 12:05, Andreas Hindborg wrote:
>>>> +//! implementations of the `Operations` trait.
>>>> +//!
>>>> +//! IO requests are passed to the driver as [`Request`] references. The
>>>> +//! `Request` type is a wrapper around the C `struct request`. The driver must
>>>> +//! mark start of request processing by calling [`Request::start`] and end of
>>>> +//! processing by calling one of the [`Request::end`], methods. Failure to do so
>>>> +//! can lead to IO failures.
>>>
>>> I am unfamiliar with this, what are "IO failures"?
>>> Do you think that it might be better to change the API to use a
>>> callback? So instead of calling start and end, you would do
>>>
>>>       request.handle(|req| {
>>>           // do the stuff that would be done between start and end
>>>       });
>>>
>>> I took a quick look at the rnull driver and there you are calling
>>> `Request::end_ok` from a different function. So my suggestion might not
>>> be possible, since you really need the freedom.
>>>
>>> Do you think that a guard approach might work better? ie `start` returns
>>> a guard that when dropped will call `end` and you need the guard to
>>> operate on the request.
>> 
>> I don't think that would fit, since the driver might not complete the
>> request immediately. We might be able to call `start` on behalf of the
>> driver.
>> 
>> At any rate, since the request is reference counted now, we can
>> automatically fail a request when the last reference is dropped and it
>> was not marked successfully completed. I would need to measure the
>> performance implications of such a feature.
>
> Are there cases where you still need access to the request after you
> have called `end`?

In general no, there is no need to handle the request after calling end.
C drivers are not allowed to, because this transfers ownership of the
request back to the block layer. This patch series defer the transfer of
ownership to the point when the ARef<Request> refcount goes to zero, so
there should be no danger associated with touching the `Request` after
end.

> If no, I think it would be better for the request to
> be consumed by the `end` function.
> This is a bit difficult with `ARef`, since the user can just clone it
> though... Do you think that it might be necessary to clone requests?

Looking into the details now I see that calling `Request::end` more than
once will trigger UAF, because C code decrements the refcount on the
request. When we have `ARef<Request>` around, that is a problem. It
probably also messes with other things in C land. Good catch.

I did implement `Request::end` to consume the request at one point
before I fell back on reference counting. It works fine for simple
drivers. However, most drivers will need to use the block layer tag set
service, that allows conversion of an integer id to a request pointer.
The abstraction for this feature is not part of this patch set. But the
block layer manages a mapping of integer to request mapping, and drivers
typically use this to identify the request that corresponds to
completion messages that arrive from hardware. When drivers are able to
turn integers into requests like this, consuming the request in the call
to `end` makes little sense (because we can just construct more).

What I do now is issue the an `Option<ARef<Request>>` with
`bindings::req_ref_inc_not_zero(rq_ptr)`, to make sure that the request
is currently owned by the driver.

I guess we can check the absolute value of the refcount, and only issue
a request handle if the count matches what we expect. Then we can be certain
that the handle is unique, and we can require transfer of ownership of
the handle to `Request::end` to make sure it can never be called more
than once.

Another option is to error out in `Request::end` if the
refcount is not what we expect.

>
> Also what happens if I call `end_ok` and then `end_err` or vice versa?

That would be similar to calling end twice.

>
>>>> +    pub fn data_ref(&self) -> &T::RequestData {
>>>> +        let request_ptr = self.0.get().cast::<bindings::request>();
>>>> +
>>>> +        // SAFETY: `request_ptr` is a valid `struct request` because `ARef` is
>>>> +        // `repr(transparent)`
>>>> +        let p: *mut c_void = unsafe { bindings::blk_mq_rq_to_pdu(request_ptr) };
>>>> +
>>>> +        let p = p.cast::<T::RequestData>();
>>>> +
>>>> +        // SAFETY: By C API contract, `p` is initialized by a call to
>>>> +        // `OperationsVTable::init_request_callback()`. By existence of `&self`
>>>> +        // it must be valid for use as a shared reference.
>>>> +        unsafe { &*p }
>>>> +    }
>>>> +}
>>>> +
>>>> +// SAFETY: It is impossible to obtain an owned or mutable `Request`, so we can
>>>
>>> What do you mean by "mutable `Request`"? There is the function to obtain
>>> a `&mut Request`.
>> 
>> The idea behind this comment is that it is not possible to have an owned
>> `Request` instance. You can only ever have something that will deref
>> (shared) to `Request`. Construction of the `Request` type is not
>> possible in safe driver code. At least that is the intention.
>> 
>> The `from_ptr_mut` is unsafe, and could be downgraded to
>> `from_ptr`, since `Operations::complete` takes a shared reference
>> anyway. Bottom line is that user code does not handle `&mut Request`.
>
> Ah I see what you mean. But the user is able to have an `ARef<Request>`.
> Which you own, if it is the only refcount currently held on that
> request. When you drop it, you will run the destructor of the request.
>
> A more suitable safety comment would be "SAFETY: A `struct request` may
> be destroyed from any thread.".

I see, I will update the comment.


BR Andreas

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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-04-03  8:46     ` Andreas Hindborg
@ 2024-04-03 19:37       ` Benno Lossin
  2024-04-04  5:44         ` Andreas Hindborg
  0 siblings, 1 reply; 26+ messages in thread
From: Benno Lossin @ 2024-04-03 19:37 UTC (permalink / raw)
  To: Andreas Hindborg
  Cc: Jens Axboe, Christoph Hellwig, Keith Busch, Damien Le Moal,
	Bart Van Assche, Hannes Reinecke, linux-block@vger.kernel.org,
	Andreas Hindborg, Wedson Almeida Filho, Niklas Cassel, Greg KH,
	Matthew Wilcox, Miguel Ojeda, Alex Gaynor, Boqun Feng, Gary Guo,
	Björn Roy Baron, Alice Ryhl, Chaitanya Kulkarni,
	Luis Chamberlain, Yexuan Yang, Sergio González Collado,
	Joel Granados, Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com, Ming Lei

On 03.04.24 10:46, Andreas Hindborg wrote:
> Benno Lossin <benno.lossin@proton.me> writes:
> 
>> On 23.03.24 07:32, Andreas Hindborg wrote:
>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>> On 3/13/24 12:05, Andreas Hindborg wrote:
>>>>> +//! implementations of the `Operations` trait.
>>>>> +//!
>>>>> +//! IO requests are passed to the driver as [`Request`] references. The
>>>>> +//! `Request` type is a wrapper around the C `struct request`. The driver must
>>>>> +//! mark start of request processing by calling [`Request::start`] and end of
>>>>> +//! processing by calling one of the [`Request::end`], methods. Failure to do so
>>>>> +//! can lead to IO failures.
>>>>
>>>> I am unfamiliar with this, what are "IO failures"?
>>>> Do you think that it might be better to change the API to use a
>>>> callback? So instead of calling start and end, you would do
>>>>
>>>>        request.handle(|req| {
>>>>            // do the stuff that would be done between start and end
>>>>        });
>>>>
>>>> I took a quick look at the rnull driver and there you are calling
>>>> `Request::end_ok` from a different function. So my suggestion might not
>>>> be possible, since you really need the freedom.
>>>>
>>>> Do you think that a guard approach might work better? ie `start` returns
>>>> a guard that when dropped will call `end` and you need the guard to
>>>> operate on the request.
>>>
>>> I don't think that would fit, since the driver might not complete the
>>> request immediately. We might be able to call `start` on behalf of the
>>> driver.
>>>
>>> At any rate, since the request is reference counted now, we can
>>> automatically fail a request when the last reference is dropped and it
>>> was not marked successfully completed. I would need to measure the
>>> performance implications of such a feature.
>>
>> Are there cases where you still need access to the request after you
>> have called `end`?
> 
> In general no, there is no need to handle the request after calling end.
> C drivers are not allowed to, because this transfers ownership of the
> request back to the block layer. This patch series defer the transfer of
> ownership to the point when the ARef<Request> refcount goes to zero, so
> there should be no danger associated with touching the `Request` after
> end.
> 
>> If no, I think it would be better for the request to
>> be consumed by the `end` function.
>> This is a bit difficult with `ARef`, since the user can just clone it
>> though... Do you think that it might be necessary to clone requests?
> 
> Looking into the details now I see that calling `Request::end` more than
> once will trigger UAF, because C code decrements the refcount on the
> request. When we have `ARef<Request>` around, that is a problem. It
> probably also messes with other things in C land. Good catch.
> 
> I did implement `Request::end` to consume the request at one point
> before I fell back on reference counting. It works fine for simple
> drivers. However, most drivers will need to use the block layer tag set
> service, that allows conversion of an integer id to a request pointer.
> The abstraction for this feature is not part of this patch set. But the
> block layer manages a mapping of integer to request mapping, and drivers
> typically use this to identify the request that corresponds to
> completion messages that arrive from hardware. When drivers are able to
> turn integers into requests like this, consuming the request in the call
> to `end` makes little sense (because we can just construct more).

How do you ensure that this is fine?:

     let r1 = tagset.get(0);
     let r2 = tagset.get(0);
     r1.end_ok();
     r2.do_something_that_would_only_be_done_while_active();

One thing that comes to my mind would be to only give out `&Request`
from the tag set. And to destroy, you could have a separate operation
that also removes the request from the tag set. (I am thinking of a tag
set as a `HashMap<u64, Request>`.

> 
> What I do now is issue the an `Option<ARef<Request>>` with
> `bindings::req_ref_inc_not_zero(rq_ptr)`, to make sure that the request
> is currently owned by the driver.
> 
> I guess we can check the absolute value of the refcount, and only issue
> a request handle if the count matches what we expect. Then we can be certain
> that the handle is unique, and we can require transfer of ownership of
> the handle to `Request::end` to make sure it can never be called more
> than once.
> 
> Another option is to error out in `Request::end` if the
> refcount is not what we expect.

I am a bit confused, why does the refcount matter in this case? Can't
the user just have multiple `ARef`s?

I think it would be weird to use `ARef<Request>` if you expect the
refcount to be 1. Maybe the API should be different?
As I understand it, a request has the following life cycle (please
correct me if I am wrong):
1. A new request is created, it is given to the driver via `queue_rq`.
2. The driver can now decide what to do with it (theoretically it can
    store it somewhere and later do something with it), but it should at
    some point call `Request::start`.
3. Work happens and eventually the driver calls `Request::end`.

To me this does not seem like something where we need a refcount (we
still might need one for safety, but it does not need to be exposed to
the user).

-- 
Cheers,
Benno


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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-04-03 19:37       ` Benno Lossin
@ 2024-04-04  5:44         ` Andreas Hindborg
  2024-04-04  8:46           ` Benno Lossin
  0 siblings, 1 reply; 26+ messages in thread
From: Andreas Hindborg @ 2024-04-04  5:44 UTC (permalink / raw)
  To: Benno Lossin
  Cc: Jens Axboe, Christoph Hellwig, Keith Busch, Damien Le Moal,
	Bart Van Assche, Hannes Reinecke, linux-block@vger.kernel.org,
	Andreas Hindborg, Wedson Almeida Filho, Niklas Cassel, Greg KH,
	Matthew Wilcox, Miguel Ojeda, Alex Gaynor, Boqun Feng, Gary Guo,
	Björn Roy Baron, Alice Ryhl, Chaitanya Kulkarni,
	Luis Chamberlain, Yexuan Yang, Sergio González Collado,
	Joel Granados, Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com, Ming Lei

Benno Lossin <benno.lossin@proton.me> writes:

> On 03.04.24 10:46, Andreas Hindborg wrote:
>> Benno Lossin <benno.lossin@proton.me> writes:
>> 
>>> On 23.03.24 07:32, Andreas Hindborg wrote:
>>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>>> On 3/13/24 12:05, Andreas Hindborg wrote:
>>>>>> +//! implementations of the `Operations` trait.
>>>>>> +//!
>>>>>> +//! IO requests are passed to the driver as [`Request`] references. The
>>>>>> +//! `Request` type is a wrapper around the C `struct request`. The driver must
>>>>>> +//! mark start of request processing by calling [`Request::start`] and end of
>>>>>> +//! processing by calling one of the [`Request::end`], methods. Failure to do so
>>>>>> +//! can lead to IO failures.
>>>>>
>>>>> I am unfamiliar with this, what are "IO failures"?
>>>>> Do you think that it might be better to change the API to use a
>>>>> callback? So instead of calling start and end, you would do
>>>>>
>>>>>        request.handle(|req| {
>>>>>            // do the stuff that would be done between start and end
>>>>>        });
>>>>>
>>>>> I took a quick look at the rnull driver and there you are calling
>>>>> `Request::end_ok` from a different function. So my suggestion might not
>>>>> be possible, since you really need the freedom.
>>>>>
>>>>> Do you think that a guard approach might work better? ie `start` returns
>>>>> a guard that when dropped will call `end` and you need the guard to
>>>>> operate on the request.
>>>>
>>>> I don't think that would fit, since the driver might not complete the
>>>> request immediately. We might be able to call `start` on behalf of the
>>>> driver.
>>>>
>>>> At any rate, since the request is reference counted now, we can
>>>> automatically fail a request when the last reference is dropped and it
>>>> was not marked successfully completed. I would need to measure the
>>>> performance implications of such a feature.
>>>
>>> Are there cases where you still need access to the request after you
>>> have called `end`?
>> 
>> In general no, there is no need to handle the request after calling end.
>> C drivers are not allowed to, because this transfers ownership of the
>> request back to the block layer. This patch series defer the transfer of
>> ownership to the point when the ARef<Request> refcount goes to zero, so
>> there should be no danger associated with touching the `Request` after
>> end.
>> 
>>> If no, I think it would be better for the request to
>>> be consumed by the `end` function.
>>> This is a bit difficult with `ARef`, since the user can just clone it
>>> though... Do you think that it might be necessary to clone requests?
>> 
>> Looking into the details now I see that calling `Request::end` more than
>> once will trigger UAF, because C code decrements the refcount on the
>> request. When we have `ARef<Request>` around, that is a problem. It
>> probably also messes with other things in C land. Good catch.
>> 
>> I did implement `Request::end` to consume the request at one point
>> before I fell back on reference counting. It works fine for simple
>> drivers. However, most drivers will need to use the block layer tag set
>> service, that allows conversion of an integer id to a request pointer.
>> The abstraction for this feature is not part of this patch set. But the
>> block layer manages a mapping of integer to request mapping, and drivers
>> typically use this to identify the request that corresponds to
>> completion messages that arrive from hardware. When drivers are able to
>> turn integers into requests like this, consuming the request in the call
>> to `end` makes little sense (because we can just construct more).
>
> How do you ensure that this is fine?:
>
>      let r1 = tagset.get(0);
>      let r2 = tagset.get(0);
>      r1.end_ok();
>      r2.do_something_that_would_only_be_done_while_active();
>
> One thing that comes to my mind would be to only give out `&Request`
> from the tag set. And to destroy, you could have a separate operation
> that also removes the request from the tag set. (I am thinking of a tag
> set as a `HashMap<u64, Request>`.

This would be similar to

  let r1 = tagset.get(0)?;
  ler r2 = r1.clone();
  r1.end_ok();
  r2.do_something_requires_active();

but it is not a problem because we do not implement any actions that are
illegal in that position (outside of `end` - that _is_ a problem).


>> 
>> What I do now is issue the an `Option<ARef<Request>>` with
>> `bindings::req_ref_inc_not_zero(rq_ptr)`, to make sure that the request
>> is currently owned by the driver.
>> 
>> I guess we can check the absolute value of the refcount, and only issue
>> a request handle if the count matches what we expect. Then we can be certain
>> that the handle is unique, and we can require transfer of ownership of
>> the handle to `Request::end` to make sure it can never be called more
>> than once.
>> 
>> Another option is to error out in `Request::end` if the
>> refcount is not what we expect.
>
> I am a bit confused, why does the refcount matter in this case? Can't
> the user just have multiple `ARef`s?

Because we want to assert that we are consuming the last handle to the
request. After we do that, the user cannot call `Request::end` again.
`TagSet::get` will not issue a request reference if the request is not
in flight. Although there might be a race condition to watch out for.

When the block layer hands over ownership to Rust, the reference count
is 1. The first `ARef<Request>` we create increments the count to 2. To
complete the request, we must have ownership of all reference counts
above 1. The block layer takes the last reference count when it takes
back ownership of the request.

> I think it would be weird to use `ARef<Request>` if you expect the
> refcount to be 1.

Yes, that would require a custom smart pointer with a `try_into_unique`
method that succeeds when the refcount is exactly 2. It would consume
the instance and decrement the refcount to 1. But as I said, there is a
potential race with `TagSet::get` when the refcount is 1 that needs to
be handled.

> Maybe the API should be different?

I needs to change a little, yes.

> As I understand it, a request has the following life cycle (please
> correct me if I am wrong):
> 1. A new request is created, it is given to the driver via `queue_rq`.
> 2. The driver can now decide what to do with it (theoretically it can
>     store it somewhere and later do something with it), but it should at
>     some point call `Request::start`.
> 3. Work happens and eventually the driver calls `Request::end`.
>
> To me this does not seem like something where we need a refcount (we
> still might need one for safety, but it does not need to be exposed to
> the user).

It would not need to be exposed to the user, other than a) ending a request
can fail OR b) `TagSet::get` can fail.

a) would require that ending a request must be done with a unique
reference. This could be done by the user by the user calling
`try_into_unique` or by making the `end` method fallible.

b) would make the reference handle `!Clone` and add a failure mode to
`TagSet::get`, so it fails to construct a `Request` handle if there are
already one in existence.

I gravitate towards a) because it allows the user to clone the Request
reference without adding an additional `Arc`.


BR Andreas

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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-04-04  5:44         ` Andreas Hindborg
@ 2024-04-04  8:46           ` Benno Lossin
  2024-04-04  9:30             ` Andreas Hindborg
  0 siblings, 1 reply; 26+ messages in thread
From: Benno Lossin @ 2024-04-04  8:46 UTC (permalink / raw)
  To: Andreas Hindborg
  Cc: Jens Axboe, Christoph Hellwig, Keith Busch, Damien Le Moal,
	Bart Van Assche, Hannes Reinecke, linux-block@vger.kernel.org,
	Andreas Hindborg, Wedson Almeida Filho, Niklas Cassel, Greg KH,
	Matthew Wilcox, Miguel Ojeda, Alex Gaynor, Boqun Feng, Gary Guo,
	Björn Roy Baron, Alice Ryhl, Chaitanya Kulkarni,
	Luis Chamberlain, Yexuan Yang, Sergio González Collado,
	Joel Granados, Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com, Ming Lei

On 04.04.24 07:44, Andreas Hindborg wrote:
> Benno Lossin <benno.lossin@proton.me> writes:
> 
>> On 03.04.24 10:46, Andreas Hindborg wrote:
>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>
>>>> On 23.03.24 07:32, Andreas Hindborg wrote:
>>>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>>>> On 3/13/24 12:05, Andreas Hindborg wrote:
>>>>>>> +//! implementations of the `Operations` trait.
>>>>>>> +//!
>>>>>>> +//! IO requests are passed to the driver as [`Request`] references. The
>>>>>>> +//! `Request` type is a wrapper around the C `struct request`. The driver must
>>>>>>> +//! mark start of request processing by calling [`Request::start`] and end of
>>>>>>> +//! processing by calling one of the [`Request::end`], methods. Failure to do so
>>>>>>> +//! can lead to IO failures.
>>>>>>
>>>>>> I am unfamiliar with this, what are "IO failures"?
>>>>>> Do you think that it might be better to change the API to use a
>>>>>> callback? So instead of calling start and end, you would do
>>>>>>
>>>>>>         request.handle(|req| {
>>>>>>             // do the stuff that would be done between start and end
>>>>>>         });
>>>>>>
>>>>>> I took a quick look at the rnull driver and there you are calling
>>>>>> `Request::end_ok` from a different function. So my suggestion might not
>>>>>> be possible, since you really need the freedom.
>>>>>>
>>>>>> Do you think that a guard approach might work better? ie `start` returns
>>>>>> a guard that when dropped will call `end` and you need the guard to
>>>>>> operate on the request.
>>>>>
>>>>> I don't think that would fit, since the driver might not complete the
>>>>> request immediately. We might be able to call `start` on behalf of the
>>>>> driver.
>>>>>
>>>>> At any rate, since the request is reference counted now, we can
>>>>> automatically fail a request when the last reference is dropped and it
>>>>> was not marked successfully completed. I would need to measure the
>>>>> performance implications of such a feature.
>>>>
>>>> Are there cases where you still need access to the request after you
>>>> have called `end`?
>>>
>>> In general no, there is no need to handle the request after calling end.
>>> C drivers are not allowed to, because this transfers ownership of the
>>> request back to the block layer. This patch series defer the transfer of
>>> ownership to the point when the ARef<Request> refcount goes to zero, so
>>> there should be no danger associated with touching the `Request` after
>>> end.
>>>
>>>> If no, I think it would be better for the request to
>>>> be consumed by the `end` function.
>>>> This is a bit difficult with `ARef`, since the user can just clone it
>>>> though... Do you think that it might be necessary to clone requests?
>>>
>>> Looking into the details now I see that calling `Request::end` more than
>>> once will trigger UAF, because C code decrements the refcount on the
>>> request. When we have `ARef<Request>` around, that is a problem. It
>>> probably also messes with other things in C land. Good catch.
>>>
>>> I did implement `Request::end` to consume the request at one point
>>> before I fell back on reference counting. It works fine for simple
>>> drivers. However, most drivers will need to use the block layer tag set
>>> service, that allows conversion of an integer id to a request pointer.
>>> The abstraction for this feature is not part of this patch set. But the
>>> block layer manages a mapping of integer to request mapping, and drivers
>>> typically use this to identify the request that corresponds to
>>> completion messages that arrive from hardware. When drivers are able to
>>> turn integers into requests like this, consuming the request in the call
>>> to `end` makes little sense (because we can just construct more).
>>
>> How do you ensure that this is fine?:
>>
>>       let r1 = tagset.get(0);
>>       let r2 = tagset.get(0);
>>       r1.end_ok();
>>       r2.do_something_that_would_only_be_done_while_active();
>>
>> One thing that comes to my mind would be to only give out `&Request`
>> from the tag set. And to destroy, you could have a separate operation
>> that also removes the request from the tag set. (I am thinking of a tag
>> set as a `HashMap<u64, Request>`.
> 
> This would be similar to
> 
>    let r1 = tagset.get(0)?;
>    ler r2 = r1.clone();
>    r1.end_ok();
>    r2.do_something_requires_active();
> 
> but it is not a problem because we do not implement any actions that are
> illegal in that position (outside of `end` - that _is_ a problem).

Makes sense, but I think it's a bit weird to still be able to access it
after `end`ing.

> 
> 
>>>
>>> What I do now is issue the an `Option<ARef<Request>>` with
>>> `bindings::req_ref_inc_not_zero(rq_ptr)`, to make sure that the request
>>> is currently owned by the driver.
>>>
>>> I guess we can check the absolute value of the refcount, and only issue
>>> a request handle if the count matches what we expect. Then we can be certain
>>> that the handle is unique, and we can require transfer of ownership of
>>> the handle to `Request::end` to make sure it can never be called more
>>> than once.
>>>
>>> Another option is to error out in `Request::end` if the
>>> refcount is not what we expect.
>>
>> I am a bit confused, why does the refcount matter in this case? Can't
>> the user just have multiple `ARef`s?
> 
> Because we want to assert that we are consuming the last handle to the
> request. After we do that, the user cannot call `Request::end` again.
> `TagSet::get` will not issue a request reference if the request is not
> in flight. Although there might be a race condition to watch out for.
> 
> When the block layer hands over ownership to Rust, the reference count
> is 1. The first `ARef<Request>` we create increments the count to 2. To
> complete the request, we must have ownership of all reference counts
> above 1. The block layer takes the last reference count when it takes
> back ownership of the request.
> 
>> I think it would be weird to use `ARef<Request>` if you expect the
>> refcount to be 1.
> 
> Yes, that would require a custom smart pointer with a `try_into_unique`
> method that succeeds when the refcount is exactly 2. It would consume
> the instance and decrement the refcount to 1. But as I said, there is a
> potential race with `TagSet::get` when the refcount is 1 that needs to
> be handled.
> 
>> Maybe the API should be different?
> 
> I needs to change a little, yes.
> 
>> As I understand it, a request has the following life cycle (please
>> correct me if I am wrong):
>> 1. A new request is created, it is given to the driver via `queue_rq`.
>> 2. The driver can now decide what to do with it (theoretically it can
>>      store it somewhere and later do something with it), but it should at
>>      some point call `Request::start`.
>> 3. Work happens and eventually the driver calls `Request::end`.
>>
>> To me this does not seem like something where we need a refcount (we
>> still might need one for safety, but it does not need to be exposed to
>> the user).
> 
> It would not need to be exposed to the user, other than a) ending a request
> can fail OR b) `TagSet::get` can fail.
> 
> a) would require that ending a request must be done with a unique
> reference. This could be done by the user by the user calling
> `try_into_unique` or by making the `end` method fallible.
> 
> b) would make the reference handle `!Clone` and add a failure mode to
> `TagSet::get`, so it fails to construct a `Request` handle if there are
> already one in existence.
> 
> I gravitate towards a) because it allows the user to clone the Request
> reference without adding an additional `Arc`.

This confuses me a little, since I thought that `TagSet::get` returns
`Option<ARef<Request>>`. (I tried to find the abstractions in your
github, but I did not find them)

I think that this could work: `queue_rq` takes a `OwnedRequest`, which
the user can store in a `TagSet`, transferring ownership. `TagSet::get`
returns `Option<&Request>` and you can call `TagSet::remove` to get
`Option<OwnedRequest>`. `OwnedRequest::end` consumes `self`.
With this pattern we also do not need to take an additional refcount.

-- 
Cheers,
Benno


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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-04-04  8:46           ` Benno Lossin
@ 2024-04-04  9:30             ` Andreas Hindborg
  2024-04-04 13:20               ` Benno Lossin
  0 siblings, 1 reply; 26+ messages in thread
From: Andreas Hindborg @ 2024-04-04  9:30 UTC (permalink / raw)
  To: Benno Lossin
  Cc: Jens Axboe, Christoph Hellwig, Keith Busch, Damien Le Moal,
	Bart Van Assche, Hannes Reinecke, linux-block@vger.kernel.org,
	Andreas Hindborg, Wedson Almeida Filho, Niklas Cassel, Greg KH,
	Matthew Wilcox, Miguel Ojeda, Alex Gaynor, Boqun Feng, Gary Guo,
	Björn Roy Baron, Alice Ryhl, Chaitanya Kulkarni,
	Luis Chamberlain, Yexuan Yang, Sergio González Collado,
	Joel Granados, Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com, Ming Lei

Benno Lossin <benno.lossin@proton.me> writes:

> On 04.04.24 07:44, Andreas Hindborg wrote:
>> Benno Lossin <benno.lossin@proton.me> writes:
>> 
>>> On 03.04.24 10:46, Andreas Hindborg wrote:
>>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>>
>>>>> On 23.03.24 07:32, Andreas Hindborg wrote:
>>>>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>>>>> On 3/13/24 12:05, Andreas Hindborg wrote:
>>>>>>>> +//! implementations of the `Operations` trait.
>>>>>>>> +//!
>>>>>>>> +//! IO requests are passed to the driver as [`Request`] references. The
>>>>>>>> +//! `Request` type is a wrapper around the C `struct request`. The driver must
>>>>>>>> +//! mark start of request processing by calling [`Request::start`] and end of
>>>>>>>> +//! processing by calling one of the [`Request::end`], methods. Failure to do so
>>>>>>>> +//! can lead to IO failures.
>>>>>>>
>>>>>>> I am unfamiliar with this, what are "IO failures"?
>>>>>>> Do you think that it might be better to change the API to use a
>>>>>>> callback? So instead of calling start and end, you would do
>>>>>>>
>>>>>>>         request.handle(|req| {
>>>>>>>             // do the stuff that would be done between start and end
>>>>>>>         });
>>>>>>>
>>>>>>> I took a quick look at the rnull driver and there you are calling
>>>>>>> `Request::end_ok` from a different function. So my suggestion might not
>>>>>>> be possible, since you really need the freedom.
>>>>>>>
>>>>>>> Do you think that a guard approach might work better? ie `start` returns
>>>>>>> a guard that when dropped will call `end` and you need the guard to
>>>>>>> operate on the request.
>>>>>>
>>>>>> I don't think that would fit, since the driver might not complete the
>>>>>> request immediately. We might be able to call `start` on behalf of the
>>>>>> driver.
>>>>>>
>>>>>> At any rate, since the request is reference counted now, we can
>>>>>> automatically fail a request when the last reference is dropped and it
>>>>>> was not marked successfully completed. I would need to measure the
>>>>>> performance implications of such a feature.
>>>>>
>>>>> Are there cases where you still need access to the request after you
>>>>> have called `end`?
>>>>
>>>> In general no, there is no need to handle the request after calling end.
>>>> C drivers are not allowed to, because this transfers ownership of the
>>>> request back to the block layer. This patch series defer the transfer of
>>>> ownership to the point when the ARef<Request> refcount goes to zero, so
>>>> there should be no danger associated with touching the `Request` after
>>>> end.
>>>>
>>>>> If no, I think it would be better for the request to
>>>>> be consumed by the `end` function.
>>>>> This is a bit difficult with `ARef`, since the user can just clone it
>>>>> though... Do you think that it might be necessary to clone requests?
>>>>
>>>> Looking into the details now I see that calling `Request::end` more than
>>>> once will trigger UAF, because C code decrements the refcount on the
>>>> request. When we have `ARef<Request>` around, that is a problem. It
>>>> probably also messes with other things in C land. Good catch.
>>>>
>>>> I did implement `Request::end` to consume the request at one point
>>>> before I fell back on reference counting. It works fine for simple
>>>> drivers. However, most drivers will need to use the block layer tag set
>>>> service, that allows conversion of an integer id to a request pointer.
>>>> The abstraction for this feature is not part of this patch set. But the
>>>> block layer manages a mapping of integer to request mapping, and drivers
>>>> typically use this to identify the request that corresponds to
>>>> completion messages that arrive from hardware. When drivers are able to
>>>> turn integers into requests like this, consuming the request in the call
>>>> to `end` makes little sense (because we can just construct more).
>>>
>>> How do you ensure that this is fine?:
>>>
>>>       let r1 = tagset.get(0);
>>>       let r2 = tagset.get(0);
>>>       r1.end_ok();
>>>       r2.do_something_that_would_only_be_done_while_active();
>>>
>>> One thing that comes to my mind would be to only give out `&Request`
>>> from the tag set. And to destroy, you could have a separate operation
>>> that also removes the request from the tag set. (I am thinking of a tag
>>> set as a `HashMap<u64, Request>`.
>> 
>> This would be similar to
>> 
>>    let r1 = tagset.get(0)?;
>>    ler r2 = r1.clone();
>>    r1.end_ok();
>>    r2.do_something_requires_active();
>> 
>> but it is not a problem because we do not implement any actions that are
>> illegal in that position (outside of `end` - that _is_ a problem).
>
> Makes sense, but I think it's a bit weird to still be able to access it
> after `end`ing.

Yes, that is true.

>
>> 
>> 
>>>>
>>>> What I do now is issue the an `Option<ARef<Request>>` with
>>>> `bindings::req_ref_inc_not_zero(rq_ptr)`, to make sure that the request
>>>> is currently owned by the driver.
>>>>
>>>> I guess we can check the absolute value of the refcount, and only issue
>>>> a request handle if the count matches what we expect. Then we can be certain
>>>> that the handle is unique, and we can require transfer of ownership of
>>>> the handle to `Request::end` to make sure it can never be called more
>>>> than once.
>>>>
>>>> Another option is to error out in `Request::end` if the
>>>> refcount is not what we expect.
>>>
>>> I am a bit confused, why does the refcount matter in this case? Can't
>>> the user just have multiple `ARef`s?
>> 
>> Because we want to assert that we are consuming the last handle to the
>> request. After we do that, the user cannot call `Request::end` again.
>> `TagSet::get` will not issue a request reference if the request is not
>> in flight. Although there might be a race condition to watch out for.
>> 
>> When the block layer hands over ownership to Rust, the reference count
>> is 1. The first `ARef<Request>` we create increments the count to 2. To
>> complete the request, we must have ownership of all reference counts
>> above 1. The block layer takes the last reference count when it takes
>> back ownership of the request.
>> 
>>> I think it would be weird to use `ARef<Request>` if you expect the
>>> refcount to be 1.
>> 
>> Yes, that would require a custom smart pointer with a `try_into_unique`
>> method that succeeds when the refcount is exactly 2. It would consume
>> the instance and decrement the refcount to 1. But as I said, there is a
>> potential race with `TagSet::get` when the refcount is 1 that needs to
>> be handled.
>> 
>>> Maybe the API should be different?
>> 
>> I needs to change a little, yes.
>> 
>>> As I understand it, a request has the following life cycle (please
>>> correct me if I am wrong):
>>> 1. A new request is created, it is given to the driver via `queue_rq`.
>>> 2. The driver can now decide what to do with it (theoretically it can
>>>      store it somewhere and later do something with it), but it should at
>>>      some point call `Request::start`.
>>> 3. Work happens and eventually the driver calls `Request::end`.
>>>
>>> To me this does not seem like something where we need a refcount (we
>>> still might need one for safety, but it does not need to be exposed to
>>> the user).
>> 
>> It would not need to be exposed to the user, other than a) ending a request
>> can fail OR b) `TagSet::get` can fail.
>> 
>> a) would require that ending a request must be done with a unique
>> reference. This could be done by the user by the user calling
>> `try_into_unique` or by making the `end` method fallible.
>> 
>> b) would make the reference handle `!Clone` and add a failure mode to
>> `TagSet::get`, so it fails to construct a `Request` handle if there are
>> already one in existence.
>> 
>> I gravitate towards a) because it allows the user to clone the Request
>> reference without adding an additional `Arc`.
>
> This confuses me a little, since I thought that `TagSet::get` returns
> `Option<ARef<Request>>`.

It does, but in the current implementation the failure mode returning
`None` is triggered when the refcount is zero, meaning that the request
corresponding to that tag is not currently owned by the driver. For
solution b) we would change the type to be
`Option<CustomSmartPointerHandleThing<Request>>`.

> (I tried to find the abstractions in your
> github, but I did not find them)

It's here [1]. It was introduced in the `rnvme-v6.8` branch.

> I think that this could work: `queue_rq` takes a `OwnedRequest`, which
> the user can store in a `TagSet`, transferring ownership. `TagSet::get`
> returns `Option<&Request>` and you can call `TagSet::remove` to get
> `Option<OwnedRequest>`. `OwnedRequest::end` consumes `self`.
> With this pattern we also do not need to take an additional refcount.

It would, but the `TagSet` is just a wrapper for the C block layer
`strugt blk_mq_tag_set`. This is a highly optimized data structure and
tag mapping is done before the driver sees the request. I would like to
reuse that logic.

We could implement what you suggest anyhow, but I would not want to that
additional logic to the hot path.


BR Andreas


[1] https://github.com/metaspace/linux/blob/0976c869fbfae13f6d475a287ade776c730cc029/rust/kernel/block/mq/tag_set.rs#L102

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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-04-04  9:30             ` Andreas Hindborg
@ 2024-04-04 13:20               ` Benno Lossin
  2024-04-05  8:43                 ` Andreas Hindborg
  0 siblings, 1 reply; 26+ messages in thread
From: Benno Lossin @ 2024-04-04 13:20 UTC (permalink / raw)
  To: Andreas Hindborg
  Cc: Jens Axboe, Christoph Hellwig, Keith Busch, Damien Le Moal,
	Bart Van Assche, Hannes Reinecke, linux-block@vger.kernel.org,
	Andreas Hindborg, Wedson Almeida Filho, Niklas Cassel, Greg KH,
	Matthew Wilcox, Miguel Ojeda, Alex Gaynor, Boqun Feng, Gary Guo,
	Björn Roy Baron, Alice Ryhl, Chaitanya Kulkarni,
	Luis Chamberlain, Yexuan Yang, Sergio González Collado,
	Joel Granados, Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com, Ming Lei

On 04.04.24 11:30, Andreas Hindborg wrote:
> Benno Lossin <benno.lossin@proton.me> writes:
> 
>> On 04.04.24 07:44, Andreas Hindborg wrote:
>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>
>>>> On 03.04.24 10:46, Andreas Hindborg wrote:
>>>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>>>
>>>>>> On 23.03.24 07:32, Andreas Hindborg wrote:
>>>>>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>>>>>> On 3/13/24 12:05, Andreas Hindborg wrote:
>>>>>>>>> +//! implementations of the `Operations` trait.
>>>>>>>>> +//!
>>>>>>>>> +//! IO requests are passed to the driver as [`Request`] references. The
>>>>>>>>> +//! `Request` type is a wrapper around the C `struct request`. The driver must
>>>>>>>>> +//! mark start of request processing by calling [`Request::start`] and end of
>>>>>>>>> +//! processing by calling one of the [`Request::end`], methods. Failure to do so
>>>>>>>>> +//! can lead to IO failures.
>>>>>>>>
>>>>>>>> I am unfamiliar with this, what are "IO failures"?
>>>>>>>> Do you think that it might be better to change the API to use a
>>>>>>>> callback? So instead of calling start and end, you would do
>>>>>>>>
>>>>>>>>          request.handle(|req| {
>>>>>>>>              // do the stuff that would be done between start and end
>>>>>>>>          });
>>>>>>>>
>>>>>>>> I took a quick look at the rnull driver and there you are calling
>>>>>>>> `Request::end_ok` from a different function. So my suggestion might not
>>>>>>>> be possible, since you really need the freedom.
>>>>>>>>
>>>>>>>> Do you think that a guard approach might work better? ie `start` returns
>>>>>>>> a guard that when dropped will call `end` and you need the guard to
>>>>>>>> operate on the request.
>>>>>>>
>>>>>>> I don't think that would fit, since the driver might not complete the
>>>>>>> request immediately. We might be able to call `start` on behalf of the
>>>>>>> driver.
>>>>>>>
>>>>>>> At any rate, since the request is reference counted now, we can
>>>>>>> automatically fail a request when the last reference is dropped and it
>>>>>>> was not marked successfully completed. I would need to measure the
>>>>>>> performance implications of such a feature.
>>>>>>
>>>>>> Are there cases where you still need access to the request after you
>>>>>> have called `end`?
>>>>>
>>>>> In general no, there is no need to handle the request after calling end.
>>>>> C drivers are not allowed to, because this transfers ownership of the
>>>>> request back to the block layer. This patch series defer the transfer of
>>>>> ownership to the point when the ARef<Request> refcount goes to zero, so
>>>>> there should be no danger associated with touching the `Request` after
>>>>> end.
>>>>>
>>>>>> If no, I think it would be better for the request to
>>>>>> be consumed by the `end` function.
>>>>>> This is a bit difficult with `ARef`, since the user can just clone it
>>>>>> though... Do you think that it might be necessary to clone requests?
>>>>>
>>>>> Looking into the details now I see that calling `Request::end` more than
>>>>> once will trigger UAF, because C code decrements the refcount on the
>>>>> request. When we have `ARef<Request>` around, that is a problem. It
>>>>> probably also messes with other things in C land. Good catch.
>>>>>
>>>>> I did implement `Request::end` to consume the request at one point
>>>>> before I fell back on reference counting. It works fine for simple
>>>>> drivers. However, most drivers will need to use the block layer tag set
>>>>> service, that allows conversion of an integer id to a request pointer.
>>>>> The abstraction for this feature is not part of this patch set. But the
>>>>> block layer manages a mapping of integer to request mapping, and drivers
>>>>> typically use this to identify the request that corresponds to
>>>>> completion messages that arrive from hardware. When drivers are able to
>>>>> turn integers into requests like this, consuming the request in the call
>>>>> to `end` makes little sense (because we can just construct more).
>>>>
>>>> How do you ensure that this is fine?:
>>>>
>>>>        let r1 = tagset.get(0);
>>>>        let r2 = tagset.get(0);
>>>>        r1.end_ok();
>>>>        r2.do_something_that_would_only_be_done_while_active();
>>>>
>>>> One thing that comes to my mind would be to only give out `&Request`
>>>> from the tag set. And to destroy, you could have a separate operation
>>>> that also removes the request from the tag set. (I am thinking of a tag
>>>> set as a `HashMap<u64, Request>`.
>>>
>>> This would be similar to
>>>
>>>     let r1 = tagset.get(0)?;
>>>     ler r2 = r1.clone();
>>>     r1.end_ok();
>>>     r2.do_something_requires_active();
>>>
>>> but it is not a problem because we do not implement any actions that are
>>> illegal in that position (outside of `end` - that _is_ a problem).
>>
>> Makes sense, but I think it's a bit weird to still be able to access it
>> after `end`ing.
> 
> Yes, that is true.
> 
>>
>>>
>>>
>>>>>
>>>>> What I do now is issue the an `Option<ARef<Request>>` with
>>>>> `bindings::req_ref_inc_not_zero(rq_ptr)`, to make sure that the request
>>>>> is currently owned by the driver.
>>>>>
>>>>> I guess we can check the absolute value of the refcount, and only issue
>>>>> a request handle if the count matches what we expect. Then we can be certain
>>>>> that the handle is unique, and we can require transfer of ownership of
>>>>> the handle to `Request::end` to make sure it can never be called more
>>>>> than once.
>>>>>
>>>>> Another option is to error out in `Request::end` if the
>>>>> refcount is not what we expect.
>>>>
>>>> I am a bit confused, why does the refcount matter in this case? Can't
>>>> the user just have multiple `ARef`s?
>>>
>>> Because we want to assert that we are consuming the last handle to the
>>> request. After we do that, the user cannot call `Request::end` again.
>>> `TagSet::get` will not issue a request reference if the request is not
>>> in flight. Although there might be a race condition to watch out for.
>>>
>>> When the block layer hands over ownership to Rust, the reference count
>>> is 1. The first `ARef<Request>` we create increments the count to 2. To
>>> complete the request, we must have ownership of all reference counts
>>> above 1. The block layer takes the last reference count when it takes
>>> back ownership of the request.
>>>
>>>> I think it would be weird to use `ARef<Request>` if you expect the
>>>> refcount to be 1.
>>>
>>> Yes, that would require a custom smart pointer with a `try_into_unique`
>>> method that succeeds when the refcount is exactly 2. It would consume
>>> the instance and decrement the refcount to 1. But as I said, there is a
>>> potential race with `TagSet::get` when the refcount is 1 that needs to
>>> be handled.
>>>
>>>> Maybe the API should be different?
>>>
>>> I needs to change a little, yes.
>>>
>>>> As I understand it, a request has the following life cycle (please
>>>> correct me if I am wrong):
>>>> 1. A new request is created, it is given to the driver via `queue_rq`.
>>>> 2. The driver can now decide what to do with it (theoretically it can
>>>>       store it somewhere and later do something with it), but it should at
>>>>       some point call `Request::start`.
>>>> 3. Work happens and eventually the driver calls `Request::end`.
>>>>
>>>> To me this does not seem like something where we need a refcount (we
>>>> still might need one for safety, but it does not need to be exposed to
>>>> the user).
>>>
>>> It would not need to be exposed to the user, other than a) ending a request
>>> can fail OR b) `TagSet::get` can fail.
>>>
>>> a) would require that ending a request must be done with a unique
>>> reference. This could be done by the user by the user calling
>>> `try_into_unique` or by making the `end` method fallible.
>>>
>>> b) would make the reference handle `!Clone` and add a failure mode to
>>> `TagSet::get`, so it fails to construct a `Request` handle if there are
>>> already one in existence.
>>>
>>> I gravitate towards a) because it allows the user to clone the Request
>>> reference without adding an additional `Arc`.
>>
>> This confuses me a little, since I thought that `TagSet::get` returns
>> `Option<ARef<Request>>`.
> 
> It does, but in the current implementation the failure mode returning
> `None` is triggered when the refcount is zero, meaning that the request
> corresponding to that tag is not currently owned by the driver. For
> solution b) we would change the type to be
> `Option<CustomSmartPointerHandleThing<Request>>`.
> 
>> (I tried to find the abstractions in your
>> github, but I did not find them)
> 
> It's here [1]. It was introduced in the `rnvme-v6.8` branch.

Thanks for the pointer.

>> I think that this could work: `queue_rq` takes a `OwnedRequest`, which
>> the user can store in a `TagSet`, transferring ownership. `TagSet::get`
>> returns `Option<&Request>` and you can call `TagSet::remove` to get
>> `Option<OwnedRequest>`. `OwnedRequest::end` consumes `self`.
>> With this pattern we also do not need to take an additional refcount.
> 
> It would, but the `TagSet` is just a wrapper for the C block layer
> `strugt blk_mq_tag_set`. This is a highly optimized data structure and
> tag mapping is done before the driver sees the request. I would like to
> reuse that logic.
> 
> We could implement what you suggest anyhow, but I would not want to that
> additional logic to the hot path.

I overlooked an important detail: the `TagSet` is always stored in an
`Arc` (IIRC since you want to be able to share it between different
`Gendisk`s). This probably makes my suggestion impossible, since you
can't mutably borrow the `TagSet` for removal of `Request`s.
Depending on how `Request`s are associated to a `TagSet`, there might be
a way around this: I saw the `qid` parameter to the `tag_to_rq`
function, is that a unique identifier for a queue? Because in that case
we might be able to have a unique `QueueTagSetRef` with

     fn remove(&mut self, tag: u32) -> OwnedRequest;

     fn get(&self, tag: u32) -> Option<&Request>;

The `TagSet` would still be shared, only the ability to "remove" (I
don't know if you do that manually in C, if not, then this would just
remove it in the abstraction, but keep it on the C side) is unique to
the `QueueTagSetRef` struct.

But feel free to use your proposed option a), it is simpler and we can
try to make this work when you send the `TagSet` abstractions.
I just think that we should try a bit harder to make it even better.

-- 
Cheers,
Benno


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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-04-04 13:20               ` Benno Lossin
@ 2024-04-05  8:43                 ` Andreas Hindborg
  2024-04-05  9:40                   ` Benno Lossin
  0 siblings, 1 reply; 26+ messages in thread
From: Andreas Hindborg @ 2024-04-05  8:43 UTC (permalink / raw)
  To: Benno Lossin
  Cc: Jens Axboe, Christoph Hellwig, Keith Busch, Damien Le Moal,
	Bart Van Assche, Hannes Reinecke, linux-block@vger.kernel.org,
	Andreas Hindborg, Wedson Almeida Filho, Niklas Cassel, Greg KH,
	Matthew Wilcox, Miguel Ojeda, Alex Gaynor, Boqun Feng, Gary Guo,
	Björn Roy Baron, Alice Ryhl, Chaitanya Kulkarni,
	Luis Chamberlain, Yexuan Yang, Sergio González Collado,
	Joel Granados, Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com, Ming Lei

Benno Lossin <benno.lossin@proton.me> writes:

> On 04.04.24 11:30, Andreas Hindborg wrote:
>> Benno Lossin <benno.lossin@proton.me> writes:
>> 
>>> On 04.04.24 07:44, Andreas Hindborg wrote:
>>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>>
>>>>> On 03.04.24 10:46, Andreas Hindborg wrote:
>>>>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>>>>
>>>>>>> On 23.03.24 07:32, Andreas Hindborg wrote:
>>>>>>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>>>>>>> On 3/13/24 12:05, Andreas Hindborg wrote:
>>>>>>>>>> +//! implementations of the `Operations` trait.
>>>>>>>>>> +//!
>>>>>>>>>> +//! IO requests are passed to the driver as [`Request`] references. The
>>>>>>>>>> +//! `Request` type is a wrapper around the C `struct request`. The driver must
>>>>>>>>>> +//! mark start of request processing by calling [`Request::start`] and end of
>>>>>>>>>> +//! processing by calling one of the [`Request::end`], methods. Failure to do so
>>>>>>>>>> +//! can lead to IO failures.
>>>>>>>>>
>>>>>>>>> I am unfamiliar with this, what are "IO failures"?
>>>>>>>>> Do you think that it might be better to change the API to use a
>>>>>>>>> callback? So instead of calling start and end, you would do
>>>>>>>>>
>>>>>>>>>          request.handle(|req| {
>>>>>>>>>              // do the stuff that would be done between start and end
>>>>>>>>>          });
>>>>>>>>>
>>>>>>>>> I took a quick look at the rnull driver and there you are calling
>>>>>>>>> `Request::end_ok` from a different function. So my suggestion might not
>>>>>>>>> be possible, since you really need the freedom.
>>>>>>>>>
>>>>>>>>> Do you think that a guard approach might work better? ie `start` returns
>>>>>>>>> a guard that when dropped will call `end` and you need the guard to
>>>>>>>>> operate on the request.
>>>>>>>>
>>>>>>>> I don't think that would fit, since the driver might not complete the
>>>>>>>> request immediately. We might be able to call `start` on behalf of the
>>>>>>>> driver.
>>>>>>>>
>>>>>>>> At any rate, since the request is reference counted now, we can
>>>>>>>> automatically fail a request when the last reference is dropped and it
>>>>>>>> was not marked successfully completed. I would need to measure the
>>>>>>>> performance implications of such a feature.
>>>>>>>
>>>>>>> Are there cases where you still need access to the request after you
>>>>>>> have called `end`?
>>>>>>
>>>>>> In general no, there is no need to handle the request after calling end.
>>>>>> C drivers are not allowed to, because this transfers ownership of the
>>>>>> request back to the block layer. This patch series defer the transfer of
>>>>>> ownership to the point when the ARef<Request> refcount goes to zero, so
>>>>>> there should be no danger associated with touching the `Request` after
>>>>>> end.
>>>>>>
>>>>>>> If no, I think it would be better for the request to
>>>>>>> be consumed by the `end` function.
>>>>>>> This is a bit difficult with `ARef`, since the user can just clone it
>>>>>>> though... Do you think that it might be necessary to clone requests?
>>>>>>
>>>>>> Looking into the details now I see that calling `Request::end` more than
>>>>>> once will trigger UAF, because C code decrements the refcount on the
>>>>>> request. When we have `ARef<Request>` around, that is a problem. It
>>>>>> probably also messes with other things in C land. Good catch.
>>>>>>
>>>>>> I did implement `Request::end` to consume the request at one point
>>>>>> before I fell back on reference counting. It works fine for simple
>>>>>> drivers. However, most drivers will need to use the block layer tag set
>>>>>> service, that allows conversion of an integer id to a request pointer.
>>>>>> The abstraction for this feature is not part of this patch set. But the
>>>>>> block layer manages a mapping of integer to request mapping, and drivers
>>>>>> typically use this to identify the request that corresponds to
>>>>>> completion messages that arrive from hardware. When drivers are able to
>>>>>> turn integers into requests like this, consuming the request in the call
>>>>>> to `end` makes little sense (because we can just construct more).
>>>>>
>>>>> How do you ensure that this is fine?:
>>>>>
>>>>>        let r1 = tagset.get(0);
>>>>>        let r2 = tagset.get(0);
>>>>>        r1.end_ok();
>>>>>        r2.do_something_that_would_only_be_done_while_active();
>>>>>
>>>>> One thing that comes to my mind would be to only give out `&Request`
>>>>> from the tag set. And to destroy, you could have a separate operation
>>>>> that also removes the request from the tag set. (I am thinking of a tag
>>>>> set as a `HashMap<u64, Request>`.
>>>>
>>>> This would be similar to
>>>>
>>>>     let r1 = tagset.get(0)?;
>>>>     ler r2 = r1.clone();
>>>>     r1.end_ok();
>>>>     r2.do_something_requires_active();
>>>>
>>>> but it is not a problem because we do not implement any actions that are
>>>> illegal in that position (outside of `end` - that _is_ a problem).
>>>
>>> Makes sense, but I think it's a bit weird to still be able to access it
>>> after `end`ing.
>> 
>> Yes, that is true.
>> 
>>>
>>>>
>>>>
>>>>>>
>>>>>> What I do now is issue the an `Option<ARef<Request>>` with
>>>>>> `bindings::req_ref_inc_not_zero(rq_ptr)`, to make sure that the request
>>>>>> is currently owned by the driver.
>>>>>>
>>>>>> I guess we can check the absolute value of the refcount, and only issue
>>>>>> a request handle if the count matches what we expect. Then we can be certain
>>>>>> that the handle is unique, and we can require transfer of ownership of
>>>>>> the handle to `Request::end` to make sure it can never be called more
>>>>>> than once.
>>>>>>
>>>>>> Another option is to error out in `Request::end` if the
>>>>>> refcount is not what we expect.
>>>>>
>>>>> I am a bit confused, why does the refcount matter in this case? Can't
>>>>> the user just have multiple `ARef`s?
>>>>
>>>> Because we want to assert that we are consuming the last handle to the
>>>> request. After we do that, the user cannot call `Request::end` again.
>>>> `TagSet::get` will not issue a request reference if the request is not
>>>> in flight. Although there might be a race condition to watch out for.
>>>>
>>>> When the block layer hands over ownership to Rust, the reference count
>>>> is 1. The first `ARef<Request>` we create increments the count to 2. To
>>>> complete the request, we must have ownership of all reference counts
>>>> above 1. The block layer takes the last reference count when it takes
>>>> back ownership of the request.
>>>>
>>>>> I think it would be weird to use `ARef<Request>` if you expect the
>>>>> refcount to be 1.
>>>>
>>>> Yes, that would require a custom smart pointer with a `try_into_unique`
>>>> method that succeeds when the refcount is exactly 2. It would consume
>>>> the instance and decrement the refcount to 1. But as I said, there is a
>>>> potential race with `TagSet::get` when the refcount is 1 that needs to
>>>> be handled.
>>>>
>>>>> Maybe the API should be different?
>>>>
>>>> I needs to change a little, yes.
>>>>
>>>>> As I understand it, a request has the following life cycle (please
>>>>> correct me if I am wrong):
>>>>> 1. A new request is created, it is given to the driver via `queue_rq`.
>>>>> 2. The driver can now decide what to do with it (theoretically it can
>>>>>       store it somewhere and later do something with it), but it should at
>>>>>       some point call `Request::start`.
>>>>> 3. Work happens and eventually the driver calls `Request::end`.
>>>>>
>>>>> To me this does not seem like something where we need a refcount (we
>>>>> still might need one for safety, but it does not need to be exposed to
>>>>> the user).
>>>>
>>>> It would not need to be exposed to the user, other than a) ending a request
>>>> can fail OR b) `TagSet::get` can fail.
>>>>
>>>> a) would require that ending a request must be done with a unique
>>>> reference. This could be done by the user by the user calling
>>>> `try_into_unique` or by making the `end` method fallible.
>>>>
>>>> b) would make the reference handle `!Clone` and add a failure mode to
>>>> `TagSet::get`, so it fails to construct a `Request` handle if there are
>>>> already one in existence.
>>>>
>>>> I gravitate towards a) because it allows the user to clone the Request
>>>> reference without adding an additional `Arc`.
>>>
>>> This confuses me a little, since I thought that `TagSet::get` returns
>>> `Option<ARef<Request>>`.
>> 
>> It does, but in the current implementation the failure mode returning
>> `None` is triggered when the refcount is zero, meaning that the request
>> corresponding to that tag is not currently owned by the driver. For
>> solution b) we would change the type to be
>> `Option<CustomSmartPointerHandleThing<Request>>`.
>> 
>>> (I tried to find the abstractions in your
>>> github, but I did not find them)
>> 
>> It's here [1]. It was introduced in the `rnvme-v6.8` branch.
>
> Thanks for the pointer.
>
>>> I think that this could work: `queue_rq` takes a `OwnedRequest`, which
>>> the user can store in a `TagSet`, transferring ownership. `TagSet::get`
>>> returns `Option<&Request>` and you can call `TagSet::remove` to get
>>> `Option<OwnedRequest>`. `OwnedRequest::end` consumes `self`.
>>> With this pattern we also do not need to take an additional refcount.
>> 
>> It would, but the `TagSet` is just a wrapper for the C block layer
>> `strugt blk_mq_tag_set`. This is a highly optimized data structure and
>> tag mapping is done before the driver sees the request. I would like to
>> reuse that logic.
>> 
>> We could implement what you suggest anyhow, but I would not want to that
>> additional logic to the hot path.
>
> I overlooked an important detail: the `TagSet` is always stored in an
> `Arc` (IIRC since you want to be able to share it between different
> `Gendisk`s). This probably makes my suggestion impossible, since you
> can't mutably borrow the `TagSet` for removal of `Request`s.
> Depending on how `Request`s are associated to a `TagSet`, there might be
> a way around this: I saw the `qid` parameter to the `tag_to_rq`
> function, is that a unique identifier for a queue?

A tag set services a number of request queues. Each queue has a number
used to identify it within the tag set. It is unique within the tag set.

> Because in that case
> we might be able to have a unique `QueueTagSetRef` with
>
>      fn remove(&mut self, tag: u32) -> OwnedRequest;

We would not need exclusive access. The tag set remove are synchronized
internally with some fancy atomic bit flipping [1].

>
>      fn get(&self, tag: u32) -> Option<&Request>;
>
> The `TagSet` would still be shared, only the ability to "remove" (I
> don't know if you do that manually in C, if not, then this would just
> remove it in the abstraction, but keep it on the C side) is unique to
> the `QueueTagSetRef` struct.

I would not advice removing tag->request associations from the driver. I
understand your point and from the perspective of these patches it makes
sense. But it would be a major layer violation of the current block
layer architecture, as far as I can tell.

I am having trouble enough trying to justify deferred free of the
request structure as it is.

> But feel free to use your proposed option a), it is simpler and we can
> try to make this work when you send the `TagSet` abstractions.
> I just think that we should try a bit harder to make it even better.

I'll code it up a) and see how it looks (and what it costs in
performance) 👍

BR Andreas


[1] https://github.com/metaspace/linux/blob/bf25426ad5652319528c6f87b74dd026fbedb9e8/lib/sbitmap.c#L638

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

* Re: [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module
  2024-04-05  8:43                 ` Andreas Hindborg
@ 2024-04-05  9:40                   ` Benno Lossin
  0 siblings, 0 replies; 26+ messages in thread
From: Benno Lossin @ 2024-04-05  9:40 UTC (permalink / raw)
  To: Andreas Hindborg
  Cc: Jens Axboe, Christoph Hellwig, Keith Busch, Damien Le Moal,
	Bart Van Assche, Hannes Reinecke, linux-block@vger.kernel.org,
	Andreas Hindborg, Wedson Almeida Filho, Niklas Cassel, Greg KH,
	Matthew Wilcox, Miguel Ojeda, Alex Gaynor, Boqun Feng, Gary Guo,
	Björn Roy Baron, Alice Ryhl, Chaitanya Kulkarni,
	Luis Chamberlain, Yexuan Yang, Sergio González Collado,
	Joel Granados, Pankaj Raghav (Samsung), Daniel Gomez, open list,
	rust-for-linux@vger.kernel.org, lsf-pc@lists.linux-foundation.org,
	gost.dev@samsung.com, Ming Lei

On 05.04.24 10:43, Andreas Hindborg wrote:
> Benno Lossin <benno.lossin@proton.me> writes:
> 
>> On 04.04.24 11:30, Andreas Hindborg wrote:
>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>
>>>> On 04.04.24 07:44, Andreas Hindborg wrote:
>>>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>>>
>>>>>> On 03.04.24 10:46, Andreas Hindborg wrote:
>>>>>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>>>>>
>>>>>>>> On 23.03.24 07:32, Andreas Hindborg wrote:
>>>>>>>>> Benno Lossin <benno.lossin@proton.me> writes:
>>>>>>>>>> On 3/13/24 12:05, Andreas Hindborg wrote:
>>>>>>>>>>> +//! implementations of the `Operations` trait.
>>>>>>>>>>> +//!
>>>>>>>>>>> +//! IO requests are passed to the driver as [`Request`] references. The
>>>>>>>>>>> +//! `Request` type is a wrapper around the C `struct request`. The driver must
>>>>>>>>>>> +//! mark start of request processing by calling [`Request::start`] and end of
>>>>>>>>>>> +//! processing by calling one of the [`Request::end`], methods. Failure to do so
>>>>>>>>>>> +//! can lead to IO failures.
>>>>>>>>>>
>>>>>>>>>> I am unfamiliar with this, what are "IO failures"?
>>>>>>>>>> Do you think that it might be better to change the API to use a
>>>>>>>>>> callback? So instead of calling start and end, you would do
>>>>>>>>>>
>>>>>>>>>>           request.handle(|req| {
>>>>>>>>>>               // do the stuff that would be done between start and end
>>>>>>>>>>           });
>>>>>>>>>>
>>>>>>>>>> I took a quick look at the rnull driver and there you are calling
>>>>>>>>>> `Request::end_ok` from a different function. So my suggestion might not
>>>>>>>>>> be possible, since you really need the freedom.
>>>>>>>>>>
>>>>>>>>>> Do you think that a guard approach might work better? ie `start` returns
>>>>>>>>>> a guard that when dropped will call `end` and you need the guard to
>>>>>>>>>> operate on the request.
>>>>>>>>>
>>>>>>>>> I don't think that would fit, since the driver might not complete the
>>>>>>>>> request immediately. We might be able to call `start` on behalf of the
>>>>>>>>> driver.
>>>>>>>>>
>>>>>>>>> At any rate, since the request is reference counted now, we can
>>>>>>>>> automatically fail a request when the last reference is dropped and it
>>>>>>>>> was not marked successfully completed. I would need to measure the
>>>>>>>>> performance implications of such a feature.
>>>>>>>>
>>>>>>>> Are there cases where you still need access to the request after you
>>>>>>>> have called `end`?
>>>>>>>
>>>>>>> In general no, there is no need to handle the request after calling end.
>>>>>>> C drivers are not allowed to, because this transfers ownership of the
>>>>>>> request back to the block layer. This patch series defer the transfer of
>>>>>>> ownership to the point when the ARef<Request> refcount goes to zero, so
>>>>>>> there should be no danger associated with touching the `Request` after
>>>>>>> end.
>>>>>>>
>>>>>>>> If no, I think it would be better for the request to
>>>>>>>> be consumed by the `end` function.
>>>>>>>> This is a bit difficult with `ARef`, since the user can just clone it
>>>>>>>> though... Do you think that it might be necessary to clone requests?
>>>>>>>
>>>>>>> Looking into the details now I see that calling `Request::end` more than
>>>>>>> once will trigger UAF, because C code decrements the refcount on the
>>>>>>> request. When we have `ARef<Request>` around, that is a problem. It
>>>>>>> probably also messes with other things in C land. Good catch.
>>>>>>>
>>>>>>> I did implement `Request::end` to consume the request at one point
>>>>>>> before I fell back on reference counting. It works fine for simple
>>>>>>> drivers. However, most drivers will need to use the block layer tag set
>>>>>>> service, that allows conversion of an integer id to a request pointer.
>>>>>>> The abstraction for this feature is not part of this patch set. But the
>>>>>>> block layer manages a mapping of integer to request mapping, and drivers
>>>>>>> typically use this to identify the request that corresponds to
>>>>>>> completion messages that arrive from hardware. When drivers are able to
>>>>>>> turn integers into requests like this, consuming the request in the call
>>>>>>> to `end` makes little sense (because we can just construct more).
>>>>>>
>>>>>> How do you ensure that this is fine?:
>>>>>>
>>>>>>         let r1 = tagset.get(0);
>>>>>>         let r2 = tagset.get(0);
>>>>>>         r1.end_ok();
>>>>>>         r2.do_something_that_would_only_be_done_while_active();
>>>>>>
>>>>>> One thing that comes to my mind would be to only give out `&Request`
>>>>>> from the tag set. And to destroy, you could have a separate operation
>>>>>> that also removes the request from the tag set. (I am thinking of a tag
>>>>>> set as a `HashMap<u64, Request>`.
>>>>>
>>>>> This would be similar to
>>>>>
>>>>>      let r1 = tagset.get(0)?;
>>>>>      ler r2 = r1.clone();
>>>>>      r1.end_ok();
>>>>>      r2.do_something_requires_active();
>>>>>
>>>>> but it is not a problem because we do not implement any actions that are
>>>>> illegal in that position (outside of `end` - that _is_ a problem).
>>>>
>>>> Makes sense, but I think it's a bit weird to still be able to access it
>>>> after `end`ing.
>>>
>>> Yes, that is true.
>>>
>>>>
>>>>>
>>>>>
>>>>>>>
>>>>>>> What I do now is issue the an `Option<ARef<Request>>` with
>>>>>>> `bindings::req_ref_inc_not_zero(rq_ptr)`, to make sure that the request
>>>>>>> is currently owned by the driver.
>>>>>>>
>>>>>>> I guess we can check the absolute value of the refcount, and only issue
>>>>>>> a request handle if the count matches what we expect. Then we can be certain
>>>>>>> that the handle is unique, and we can require transfer of ownership of
>>>>>>> the handle to `Request::end` to make sure it can never be called more
>>>>>>> than once.
>>>>>>>
>>>>>>> Another option is to error out in `Request::end` if the
>>>>>>> refcount is not what we expect.
>>>>>>
>>>>>> I am a bit confused, why does the refcount matter in this case? Can't
>>>>>> the user just have multiple `ARef`s?
>>>>>
>>>>> Because we want to assert that we are consuming the last handle to the
>>>>> request. After we do that, the user cannot call `Request::end` again.
>>>>> `TagSet::get` will not issue a request reference if the request is not
>>>>> in flight. Although there might be a race condition to watch out for.
>>>>>
>>>>> When the block layer hands over ownership to Rust, the reference count
>>>>> is 1. The first `ARef<Request>` we create increments the count to 2. To
>>>>> complete the request, we must have ownership of all reference counts
>>>>> above 1. The block layer takes the last reference count when it takes
>>>>> back ownership of the request.
>>>>>
>>>>>> I think it would be weird to use `ARef<Request>` if you expect the
>>>>>> refcount to be 1.
>>>>>
>>>>> Yes, that would require a custom smart pointer with a `try_into_unique`
>>>>> method that succeeds when the refcount is exactly 2. It would consume
>>>>> the instance and decrement the refcount to 1. But as I said, there is a
>>>>> potential race with `TagSet::get` when the refcount is 1 that needs to
>>>>> be handled.
>>>>>
>>>>>> Maybe the API should be different?
>>>>>
>>>>> I needs to change a little, yes.
>>>>>
>>>>>> As I understand it, a request has the following life cycle (please
>>>>>> correct me if I am wrong):
>>>>>> 1. A new request is created, it is given to the driver via `queue_rq`.
>>>>>> 2. The driver can now decide what to do with it (theoretically it can
>>>>>>        store it somewhere and later do something with it), but it should at
>>>>>>        some point call `Request::start`.
>>>>>> 3. Work happens and eventually the driver calls `Request::end`.
>>>>>>
>>>>>> To me this does not seem like something where we need a refcount (we
>>>>>> still might need one for safety, but it does not need to be exposed to
>>>>>> the user).
>>>>>
>>>>> It would not need to be exposed to the user, other than a) ending a request
>>>>> can fail OR b) `TagSet::get` can fail.
>>>>>
>>>>> a) would require that ending a request must be done with a unique
>>>>> reference. This could be done by the user by the user calling
>>>>> `try_into_unique` or by making the `end` method fallible.
>>>>>
>>>>> b) would make the reference handle `!Clone` and add a failure mode to
>>>>> `TagSet::get`, so it fails to construct a `Request` handle if there are
>>>>> already one in existence.
>>>>>
>>>>> I gravitate towards a) because it allows the user to clone the Request
>>>>> reference without adding an additional `Arc`.
>>>>
>>>> This confuses me a little, since I thought that `TagSet::get` returns
>>>> `Option<ARef<Request>>`.
>>>
>>> It does, but in the current implementation the failure mode returning
>>> `None` is triggered when the refcount is zero, meaning that the request
>>> corresponding to that tag is not currently owned by the driver. For
>>> solution b) we would change the type to be
>>> `Option<CustomSmartPointerHandleThing<Request>>`.
>>>
>>>> (I tried to find the abstractions in your
>>>> github, but I did not find them)
>>>
>>> It's here [1]. It was introduced in the `rnvme-v6.8` branch.
>>
>> Thanks for the pointer.
>>
>>>> I think that this could work: `queue_rq` takes a `OwnedRequest`, which
>>>> the user can store in a `TagSet`, transferring ownership. `TagSet::get`
>>>> returns `Option<&Request>` and you can call `TagSet::remove` to get
>>>> `Option<OwnedRequest>`. `OwnedRequest::end` consumes `self`.
>>>> With this pattern we also do not need to take an additional refcount.
>>>
>>> It would, but the `TagSet` is just a wrapper for the C block layer
>>> `strugt blk_mq_tag_set`. This is a highly optimized data structure and
>>> tag mapping is done before the driver sees the request. I would like to
>>> reuse that logic.
>>>
>>> We could implement what you suggest anyhow, but I would not want to that
>>> additional logic to the hot path.
>>
>> I overlooked an important detail: the `TagSet` is always stored in an
>> `Arc` (IIRC since you want to be able to share it between different
>> `Gendisk`s). This probably makes my suggestion impossible, since you
>> can't mutably borrow the `TagSet` for removal of `Request`s.
>> Depending on how `Request`s are associated to a `TagSet`, there might be
>> a way around this: I saw the `qid` parameter to the `tag_to_rq`
>> function, is that a unique identifier for a queue?
> 
> A tag set services a number of request queues. Each queue has a number
> used to identify it within the tag set. It is unique within the tag set.
> 
>> Because in that case
>> we might be able to have a unique `QueueTagSetRef` with
>>
>>       fn remove(&mut self, tag: u32) -> OwnedRequest;
> 
> We would not need exclusive access. The tag set remove are synchronized
> internally with some fancy atomic bit flipping [1].

If we bind the ability to call `Request::end` to `OwnedRequest` and
require exclusive access to the `QueueTagSetRef`, then we could ensure
that the `end` function is only called once.

>>
>>       fn get(&self, tag: u32) -> Option<&Request>;
>>
>> The `TagSet` would still be shared, only the ability to "remove" (I
>> don't know if you do that manually in C, if not, then this would just
>> remove it in the abstraction, but keep it on the C side) is unique to
>> the `QueueTagSetRef` struct.
> 
> I would not advice removing tag->request associations from the driver. I
> understand your point and from the perspective of these patches it makes
> sense. But it would be a major layer violation of the current block
> layer architecture, as far as I can tell.

Ah I should have specified this better: we don't remove the request from
the C side, only from the `TagSet` Rust abstraction. Maybe a better name
would be `end_request` (the function would then return bool to indicate
if there was a request with that tag).

> I am having trouble enough trying to justify deferred free of the
> request structure as it is.

Using this approach, there also would not be a deferred free, as we
would call `end` immediately, right?

>> But feel free to use your proposed option a), it is simpler and we can
>> try to make this work when you send the `TagSet` abstractions.
>> I just think that we should try a bit harder to make it even better.
> 
> I'll code it up a) and see how it looks (and what it costs in
> performance) 👍

Sure.

We can also speak about this in the meeting, I have the feeling that
that would be easier than trying via mail :)

-- 
Cheers,
Benno


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

end of thread, other threads:[~2024-04-05  9:40 UTC | newest]

Thread overview: 26+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
2024-03-22 23:40 [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module Benno Lossin
2024-03-23  6:32 ` Andreas Hindborg
2024-04-02 23:09   ` Benno Lossin
2024-04-03  8:46     ` Andreas Hindborg
2024-04-03 19:37       ` Benno Lossin
2024-04-04  5:44         ` Andreas Hindborg
2024-04-04  8:46           ` Benno Lossin
2024-04-04  9:30             ` Andreas Hindborg
2024-04-04 13:20               ` Benno Lossin
2024-04-05  8:43                 ` Andreas Hindborg
2024-04-05  9:40                   ` Benno Lossin
  -- strict thread matches above, loose matches on Subject: below --
2024-03-13 11:05 [RFC PATCH 0/5] Rust block device driver API and null block driver Andreas Hindborg
2024-03-13 11:05 ` [RFC PATCH 1/5] rust: block: introduce `kernel::block::mq` module Andreas Hindborg
2024-03-13 23:55   ` Boqun Feng
2024-03-14  8:58     ` Andreas Hindborg
2024-03-14 18:55       ` Miguel Ojeda
2024-03-14 19:22         ` Andreas Hindborg
2024-03-14 19:41           ` Andreas Hindborg
2024-03-14 19:41           ` Miguel Ojeda
2024-03-14 20:56             ` Miguel Ojeda
2024-03-15  7:52             ` Andreas Hindborg
2024-03-15 12:17               ` Ming Lei
2024-03-15 12:46                 ` Andreas Hindborg
2024-03-15 15:24                   ` Ming Lei
2024-03-15 17:49                     ` Andreas Hindborg
2024-03-16 14:48                       ` Ming Lei
2024-03-16 17:27                         ` Andreas Hindborg

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).