linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH V10 0/5] fuse: Add support for passthrough read/write
@ 2020-10-26 12:50 Alessio Balsini
  2020-10-26 12:50 ` [PATCH V10 1/5] fuse: Definitions and ioctl() for passthrough Alessio Balsini
                   ` (5 more replies)
  0 siblings, 6 replies; 13+ messages in thread
From: Alessio Balsini @ 2020-10-26 12:50 UTC (permalink / raw)
  To: Miklos Szeredi
  Cc: Akilesh Kailash, Amir Goldstein, Antonio SJ Musumeci,
	David Anderson, Giuseppe Scrivano, Jann Horn, Jens Axboe,
	Martijn Coenen, Palmer Dabbelt, Paul Lawrence, Stefano Duo,
	Zimuzo Ezeozue, fuse-devel, kernel-team, linux-fsdevel,
	linux-kernel

This is the 10th version of the series. Please find the changelog at the
bottom of this cover letter.

Add support for file system passthrough read/write of files when enabled in
userspace through the option FUSE_PASSTHROUGH.

There are file systems based on FUSE that are intended to enforce special
policies or trigger complicated decision makings at the file operations
level. Android, for example, uses FUSE to enforce fine-grained access
policies that also depend on the file contents.
Sometimes it happens that at open or create time a file is identified as
not requiring additional checks for consequent reads/writes, thus FUSE
would simply act as a passive bridge between the process accessing the FUSE
file system and the lower file system. Splicing and caching help reduce the
FUSE overhead, but there are still read/write operations forwarded to the
userspace FUSE daemon that could be avoided.

This series has been inspired by the original patches from Nikhilesh Reddy,
the idea and code of which has been elaborated and improved thanks to the
community support.

When the FUSE_PASSTHROUGH capability is enabled, the FUSE daemon may decide
while handling the open/create operations, if the given file can be
accessed in passthrough mode. This means that all the further read and
write operations would be forwarded by the kernel directly to the lower
file system using the VFS layer rather than to the FUSE daemon.
All the requests other than reads or writes are still handled by the
userspace FUSE daemon.
This allows for improved performance on reads and writes, especially in the
case of reads at random offsets, for which no (readahead) caching mechanism
would help.
Benchmarks show improved performance that is close to native file system
access when doing massive manipulations on a single opened file, especially
in the case of random reads, for which the bandwidth increased by almost 2X
or sequential writes for which the improvement is close to 3X.

The creation of this direct connection (passthrough) between FUSE file
objects and file objects in the lower file system happens in a way that
reminds of passing file descriptors via sockets:
- a process requests the opening of a file handled by FUSE, so the kernel
  forwards the request to the FUSE daemon;
- the FUSE daemon opens the target file in the lower file system, getting
  its file descriptor;
- the FUSE daemon also decides according to its internal policies if
  passthrough can be enabled for that file, and, if so, can perform a
  FUSE_DEV_IOC_PASSTHROUGH_OPEN ioctl() on /dev/fuse, passing the file
  descriptor obtained at the previous step and the fuse_req unique
  identifier;
- the kernel translates the file descriptor to the file pointer navigating
  through the opened files of the "current" process and temporarily stores
  it in the associated open/create fuse_req's passthrough_filp;
- when the FUSE daemon has done with the request and it's time for the
  kernel to close it, it checks if the passthrough_filp is available and in
  case updates the additional field in the fuse_file owned by the process
  accessing the FUSE file system.
From now on, all the read/write operations performed by that process will
be redirected to the corresponding lower file system file by creating new
VFS requests.
Since the read/write operation to the lower file system is executed with
the current process's credentials, it might happen that it does not have
enough privileges to succeed. For this reason, the process temporarily
receives the same credentials as the FUSE daemon, that are reverted as soon
as the read/write operation completes, emulating the behavior of the
request to be performed by the FUSE daemon itself. This solution has been
inspired by the way overlayfs handles read/write operations.
Asynchronous IO is supported as well, handled by creating separate AIO
requests for the lower file system that will be internally tracked by FUSE,
that intercepts and propagates their completion through an internal
ki_completed callback similar to the current implementation of overlayfs.
The ioctl() has been designed taking as a reference and trying to converge
to the fuse2 implementation. For example, the fuse_passthrough_out data
structure has extra fields that will allow for further extensions of the
feature.


    Performance on SSD

What follows has been performed with this change [V6] rebased on top of
vanilla v5.8 Linux kernel, using a custom passthrough_hp FUSE daemon that
enables pass-through for each file that is opened during both "open" and
"create". Tests were run on an Intel Xeon E5-2678V3, 32GiB of RAM, with an
ext4-formatted SSD as the lower file system, with no special tuning, e.g.,
all the involved processes are SCHED_OTHER, ondemand is the frequency
governor with no frequency restrictions, and turbo-boost, as well as
p-state, are active. This is because I noticed that, for such high-level
benchmarks, results consistency was minimally affected by these features.
The source code of the updated libfuse library and passthrough_hp is shared
at the following repository:

    https://github.com/balsini/libfuse/tree/fuse-passthrough-stable-v.3.9.4

Two different kinds of benchmarks were done for this change, the first set
of tests evaluates the bandwidth improvements when manipulating a huge
single file, the second set of tests verify that no performance regressions
were introduced when handling many small files.

The first benchmarks were done by running FIO (fio-3.21) with:
- bs=4Ki;
- file size: 50Gi;
- ioengine: sync;
- fsync_on_close: true.
The target file has been chosen large enough to avoid it to be entirely
loaded into the page cache.
Results are presented in the following table:

+-----------+--------+-------------+--------+
| Bandwidth |  FUSE  |     FUSE    |  Bind  |
|  (KiB/s)  |        | passthrough |  mount |
+-----------+--------+-------------+--------+
| read      | 468897 |      502085 | 516830 |
+-----------+--------+-------------+--------+
| randread  |  15773 |       26632 |  21386 |
+-----------+--------+-------------+--------+
| write     |  58185 |      141272 | 141671 |
+-----------+--------+-------------+--------+
| randwrite |  59892 |       75236 |  76486 |
+-----------+--------+-------------+--------+

The higher FUSE passthrough performance compared to bind mount in the case
of randread has been identified as the result or SSD performance
fluctuations when dealing with random offsets. Updated results are reported
below, with the measurements performed on a lower file system created on
top of a RAM block device.

As long as this patch has the primary objective of improving bandwidth,
another set of tests has been performed to see how this behaves on a
totally different scenario that involves accessing many small files. For
this purpose, measuring the build time of the Linux kernel has been chosen
as a well-known workload. The kernel has been built with as many processes
as the number of logical CPUs (-j $(nproc)), that besides being a
reasonable number, is also enough to saturate the processor’s utilization
thanks to the additional FUSE daemon’s threads, making it even harder to
get closer to the native file system performance.
The following table shows the total build times in the different
configurations:

+------------------+--------------+-----------+
|                  | AVG duration |  Standard |
|                  |     (sec)    | deviation |
+------------------+--------------+-----------+
| FUSE             |      144.566 |     0.697 |
+------------------+--------------+-----------+
| FUSE passthrough |      133.820 |     0.341 |
+------------------+--------------+-----------+
| Raw              |      109.423 |     0.724 |
+------------------+--------------+-----------+

Similar performance measurements were performed with the current version of
the patch, the results of which are comparable with what is shown above.


    Performance on RAM block device

Getting rid of the discrete storage device removes a huge component of
slowness, highlighting the performance difference of the software parts
(and probably goodness of CPU cache and its coherence/invalidation
mechanisms).
What follows has been performed with this change [V10] rebased on top of
vanilla v5.8 Linux kernel.

More specifically, out of my system's 32 GiB of RAM, I reserved 24 for
/dev/ram0, which has been formatted as ext4. That file system has been
completely filled and then cleaned up before running the benchmarks to make
sure all the memory addresses were marked as used and removed from the page
cache.

The following tests were ran using fio-3.23 with the following
configuration:
- bs=4Ki
- size=20Gi
- ioengine=sync
- fsync_on_close=1
- randseed=0
- create_only=0 (set to 1 during a first dry run to create the test
  file)

As for the tool configuration, the following benchmarks would perform a
single open operation each, focusing on just the read/write performance.

The file size of 20 GiB has been chosen to not completely fit the page
cache.

As mentioned in my previous email, all the caches were dropped before
running every benchmark with

  echo 3 > /proc/sys/vm/drop_caches

All the benchmarks were run 10 times, with 1 minute cool down between each
run.

Here the updated results for this patch set:

+-----------+-------------+-------------+-------------+
|           |             | FUSE        |             |
| MiB/s     | FUSE        | passthrough | native      |
+-----------+-------------+-------------+-------------+
| read      | 1341(±4.2%) | 1485(±1.1%) |  1634(±.5%) |
+-----------+-------------+-------------+-------------+
| write     |   49(±2.1%) | 1304(±2.6%) | 1363(±3.0%) |
+-----------+-------------+-------------+-------------+
| randread  |   43(±1.3%) | 643(±11.1%) |  715(±1.1%) |
+-----------+-------------+-------------+-------------+
| randwrite |  27(±39.9%) |  763(±1.1%) |  790(±1.0%) |
+-----------+-------------+-------------+-------------+

This table shows that FUSE, except for the sequential reads, is left behind
FUSE passthrough and native performance. The extremely good FUSE
performance for sequential reads is the result of a great read-ahead
mechanism, that has been easy to prove by showing that performance dropped
after setting read_ahead_kb to 0.
Except for FUSE randwrite and passthrough randread with respectively ~40%
and ~11% standard deviations, all the other results are relatively stable.
Nevertheless, these two standard deviation exceptions are not sufficient to
invalidate the results, which are still showing clear performance benefits.
I'm also kind of happy to see that passthrough, that for each read/write
operation traverses the VFS layer twice, now maintains consistent slightly
lower performance than native.

Further testing and performance evaluations are welcome.


    Description of the series

Patch 1 introduces the data structures and function signatures required
both for the communication with userspace and for the internal kernel use.

Patch 2 introduces the ioctl() and initialization and release functions for
FUSE passthrough.

Patch 3 enables the synchronous read and write operations for those FUSE
files for which the passthrough functionality is enabled.

Patch 4 extends the read and write operations to also support asynchronous
IO.

Patch 5 allows FUSE passthrough to target files for which the requesting
process would not have direct access to, by temporarily performing a
credentials switch to the credentials of the FUSE daemon that issued the
FUSE passthrough ioctl().


    Changelog

Changes in v10:
* UAPI updated: ioctl() now returns an ID that will be used at
  open/create response time to reference the passthrough file
* Synchronous read/write_iter functions does not return silly errors (fixed
  in aio patch)
* FUSE daemon credentials updated at ioctl() time instead of mount time
* Updated benchmark results with RAM block device
  [Requested by Miklos Szeredi]

Changes in v9:
* Switched to using VFS instead of direct lower FS file ops
  [Attempt to address a request from Jens Axboe, Jann Horn, Amir
Goldstein]
* Removal of useless included aio.h header
  [Proposed by Jens Axboe]

Changes in v8:
* aio requests now use kmalloc/kfree, instead of kmem_cache
* Switched to call_{read,write}_iter in AIO
* Revisited attributes copy
* Passthrough can only be enabled via ioctl(), fixing the security issue
  spotted by Jann
* Use an extensible fuse_passthrough_out data structure
  [Attempt to address a request from Nikolaus Rath, Amir Goldstein and
  Miklos Szeredi]

Changes in v7:
* Full handling of aio requests as done in overlayfs (update commit
* message).
* s/fget_raw/fget.
* Open fails in case of passthrough errors, emitting warning messages.
  [Proposed by Jann Horn]
