1
0
Fork 0

Merge branch 'for-linus' of git://git.kernel.org/pub/scm/linux/kernel/git/mszeredi/fuse

Pull fuse updates from Miklos Szeredi:
 "This is the start of improving fuse scalability.

  An input queue and a processing queue is split out from the monolithic
  fuse connection, each of those having their own spinlock.  The end of
  the patchset adds the ability to clone a fuse connection.  This means,
  that instead of having to read/write requests/answers on a single fuse
  device fd, the fuse daemon can have multiple distinct file descriptors
  open.  Each of those can be used to receive requests and send answers,
  currently the only constraint is that a request must be answered on
  the same fd as it was read from.

  This can be extended further to allow binding a device clone to a
  specific CPU or NUMA node.

  Based on a patchset by Srinivas Eeda and Ashish Samant.  Thanks to
  Ashish for the review of this series"

* 'for-linus' of git://git.kernel.org/pub/scm/linux/kernel/git/mszeredi/fuse: (40 commits)
  fuse: update MAINTAINERS entry
  fuse: separate pqueue for clones
  fuse: introduce per-instance fuse_dev structure
  fuse: device fd clone
  fuse: abort: no fc->lock needed for request ending
  fuse: no fc->lock for pqueue parts
  fuse: no fc->lock in request_end()
  fuse: cleanup request_end()
  fuse: request_end(): do once
  fuse: add req flag for private list
  fuse: pqueue locking
  fuse: abort: group pqueue accesses
  fuse: cleanup fuse_dev_do_read()
  fuse: move list_del_init() from request_end() into callers
  fuse: duplicate ->connected in pqueue
  fuse: separate out processing queue
  fuse: simplify request_wait()
  fuse: no fc->lock for iqueue parts
  fuse: allow interrupt queuing without fc->lock
  fuse: iqueue locking
  ...
hifive-unleashed-5.1
Linus Torvalds 2015-07-02 11:21:26 -07:00
commit a7ba4bf5e7
8 changed files with 635 additions and 494 deletions

View File

@ -321,6 +321,7 @@ Code Seq#(hex) Include File Comments
0xDB 00-0F drivers/char/mwave/mwavepub.h
0xDD 00-3F ZFCP device driver see drivers/s390/scsi/
<mailto:aherrman@de.ibm.com>
0xE5 00-3F linux/fuse.h
0xEC 00-01 drivers/platform/chrome/cros_ec_dev.h ChromeOS EC driver
0xF3 00-3F drivers/usb/misc/sisusbvga/sisusb.h sisfb (in development)
<mailto:thomas@winischhofer.net>

View File

@ -4430,9 +4430,11 @@ FUSE: FILESYSTEM IN USERSPACE
M: Miklos Szeredi <miklos@szeredi.hu>
L: fuse-devel@lists.sourceforge.net
W: http://fuse.sourceforge.net/
T: git git://git.kernel.org/pub/scm/linux/kernel/git/mszeredi/fuse.git
S: Maintained
F: fs/fuse/
F: include/uapi/linux/fuse.h
F: Documentation/filesystems/fuse.txt
FUTURE DOMAIN TMC-16x0 SCSI DRIVER (16-bit)
M: Rik Faith <faith@cs.unc.edu>

View File