* Create new local kiocb, getting rid of the previously proposed ki_filp
* swapping.
  [Proposed by Jann Horn and Jens Axboe]
* Code polishing.

Changes in v6:
* Port to kernel v5.8:
  * fuse_file_{read,write}_iter() changed since the v5 of this patch was
  * proposed.
* Simplify fuse_simple_request().
* Merge fuse_passthrough.h into fuse_i.h
* Refactor of passthrough.c:
  * Remove BUG_ON()s.
  * Simplified error checking and request arguments indexing.
  * Use call_{read,write}_iter() utility functions.
  * Remove get_file() and fputs() during read/write: handle the extra
  * FUSE references to the lower file object when the fuse_file is
  * created/deleted.
  [Proposed by Jann Horn]

Changes in v5:
* Fix the check when setting the passthrough file.
  [Found when testing by Mike Shal]

Changes in v3 and v4:
* Use the fs_stack_depth to prevent further stacking and a minor fix.
  [Proposed by Jann Horn]

Changes in v2:
* Changed the feature name to passthrough from stacked_io.
  [Proposed by Linus Torvalds]


Alessio Balsini (5):
  fuse: Definitions and ioctl() for passthrough
  fuse: Passthrough initialization and release
  fuse: Introduce synchronous read and write for passthrough
  fuse: Handle asynchronous read and write in passthrough
  fuse: Use daemon creds in passthrough mode

 fs/fuse/Makefile          |   1 +
 fs/fuse/dev.c             |  40 +++++--
 fs/fuse/dir.c             |   1 +
 fs/fuse/file.c            |  12 +-
 fs/fuse/fuse_i.h          |  31 +++++
 fs/fuse/inode.c           |  23 +++-
 fs/fuse/passthrough.c     | 245 ++++++++++++++++++++++++++++++++++++++
 include/uapi/linux/fuse.h |  13 +-
 8 files changed, 349 insertions(+), 17 deletions(-)
 create mode 100644 fs/fuse/passthrough.c

-- 
2.29.0.rc1.297.gfa9743e501-goog


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

* [PATCH V10 1/5] fuse: Definitions and ioctl() for passthrough
  2020-10-26 12:50 [PATCH V10 0/5] fuse: Add support for passthrough read/write Alessio Balsini
@ 2020-10-26 12:50 ` Alessio Balsini
  2020-10-26 12:50 ` [PATCH V10 2/5] fuse: Passthrough initialization and release Alessio Balsini
                   ` (4 subsequent siblings)
  5 siblings, 0 replies; 13+ messages in thread
From: Alessio Balsini @ 2020-10-26 12:50 UTC (permalink / raw)
  To: Miklos Szeredi
  Cc: Akilesh Kailash, Amir Goldstein, Antonio SJ Musumeci,
	David Anderson, Giuseppe Scrivano, Jann Horn, Jens Axboe,
	Martijn Coenen, Palmer Dabbelt, Paul Lawrence, Stefano Duo,
	Zimuzo Ezeozue, fuse-devel, kernel-team, linux-fsdevel,
	linux-kernel

Expose the FUSE_PASSTHROUGH interface to userspace and declare all the
basic data structures and functions as the skeleton on top of which the
FUSE passthrough functionality will be built.

As part of this, introduce the new FUSE passthrough ioctl(), which allows
the FUSE daemon to specify a direct connection between a FUSE file and a
lower file system file. Such ioctl() requires userspace to pass the file
descriptor of one of its opened files through the fuse_passthrough_out data
structure introduced in this patch. This structure includes extra fields
for possible future extensions.
Also, add the passthrough functions for the set-up and tear-down of the
data structures and locks that will be used both when fuse_conns and
fuse_files are created/deleted.

Signed-off-by: Alessio Balsini <balsini@android.com>
---
 fs/fuse/Makefile          |  1 +
 fs/fuse/dev.c             | 40 ++++++++++++++++++++++++++++-----------
 fs/fuse/dir.c             |  1 +
 fs/fuse/file.c            |  4 +++-
 fs/fuse/fuse_i.h          | 26 +++++++++++++++++++++++++
 fs/fuse/inode.c           | 18 +++++++++++++++++-
 fs/fuse/passthrough.c     | 21 ++++++++++++++++++++
 include/uapi/linux/fuse.h | 13 +++++++++++--
 8 files changed, 109 insertions(+), 15 deletions(-)
 create mode 100644 fs/fuse/passthrough.c

diff --git a/fs/fuse/Makefile b/fs/fuse/Makefile
index 3e8cebfb59b7..6971454a2bdf 100644
--- a/fs/fuse/Makefile
+++ b/fs/fuse/Makefile
@@ -8,4 +8,5 @@ obj-$(CONFIG_CUSE) += cuse.o
 obj-$(CONFIG_VIRTIO_FS) += virtiofs.o
 
 fuse-objs := dev.o dir.o file.o inode.o control.o xattr.o acl.o readdir.o
+fuse-objs += passthrough.o
 virtiofs-y += virtio_fs.o
diff --git a/fs/fuse/dev.c b/fs/fuse/dev.c
index 02b3c36b3676..bcf1da0260bc 100644
--- a/fs/fuse/dev.c
+++ b/fs/fuse/dev.c
@@ -2222,37 +2222,55 @@ static int fuse_device_clone(struct fuse_conn *fc, struct file *new)
 static long fuse_dev_ioctl(struct file *file, unsigned int cmd,
 			   unsigned long arg)
 {
-	int err = -ENOTTY;
-
-	if (cmd == FUSE_DEV_IOC_CLONE) {
-		int oldfd;
+	int res;
+	int oldfd;
+	struct fuse_dev *fud;
+	struct fuse_passthrough_out pto;
 
-		err = -EFAULT;
-		if (!get_user(oldfd, (__u32 __user *) arg)) {
+	switch (cmd) {
+	case FUSE_DEV_IOC_CLONE:
+		res = -EFAULT;
+		if (!get_user(oldfd, (__u32 __user *)arg)) {
 			struct file *old = fget(oldfd);
 
-			err = -EINVAL;
+			res = -EINVAL;
 			if (old) {
-				struct fuse_dev *fud = NULL;
+				fud = NULL;
 
 				/*
 				 * Check against file->f_op because CUSE
 				 * uses the same ioctl handler.
 				 */
 				if (old->f_op == file->f_op &&
-				    old->f_cred->user_ns == file->f_cred->user_ns)
+				    old->f_cred->user_ns ==
+					    file->f_cred->user_ns)
 					fud = fuse_get_dev(old);
 
 				if (fud) {
 					mutex_lock(&fuse_mutex);
-					err = fuse_device_clone(fud->fc, file);
+					res = fuse_device_clone(fud->fc, file);
 					mutex_unlock(&fuse_mutex);
 				}
 				fput(old);
 			}
 		}
+		break;
+	case FUSE_DEV_IOC_PASSTHROUGH_OPEN:
+		res = -EFAULT;
+		if (!copy_from_user(&pto,
+				    (struct fuse_passthrough_out __user *)arg,
+				    sizeof(pto))) {
+			res = -EINVAL;
+			fud = fuse_get_dev(file);
+			if (fud)
+				res = fuse_passthrough_open(fud, &pto);
+		}
+		break;
+	default:
+		res = -ENOTTY;
+		break;
 	}
-	return err;
+	return res;
 }
 
 const struct file_operations fuse_dev_operations = {
diff --git a/fs/fuse/dir.c b/fs/fuse/dir.c
index 26f028bc760b..875799959e33 100644
--- a/fs/fuse/dir.c
+++ b/fs/fuse/dir.c
@@ -489,6 +489,7 @@ static int fuse_create_open(struct inode *dir, struct dentry *entry,
 	ff->fh = outopen.fh;
 	ff->nodeid = outentry.nodeid;
 	ff->open_flags = outopen.open_flags;
+	fuse_passthrough_setup(fc, ff, &outopen);
 	inode = fuse_iget(dir->i_sb, outentry.nodeid, outentry.generation,
 			  &outentry.attr, entry_attr_timeout(&outentry), 0);
 	if (!inode) {
diff --git a/fs/fuse/file.c b/fs/fuse/file.c
index 83d917f7e542..84daaf084197 100644
--- a/fs/fuse/file.c
+++ b/fs/fuse/file.c
@@ -151,7 +151,7 @@ int fuse_do_open(struct fuse_conn *fc, u64 nodeid, struct file *file,
 		if (!err) {
 			ff->fh = outarg.fh;
 			ff->open_flags = outarg.open_flags;
-
+			fuse_passthrough_setup(fc, ff, &outarg);
 		} else if (err != -ENOSYS) {
 			fuse_file_free(ff);
 			return err;
@@ -281,6 +281,8 @@ void fuse_release_common(struct file *file, bool isdir)
 	struct fuse_release_args *ra = ff->release_args;
 	int opcode = isdir ? FUSE_RELEASEDIR : FUSE_RELEASE;
 
+	fuse_passthrough_release(&ff->passthrough);
+
 	fuse_prepare_release(fi, ff, file->f_flags, opcode);
 
 	if (ff->flock) {
diff --git a/fs/fuse/fuse_i.h b/fs/fuse/fuse_i.h
index 740a8a7d7ae6..32da45ce86e0 100644
--- a/fs/fuse/fuse_i.h
+++ b/fs/fuse/fuse_i.h
@@ -163,6 +163,14 @@ enum {
 struct fuse_conn;
 struct fuse_release_args;
 
+/**
+ * Reference to lower filesystem file for read/write operations handled in
+ * passthrough mode
+ */
+struct fuse_passthrough {
+	struct file *filp;
+};
+
 /** FUSE specific file data */
 struct fuse_file {
 	/** Fuse connection for this file */
@@ -208,6 +216,9 @@ struct fuse_file {
 
 	} readdir;
 
+	/** Container for data related to the passthrough functionality */
+	struct fuse_passthrough passthrough;
+
 	/** RB node to be linked on fuse_conn->polled_files */
 	struct rb_node polled_node;
 
@@ -720,6 +731,9 @@ struct fuse_conn {
 	/* Do not show mount options */
 	unsigned int no_mount_options:1;
 
+	/** Passthrough mode for read/write IO */
+	unsigned int passthrough:1;
+
 	/** The number of requests waiting for completion */
 	atomic_t num_waiting;
 
@@ -755,6 +769,12 @@ struct fuse_conn {
 
 	/** List of device instances belonging to this connection */
 	struct list_head devices;
+
+	/** IDR for passthrough requests */
+	struct idr passthrough_req;
+
+	/** Protects passthrough_req */
+	spinlock_t passthrough_req_lock;
 };
 
 static inline struct fuse_conn *get_fuse_conn_super(struct super_block *sb)
@@ -1093,4 +1113,10 @@ unsigned int fuse_len_args(unsigned int numargs, struct fuse_arg *args);
 u64 fuse_get_unique(struct fuse_iqueue *fiq);
 void fuse_free_conn(struct fuse_conn *fc);
 
+int fuse_passthrough_open(struct fuse_dev *fud,
+			  struct fuse_passthrough_out *pto);
+int fuse_passthrough_setup(struct fuse_conn *fc, struct fuse_file *ff,
+			   struct fuse_open_out *openarg);
+void fuse_passthrough_release(struct fuse_passthrough *passthrough);
+
 #endif /* _FS_FUSE_I_H */
diff --git a/fs/fuse/inode.c b/fs/fuse/inode.c
index bba747520e9b..6738dd5ff5d2 100644
--- a/fs/fuse/inode.c
+++ b/fs/fuse/inode.c
@@ -621,6 +621,7 @@ void fuse_conn_init(struct fuse_conn *fc, struct user_namespace *user_ns,
 	memset(fc, 0, sizeof(*fc));
 	spin_lock_init(&fc->lock);
 	spin_lock_init(&fc->bg_lock);
+	spin_lock_init(&fc->passthrough_req_lock);
 	init_rwsem(&fc->killsb);
 	refcount_set(&fc->count, 1);
 	atomic_set(&fc->dev_count, 1);
@@ -629,6 +630,7 @@ void fuse_conn_init(struct fuse_conn *fc, struct user_namespace *user_ns,
 	INIT_LIST_HEAD(&fc->bg_queue);
 	INIT_LIST_HEAD(&fc->entry);
 	INIT_LIST_HEAD(&fc->devices);
+	idr_init(&fc->passthrough_req);
 	atomic_set(&fc->num_waiting, 0);
 	fc->max_background = FUSE_DEFAULT_MAX_BACKGROUND;
 	fc->congestion_threshold = FUSE_DEFAULT_CONGESTION_THRESHOLD;
@@ -965,6 +967,12 @@ static void process_init_reply(struct fuse_conn *fc, struct fuse_args *args,
 					min_t(unsigned int, FUSE_MAX_MAX_PAGES,
 					max_t(unsigned int, arg->max_pages, 1));
 			}
+			if (arg->flags & FUSE_PASSTHROUGH) {
+				fc->passthrough = 1;
+				/* Prevent further stacking */
+				fc->sb->s_stack_depth =
+					FILESYSTEM_MAX_STACK_DEPTH;
+			}
 		} else {
 			ra_pages = fc->max_read / PAGE_SIZE;
 			fc->no_lock = 1;
@@ -1002,7 +1010,8 @@ void fuse_send_init(struct fuse_conn *fc)
 		FUSE_WRITEBACK_CACHE | FUSE_NO_OPEN_SUPPORT |
 		FUSE_PARALLEL_DIROPS | FUSE_HANDLE_KILLPRIV | FUSE_POSIX_ACL |
 		FUSE_ABORT_ERROR | FUSE_MAX_PAGES | FUSE_CACHE_SYMLINKS |
-		FUSE_NO_OPENDIR_SUPPORT | FUSE_EXPLICIT_INVAL_DATA;
+		FUSE_NO_OPENDIR_SUPPORT | FUSE_EXPLICIT_INVAL_DATA |
+		FUSE_PASSTHROUGH;
 	ia->args.opcode = FUSE_INIT;
 	ia->args.in_numargs = 1;
 	ia->args.in_args[0].size = sizeof(ia->in);
@@ -1023,9 +1032,16 @@ void fuse_send_init(struct fuse_conn *fc)
 }
 EXPORT_SYMBOL_GPL(fuse_send_init);
 
+static int free_fuse_passthrough(int id, void *p, void *data)
+{
+	return 0;
+}
+
 void fuse_free_conn(struct fuse_conn *fc)
 {
 	WARN_ON(!list_empty(&fc->devices));
+	idr_for_each(&fc->passthrough_req, free_fuse_passthrough, NULL);
+	idr_destroy(&fc->passthrough_req);
 	kfree_rcu(fc, rcu);
 }
 EXPORT_SYMBOL_GPL(fuse_free_conn);
diff --git a/fs/fuse/passthrough.c b/fs/fuse/passthrough.c
new file mode 100644
index 000000000000..594060c654f8
--- /dev/null
+++ b/fs/fuse/passthrough.c
@@ -0,0 +1,21 @@
+// SPDX-License-Identifier: GPL-2.0
+
+#include "fuse_i.h"
+
+#include <linux/fuse.h>
+
+int fuse_passthrough_open(struct fuse_dev *fud,
+			  struct fuse_passthrough_out *pto)
+{
+	return -EINVAL;
+}
+
+int fuse_passthrough_setup(struct fuse_conn *fc, struct fuse_file *ff,
+			   struct fuse_open_out *openarg)
+{
+	return -EINVAL;
+}
+
+void fuse_passthrough_release(struct fuse_passthrough *passthrough)
+{
+}
diff --git a/include/uapi/linux/fuse.h b/include/uapi/linux/fuse.h
index 373cada89815..ae06efb25d18 100644
--- a/include/uapi/linux/fuse.h
+++ b/include/uapi/linux/fuse.h
@@ -342,6 +342,7 @@ struct fuse_file_lock {
 #define FUSE_NO_OPENDIR_SUPPORT (1 << 24)
 #define FUSE_EXPLICIT_INVAL_DATA (1 << 25)
 #define FUSE_MAP_ALIGNMENT	(1 << 26)
+#define FUSE_PASSTHROUGH	(1 << 27)
 
 /**
  * CUSE INIT request/reply flags
@@ -591,7 +592,7 @@ struct fuse_create_in {
 struct fuse_open_out {
 	uint64_t	fh;
 	uint32_t	open_flags;
-	uint32_t	padding;
+	uint32_t	passthrough_fh;
 };
 
 struct fuse_release_in {
@@ -794,6 +795,13 @@ struct fuse_in_header {
 	uint32_t	padding;
 };
 
+struct fuse_passthrough_out {
+	uint32_t	fd;
+	/* For future implementation */
+	uint32_t	len;
+	void		*vec;
+};
+
 struct fuse_out_header {
 	uint32_t	len;
 	int32_t		error;
@@ -869,7 +877,8 @@ struct fuse_notify_retrieve_in {
 };
 
 /* Device ioctls: */
-#define FUSE_DEV_IOC_CLONE	_IOR(229, 0, uint32_t)
+#define FUSE_DEV_IOC_CLONE		_IOR(229, 0, uint32_t)
+#define FUSE_DEV_IOC_PASSTHROUGH_OPEN	_IOW(229, 1, struct fuse_passthrough_out)
 
 struct fuse_lseek_in {
 	uint64_t	fh;
-- 
2.29.0.rc1.297.gfa9743e501-goog


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

* [PATCH V10 2/5] fuse: Passthrough initialization and release
  2020-10-26 12:50 [PATCH V10 0/5] fuse: Add support for passthrough read/write Alessio Balsini
  2020-10-26 12:50 ` [PATCH V10 1/5] fuse: Definitions and ioctl() for passthrough Alessio Balsini