@ -489,6 +489,7 @@ static void cuse_fc_release(struct fuse_conn *fc)
*/
static int cuse_channel_open(struct inode *inode, struct file *file)
{
struct fuse_dev *fud;
struct cuse_conn *cc;
int rc;
@ -499,17 +500,22 @@ static int cuse_channel_open(struct inode *inode, struct file *file)
fuse_conn_init(&cc->fc);
fud = fuse_dev_alloc(&cc->fc);
if (!fud) {
kfree(cc);
return -ENOMEM;
}
INIT_LIST_HEAD(&cc->list);
cc->fc.release = cuse_fc_release;
cc->fc.connected = 1;
cc->fc.initialized = 1;
rc = cuse_send_init(cc);
if (rc) {
fuse_conn_put(&cc->fc);
fuse_dev_free(fud);
return rc;
}
file->private_data = &cc->fc; /* channel owns base reference to cc */
file->private_data = fud;
return 0;
}
@ -527,7 +533,8 @@ static int cuse_channel_open(struct inode *inode, struct file *file)
*/
static int cuse_channel_release(struct inode *inode, struct file *file)
{
struct cuse_conn *cc = fc_to_cc(file->private_data);
struct fuse_dev *fud = file->private_data;
struct cuse_conn *cc = fc_to_cc(fud->fc);
int rc;
/* remove from the conntbl, no more access from this point on */

File diff suppressed because it is too large Load Diff

View File

@ -96,17 +96,17 @@ static void fuse_file_put(struct fuse_file *ff, bool sync)
* Drop the release request when client does not
* implement 'open'
*/
req->background = 0;
__clear_bit(FR_BACKGROUND, &req->flags);
iput(req->misc.release.inode);
fuse_put_request(ff->fc, req);
} else if (sync) {
req->background = 0;
__clear_bit(FR_BACKGROUND, &req->flags);
fuse_request_send(ff->fc, req);
iput(req->misc.release.inode);
fuse_put_request(ff->fc, req);
} else {
req->end = fuse_release_end;
req->background = 1;
__set_bit(FR_BACKGROUND, &req->flags);
fuse_request_send_background(ff->fc, req);
}
kfree(ff);
@ -299,8 +299,8 @@ void fuse_sync_release(struct fuse_file *ff, int flags)
{
WARN_ON(atomic_read(&ff->count) > 1);
fuse_prepare_release(ff, flags, FUSE_RELEASE);
ff->reserved_req->force = 1;
ff->reserved_req->background = 0;
__set_bit(FR_FORCE, &ff->reserved_req->flags);
__clear_bit(FR_BACKGROUND, &ff->reserved_req->flags);
fuse_request_send(ff->fc, ff->reserved_req);
fuse_put_request(ff->fc, ff->reserved_req);
kfree(ff);
@ -426,7 +426,7 @@ static int fuse_flush(struct file *file, fl_owner_t id)
req->in.numargs = 1;
req->in.args[0].size = sizeof(inarg);
req->in.args[0].value = &inarg;
req->force = 1;
__set_bit(FR_FORCE, &req->flags);
fuse_request_send(fc, req);
err = req->out.h.error;
fuse_put_request(fc, req);
@ -1611,7 +1611,8 @@ static int fuse_writepage_locked(struct page *page)
if (!req)
goto err;
req->background = 1; /* writeback always goes to bg_queue */
/* writeback always goes to bg_queue */
__set_bit(FR_BACKGROUND, &req->flags);
tmp_page = alloc_page(GFP_NOFS | __GFP_HIGHMEM);
if (!tmp_page)
goto err_free;
@ -1742,8 +1743,7 @@ static bool fuse_writepage_in_flight(struct fuse_req *new_req,
}
}
if (old_req->num_pages == 1 && (old_req->state == FUSE_REQ_INIT ||
old_req->state == FUSE_REQ_PENDING)) {
if (old_req->num_pages == 1 && test_bit(FR_PENDING, &old_req->flags)) {
struct backing_dev_info *bdi = inode_to_bdi(page->mapping->host);
copy_highpage(old_req->pages[0], page);
@ -1830,7 +1830,7 @@ static int fuse_writepages_fill(struct page *page,
req->misc.write.in.write_flags |= FUSE_WRITE_CACHE;
req->misc.write.next = NULL;
req->in.argpages = 1;
req->background = 1;
__set_bit(FR_BACKGROUND, &req->flags);
req->num_pages = 0;
req->end = fuse_writepage_end;
req->inode = inode;

View File

@ -241,16 +241,6 @@ struct fuse_args {
#define FUSE_ARGS(args) struct fuse_args args = {}
/** The request state */
enum fuse_req_state {
FUSE_REQ_INIT = 0,
FUSE_REQ_PENDING,
FUSE_REQ_READING,
FUSE_REQ_SENT,
FUSE_REQ_WRITING,
FUSE_REQ_FINISHED
};
/** The request IO state (for asynchronous processing) */
struct fuse_io_priv {
int async;
@ -266,8 +256,41 @@ struct fuse_io_priv {
struct completion *done;
};
/**
* Request flags
*
* FR_ISREPLY: set if the request has reply
* FR_FORCE: force sending of the request even if interrupted
* FR_BACKGROUND: request is sent in the background
* FR_WAITING: request is counted as "waiting"
* FR_ABORTED: the request was aborted
* FR_INTERRUPTED: the request has been interrupted
* FR_LOCKED: data is being copied to/from the request
* FR_PENDING: request is not yet in userspace
* FR_SENT: request is in userspace, waiting for an answer
* FR_FINISHED: request is finished
* FR_PRIVATE: request is on private list
*/
enum fuse_req_flag {
FR_ISREPLY,
FR_FORCE,
FR_BACKGROUND,
FR_WAITING,
FR_ABORTED,
FR_INTERRUPTED,
FR_LOCKED,
FR_PENDING,
FR_SENT,
FR_FINISHED,
FR_PRIVATE,
};
/**
* A request to the client
*
* .waitq.lock protects the following fields:
* - FR_ABORTED
* - FR_LOCKED (may also be modified under fc->lock, tested under both)
*/
struct fuse_req {
/** This can be on either pending processing or io lists in
@ -283,35 +306,8 @@ struct fuse_req {
/** Unique ID for the interrupt request */
u64 intr_unique;
/*
* The following bitfields are either set once before the
* request is queued or setting/clearing them is protected by
* fuse_conn->lock
*/
/** True if the request has reply */
unsigned isreply:1;
/** Force sending of the request even if interrupted */
unsigned force:1;
/** The request was aborted */
unsigned aborted:1;
/** Request is sent in the background */
unsigned background:1;
/** The request has been interrupted */
unsigned interrupted:1;
/** Data is being copied to/from the request */
unsigned locked:1;
/** Request is counted as "waiting" */
unsigned waiting:1;
/** State of the request */
enum fuse_req_state state;
/* Request flags, updated with test/set/clear_bit() */
unsigned long flags;
/** The request input */
struct fuse_in in;
@ -380,6 +376,61 @@ struct fuse_req {
struct file *stolen_file;
};
struct fuse_iqueue {
/** Connection established */
unsigned connected;
/** Readers of the connection are waiting on this */
wait_queue_head_t waitq;
/** The next unique request id */
u64 reqctr;
/** The list of pending requests */
struct list_head pending;
/** Pending interrupts */
struct list_head interrupts;
/** Queue of pending forgets */
struct fuse_forget_link forget_list_head;
struct fuse_forget_link *forget_list_tail;
/** Batching of FORGET requests (positive indicates FORGET batch) */
int forget_batch;
/** O_ASYNC requests */
struct fasync_struct *fasync;
};
struct fuse_pqueue {
/** Connection established */
unsigned connected;
/** Lock protecting accessess to members of this structure */
spinlock_t lock;
/** The list of requests being processed */
struct list_head processing;
/** The list of requests under I/O */
struct list_head io;
};
/**
* Fuse device instance
*/
struct fuse_dev {
/** Fuse connection for this device */
struct fuse_conn *fc;
/** Processing queue */
struct fuse_pqueue pq;
/** list entry on fc->devices */
struct list_head entry;
};
/**
* A Fuse connection.
*
@ -394,6 +445,9 @@ struct fuse_conn {
/** Refcount */
atomic_t count;
/** Number of fuse_dev's */
atomic_t dev_count;
struct rcu_head rcu;
/** The user id for this mount */
@ -411,17 +465,8 @@ struct fuse_conn {
/** Maximum write size */
unsigned max_write;
/** Readers of the connection are waiting on this */
wait_queue_head_t waitq;
/** The list of pending requests */
struct list_head pending;
/** The list of requests being processed */
struct list_head processing;
/** The list of requests under I/O */
struct list_head io;
/** Input queue */
struct fuse_iqueue iq;
/** The next unique kernel file handle */
u64 khctr;
@ -444,16 +489,6 @@ struct fuse_conn {
/** The list of background requests set aside for later queuing */
struct list_head bg_queue;
/** Pending interrupts */
struct list_head interrupts;
/** Queue of pending forgets */
struct fuse_forget_link forget_list_head;
struct fuse_forget_link *forget_list_tail;
/** Batching of FORGET requests (positive indicates FORGET batch) */
int forget_batch;
/** Flag indicating that INIT reply has been received. Allocating
* any fuse request will be suspended until the flag is set */
int initialized;
@ -469,9 +504,6 @@ struct fuse_conn {
/** waitq for reserved requests */
wait_queue_head_t reserved_req_waitq;
/** The next unique request id */
u64 reqctr;
/** Connection established, cleared on umount, connection
abort and device release */
unsigned connected;
@ -594,9 +626,6 @@ struct fuse_conn {
/** number of dentries used in the above array */
int ctl_ndents;
/** O_ASYNC requests */
struct fasync_struct *fasync;
/** Key for lock owner ID scrambling */
u32 scramble_key[4];
@ -614,6 +643,9 @@ struct fuse_conn {
/** Read/write semaphore to hold when accessing sb. */
struct rw_semaphore killsb;
/** List of device instances belonging to this connection */
struct list_head devices;
};
static inline struct fuse_conn *get_fuse_conn_super(struct super_block *sb)
@ -826,6 +858,9 @@ void fuse_conn_init(struct fuse_conn *fc);
*/
void fuse_conn_put(struct fuse_conn *fc);
struct fuse_dev *fuse_dev_alloc(struct fuse_conn *fc);
void fuse_dev_free(struct fuse_dev *fud);
/**
* Add connection to control filesystem
*/

View File

@ -362,8 +362,8 @@ static void fuse_send_destroy(struct fuse_conn *fc)
if (req && fc->conn_init) {
fc->destroy_req = NULL;
req->in.h.opcode = FUSE_DESTROY;
req->force = 1;
req->background = 0;
__set_bit(FR_FORCE, &req->flags);
__clear_bit(FR_BACKGROUND, &req->flags);
fuse_request_send(fc, req);
fuse_put_request(fc, req);
}
@ -567,30 +567,46 @@ static int fuse_show_options(struct seq_file *m, struct dentry *root)
return 0;
}
static void fuse_iqueue_init(struct fuse_iqueue *fiq)
{
memset(fiq, 0, sizeof(struct fuse_iqueue));
init_waitqueue_head(&fiq->waitq);
INIT_LIST_HEAD(&fiq->pending);
INIT_LIST_HEAD(&fiq->interrupts);
fiq->forget_list_tail = &fiq->forget_list_head;
fiq->connected = 1;
}
static void fuse_pqueue_init(struct fuse_pqueue *fpq)
{
memset(fpq, 0, sizeof(struct fuse_pqueue));
spin_lock_init(&fpq->lock);
INIT_LIST_HEAD(&fpq->processing);
INIT_LIST_HEAD(&fpq->io);
fpq->connected = 1;
}
void fuse_conn_init(struct fuse_conn *fc)
{
memset(fc, 0, sizeof(*fc));
spin_lock_init(&fc->lock);
init_rwsem(&fc->killsb);
atomic_set(&fc->count, 1);
init_waitqueue_head(&fc->waitq);
atomic_set(&fc->dev_count, 1);
init_waitqueue_head(&fc->blocked_waitq);
init_waitqueue_head(&fc->reserved_req_waitq);
INIT_LIST_HEAD(&fc->pending);
INIT_LIST_HEAD(&fc->processing);
INIT_LIST_HEAD(&fc->io);
INIT_LIST_HEAD(&fc->interrupts);
fuse_iqueue_init(&fc->iq);
INIT_LIST_HEAD(&fc->bg_queue);
INIT_LIST_HEAD(&fc->entry);
fc->forget_list_tail = &fc->forget_list_head;
INIT_LIST_HEAD(&fc->devices);
atomic_set(&fc->num_waiting, 0);
fc->max_background = FUSE_DEFAULT_MAX_BACKGROUND;
fc->congestion_threshold = FUSE_DEFAULT_CONGESTION_THRESHOLD;
fc->khctr = 0;
fc->polled_files = RB_ROOT;
fc->reqctr = 0;
fc->blocked = 0;
fc->initialized = 0;
fc->connected = 1;
fc->attr_version = 1;
get_random_bytes(&fc->scramble_key, sizeof(fc->scramble_key));
}
@ -930,6 +946,7 @@ static void fuse_send_init(struct fuse_conn *fc, struct fuse_req *req)
static void fuse_free_conn(struct fuse_conn *fc)
{
WARN_ON(!list_empty(&fc->devices));
kfree_rcu(fc, rcu);
}
@ -975,8 +992,42 @@ static int fuse_bdi_init(struct fuse_conn *fc, struct super_block *sb)
return 0;
}
struct fuse_dev *fuse_dev_alloc(struct fuse_conn *fc)
{
struct fuse_dev *fud;
fud = kzalloc(sizeof(struct fuse_dev), GFP_KERNEL);
if (fud) {
fud->fc = fuse_conn_get(fc);
fuse_pqueue_init(&fud->pq);
spin_lock(&fc->lock);
list_add_tail(&fud->entry, &fc->devices);
spin_unlock(&fc->lock);
}
return fud;
}
EXPORT_SYMBOL_GPL(fuse_dev_alloc);
void fuse_dev_free(struct fuse_dev *fud)
{
struct fuse_conn *fc = fud->fc;
if (fc) {
spin_lock(&fc->lock);
list_del(&fud->entry);
spin_unlock(&fc->lock);
fuse_conn_put(fc);
}
kfree(fud);
}
EXPORT_SYMBOL_GPL(fuse_dev_free);
static int fuse_fill_super(struct super_block *sb, void *data, int silent)
{
struct fuse_dev *fud;
struct fuse_conn *fc;
struct inode *root;
struct fuse_mount_data d;
@ -1026,12 +1077,17 @@ static int fuse_fill_super(struct super_block *sb, void *data, int silent)
goto err_fput;
fuse_conn_init(fc);
fc->release = fuse_free_conn;
fud = fuse_dev_alloc(fc);
if (!fud)
goto err_put_conn;
fc->dev = sb->s_dev;
fc->sb = sb;
err = fuse_bdi_init(fc, sb);
if (err)
goto err_put_conn;
goto err_dev_free;
sb->s_bdi = &fc->bdi;
@ -1040,7 +1096,6 @@ static int fuse_fill_super(struct super_block *sb, void *data, int silent)
fc->dont_mask = 1;
sb->s_flags |= MS_POSIXACL;
fc->release = fuse_free_conn;
fc->flags = d.flags;
fc->user_id = d.user_id;
fc->group_id = d.group_id;
@ -1053,14 +1108,14 @@ static int fuse_fill_super(struct super_block *sb, void *data, int silent)
root = fuse_get_root_inode(sb, d.rootmode);
root_dentry = d_make_root(root);
if (!root_dentry)
goto err_put_conn;
goto err_dev_free;
/* only now - we want root dentry with NULL ->d_op */
sb->s_d_op = &fuse_dentry_operations;
init_req = fuse_request_alloc(0);
if (!init_req)
goto err_put_root;
init_req->background = 1;
__set_bit(FR_BACKGROUND, &init_req->flags);
if (is_bdev) {
fc->destroy_req = fuse_request_alloc(0);
@ -1079,8 +1134,7 @@ static int fuse_fill_super(struct super_block *sb, void *data, int silent)
list_add_tail(&fc->entry, &fuse_conn_list);
sb->s_root = root_dentry;
fc->connected = 1;
file->private_data = fuse_conn_get(fc);
file->private_data = fud;
mutex_unlock(&fuse_mutex);
/*
* atomic_dec_and_test() in fput() provides the necessary
@ -1099,6 +1153,8 @@ static int fuse_fill_super(struct super_block *sb, void *data, int silent)
fuse_request_free(init_req);
err_put_root:
dput(root_dentry);
err_dev_free:
fuse_dev_free(fud);
err_put_conn:
fuse_bdi_destroy(fc);
fuse_conn_put(fc);

View File

@ -755,4 +755,7 @@ struct fuse_notify_retrieve_in {
uint64_t dummy4;
};
/* Device ioctls: */
#define FUSE_DEV_IOC_CLONE _IOR(229, 0, uint32_t)
#endif /* _LINUX_FUSE_H */