@ 2020-10-26 12:50 ` Alessio Balsini
  2020-11-26 13:33   ` Peng Tao
       [not found]   ` <3bf58b6f-c7eb-7baa-384d-ae0830d8bceb@tcl.com>
  2020-10-26 12:50 ` [PATCH V10 3/5] fuse: Introduce synchronous read and write for passthrough Alessio Balsini
                   ` (3 subsequent siblings)
  5 siblings, 2 replies; 13+ messages in thread
From: Alessio Balsini @ 2020-10-26 12:50 UTC (permalink / raw)
  To: Miklos Szeredi
  Cc: Akilesh Kailash, Amir Goldstein, Antonio SJ Musumeci,
	David Anderson, Giuseppe Scrivano, Jann Horn, Jens Axboe,
	Martijn Coenen, Palmer Dabbelt, Paul Lawrence, Stefano Duo,
	Zimuzo Ezeozue, fuse-devel, kernel-team, linux-fsdevel,
	linux-kernel

Implement the FUSE passthrough ioctl() that associates the lower
(passthrough) file system file with the fuse_file.

The file descriptor passed to the ioctl() by the FUSE daemon is used to
access the relative file pointer, that will be copied to the fuse_file data
structure to consolidate the link between the FUSE and lower file system.

To enable the passthrough mode, userspace triggers the
FUSE_DEV_IOC_PASSTHROUGH_OPEN ioctl() and, if the call succeeds,
receives back an identifier that will be used at open/create response
time in the fuse_open_out field to associate the FUSE file to the lower
file system file.
The value returned by the ioctl() to userspace can be:
- > 0: success, the identifier can be used as part of an open/create
  reply.
- < 0: an error occurred.
The value 0 has been left unused for backward compatibility: the
fuse_open_out field that is used to pass the passthrough_fh back to the
kernel uses the same bits that were previously as struct padding,
zero-initialized in the common libfuse implementation. Removing the 0
value fixes the ambiguity between the case in which 0 corresponds to a
real passthrough_fh or a missing implementation, simplifying the
userspace implementation.

For the passthrough mode to be successfully activated, the lower file
system file must implement both read_ and write_iter file operations.
This extra check avoids special pseudo files to be targeted for this
feature.
Passthrough comes with another limitation: no further file system stacking
is allowed for those FUSE file systems using passthrough.

Signed-off-by: Alessio Balsini <balsini@android.com>
---
 fs/fuse/inode.c       |  5 +++
 fs/fuse/passthrough.c | 80 +++++++++++++++++++++++++++++++++++++++++--
 2 files changed, 83 insertions(+), 2 deletions(-)

diff --git a/fs/fuse/inode.c b/fs/fuse/inode.c
index 6738dd5ff5d2..1e94c54d1455 100644
--- a/fs/fuse/inode.c
+++ b/fs/fuse/inode.c
@@ -1034,6 +1034,11 @@ EXPORT_SYMBOL_GPL(fuse_send_init);
 
 static int free_fuse_passthrough(int id, void *p, void *data)
 {
+	struct fuse_passthrough *passthrough = (struct fuse_passthrough *)p;
+
+	fuse_passthrough_release(passthrough);
+	kfree(p);
+
 	return 0;
 }
 
diff --git a/fs/fuse/passthrough.c b/fs/fuse/passthrough.c
index 594060c654f8..a135c955cc33 100644
--- a/fs/fuse/passthrough.c
+++ b/fs/fuse/passthrough.c
@@ -3,19 +3,95 @@
 #include "fuse_i.h"
 
 #include <linux/fuse.h>
+#include <linux/idr.h>
 
 int fuse_passthrough_open(struct fuse_dev *fud,
 			  struct fuse_passthrough_out *pto)
 {
-	return -EINVAL;
+	int res;
+	struct file *passthrough_filp;
+	struct fuse_conn *fc = fud->fc;
+	struct fuse_passthrough *passthrough;
+
+	if (!fc->passthrough)
+		return -EPERM;
+
+	/* This field is reserved for future implementation */
+	if (pto->len != 0)
+		return -EINVAL;
+
+	passthrough_filp = fget(pto->fd);
+	if (!passthrough_filp) {
+		pr_err("FUSE: invalid file descriptor for passthrough.\n");
+		return -EBADF;
+	}
+
+	if (!passthrough_filp->f_op->read_iter ||
+	    !passthrough_filp->f_op->write_iter) {
+		pr_err("FUSE: passthrough file misses file operations.\n");
+		return -EBADF;
+	}
+
+	passthrough = kmalloc(sizeof(struct fuse_passthrough), GFP_KERNEL);
+	if (!passthrough)
+		return -ENOMEM;
+
+	passthrough->filp = passthrough_filp;
+
+	idr_preload(GFP_KERNEL);
+	spin_lock(&fc->passthrough_req_lock);
+	res = idr_alloc(&fc->passthrough_req, passthrough, 1, 0, GFP_ATOMIC);
+	spin_unlock(&fc->passthrough_req_lock);
+	idr_preload_end();
+	if (res <= 0) {
+		fuse_passthrough_release(passthrough);
+		kfree(passthrough);
+	}
+
+	return res;
 }
 
 int fuse_passthrough_setup(struct fuse_conn *fc, struct fuse_file *ff,
 			   struct fuse_open_out *openarg)
 {
-	return -EINVAL;
+	struct inode *passthrough_inode;
+	struct super_block *passthrough_sb;
+	struct fuse_passthrough *passthrough;
+	int passthrough_fh = openarg->passthrough_fh;
+
+	if (!fc->passthrough)
+		return -EPERM;
+
+	/* Default case, passthrough is not requested */
+	if (passthrough_fh <= 0)
+		return -EINVAL;
+
+	spin_lock(&fc->passthrough_req_lock);
+	passthrough = idr_remove(&fc->passthrough_req, passthrough_fh);
+	spin_unlock(&fc->passthrough_req_lock);
+
+	if (!passthrough)
+		return -EINVAL;
+
+	passthrough_inode = file_inode(passthrough->filp);
+	passthrough_sb = passthrough_inode->i_sb;
+	if (passthrough_sb->s_stack_depth >= FILESYSTEM_MAX_STACK_DEPTH) {
+		pr_err("FUSE: fs stacking depth exceeded for passthrough\n");
+		fuse_passthrough_release(passthrough);
+		kfree(passthrough);
+		return -EINVAL;
+	}
+
+	ff->passthrough = *passthrough;
+	kfree(passthrough);
+
+	return 0;
 }
 
 void fuse_passthrough_release(struct fuse_passthrough *passthrough)
 {
+	if (passthrough->filp) {
+		fput(passthrough->filp);
+		passthrough->filp = NULL;
+	}
 }
-- 
2.29.0.rc1.297.gfa9743e501-goog


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

* [PATCH V10 3/5] fuse: Introduce synchronous read and write for passthrough
  2020-10-26 12:50 [PATCH V10 0/5] fuse: Add support for passthrough read/write Alessio Balsini
  2020-10-26 12:50 ` [PATCH V10 1/5] fuse: Definitions and ioctl() for passthrough Alessio Balsini
  2020-10-26 12:50 ` [PATCH V10 2/5] fuse: Passthrough initialization and release Alessio Balsini
@ 2020-10-26 12:50 ` Alessio Balsini
  2020-10-26 12:50 ` [PATCH V10 4/5] fuse: Handle asynchronous read and write in passthrough Alessio Balsini
                   ` (2 subsequent siblings)
  5 siblings, 0 replies; 13+ messages in thread
From: Alessio Balsini @ 2020-10-26 12:50 UTC (permalink / raw)
  To: Miklos Szeredi
  Cc: Akilesh Kailash, Amir Goldstein, Antonio SJ Musumeci,
	David Anderson, Giuseppe Scrivano, Jann Horn, Jens Axboe,
	Martijn Coenen, Palmer Dabbelt, Paul Lawrence, Stefano Duo,
	Zimuzo Ezeozue, fuse-devel, kernel-team, linux-fsdevel,
	linux-kernel

All the read and write operations performed on fuse_files which have the
passthrough feature enabled are forwarded to the associated lower file
system file via VFS.

Sending the request directly to the lower file system avoids the userspace
round-trip that, because of possible context switches and additional
operations might reduce the overall performance, especially in those cases
where caching doesn't help, for example in reads at random offsets.

Verifying if a fuse_file has a lower file system file associated with can
be done by checking the validity of its passthrough_filp pointer. This
pointer is not NULL only if passthrough has been successfully enabled via
the appropriate ioctl().
When a read/write operation is requested for a FUSE file with passthrough
enabled, a new equivalent VFS request is generated, which instead targets
the lower file system file.
The VFS layer performs additional checks that allow for safer operations
but may cause the operation to fail if the process accessing the FUSE file
system does not have access to the lower file system.

This change only implements synchronous requests in passthrough, returning
an error in the case of asynchronous operations, yet covering the majority
of the use cases.

Signed-off-by: Alessio Balsini <balsini@android.com>
---
 fs/fuse/file.c        |  8 +++--
 fs/fuse/fuse_i.h      |  2 ++
 fs/fuse/passthrough.c | 70 +++++++++++++++++++++++++++++++++++++++++++
 3 files changed, 78 insertions(+), 2 deletions(-)

diff --git a/fs/fuse/file.c b/fs/fuse/file.c
index 84daaf084197..f7a12489c0ef 100644
--- a/fs/fuse/file.c
+++ b/fs/fuse/file.c
@@ -1545,7 +1545,9 @@ static ssize_t fuse_file_read_iter(struct kiocb *iocb, struct iov_iter *to)
 	if (is_bad_inode(file_inode(file)))
 		return -EIO;
 
-	if (!(ff->open_flags & FOPEN_DIRECT_IO))
+	if (ff->passthrough.filp)
+		return fuse_passthrough_read_iter(iocb, to);
+	else if (!(ff->open_flags & FOPEN_DIRECT_IO))
 		return fuse_cache_read_iter(iocb, to);
 	else
 		return fuse_direct_read_iter(iocb, to);
@@ -1559,7 +1561,9 @@ static ssize_t fuse_file_write_iter(struct kiocb *iocb, struct iov_iter *from)
 	if (is_bad_inode(file_inode(file)))
 		return -EIO;
 
-	if (!(ff->open_flags & FOPEN_DIRECT_IO))
+	if (ff->passthrough.filp)
+		return fuse_passthrough_write_iter(iocb, from);
+	else if (!(ff->open_flags & FOPEN_DIRECT_IO))
 		return fuse_cache_write_iter(iocb, from);
 	else
 		return fuse_direct_write_iter(iocb, from);
diff --git a/fs/fuse/fuse_i.h b/fs/fuse/fuse_i.h
index 32da45ce86e0..a888d3df5877 100644
--- a/fs/fuse/fuse_i.h
+++ b/fs/fuse/fuse_i.h
@@ -1118,5 +1118,7 @@ int fuse_passthrough_open(struct fuse_dev *fud,
 int fuse_passthrough_setup(struct fuse_conn *fc, struct fuse_file *ff,
 			   struct fuse_open_out *openarg);
 void fuse_passthrough_release(struct fuse_passthrough *passthrough);
+ssize_t fuse_passthrough_read_iter(struct kiocb *iocb, struct iov_iter *to);
+ssize_t fuse_passthrough_write_iter(struct kiocb *iocb, struct iov_iter *from);
 
 #endif /* _FS_FUSE_I_H */
diff --git a/fs/fuse/passthrough.c b/fs/fuse/passthrough.c
index a135c955cc33..5a78cb336db4 100644
--- a/fs/fuse/passthrough.c
+++ b/fs/fuse/passthrough.c
@@ -4,6 +4,76 @@
 
 #include <linux/fuse.h>
 #include <linux/idr.h>
+#include <linux/uio.h>
+
+static void fuse_copyattr(struct file *dst_file, struct file *src_file)
+{
+	struct inode *dst = file_inode(dst_file);
+	struct inode *src = file_inode(src_file);
+
+	i_size_write(dst, i_size_read(src));
+}
+
+static inline rwf_t iocb_to_rw_flags(int ifl)
+{
+	rwf_t flags = 0;
+
+	if (ifl & IOCB_APPEND)
+		flags |= RWF_APPEND;
+	if (ifl & IOCB_DSYNC)
+		flags |= RWF_DSYNC;
+	if (ifl & IOCB_HIPRI)
+		flags |= RWF_HIPRI;
+	if (ifl & IOCB_NOWAIT)
+		flags |= RWF_NOWAIT;
+	if (ifl & IOCB_SYNC)
+		flags |= RWF_SYNC;
+
+	return flags;
+}
+
+ssize_t fuse_passthrough_read_iter(struct kiocb *iocb_fuse,
+				   struct iov_iter *iter)
+{
+	ssize_t ret;
+	struct file *fuse_filp = iocb_fuse->ki_filp;
+	struct fuse_file *ff = fuse_filp->private_data;
+	struct file *passthrough_filp = ff->passthrough.filp;
+
+	if (!iov_iter_count(iter))
+		return 0;
+
+	ret = vfs_iter_read(passthrough_filp, iter, &iocb_fuse->ki_pos,
+			    iocb_to_rw_flags(iocb_fuse->ki_flags));
+
+	return ret;
+}
+
+ssize_t fuse_passthrough_write_iter(struct kiocb *iocb_fuse,
+				    struct iov_iter *iter)
+{
+	ssize_t ret;
+	struct file *fuse_filp = iocb_fuse->ki_filp;
+	struct fuse_file *ff = fuse_filp->private_data;
+	struct inode *fuse_inode = file_inode(fuse_filp);
+	struct file *passthrough_filp = ff->passthrough.filp;
+
+	if (!iov_iter_count(iter))
+		return 0;
+
+	inode_lock(fuse_inode);
+
+	file_start_write(passthrough_filp);
+	ret = vfs_iter_write(passthrough_filp, iter, &iocb_fuse->ki_pos,
+			     iocb_to_rw_flags(iocb_fuse->ki_flags));
+	file_end_write(passthrough_filp);
+	if (ret > 0)
+		fuse_copyattr(fuse_filp, passthrough_filp);
+
+	inode_unlock(fuse_inode);
+
+	return ret;
+}
 
 int fuse_passthrough_open(struct fuse_dev *fud,
 			  struct fuse_passthrough_out *pto)
-- 
2.29.0.rc1.297.gfa9743e501-goog


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

* [PATCH V10 4/5] fuse: Handle asynchronous read and write in passthrough
  2020-10-26 12:50 [PATCH V10 0/5] fuse: Add support for passthrough read/write Alessio Balsini
                   ` (2 preceding siblings ...)
  2020-10-26 12:50 ` [PATCH V10 3/5] fuse: Introduce synchronous read and write for passthrough Alessio Balsini
@ 2020-10-26 12:50 ` Alessio Balsini
  2020-10-26 12:50 ` [PATCH V10 5/5] fuse: Use daemon creds in passthrough mode Alessio Balsini
  2020-11-28  2:10 ` [PATCH V10 0/5] fuse: Add support for passthrough read/write Peng Tao
  5 siblings, 0 replies; 13+ messages in thread
From: Alessio Balsini @ 2020-10-26 12:50 UTC (permalink / raw)
  To: Miklos Szeredi
  Cc: Akilesh Kailash, Amir Goldstein, Antonio SJ Musumeci,
	David Anderson, Giuseppe Scrivano, Jann Horn, Jens Axboe,
	Martijn Coenen, Palmer Dabbelt, Paul Lawrence, Stefano Duo,
	Zimuzo Ezeozue, fuse-devel, kernel-team, linux-fsdevel,
	linux-kernel

Extend the passthrough feature by handling asynchronous IO both for read
and write operations.

When an AIO request is received, if the request targets a FUSE file with
the passthrough functionality enabled, a new identical AIO request is
created. The new request targets the lower file system file and gets
assigned a special FUSE passthrough AIO completion callback.
When the lower file system AIO request is completed, the FUSE passthrough
AIO completion callback is executed and propagates the completion signal to
the FUSE AIO request by triggering its completion callback as well.

Signed-off-by: Alessio Balsini <balsini@android.com>
---
 fs/fuse/passthrough.c | 85 ++++++++++++++++++++++++++++++++++++++-----
 1 file changed, 76 insertions(+), 9 deletions(-)

diff --git a/fs/fuse/passthrough.c b/fs/fuse/passthrough.c
index 5a78cb336db4..10b6872cdaa7 100644
--- a/fs/fuse/passthrough.c
+++ b/fs/fuse/passthrough.c
@@ -6,6 +6,11 @@
 #include <linux/idr.h>
 #include <linux/uio.h>
 
+struct fuse_aio_req {
+	struct kiocb iocb;
+	struct kiocb *iocb_fuse;
+};
+
 static void fuse_copyattr(struct file *dst_file, struct file *src_file)
 {
 	struct inode *dst = file_inode(dst_file);
@@ -32,6 +37,32 @@ static inline rwf_t iocb_to_rw_flags(int ifl)
 	return flags;
 }
 
+static void fuse_aio_cleanup_handler(struct fuse_aio_req *aio_req)
+{
+	struct kiocb *iocb = &aio_req->iocb;
+	struct kiocb *iocb_fuse = aio_req->iocb_fuse;
+
+	if (iocb->ki_flags & IOCB_WRITE) {
+		__sb_writers_acquired(file_inode(iocb->ki_filp)->i_sb,
+				      SB_FREEZE_WRITE);
+		file_end_write(iocb->ki_filp);
+		fuse_copyattr(iocb_fuse->ki_filp, iocb->ki_filp);
+	}
+
+	iocb_fuse->ki_pos = iocb->ki_pos;
+	kfree(aio_req);
+}
+
+static void fuse_aio_rw_complete(struct kiocb *iocb, long res, long res2)
+{
+	struct fuse_aio_req *aio_req =
+		container_of(iocb, struct fuse_aio_req, iocb);
+	struct kiocb *iocb_fuse = aio_req->iocb_fuse;
+
+	fuse_aio_cleanup_handler(aio_req);
+	iocb_fuse->ki_complete(iocb_fuse, res, res2);
+}
+
 ssize_t fuse_passthrough_read_iter(struct kiocb *iocb_fuse,
 				   struct iov_iter *iter)
 {
@@ -43,8 +74,23 @@ ssize_t fuse_passthrough_read_iter(struct kiocb *iocb_fuse,
 	if (!iov_iter_count(iter))
 		return 0;
 
-	ret = vfs_iter_read(passthrough_filp, iter, &iocb_fuse->ki_pos,
-			    iocb_to_rw_flags(iocb_fuse->ki_flags));
+	if (is_sync_kiocb(iocb_fuse)) {
+		ret = vfs_iter_read(passthrough_filp, iter, &iocb_fuse->ki_pos,
+				    iocb_to_rw_flags(iocb_fuse->ki_flags));
+	} else {
+		struct fuse_aio_req *aio_req;
+
+		aio_req = kmalloc(sizeof(struct fuse_aio_req), GFP_KERNEL);
+		if (!aio_req)
+			return -ENOMEM;
+
+		aio_req->iocb_fuse = iocb_fuse;
+		kiocb_clone(&aio_req->iocb, iocb_fuse, passthrough_filp);
+		aio_req->iocb.ki_complete = fuse_aio_rw_complete;
+		ret = call_read_iter(passthrough_filp, &aio_req->iocb, iter);
+		if (ret != -EIOCBQUEUED)
+			fuse_aio_cleanup_handler(aio_req);
+	}
 
 	return ret;
 }
@@ -57,19 +103,40 @@ ssize_t fuse_passthrough_write_iter(struct kiocb *iocb_fuse,
 	struct fuse_file *ff = fuse_filp->private_data;
 	struct inode *fuse_inode = file_inode(fuse_filp);
 	struct file *passthrough_filp = ff->passthrough.filp;
+	struct inode *passthrough_inode = file_inode(passthrough_filp);
 
 	if (!iov_iter_count(iter))
 		return 0;
 
 	inode_lock(fuse_inode);
 
-	file_start_write(passthrough_filp);
-	ret = vfs_iter_write(passthrough_filp, iter, &iocb_fuse->ki_pos,
-			     iocb_to_rw_flags(iocb_fuse->ki_flags));
-	file_end_write(passthrough_filp);
-	if (ret > 0)
-		fuse_copyattr(fuse_filp, passthrough_filp);
-
+	if (is_sync_kiocb(iocb_fuse)) {
+		file_start_write(passthrough_filp);
+		ret = vfs_iter_write(passthrough_filp, iter, &iocb_fuse->ki_pos,
+				     iocb_to_rw_flags(iocb_fuse->ki_flags));
+		file_end_write(passthrough_filp);
+		if (ret > 0)
+			fuse_copyattr(fuse_filp, passthrough_filp);
+	} else {
+		struct fuse_aio_req *aio_req;
+
+		aio_req = kmalloc(sizeof(struct fuse_aio_req), GFP_KERNEL);
+		if (!aio_req) {
+			ret = -ENOMEM;
+			goto out;
+		}
+
+		file_start_write(passthrough_filp);
+		__sb_writers_release(passthrough_inode->i_sb, SB_FREEZE_WRITE);
+
+		aio_req->iocb_fuse = iocb_fuse;
+		kiocb_clone(&aio_req->iocb, iocb_fuse, passthrough_filp);
+		aio_req->iocb.ki_complete = fuse_aio_rw_complete;
+		ret = call_write_iter(passthrough_filp, &aio_req->iocb, iter);
+		if (ret != -EIOCBQUEUED)
+			fuse_aio_cleanup_handler(aio_req);
+	}
+out:
 	inode_unlock(fuse_inode);
 
 	return ret;
-- 
2.29.0.rc1.297.gfa9743e501-goog


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

* [PATCH V10 5/5] fuse: Use daemon creds in passthrough mode
  2020-10-26 12:50 [PATCH V10 0/5] fuse: Add support for passthrough read/write Alessio Balsini
                   ` (3 preceding siblings ...)
  2020-10-26 12:50 ` [PATCH V10 4/5] fuse: Handle asynchronous read and write in passthrough Alessio Balsini
@ 2020-10-26 12:50 ` Alessio Balsini
  2020-11-28  2:10 ` [PATCH V10 0/5] fuse: Add support for passthrough read/write Peng Tao
  5 siblings, 0 replies; 13+ messages in thread
From: Alessio Balsini @ 2020-10-26 12:50 UTC (permalink / raw)
  To: Miklos Szeredi
  Cc: Akilesh Kailash, Amir Goldstein, Antonio SJ Musumeci,
	David Anderson, Giuseppe Scrivano, Jann Horn, Jens Axboe,
	Martijn Coenen, Palmer Dabbelt, Paul Lawrence, Stefano Duo,
	Zimuzo Ezeozue, fuse-devel, kernel-team, linux-fsdevel,
	linux-kernel

When using FUSE passthrough, read/write operations are directly forwarded
to the lower file system file through VFS, but there is no guarantee that
the process that is triggering the request has the right permissions to
access the lower file system. This would cause the read/write access to
fail.

In passthrough file systems, where the FUSE daemon is responsible for the
enforcement of the lower file system access policies, often happens that
the process dealing with the FUSE file system doesn't have access to the
lower file system.
Being the FUSE daemon in charge of implementing the FUSE file operations,
that in the case of read/write operations usually simply results in the
copy of memory buffers from/to the lower file system respectively, these
operations are executed with the FUSE daemon privileges.

This patch adds a reference to the FUSE daemon credentials, referenced at
FUSE_DEV_IOC_PASSTHROUGH_OPEN ioctl() time so that they can be used to
temporarily raise the user credentials when accessing lower file system
files in passthrough.
The process accessing the FUSE file with passthrough enabled temporarily
receives the privileges of the FUSE daemon while performing read/write
operations. Similar behavior is implemented in overlayfs.
These privileges will be reverted as soon as the IO operation completes.
This feature does not provide any higher security privileges to those
processes accessing the FUSE file system with passthrough enabled. This is
because it is still the FUSE daemon responsible for enabling or not the
passthrough feature at file open time, and should enable the feature only
after appropriate access policy checks.

Signed-off-by: Alessio Balsini <balsini@android.com>
---
 fs/fuse/fuse_i.h      |  5 ++++-
 fs/fuse/passthrough.c | 11 +++++++++++
 2 files changed, 15 insertions(+), 1 deletion(-)

diff --git a/fs/fuse/fuse_i.h b/fs/fuse/fuse_i.h
index a888d3df5877..59e033a59551 100644
--- a/fs/fuse/fuse_i.h
+++ b/fs/fuse/fuse_i.h
@@ -165,10 +165,13 @@ struct fuse_release_args;
 
 /**
  * Reference to lower filesystem file for read/write operations handled in
- * passthrough mode
+ * passthrough mode.
+ * This struct also tracks the credentials to be used for handling read/write
+ * operations.
  */
 struct fuse_passthrough {
 	struct file *filp;
+	struct cred *cred;
 };
 
 /** FUSE specific file data */
diff --git a/fs/fuse/passthrough.c b/fs/fuse/passthrough.c
index 10b6872cdaa7..ab81dd8f010b 100644
--- a/fs/fuse/passthrough.c
+++ b/fs/fuse/passthrough.c
@@ -67,6 +67,7 @@ ssize_t fuse_passthrough_read_iter(struct kiocb *iocb_fuse,
 				   struct iov_iter *iter)
 {
 	ssize_t ret;
+	const struct cred *old_cred;
 	struct file *fuse_filp = iocb_fuse->ki_filp;
 	struct fuse_file *ff = fuse_filp->private_data;
 	struct file *passthrough_filp = ff->passthrough.filp;
@@ -74,6 +75,7 @@ ssize_t fuse_passthrough_read_iter(struct kiocb *iocb_fuse,
 	if (!iov_iter_count(iter))
 		return 0;
 
+	old_cred = override_creds(ff->passthrough.cred);
 	if (is_sync_kiocb(iocb_fuse)) {
 		ret = vfs_iter_read(passthrough_filp, iter, &iocb_fuse->ki_pos,
 				    iocb_to_rw_flags(iocb_fuse->ki_flags));
@@ -91,6 +93,7 @@ ssize_t fuse_passthrough_read_iter(struct kiocb *iocb_fuse,
 		if (ret != -EIOCBQUEUED)
 			fuse_aio_cleanup_handler(aio_req);
 	}
+	revert_creds(old_cred);
 
 	return ret;
 }
@@ -99,6 +102,7 @@ ssize_t fuse_passthrough_write_iter(struct kiocb *iocb_fuse,
 				    struct iov_iter *iter)
 {
 	ssize_t ret;
+	const struct cred *old_cred;
 	struct file *fuse_filp = iocb_fuse->ki_filp;
 	struct fuse_file *ff = fuse_filp->private_data;
 	struct inode *fuse_inode = file_inode(fuse_filp);
@@ -110,6 +114,7 @@ ssize_t fuse_passthrough_write_iter(struct kiocb *iocb_fuse,
 
 	inode_lock(fuse_inode);
 
+	old_cred = override_creds(ff->passthrough.cred);
 	if (is_sync_kiocb(iocb_fuse)) {
 		file_start_write(passthrough_filp);
 		ret = vfs_iter_write(passthrough_filp, iter, &iocb_fuse->ki_pos,
@@ -137,6 +142,7 @@ ssize_t fuse_passthrough_write_iter(struct kiocb *iocb_fuse,
 			fuse_aio_cleanup_handler(aio_req);
 	}
 out:
+	revert_creds(old_cred);
 	inode_unlock(fuse_inode);
 
 	return ret;
@@ -174,6 +180,7 @@ int fuse_passthrough_open(struct fuse_dev *fud,
 		return -ENOMEM;
 
 	passthrough->filp = passthrough_filp;
+	passthrough->cred = prepare_creds();
 
 	idr_preload(GFP_KERNEL);
 	spin_lock(&fc->passthrough_req_lock);
@@ -231,4 +238,8 @@ void fuse_passthrough_release(struct fuse_passthrough *passthrough)
 		fput(passthrough->filp);
 		passthrough->filp = NULL;
 	}
+	if (passthrough->cred) {
+		put_cred(passthrough->cred);
+		passthrough->cred = NULL;
+	}
 }
-- 
2.29.0.rc1.297.gfa9743e501-goog


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

* Re: [PATCH V10 2/5] fuse: Passthrough initialization and release
  2020-10-26 12:50 ` [PATCH V10 2/5] fuse: Passthrough initialization and release Alessio Balsini
@ 2020-11-26 13:33   ` Peng Tao
  2020-11-27 13:41     ` Alessio Balsini
       [not found]   ` <3bf58b6f-c7eb-7baa-384d-ae0830d8bceb@tcl.com>
  1 sibling, 1 reply; 13+ messages in thread
From: Peng Tao @ 2020-11-26 13:33 UTC (permalink / raw)
  To: Alessio Balsini
  Cc: Miklos Szeredi, Akilesh Kailash, Amir Goldstein,
	Antonio SJ Musumeci, David Anderson, Giuseppe Scrivano,
	Jann Horn, Jens Axboe, Martijn Coenen, Palmer Dabbelt,
	Paul Lawrence, Stefano Duo, Zimuzo Ezeozue, fuse-devel,
	kernel-team, linux-fsdevel, Linux Kernel Mailing List

On Tue, Oct 27, 2020 at 12:19 AM Alessio Balsini <balsini@android.com> wrote:
>
> Implement the FUSE passthrough ioctl() that associates the lower
> (passthrough) file system file with the fuse_file.
>
> The file descriptor passed to the ioctl() by the FUSE daemon is used to
> access the relative file pointer, that will be copied to the fuse_file data
> structure to consolidate the link between the FUSE and lower file system.
>
> To enable the passthrough mode, userspace triggers the
> FUSE_DEV_IOC_PASSTHROUGH_OPEN ioctl() and, if the call succeeds,
> receives back an identifier that will be used at open/create response
> time in the fuse_open_out field to associate the FUSE file to the lower
> file system file.
> The value returned by the ioctl() to userspace can be:
> - > 0: success, the identifier can be used as part of an open/create
>   reply.
> - < 0: an error occurred.
> The value 0 has been left unused for backward compatibility: the
> fuse_open_out field that is used to pass the passthrough_fh back to the
> kernel uses the same bits that were previously as struct padding,
> zero-initialized in the common libfuse implementation. Removing the 0
> value fixes the ambiguity between the case in which 0 corresponds to a
> real passthrough_fh or a missing implementation, simplifying the
> userspace implementation.
>
> For the passthrough mode to be successfully activated, the lower file
> system file must implement both read_ and write_iter file operations.
> This extra check avoids special pseudo files to be targeted for this
> feature.
> Passthrough comes with another limitation: no further file system stacking
> is allowed for those FUSE file systems using passthrough.
>
> Signed-off-by: Alessio Balsini <balsini@android.com>
> ---
>  fs/fuse/inode.c       |  5 +++
>  fs/fuse/passthrough.c | 80 +++++++++++++++++++++++++++++++++++++++++--
>  2 files changed, 83 insertions(+), 2 deletions(-)
>
> diff --git a/fs/fuse/inode.c b/fs/fuse/inode.c
> index 6738dd5ff5d2..1e94c54d1455 100644
> --- a/fs/fuse/inode.c
> +++ b/fs/fuse/inode.c
> @@ -1034,6 +1034,11 @@ EXPORT_SYMBOL_GPL(fuse_send_init);
>
>  static int free_fuse_passthrough(int id, void *p, void *data)
>  {
> +       struct fuse_passthrough *passthrough = (struct fuse_passthrough *)p;
> +
> +       fuse_passthrough_release(passthrough);
> +       kfree(p);
> +
>         return 0;
>  }
>
> diff --git a/fs/fuse/passthrough.c b/fs/fuse/passthrough.c
> index 594060c654f8..a135c955cc33 100644
> --- a/fs/fuse/passthrough.c
> +++ b/fs/fuse/passthrough.c
> @@ -3,19 +3,95 @@
>  #include "fuse_i.h"
>
>  #include <linux/fuse.h>
> +#include <linux/idr.h>
>
>  int fuse_passthrough_open(struct fuse_dev *fud,
>                           struct fuse_passthrough_out *pto)
>  {
> -       return -EINVAL;
> +       int res;
> +       struct file *passthrough_filp;
> +       struct fuse_conn *fc = fud->fc;
> +       struct fuse_passthrough *passthrough;
> +
> +       if (!fc->passthrough)
> +               return -EPERM;
> +
> +       /* This field is reserved for future implementation */
> +       if (pto->len != 0)
> +               return -EINVAL;
> +
> +       passthrough_filp = fget(pto->fd);
> +       if (!passthrough_filp) {
> +               pr_err("FUSE: invalid file descriptor for passthrough.\n");
> +               return -EBADF;
> +       }
> +
> +       if (!passthrough_filp->f_op->read_iter ||
> +           !passthrough_filp->f_op->write_iter) {
> +               pr_err("FUSE: passthrough file misses file operations.\n");
> +               return -EBADF;
> +       }
> +
> +       passthrough = kmalloc(sizeof(struct fuse_passthrough), GFP_KERNEL);
> +       if (!passthrough)
> +               return -ENOMEM;
> +
> +       passthrough->filp = passthrough_filp;
> +
> +       idr_preload(GFP_KERNEL);
> +       spin_lock(&fc->passthrough_req_lock);
> +       res = idr_alloc(&fc->passthrough_req, passthrough, 1, 0, GFP_ATOMIC);
> +       spin_unlock(&fc->passthrough_req_lock);
> +       idr_preload_end();
> +       if (res <= 0) {
> +               fuse_passthrough_release(passthrough);
> +               kfree(passthrough);
> +       }
> +
> +       return res;
>  }
>
>  int fuse_passthrough_setup(struct fuse_conn *fc, struct fuse_file *ff,
>                            struct fuse_open_out *openarg)
>  {
> -       return -EINVAL;
> +       struct inode *passthrough_inode;
> +       struct super_block *passthrough_sb;
> +       struct fuse_passthrough *passthrough;
> +       int passthrough_fh = openarg->passthrough_fh;
> +
> +       if (!fc->passthrough)
> +               return -EPERM;
> +
> +       /* Default case, passthrough is not requested */
> +       if (passthrough_fh <= 0)
> +               return -EINVAL;
> +
> +       spin_lock(&fc->passthrough_req_lock);
> +       passthrough = idr_remove(&fc->passthrough_req, passthrough_fh);
> +       spin_unlock(&fc->passthrough_req_lock);
> +
> +       if (!passthrough)
> +               return -EINVAL;
> +
> +       passthrough_inode = file_inode(passthrough->filp);
> +       passthrough_sb = passthrough_inode->i_sb;
> +       if (passthrough_sb->s_stack_depth >= FILESYSTEM_MAX_STACK_DEPTH) {
Hi Alessio,

passthrough_sb is the underlying filesystem superblock, right? It
seems to prevent fuse passthrough fs from stacking on another fully
stacked file system, instead of preventing other file systems from
stacking on this fuse passthrough file system. Am I misunderstanding
it?

Cheers,
Tao
--
Into Sth. Rich & Strange

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

* Re: [PATCH V10 2/5] fuse: Passthrough initialization and release
  2020-11-26 13:33   ` Peng Tao
@ 2020-11-27 13:41     ` Alessio Balsini
  2020-11-28  1:57       ` Peng Tao
  0 siblings, 1 reply; 13+ messages in thread
From: Alessio Balsini @ 2020-11-27 13:41 UTC (permalink / raw)
  To: Peng Tao
  Cc: Alessio Balsini, Miklos Szeredi, Akilesh Kailash, Amir Goldstein,
	Antonio SJ Musumeci, David Anderson, Giuseppe Scrivano,
	Jann Horn, Jens Axboe, Martijn Coenen, Palmer Dabbelt,
	Paul Lawrence, Stefano Duo, Zimuzo Ezeozue, fuse-devel,
	kernel-team, linux-fsdevel, Linux Kernel Mailing List

Hi Peng,

Thanks for the heads up!

On Thu, Nov 26, 2020 at 09:33:34PM +0800, Peng Tao wrote:
> On Tue, Oct 27, 2020 at 12:19 AM Alessio Balsini <balsini@android.com> wrote:
> > [...]
> >  int fuse_passthrough_setup(struct fuse_conn *fc, struct fuse_file *ff,
> >                            struct fuse_open_out *openarg)
> >  {
> > -       return -EINVAL;
> > +       struct inode *passthrough_inode;
> > +       struct super_block *passthrough_sb;
> > +       struct fuse_passthrough *passthrough;
> > +       int passthrough_fh = openarg->passthrough_fh;
> > +
> > +       if (!fc->passthrough)
> > +               return -EPERM;
> > +
> > +       /* Default case, passthrough is not requested */
> > +       if (passthrough_fh <= 0)
> > +               return -EINVAL;
> > +
> > +       spin_lock(&fc->passthrough_req_lock);
> > +       passthrough = idr_remove(&fc->passthrough_req, passthrough_fh);
> > +       spin_unlock(&fc->passthrough_req_lock);
> > +
> > +       if (!passthrough)
> > +               return -EINVAL;
> > +
> > +       passthrough_inode = file_inode(passthrough->filp);
> > +       passthrough_sb = passthrough_inode->i_sb;
> > +       if (passthrough_sb->s_stack_depth >= FILESYSTEM_MAX_STACK_DEPTH) {
> Hi Alessio,
> 
> passthrough_sb is the underlying filesystem superblock, right? It
> seems to prevent fuse passthrough fs from stacking on another fully
> stacked file system, instead of preventing other file systems from
> stacking on this fuse passthrough file system. Am I misunderstanding
> it?

Correct, this checks the stacking depth on the lower filesystem.
This is an intended behavior to avoid the stacking of multiple FUSE
passthrough filesystems, and works because when a FUSE connection has
the passthrough feature activated, the file system updates its
s_stack_depth to FILESYSTEM_MAX_STACK_DEPTH in process_init_reply()
(PATCH 1/5), avoiding further stacking.

Do you see issues with that?

What I'm now thinking is that fuse_passthrough_open would probably be a
better place for this check, so that the ioctl() would fail and the user
space daemon may decide to skip passthrough and use legacy FUSE access
for that file (or, at least, be aware that something went wrong).

A more aggressive approach would be instead to move the stacking depth
check to fuse_fill_super_common, where we can update s_stack_depth to
lower-fs depth+1 and fail if passthrough is active and s_stack_depth is
greater than FILESYSTEM_MAX_STACK_DEPTH.

What do you think?

Thanks,
Alessio


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

* Re: [PATCH V10 2/5] fuse: Passthrough initialization and release
  2020-11-27 13:41     ` Alessio Balsini
@ 2020-11-28  1:57       ` Peng Tao
  2020-12-16 16:46         ` Alessio Balsini
  0 siblings, 1 reply; 13+ messages in thread
From: Peng Tao @ 2020-11-28  1:57 UTC (permalink / raw)
  To: Alessio Balsini
  Cc: Miklos Szeredi, Akilesh Kailash, Amir Goldstein,
	Antonio SJ Musumeci, David Anderson, Giuseppe Scrivano,
	Jann Horn, Jens Axboe, Martijn Coenen, Palmer Dabbelt,
	Paul Lawrence, Stefano Duo, Zimuzo Ezeozue, fuse-devel,
	kernel-team, linux-fsdevel, Linux Kernel Mailing List

On Fri, Nov 27, 2020 at 9:41 PM Alessio Balsini <balsini@android.com> wrote:
>
> Hi Peng,
>
> Thanks for the heads up!
>
> On Thu, Nov 26, 2020 at 09:33:34PM +0800, Peng Tao wrote:
> > On Tue, Oct 27, 2020 at 12:19 AM Alessio Balsini <balsini@android.com> wrote:
> > > [...]
> > >  int fuse_passthrough_setup(struct fuse_conn *fc, struct fuse_file *ff,
> > >                            struct fuse_open_out *openarg)
> > >  {
> > > -       return -EINVAL;
> > > +       struct inode *passthrough_inode;
> > > +       struct super_block *passthrough_sb;
> > > +       struct fuse_passthrough *passthrough;
> > > +       int passthrough_fh = openarg->passthrough_fh;
> > > +
> > > +       if (!fc->passthrough)
> > > +               return -EPERM;
> > > +
> > > +       /* Default case, passthrough is not requested */
> > > +       if (passthrough_fh <= 0)
> > > +               return -EINVAL;
> > > +
> > > +       spin_lock(&fc->passthrough_req_lock);
> > > +       passthrough = idr_remove(&fc->passthrough_req, passthrough_fh);
> > > +       spin_unlock(&fc->passthrough_req_lock);
> > > +
> > > +       if (!passthrough)
> > > +               return -EINVAL;
> > > +
> > > +       passthrough_inode = file_inode(passthrough->filp);
> > > +       passthrough_sb = passthrough_inode->i_sb;
> > > +       if (passthrough_sb->s_stack_depth >= FILESYSTEM_MAX_STACK_DEPTH) {
> > Hi Alessio,
> >
> > passthrough_sb is the underlying filesystem superblock, right? It
> > seems to prevent fuse passthrough fs from stacking on another fully
> > stacked file system, instead of preventing other file systems from
> > stacking on this fuse passthrough file system. Am I misunderstanding
> > it?
>
> Correct, this checks the stacking depth on the lower filesystem.
> This is an intended behavior to avoid the stacking of multiple FUSE
> passthrough filesystems, and works because when a FUSE connection has
> the passthrough feature activated, the file system updates its
> s_stack_depth to FILESYSTEM_MAX_STACK_DEPTH in process_init_reply()
> (PATCH 1/5), avoiding further stacking.
>
> Do you see issues with that?
I'm considering a use case where a fuse passthrough file system is
stacked on top of an overlayfs and/or an ecryptfs. The underlying
s_stack_depth FILESYSTEM_MAX_STACK_DEPTH is rejected here so it is
possible to have an overlayfs or an ecryptfs underneath but not both
(with current FILESYSTEM_MAX_STACK_DEPTH being 2). How about changing
passthrough fuse sb s_stack_depth to FILESYSTEM_MAX_STACK_DEPTH + 1 in
PATCH 1/5, and allow passthrough_sb->s_stack_depth to be
FILESYSTEM_MAX_STACK_DEPTH here? So that existing kernel stacking file
system setups can all work as the underlying file system, while the
stacking of multiple FUSE passthrough filesystems is still blocked.

>
> What I'm now thinking is that fuse_passthrough_open would probably be a
> better place for this check, so that the ioctl() would fail and the user
> space daemon may decide to skip passthrough and use legacy FUSE access
> for that file (or, at least, be aware that something went wrong).
>
+1, fuse_passthrough_open seems to be a better place for the check.

> A more aggressive approach would be instead to move the stacking depth
> check to fuse_fill_super_common, where we can update s_stack_depth to
> lower-fs depth+1 and fail if passthrough is active and s_stack_depth is
> greater than FILESYSTEM_MAX_STACK_DEPTH.
>
The lower layer files/directories might actually spread on different
file systems. I'm not sure if s_stack_depth check is still possible at
mount time. Even if we can calculate the substree s_stack_depth, it is
still more flexible to determine on a per inode basis, right?

Cheers,
Tao
--
Into Sth. Rich & Strange

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

* Re: [PATCH V10 0/5] fuse: Add support for passthrough read/write
  2020-10-26 12:50 [PATCH V10 0/5] fuse: Add support for passthrough read/write Alessio Balsini
                   ` (4 preceding siblings ...)
  2020-10-26 12:50 ` [PATCH V10 5/5] fuse: Use daemon creds in passthrough mode Alessio Balsini
@ 2020-11-28  2:10 ` Peng Tao
  2020-11-30 11:08   ` Alessio Balsini
  5 siblings, 1 reply; 13+ messages in thread
From: Peng Tao @ 2020-11-28  2:10 UTC (permalink / raw)
  To: Alessio Balsini
  Cc: Miklos Szeredi, Akilesh Kailash, Amir Goldstein,
	Antonio SJ Musumeci, David Anderson, Giuseppe Scrivano,
	Jann Horn, Jens Axboe, Martijn Coenen, Palmer Dabbelt,
	Paul Lawrence, Stefano Duo, Zimuzo Ezeozue, fuse-devel,
	kernel-team, linux-fsdevel, Linux Kernel Mailing List

On Tue, Oct 27, 2020 at 1:00 AM Alessio Balsini <balsini@android.com> wrote:
>
> This is the 10th version of the series. Please find the changelog at the
> bottom of this cover letter.
>
> Add support for file system passthrough read/write of files when enabled in
> userspace through the option FUSE_PASSTHROUGH.
>
> There are file systems based on FUSE that are intended to enforce special
> policies or trigger complicated decision makings at the file operations
> level. Android, for example, uses FUSE to enforce fine-grained access
> policies that also depend on the file contents.
> Sometimes it happens that at open or create time a file is identified as
> not requiring additional checks for consequent reads/writes, thus FUSE
> would simply act as a passive bridge between the process accessing the FUSE
> file system and the lower file system. Splicing and caching help reduce the
> FUSE overhead, but there are still read/write operations forwarded to the
> userspace FUSE daemon that could be avoided.
>
> This series has been inspired by the original patches from Nikhilesh Reddy,
> the idea and code of which has been elaborated and improved thanks to the
> community support.
>
> When the FUSE_PASSTHROUGH capability is enabled, the FUSE daemon may decide
> while handling the open/create operations, if the given file can be
> accessed in passthrough mode. This means that all the further read and
> write operations would be forwarded by the kernel directly to the lower
> file system using the VFS layer rather than to the FUSE daemon.
> All the requests other than reads or writes are still handled by the
> userspace FUSE daemon.
> This allows for improved performance on reads and writes, especially in the
> case of reads at random offsets, for which no (readahead) caching mechanism
> would help.
> Benchmarks show improved performance that is close to native file system
> access when doing massive manipulations on a single opened file, especially
> in the case of random reads, for which the bandwidth increased by almost 2X
> or sequential writes for which the improvement is close to 3X.
>
> The creation of this direct connection (passthrough) between FUSE file
> objects and file objects in the lower file system happens in a way that
> reminds of passing file descriptors via sockets:
> - a process requests the opening of a file handled by FUSE, so the kernel
>   forwards the request to the FUSE daemon;
> - the FUSE daemon opens the target file in the lower file system, getting
>   its file descriptor;
> - the FUSE daemon also decides according to its internal policies if
>   passthrough can be enabled for that file, and, if so, can perform a
>   FUSE_DEV_IOC_PASSTHROUGH_OPEN ioctl() on /dev/fuse, passing the file
>   descriptor obtained at the previous step and the fuse_req unique
>   identifier;
> - the kernel translates the file descriptor to the file pointer navigating
>   through the opened files of the "current" process and temporarily stores
>   it in the associated open/create fuse_req's passthrough_filp;
> - when the FUSE daemon has done with the request and it's time for the
>   kernel to close it, it checks if the passthrough_filp is available and in
>   case updates the additional field in the fuse_file owned by the process
>   accessing the FUSE file system.
> From now on, all the read/write operations performed by that process will
> be redirected to the corresponding lower file system file by creating new
> VFS requests.
> Since the read/write operation to the lower file system is executed with
> the current process's credentials, it might happen that it does not have
> enough privileges to succeed. For this reason, the process temporarily
> receives the same credentials as the FUSE daemon, that are reverted as soon
> as the read/write operation completes, emulating the behavior of the
> request to be performed by the FUSE daemon itself. This solution has been
> inspired by the way overlayfs handles read/write operations.
> Asynchronous IO is supported as well, handled by creating separate AIO
> requests for the lower file system that will be internally tracked by FUSE,
> that intercepts and propagates their completion through an internal
> ki_completed callback similar to the current implementation of overlayfs.
> The ioctl() has been designed taking as a reference and trying to converge
> to the fuse2 implementation. For example, the fuse_passthrough_out data
> structure has extra fields that will allow for further extensions of the
> feature.
>
>
>     Performance on SSD
>
> What follows has been performed with this change [V6] rebased on top of
> vanilla v5.8 Linux kernel, using a custom passthrough_hp FUSE daemon that
> enables pass-through for each file that is opened during both "open" and
> "create". Tests were run on an Intel Xeon E5-2678V3, 32GiB of RAM, with an
> ext4-formatted SSD as the lower file system, with no special tuning, e.g.,
> all the involved processes are SCHED_OTHER, ondemand is the frequency
> governor with no frequency restrictions, and turbo-boost, as well as
> p-state, are active. This is because I noticed that, for such high-level
> benchmarks, results consistency was minimally affected by these features.
> The source code of the updated libfuse library and passthrough_hp is shared
> at the following repository:
>
>     https://github.com/balsini/libfuse/tree/fuse-passthrough-stable-v.3.9.4
The libfuse changes are not updated with the latest ioctl UAPI change yet.

> * UAPI updated: ioctl() now returns an ID that will be used at
>   open/create response time to reference the passthrough file

Cheers,
Tao
-- 
Into Sth. Rich & Strange

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

* Re: [PATCH V10 0/5] fuse: Add support for passthrough read/write
  2020-11-28  2:10 ` [PATCH V10 0/5] fuse: Add support for passthrough read/write Peng Tao
@ 2020-11-30 11:08   ` Alessio Balsini
  0 siblings, 0 replies; 13+ messages in thread
From: Alessio Balsini @ 2020-11-30 11:08 UTC (permalink / raw)
  To: Peng Tao
  Cc: Alessio Balsini, Miklos Szeredi, Akilesh Kailash, Amir Goldstein,
	Antonio SJ Musumeci, David Anderson, Giuseppe Scrivano,
	Jann Horn, Jens Axboe, Martijn Coenen, Palmer Dabbelt,
	Paul Lawrence, Stefano Duo, Zimuzo Ezeozue, fuse-devel,
	kernel-team, linux-fsdevel, Linux Kernel Mailing List

On Sat, Nov 28, 2020 at 10:10:37AM +0800, Peng Tao wrote:
> On Tue, Oct 27, 2020 at 1:00 AM Alessio Balsini <balsini@android.com> wrote:
> >
> > This is the 10th version of the series. Please find the changelog at the
> > bottom of this cover letter.
> >
> > Add support for file system passthrough read/write of files when enabled in
> > userspace through the option FUSE_PASSTHROUGH.
> >
> > There are file systems based on FUSE that are intended to enforce special
> > policies or trigger complicated decision makings at the file operations
> > level. Android, for example, uses FUSE to enforce fine-grained access
> > policies that also depend on the file contents.
> > Sometimes it happens that at open or create time a file is identified as
> > not requiring additional checks for consequent reads/writes, thus FUSE
> > would simply act as a passive bridge between the process accessing the FUSE
> > file system and the lower file system. Splicing and caching help reduce the
> > FUSE overhead, but there are still read/write operations forwarded to the
> > userspace FUSE daemon that could be avoided.
> >
> > This series has been inspired by the original patches from Nikhilesh Reddy,
> > the idea and code of which has been elaborated and improved thanks to the
> > community support.
> >
> > When the FUSE_PASSTHROUGH capability is enabled, the FUSE daemon may decide
> > while handling the open/create operations, if the given file can be
> > accessed in passthrough mode. This means that all the further read and
> > write operations would be forwarded by the kernel directly to the lower
> > file system using the VFS layer rather than to the FUSE daemon.
> > All the requests other than reads or writes are still handled by the
> > userspace FUSE daemon.
> > This allows for improved performance on reads and writes, especially in the
> > case of reads at random offsets, for which no (readahead) caching mechanism
> > would help.
> > Benchmarks show improved performance that is close to native file system
> > access when doing massive manipulations on a single opened file, especially
> > in the case of random reads, for which the bandwidth increased by almost 2X
> > or sequential writes for which the improvement is close to 3X.
> >
> > The creation of this direct connection (passthrough) between FUSE file
> > objects and file objects in the lower file system happens in a way that
> > reminds of passing file descriptors via sockets:
> > - a process requests the opening of a file handled by FUSE, so the kernel
> >   forwards the request to the FUSE daemon;
> > - the FUSE daemon opens the target file in the lower file system, getting
> >   its file descriptor;
> > - the FUSE daemon also decides according to its internal policies if
> >   passthrough can be enabled for that file, and, if so, can perform a
> >   FUSE_DEV_IOC_PASSTHROUGH_OPEN ioctl() on /dev/fuse, passing the file
> >   descriptor obtained at the previous step and the fuse_req unique
> >   identifier;
> > - the kernel translates the file descriptor to the file pointer navigating
> >   through the opened files of the "current" process and temporarily stores
> >   it in the associated open/create fuse_req's passthrough_filp;
> > - when the FUSE daemon has done with the request and it's time for the
> >   kernel to close it, it checks if the passthrough_filp is available and in
> >   case updates the additional field in the fuse_file owned by the process
> >   accessing the FUSE file system.
> > From now on, all the read/write operations performed by that process will
> > be redirected to the corresponding lower file system file by creating new
> > VFS requests.
> > Since the read/write operation to the lower file system is executed with
> > the current process's credentials, it might happen that it does not have
> > enough privileges to succeed. For this reason, the process temporarily
> > receives the same credentials as the FUSE daemon, that are reverted as soon
> > as the read/write operation completes, emulating the behavior of the
> > request to be performed by the FUSE daemon itself. This solution has been
> > inspired by the way overlayfs handles read/write operations.
> > Asynchronous IO is supported as well, handled by creating separate AIO
> > requests for the lower file system that will be internally tracked by FUSE,
> > that intercepts and propagates their completion through an internal
> > ki_completed callback similar to the current implementation of overlayfs.
> > The ioctl() has been designed taking as a reference and trying to converge
> > to the fuse2 implementation. For example, the fuse_passthrough_out data
> > structure has extra fields that will allow for further extensions of the
> > feature.
> >
> >
> >     Performance on SSD
> >
> > What follows has been performed with this change [V6] rebased on top of
> > vanilla v5.8 Linux kernel, using a custom passthrough_hp FUSE daemon that
> > enables pass-through for each file that is opened during both "open" and
> > "create". Tests were run on an Intel Xeon E5-2678V3, 32GiB of RAM, with an
> > ext4-formatted SSD as the lower file system, with no special tuning, e.g.,
> > all the involved processes are SCHED_OTHER, ondemand is the frequency
> > governor with no frequency restrictions, and turbo-boost, as well as
> > p-state, are active. This is because I noticed that, for such high-level
> > benchmarks, results consistency was minimally affected by these features.
> > The source code of the updated libfuse library and passthrough_hp is shared
> > at the following repository:
> >
> >     https://github.com/balsini/libfuse/tree/fuse-passthrough-stable-v.3.9.4
> The libfuse changes are not updated with the latest ioctl UAPI change yet.
> 
> > * UAPI updated: ioctl() now returns an ID that will be used at
> >   open/create response time to reference the passthrough file
> 
> Cheers,
> Tao
> -- 
> Into Sth. Rich & Strange

Hi Tao,

You are right, sorry, this is the correct branch that uses the
FUSE_DEV_IOC_PASSTHROUGH_OPEN ioctl with the current patch set:

  https://github.com/balsini/libfuse/tree/fuse-passthrough-v10-linux-5.8-v.3.9.4

I think I'll stick to this libfuse branch naming pattern from now on. :)

Thanks,
Alessio

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

* Re: [PATCH V10 2/5] fuse: Passthrough initialization and release
  2020-11-28  1:57       ` Peng Tao
@ 2020-12-16 16:46         ` Alessio Balsini
  0 siblings, 0 replies; 13+ messages in thread
From: Alessio Balsini @ 2020-12-16 16:46 UTC (permalink / raw)
  To: Peng Tao
  Cc: Alessio Balsini, Miklos Szeredi, Akilesh Kailash, Amir Goldstein,
	Antonio SJ Musumeci, David Anderson, Giuseppe Scrivano,
	Jann Horn, Jens Axboe, Martijn Coenen, Palmer Dabbelt,
	Paul Lawrence, Stefano Duo, Zimuzo Ezeozue, fuse-devel,
	kernel-team, linux-fsdevel, Linux Kernel Mailing List

Hi Tao,

On Sat, Nov 28, 2020 at 09:57:31AM +0800, Peng Tao wrote:
> On Fri, Nov 27, 2020 at 9:41 PM Alessio Balsini <balsini@android.com> wrote:
> >
> > Hi Peng,
> >
> > Thanks for the heads up!
> >
> > On Thu, Nov 26, 2020 at 09:33:34PM +0800, Peng Tao wrote:
> > > On Tue, Oct 27, 2020 at 12:19 AM Alessio Balsini <balsini@android.com> wrote:
> > > > [...]
> > > >  int fuse_passthrough_setup(struct fuse_conn *fc, struct fuse_file *ff,
> > > >                            struct fuse_open_out *openarg)
> > > >  {
> > > > -       return -EINVAL;
> > > > +       struct inode *passthrough_inode;
> > > > +       struct super_block *passthrough_sb;
> > > > +       struct fuse_passthrough *passthrough;
> > > > +       int passthrough_fh = openarg->passthrough_fh;
> > > > +
> > > > +       if (!fc->passthrough)
> > > > +               return -EPERM;
> > > > +
> > > > +       /* Default case, passthrough is not requested */
> > > > +       if (passthrough_fh <= 0)
> > > > +               return -EINVAL;
> > > > +
> > > > +       spin_lock(&fc->passthrough_req_lock);
> > > > +       passthrough = idr_remove(&fc->passthrough_req, passthrough_fh);
> > > > +       spin_unlock(&fc->passthrough_req_lock);
> > > > +
> > > > +       if (!passthrough)
> > > > +               return -EINVAL;
> > > > +
> > > > +       passthrough_inode = file_inode(passthrough->filp);
> > > > +       passthrough_sb = passthrough_inode->i_sb;
> > > > +       if (passthrough_sb->s_stack_depth >= FILESYSTEM_MAX_STACK_DEPTH) {
> > > Hi Alessio,
> > >
> > > passthrough_sb is the underlying filesystem superblock, right? It
> > > seems to prevent fuse passthrough fs from stacking on another fully
> > > stacked file system, instead of preventing other file systems from
> > > stacking on this fuse passthrough file system. Am I misunderstanding
> > > it?
> >
> > Correct, this checks the stacking depth on the lower filesystem.
> > This is an intended behavior to avoid the stacking of multiple FUSE
> > passthrough filesystems, and works because when a FUSE connection has
> > the passthrough feature activated, the file system updates its
> > s_stack_depth to FILESYSTEM_MAX_STACK_DEPTH in process_init_reply()
> > (PATCH 1/5), avoiding further stacking.
> >
> > Do you see issues with that?
> I'm considering a use case where a fuse passthrough file system is
> stacked on top of an overlayfs and/or an ecryptfs. The underlying
> s_stack_depth FILESYSTEM_MAX_STACK_DEPTH is rejected here so it is
> possible to have an overlayfs or an ecryptfs underneath but not both
> (with current FILESYSTEM_MAX_STACK_DEPTH being 2). How about changing
> passthrough fuse sb s_stack_depth to FILESYSTEM_MAX_STACK_DEPTH + 1 in
> PATCH 1/5, and allow passthrough_sb->s_stack_depth to be
> FILESYSTEM_MAX_STACK_DEPTH here? So that existing kernel stacking file
> system setups can all work as the underlying file system, while the
> stacking of multiple FUSE passthrough filesystems is still blocked.
> 


Sounds like a good idea, I'll think about it a bit more and if
everything's all right I'll post the new patchset.


> >
> > What I'm now thinking is that fuse_passthrough_open would probably be a
> > better place for this check, so that the ioctl() would fail and the user
> > space daemon may decide to skip passthrough and use legacy FUSE access
> > for that file (or, at least, be aware that something went wrong).
> >
> +1, fuse_passthrough_open seems to be a better place for the check.
> 
> > A more aggressive approach would be instead to move the stacking depth
> > check to fuse_fill_super_common, where we can update s_stack_depth to
> > lower-fs depth+1 and fail if passthrough is active and s_stack_depth is
> > greater than FILESYSTEM_MAX_STACK_DEPTH.
> >
> The lower layer files/directories might actually spread on different
> file systems. I'm not sure if s_stack_depth check is still possible at
> mount time. Even if we can calculate the substree s_stack_depth, it is
> still more flexible to determine on a per inode basis, right?
> 
> Cheers,
> Tao
> --
> Into Sth. Rich & Strange


Fair enough. The per-inode check is definitely the right way to proceed.

Thanks a lot for you feedback!
Alessio


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

* Re: [PATCH V10 2/5] fuse: Passthrough initialization and release
       [not found]   ` <3bf58b6f-c7eb-7baa-384d-ae0830d8bceb@tcl.com>
@ 2020-12-16 16:55     ` Alessio Balsini
  0 siblings, 0 replies; 13+ messages in thread
From: Alessio Balsini @ 2020-12-16 16:55 UTC (permalink / raw)
  To: wu-yan
  Cc: balsini, akailash, amir73il, axboe, duostefano93, dvander,
	fuse-devel, gscrivan, jannh, kernel-team, linux-fsdevel,
	linux-kernel, maco, miklos, palmer, paullawrence, trapexit,
	zezeozue

On Wed, Dec 16, 2020 at 09:32:51PM +0800, wu-yan@tcl.com wrote:
> Hi Alessio,
> 
> It may cause file reference counter leak in fuse_passthrough_open. If the
> passthrough_filp
> 
> not implement read_iter/write_iter or passthrough struct allocated failed,
> the reference counter get in fget(pro->fd) not released and cause leak.
> 
> Cheers,
> 
> yanwu
> 


Hi yanwu,

Nice catch, this bug was introduced in v10 and will be fixed in the next
patch set.

Cheers,
Alessio

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

end of thread, other threads:[~2020-12-16 16:55 UTC | newest]

Thread overview: 13+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-10-26 12:50 [PATCH V10 0/5] fuse: Add support for passthrough read/write Alessio Balsini
2020-10-26 12:50 ` [PATCH V10 1/5] fuse: Definitions and ioctl() for passthrough Alessio Balsini
2020-10-26 12:50 ` [PATCH V10 2/5] fuse: Passthrough initialization and release Alessio Balsini
2020-11-26 13:33   ` Peng Tao
2020-11-27 13:41     ` Alessio Balsini
2020-11-28  1:57       ` Peng Tao
2020-12-16 16:46         ` Alessio Balsini
     [not found]   ` <3bf58b6f-c7eb-7baa-384d-ae0830d8bceb@tcl.com>
2020-12-16 16:55     ` Alessio Balsini
2020-10-26 12:50 ` [PATCH V10 3/5] fuse: Introduce synchronous read and write for passthrough Alessio Balsini
2020-10-26 12:50 ` [PATCH V10 4/5] fuse: Handle asynchronous read and write in passthrough Alessio Balsini
2020-10-26 12:50 ` [PATCH V10 5/5] fuse: Use daemon creds in passthrough mode Alessio Balsini
2020-11-28  2:10 ` [PATCH V10 0/5] fuse: Add support for passthrough read/write Peng Tao
2020-11-30 11:08   ` Alessio Balsini

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