Merge branch 'iomap-4.10-directio' into for-next
This commit is contained in:
commit
5f1c6d28cf
49
block/bio.c
49
block/bio.c
|
@ -847,6 +847,55 @@ done:
|
|||
}
|
||||
EXPORT_SYMBOL(bio_add_page);
|
||||
|
||||
/**
|
||||
* bio_iov_iter_get_pages - pin user or kernel pages and add them to a bio
|
||||
* @bio: bio to add pages to
|
||||
* @iter: iov iterator describing the region to be mapped
|
||||
*
|
||||
* Pins as many pages from *iter and appends them to @bio's bvec array. The
|
||||
* pages will have to be released using put_page() when done.
|
||||
*/
|
||||
int bio_iov_iter_get_pages(struct bio *bio, struct iov_iter *iter)
|
||||
{
|
||||
unsigned short nr_pages = bio->bi_max_vecs - bio->bi_vcnt;
|
||||
struct bio_vec *bv = bio->bi_io_vec + bio->bi_vcnt;
|
||||
struct page **pages = (struct page **)bv;
|
||||
size_t offset, diff;
|
||||
ssize_t size;
|
||||
|
||||
size = iov_iter_get_pages(iter, pages, LONG_MAX, nr_pages, &offset);
|
||||
if (unlikely(size <= 0))
|
||||
return size ? size : -EFAULT;
|
||||
nr_pages = (size + offset + PAGE_SIZE - 1) / PAGE_SIZE;
|
||||
|
||||
/*
|
||||
* Deep magic below: We need to walk the pinned pages backwards
|
||||
* because we are abusing the space allocated for the bio_vecs
|
||||
* for the page array. Because the bio_vecs are larger than the
|
||||
* page pointers by definition this will always work. But it also
|
||||
* means we can't use bio_add_page, so any changes to it's semantics
|
||||
* need to be reflected here as well.
|
||||
*/
|
||||
bio->bi_iter.bi_size += size;
|
||||
bio->bi_vcnt += nr_pages;
|
||||
|
||||
diff = (nr_pages * PAGE_SIZE - offset) - size;
|
||||
while (nr_pages--) {
|
||||
bv[nr_pages].bv_page = pages[nr_pages];
|
||||
bv[nr_pages].bv_len = PAGE_SIZE;
|
||||
bv[nr_pages].bv_offset = 0;
|
||||
}
|
||||
|
||||
bv[0].bv_offset += offset;
|
||||
bv[0].bv_len -= offset;
|
||||
if (diff)
|
||||
bv[bio->bi_vcnt - 1].bv_len -= diff;
|
||||
|
||||
iov_iter_advance(iter, size);
|
||||
return 0;
|
||||
}
|
||||
EXPORT_SYMBOL_GPL(bio_iov_iter_get_pages);
|
||||
|
||||
struct submit_bio_ret {
|
||||
struct completion event;
|
||||
int error;
|
||||
|
|
|
@ -554,7 +554,7 @@ static inline int dio_bio_reap(struct dio *dio, struct dio_submit *sdio)
|
|||
* filesystems that don't need it and also allows us to create the workqueue
|
||||
* late enough so the we can include s_id in the name of the workqueue.
|
||||
*/
|
||||
static int sb_init_dio_done_wq(struct super_block *sb)
|
||||
int sb_init_dio_done_wq(struct super_block *sb)
|
||||
{
|
||||
struct workqueue_struct *old;
|
||||
struct workqueue_struct *wq = alloc_workqueue("dio/%s",
|
||||
|
|
|
@ -184,3 +184,6 @@ typedef loff_t (*iomap_actor_t)(struct inode *inode, loff_t pos, loff_t len,
|
|||
loff_t iomap_apply(struct inode *inode, loff_t pos, loff_t length,
|
||||
unsigned flags, struct iomap_ops *ops, void *data,
|
||||
iomap_actor_t actor);
|
||||
|
||||
/* direct-io.c: */
|
||||
int sb_init_dio_done_wq(struct super_block *sb);
|
||||
|
|
373
fs/iomap.c
373
fs/iomap.c
|
@ -24,6 +24,7 @@
|
|||
#include <linux/uio.h>
|
||||
#include <linux/backing-dev.h>
|
||||
#include <linux/buffer_head.h>
|
||||
#include <linux/task_io_accounting_ops.h>
|
||||
#include <linux/dax.h>
|
||||
#include "internal.h"
|
||||
|
||||
|
@ -584,3 +585,375 @@ int iomap_fiemap(struct inode *inode, struct fiemap_extent_info *fi,
|
|||
return 0;
|
||||
}
|
||||
EXPORT_SYMBOL_GPL(iomap_fiemap);
|
||||
|
||||
/*
|
||||
* Private flags for iomap_dio, must not overlap with the public ones in
|
||||
* iomap.h:
|
||||
*/
|
||||
#define IOMAP_DIO_WRITE (1 << 30)
|
||||
#define IOMAP_DIO_DIRTY (1 << 31)
|
||||
|
||||
struct iomap_dio {
|
||||
struct kiocb *iocb;
|
||||
iomap_dio_end_io_t *end_io;
|
||||
loff_t i_size;
|
||||
loff_t size;
|
||||
atomic_t ref;
|
||||
unsigned flags;
|
||||
int error;
|
||||
|
||||
union {
|
||||
/* used during submission and for synchronous completion: */
|
||||
struct {
|
||||
struct iov_iter *iter;
|
||||
struct task_struct *waiter;
|
||||
struct request_queue *last_queue;
|
||||
blk_qc_t cookie;
|
||||
} submit;
|
||||
|
||||
/* used for aio completion: */
|
||||
struct {
|
||||
struct work_struct work;
|
||||
} aio;
|
||||
};
|
||||
};
|
||||
|
||||
static ssize_t iomap_dio_complete(struct iomap_dio *dio)
|
||||
{
|
||||
struct kiocb *iocb = dio->iocb;
|
||||
ssize_t ret;
|
||||
|
||||
if (dio->end_io) {
|
||||
ret = dio->end_io(iocb,
|
||||
dio->error ? dio->error : dio->size,
|
||||
dio->flags);
|
||||
} else {
|
||||
ret = dio->error;
|
||||
}
|
||||
|
||||
if (likely(!ret)) {
|
||||
ret = dio->size;
|
||||
/* check for short read */
|
||||
if (iocb->ki_pos + ret > dio->i_size &&
|
||||
!(dio->flags & IOMAP_DIO_WRITE))
|
||||
ret = dio->i_size - iocb->ki_pos;
|
||||
iocb->ki_pos += ret;
|
||||
}
|
||||
|
||||
inode_dio_end(file_inode(iocb->ki_filp));
|
||||
kfree(dio);
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
static void iomap_dio_complete_work(struct work_struct *work)
|
||||
{
|
||||
struct iomap_dio *dio = container_of(work, struct iomap_dio, aio.work);
|
||||
struct kiocb *iocb = dio->iocb;
|
||||
bool is_write = (dio->flags & IOMAP_DIO_WRITE);
|
||||
ssize_t ret;
|
||||
|
||||
ret = iomap_dio_complete(dio);
|
||||
if (is_write && ret > 0)
|
||||
ret = generic_write_sync(iocb, ret);
|
||||
iocb->ki_complete(iocb, ret, 0);
|
||||
}
|
||||
|
||||
/*
|
||||
* Set an error in the dio if none is set yet. We have to use cmpxchg
|
||||
* as the submission context and the completion context(s) can race to
|
||||
* update the error.
|
||||
*/
|
||||
static inline void iomap_dio_set_error(struct iomap_dio *dio, int ret)
|
||||
{
|
||||
cmpxchg(&dio->error, 0, ret);
|
||||
}
|
||||
|
||||
static void iomap_dio_bio_end_io(struct bio *bio)
|
||||
{
|
||||
struct iomap_dio *dio = bio->bi_private;
|
||||
bool should_dirty = (dio->flags & IOMAP_DIO_DIRTY);
|
||||
|
||||
if (bio->bi_error)
|
||||
iomap_dio_set_error(dio, bio->bi_error);
|
||||
|
||||
if (atomic_dec_and_test(&dio->ref)) {
|
||||
if (is_sync_kiocb(dio->iocb)) {
|
||||
struct task_struct *waiter = dio->submit.waiter;
|
||||
|
||||
WRITE_ONCE(dio->submit.waiter, NULL);
|
||||
wake_up_process(waiter);
|
||||
} else if (dio->flags & IOMAP_DIO_WRITE) {
|
||||
struct inode *inode = file_inode(dio->iocb->ki_filp);
|
||||
|
||||
INIT_WORK(&dio->aio.work, iomap_dio_complete_work);
|
||||
queue_work(inode->i_sb->s_dio_done_wq, &dio->aio.work);
|
||||
} else {
|
||||
iomap_dio_complete_work(&dio->aio.work);
|
||||
}
|
||||
}
|
||||
|
||||
if (should_dirty) {
|
||||
bio_check_pages_dirty(bio);
|
||||
} else {
|
||||
struct bio_vec *bvec;
|
||||
int i;
|
||||
|
||||
bio_for_each_segment_all(bvec, bio, i)
|
||||
put_page(bvec->bv_page);
|
||||
bio_put(bio);
|
||||
}
|
||||
}
|
||||
|
||||
static blk_qc_t
|
||||
iomap_dio_zero(struct iomap_dio *dio, struct iomap *iomap, loff_t pos,
|
||||
unsigned len)
|
||||
{
|
||||
struct page *page = ZERO_PAGE(0);
|
||||
struct bio *bio;
|
||||
|
||||
bio = bio_alloc(GFP_KERNEL, 1);
|
||||
bio->bi_bdev = iomap->bdev;
|
||||
bio->bi_iter.bi_sector =
|
||||
iomap->blkno + ((pos - iomap->offset) >> 9);
|
||||
bio->bi_private = dio;
|
||||
bio->bi_end_io = iomap_dio_bio_end_io;
|
||||
|
||||
get_page(page);
|
||||
if (bio_add_page(bio, page, len, 0) != len)
|
||||
BUG();
|
||||
bio_set_op_attrs(bio, REQ_OP_WRITE, WRITE_ODIRECT);
|
||||
|
||||
atomic_inc(&dio->ref);
|
||||
return submit_bio(bio);
|
||||
}
|
||||
|
||||
static loff_t
|
||||
iomap_dio_actor(struct inode *inode, loff_t pos, loff_t length,
|
||||
void *data, struct iomap *iomap)
|
||||
{
|
||||
struct iomap_dio *dio = data;
|
||||
unsigned blkbits = blksize_bits(bdev_logical_block_size(iomap->bdev));
|
||||
unsigned fs_block_size = (1 << inode->i_blkbits), pad;
|
||||
unsigned align = iov_iter_alignment(dio->submit.iter);
|
||||
struct iov_iter iter;
|
||||
struct bio *bio;
|
||||
bool need_zeroout = false;
|
||||
int nr_pages, ret;
|
||||
|
||||
if ((pos | length | align) & ((1 << blkbits) - 1))
|
||||
return -EINVAL;
|
||||
|
||||
switch (iomap->type) {
|
||||
case IOMAP_HOLE:
|
||||
if (WARN_ON_ONCE(dio->flags & IOMAP_DIO_WRITE))
|
||||
return -EIO;
|
||||
/*FALLTHRU*/
|
||||
case IOMAP_UNWRITTEN:
|
||||
if (!(dio->flags & IOMAP_DIO_WRITE)) {
|
||||
iov_iter_zero(length, dio->submit.iter);
|
||||
dio->size += length;
|
||||
return length;
|
||||
}
|
||||
dio->flags |= IOMAP_DIO_UNWRITTEN;
|
||||
need_zeroout = true;
|
||||
break;
|
||||
case IOMAP_MAPPED:
|
||||
if (iomap->flags & IOMAP_F_SHARED)
|
||||
dio->flags |= IOMAP_DIO_COW;
|
||||
if (iomap->flags & IOMAP_F_NEW)
|
||||
need_zeroout = true;
|
||||
break;
|
||||
default:
|
||||
WARN_ON_ONCE(1);
|
||||
return -EIO;
|
||||
}
|
||||
|
||||
/*
|
||||
* Operate on a partial iter trimmed to the extent we were called for.
|
||||
* We'll update the iter in the dio once we're done with this extent.
|
||||
*/
|
||||
iter = *dio->submit.iter;
|
||||
iov_iter_truncate(&iter, length);
|
||||
|
||||
nr_pages = iov_iter_npages(&iter, BIO_MAX_PAGES);
|
||||
if (nr_pages <= 0)
|
||||
return nr_pages;
|
||||
|
||||
if (need_zeroout) {
|
||||
/* zero out from the start of the block to the write offset */
|
||||
pad = pos & (fs_block_size - 1);
|
||||
if (pad)
|
||||
iomap_dio_zero(dio, iomap, pos - pad, pad);
|
||||
}
|
||||
|
||||
do {
|
||||
if (dio->error)
|
||||
return 0;
|
||||
|
||||
bio = bio_alloc(GFP_KERNEL, nr_pages);
|
||||
bio->bi_bdev = iomap->bdev;
|
||||
bio->bi_iter.bi_sector =
|
||||
iomap->blkno + ((pos - iomap->offset) >> 9);
|
||||
bio->bi_private = dio;
|
||||
bio->bi_end_io = iomap_dio_bio_end_io;
|
||||
|
||||
ret = bio_iov_iter_get_pages(bio, &iter);
|
||||
if (unlikely(ret)) {
|
||||
bio_put(bio);
|
||||
return ret;
|
||||
}
|
||||
|
||||
if (dio->flags & IOMAP_DIO_WRITE) {
|
||||
bio_set_op_attrs(bio, REQ_OP_WRITE, WRITE_ODIRECT);
|
||||
task_io_account_write(bio->bi_iter.bi_size);
|
||||
} else {
|
||||
bio_set_op_attrs(bio, REQ_OP_READ, 0);
|
||||
if (dio->flags & IOMAP_DIO_DIRTY)
|
||||
bio_set_pages_dirty(bio);
|
||||
}
|
||||
|
||||
dio->size += bio->bi_iter.bi_size;
|
||||
pos += bio->bi_iter.bi_size;
|
||||
|
||||
nr_pages = iov_iter_npages(&iter, BIO_MAX_PAGES);
|
||||
|
||||
atomic_inc(&dio->ref);
|
||||
|
||||
dio->submit.last_queue = bdev_get_queue(iomap->bdev);
|
||||
dio->submit.cookie = submit_bio(bio);
|
||||
} while (nr_pages);
|
||||
|
||||
if (need_zeroout) {
|
||||
/* zero out from the end of the write to the end of the block */
|
||||
pad = pos & (fs_block_size - 1);
|
||||
if (pad)
|
||||
iomap_dio_zero(dio, iomap, pos, fs_block_size - pad);
|
||||
}
|
||||
|
||||
iov_iter_advance(dio->submit.iter, length);
|
||||
return length;
|
||||
}
|
||||
|
||||
ssize_t
|
||||
iomap_dio_rw(struct kiocb *iocb, struct iov_iter *iter, struct iomap_ops *ops,
|
||||
iomap_dio_end_io_t end_io)
|
||||
{
|
||||
struct address_space *mapping = iocb->ki_filp->f_mapping;
|
||||
struct inode *inode = file_inode(iocb->ki_filp);
|
||||
size_t count = iov_iter_count(iter);
|
||||
loff_t pos = iocb->ki_pos, end = iocb->ki_pos + count - 1, ret = 0;
|
||||
unsigned int flags = IOMAP_DIRECT;
|
||||
struct blk_plug plug;
|
||||
struct iomap_dio *dio;
|
||||
|
||||
lockdep_assert_held(&inode->i_rwsem);
|
||||
|
||||
if (!count)
|
||||
return 0;
|
||||
|
||||
dio = kmalloc(sizeof(*dio), GFP_KERNEL);
|
||||
if (!dio)
|
||||
return -ENOMEM;
|
||||
|
||||
dio->iocb = iocb;
|
||||
atomic_set(&dio->ref, 1);
|
||||
dio->size = 0;
|
||||
dio->i_size = i_size_read(inode);
|
||||
dio->end_io = end_io;
|
||||
dio->error = 0;
|
||||
dio->flags = 0;
|
||||
|
||||
dio->submit.iter = iter;
|
||||
if (is_sync_kiocb(iocb)) {
|
||||
dio->submit.waiter = current;
|
||||
dio->submit.cookie = BLK_QC_T_NONE;
|
||||
dio->submit.last_queue = NULL;
|
||||
}
|
||||
|
||||
if (iov_iter_rw(iter) == READ) {
|
||||
if (pos >= dio->i_size)
|
||||
goto out_free_dio;
|
||||
|
||||
if (iter->type == ITER_IOVEC)
|
||||
dio->flags |= IOMAP_DIO_DIRTY;
|
||||
} else {
|
||||
dio->flags |= IOMAP_DIO_WRITE;
|
||||
flags |= IOMAP_WRITE;
|
||||
}
|
||||
|
||||
if (mapping->nrpages) {
|
||||
ret = filemap_write_and_wait_range(mapping, iocb->ki_pos, end);
|
||||
if (ret)
|
||||
goto out_free_dio;
|
||||
|
||||
ret = invalidate_inode_pages2_range(mapping,
|
||||
iocb->ki_pos >> PAGE_SHIFT, end >> PAGE_SHIFT);
|
||||
WARN_ON_ONCE(ret);
|
||||
ret = 0;
|
||||
}
|
||||
|
||||
inode_dio_begin(inode);
|
||||
|
||||
blk_start_plug(&plug);
|
||||
do {
|
||||
ret = iomap_apply(inode, pos, count, flags, ops, dio,
|
||||
iomap_dio_actor);
|
||||
if (ret <= 0) {
|
||||
/* magic error code to fall back to buffered I/O */
|
||||
if (ret == -ENOTBLK)
|
||||
ret = 0;
|
||||
break;
|
||||
}
|
||||
pos += ret;
|
||||
} while ((count = iov_iter_count(iter)) > 0);
|
||||
blk_finish_plug(&plug);
|
||||
|
||||
if (ret < 0)
|
||||
iomap_dio_set_error(dio, ret);
|
||||
|
||||
if (ret >= 0 && iov_iter_rw(iter) == WRITE && !is_sync_kiocb(iocb) &&
|
||||
!inode->i_sb->s_dio_done_wq) {
|
||||
ret = sb_init_dio_done_wq(inode->i_sb);
|
||||
if (ret < 0)
|
||||
iomap_dio_set_error(dio, ret);
|
||||
}
|
||||
|
||||
if (!atomic_dec_and_test(&dio->ref)) {
|
||||
if (!is_sync_kiocb(iocb))
|
||||
return -EIOCBQUEUED;
|
||||
|
||||
for (;;) {
|
||||
set_current_state(TASK_UNINTERRUPTIBLE);
|
||||
if (!READ_ONCE(dio->submit.waiter))
|
||||
break;
|
||||
|
||||
if (!(iocb->ki_flags & IOCB_HIPRI) ||
|
||||
!dio->submit.last_queue ||
|
||||
!blk_poll(dio->submit.last_queue,
|
||||
dio->submit.cookie))
|
||||
io_schedule();
|
||||
}
|
||||
__set_current_state(TASK_RUNNING);
|
||||
}
|
||||
|
||||
/*
|
||||
* Try again to invalidate clean pages which might have been cached by
|
||||
* non-direct readahead, or faulted in by get_user_pages() if the source
|
||||
* of the write was an mmap'ed region of the file we're writing. Either
|
||||
* one is a pretty crazy thing to do, so we don't support it 100%. If
|
||||
* this invalidation fails, tough, the write still worked...
|
||||
*/
|
||||
if (iov_iter_rw(iter) == WRITE && mapping->nrpages) {
|
||||
ret = invalidate_inode_pages2_range(mapping,
|
||||
iocb->ki_pos >> PAGE_SHIFT, end >> PAGE_SHIFT);
|
||||
WARN_ON_ONCE(ret);
|
||||
}
|
||||
|
||||
return iomap_dio_complete(dio);
|
||||
|
||||
out_free_dio:
|
||||
kfree(dio);
|
||||
return ret;
|
||||
}
|
||||
EXPORT_SYMBOL_GPL(iomap_dio_rw);
|
||||
|
|
|
@ -37,11 +37,6 @@
|
|||
#include <linux/pagevec.h>
|
||||
#include <linux/writeback.h>
|
||||
|
||||
/* flags for direct write completions */
|
||||
#define XFS_DIO_FLAG_UNWRITTEN (1 << 0)
|
||||
#define XFS_DIO_FLAG_APPEND (1 << 1)
|
||||
#define XFS_DIO_FLAG_COW (1 << 2)
|
||||
|
||||
/*
|
||||
* structure owned by writepages passed to individual writepage calls
|
||||
*/
|
||||
|
@ -1175,45 +1170,6 @@ xfs_vm_releasepage(
|
|||
return try_to_free_buffers(page);
|
||||
}
|
||||
|
||||
/*
|
||||
* When we map a DIO buffer, we may need to pass flags to
|
||||
* xfs_end_io_direct_write to tell it what kind of write IO we are doing.
|
||||
*
|
||||
* Note that for DIO, an IO to the highest supported file block offset (i.e.
|
||||
* 2^63 - 1FSB bytes) will result in the offset + count overflowing a signed 64
|
||||
* bit variable. Hence if we see this overflow, we have to assume that the IO is
|
||||
* extending the file size. We won't know for sure until IO completion is run
|
||||
* and the actual max write offset is communicated to the IO completion
|
||||
* routine.
|
||||
*/
|
||||
static void
|
||||
xfs_map_direct(
|
||||
struct inode *inode,
|
||||
struct buffer_head *bh_result,
|
||||
struct xfs_bmbt_irec *imap,
|
||||
xfs_off_t offset,
|
||||
bool is_cow)
|
||||
{
|
||||
uintptr_t *flags = (uintptr_t *)&bh_result->b_private;
|
||||
xfs_off_t size = bh_result->b_size;
|
||||
|
||||
trace_xfs_get_blocks_map_direct(XFS_I(inode), offset, size,
|
||||
ISUNWRITTEN(imap) ? XFS_IO_UNWRITTEN : is_cow ? XFS_IO_COW :
|
||||
XFS_IO_OVERWRITE, imap);
|
||||
|
||||
if (ISUNWRITTEN(imap)) {
|
||||
*flags |= XFS_DIO_FLAG_UNWRITTEN;
|
||||
set_buffer_defer_completion(bh_result);
|
||||
} else if (is_cow) {
|
||||
*flags |= XFS_DIO_FLAG_COW;
|
||||
set_buffer_defer_completion(bh_result);
|
||||
}
|
||||
if (offset + size > i_size_read(inode) || offset + size < 0) {
|
||||
*flags |= XFS_DIO_FLAG_APPEND;
|
||||
set_buffer_defer_completion(bh_result);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* If this is O_DIRECT or the mpage code calling tell them how large the mapping
|
||||
* is, so that we can avoid repeated get_blocks calls.
|
||||
|
@ -1254,51 +1210,12 @@ xfs_map_trim_size(
|
|||
bh_result->b_size = mapping_size;
|
||||
}
|
||||
|
||||
/* Bounce unaligned directio writes to the page cache. */
|
||||
static int
|
||||
xfs_bounce_unaligned_dio_write(
|
||||
struct xfs_inode *ip,
|
||||
xfs_fileoff_t offset_fsb,
|
||||
struct xfs_bmbt_irec *imap)
|
||||
{
|
||||
struct xfs_bmbt_irec irec;
|
||||
xfs_fileoff_t delta;
|
||||
bool shared;
|
||||
bool x;
|
||||
int error;
|
||||
|
||||
irec = *imap;
|
||||
if (offset_fsb > irec.br_startoff) {
|
||||
delta = offset_fsb - irec.br_startoff;
|
||||
irec.br_blockcount -= delta;
|
||||
irec.br_startblock += delta;
|
||||
irec.br_startoff = offset_fsb;
|
||||
}
|
||||
error = xfs_reflink_trim_around_shared(ip, &irec, &shared, &x);
|
||||
if (error)
|
||||
return error;
|
||||
|
||||
/*
|
||||
* We're here because we're trying to do a directio write to a
|
||||
* region that isn't aligned to a filesystem block. If any part
|
||||
* of the extent is shared, fall back to buffered mode to handle
|
||||
* the RMW. This is done by returning -EREMCHG ("remote addr
|
||||
* changed"), which is caught further up the call stack.
|
||||
*/
|
||||
if (shared) {
|
||||
trace_xfs_reflink_bounce_dio_write(ip, imap);
|
||||
return -EREMCHG;
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
STATIC int
|
||||
__xfs_get_blocks(
|
||||
xfs_get_blocks(
|
||||
struct inode *inode,
|
||||
sector_t iblock,
|
||||
struct buffer_head *bh_result,
|
||||
int create,
|
||||
bool direct)
|
||||
int create)
|
||||
{
|
||||
struct xfs_inode *ip = XFS_I(inode);
|
||||
struct xfs_mount *mp = ip->i_mount;
|
||||
|
@ -1309,10 +1226,8 @@ __xfs_get_blocks(
|
|||
int nimaps = 1;
|
||||
xfs_off_t offset;
|
||||
ssize_t size;
|
||||
int new = 0;
|
||||
bool is_cow = false;
|
||||
|
||||
BUG_ON(create && !direct);
|
||||
BUG_ON(create);
|
||||
|
||||
if (XFS_FORCED_SHUTDOWN(mp))
|
||||
return -EIO;
|
||||
|
@ -1321,7 +1236,7 @@ __xfs_get_blocks(
|
|||
ASSERT(bh_result->b_size >= (1 << inode->i_blkbits));
|
||||
size = bh_result->b_size;
|
||||
|
||||
if (!create && offset >= i_size_read(inode))
|
||||
if (offset >= i_size_read(inode))
|
||||
return 0;
|
||||
|
||||
/*
|
||||
|
@ -1336,73 +1251,12 @@ __xfs_get_blocks(
|
|||
end_fsb = XFS_B_TO_FSB(mp, (xfs_ufsize_t)offset + size);
|
||||
offset_fsb = XFS_B_TO_FSBT(mp, offset);
|
||||
|
||||
if (create && direct && xfs_is_reflink_inode(ip)) {
|
||||
is_cow = xfs_reflink_find_cow_mapping(ip, offset, &imap);
|
||||
ASSERT(!is_cow || !isnullstartblock(imap.br_startblock));
|
||||
}
|
||||
|
||||
if (!is_cow) {
|
||||
error = xfs_bmapi_read(ip, offset_fsb, end_fsb - offset_fsb,
|
||||
&imap, &nimaps, XFS_BMAPI_ENTIRE);
|
||||
/*
|
||||
* Truncate an overwrite extent if there's a pending CoW
|
||||
* reservation before the end of this extent. This
|
||||
* forces us to come back to get_blocks to take care of
|
||||
* the CoW.
|
||||
*/
|
||||
if (create && direct && nimaps &&
|
||||
imap.br_startblock != HOLESTARTBLOCK &&
|
||||
imap.br_startblock != DELAYSTARTBLOCK &&
|
||||
!ISUNWRITTEN(&imap))
|
||||
xfs_reflink_trim_irec_to_next_cow(ip, offset_fsb,
|
||||
&imap);
|
||||
}
|
||||
error = xfs_bmapi_read(ip, offset_fsb, end_fsb - offset_fsb,
|
||||
&imap, &nimaps, XFS_BMAPI_ENTIRE);
|
||||
if (error)
|
||||
goto out_unlock;
|
||||
|
||||
/*
|
||||
* The only time we can ever safely find delalloc blocks on direct I/O
|
||||
* is a dio write to post-eof speculative preallocation. All other
|
||||
* scenarios are indicative of a problem or misuse (such as mixing
|
||||
* direct and mapped I/O).
|
||||
*
|
||||
* The file may be unmapped by the time we get here so we cannot
|
||||
* reliably fail the I/O based on mapping. Instead, fail the I/O if this
|
||||
* is a read or a write within eof. Otherwise, carry on but warn as a
|
||||
* precuation if the file happens to be mapped.
|
||||
*/
|
||||
if (direct && imap.br_startblock == DELAYSTARTBLOCK) {
|
||||
if (!create || offset < i_size_read(VFS_I(ip))) {
|
||||
WARN_ON_ONCE(1);
|
||||
error = -EIO;
|
||||
goto out_unlock;
|
||||
}
|
||||
WARN_ON_ONCE(mapping_mapped(VFS_I(ip)->i_mapping));
|
||||
}
|
||||
|
||||
/* for DAX, we convert unwritten extents directly */
|
||||
if (create &&
|
||||
(!nimaps ||
|
||||
(imap.br_startblock == HOLESTARTBLOCK ||
|
||||
imap.br_startblock == DELAYSTARTBLOCK) ||
|
||||
(IS_DAX(inode) && ISUNWRITTEN(&imap)))) {
|
||||
/*
|
||||
* xfs_iomap_write_direct() expects the shared lock. It
|
||||
* is unlocked on return.
|
||||
*/
|
||||
if (lockmode == XFS_ILOCK_EXCL)
|
||||
xfs_ilock_demote(ip, lockmode);
|
||||
|
||||
error = xfs_iomap_write_direct(ip, offset, size,
|
||||
&imap, nimaps);
|
||||
if (error)
|
||||
return error;
|
||||
new = 1;
|
||||
|
||||
trace_xfs_get_blocks_alloc(ip, offset, size,
|
||||
ISUNWRITTEN(&imap) ? XFS_IO_UNWRITTEN
|
||||
: XFS_IO_DELALLOC, &imap);
|
||||
} else if (nimaps) {
|
||||
if (nimaps) {
|
||||
trace_xfs_get_blocks_found(ip, offset, size,
|
||||
ISUNWRITTEN(&imap) ? XFS_IO_UNWRITTEN
|
||||
: XFS_IO_OVERWRITE, &imap);
|
||||
|
@ -1412,12 +1266,6 @@ __xfs_get_blocks(
|
|||
goto out_unlock;
|
||||
}
|
||||
|
||||
if (IS_DAX(inode) && create) {
|
||||
ASSERT(!ISUNWRITTEN(&imap));
|
||||
/* zeroing is not needed at a higher layer */
|
||||
new = 0;
|
||||
}
|
||||
|
||||
/* trim mapping down to size requested */
|
||||
xfs_map_trim_size(inode, iblock, bh_result, &imap, offset, size);
|
||||
|
||||
|
@ -1427,43 +1275,14 @@ __xfs_get_blocks(
|
|||
*/
|
||||
if (imap.br_startblock != HOLESTARTBLOCK &&
|
||||
imap.br_startblock != DELAYSTARTBLOCK &&
|
||||
(create || !ISUNWRITTEN(&imap))) {
|
||||
if (create && direct && !is_cow) {
|
||||
error = xfs_bounce_unaligned_dio_write(ip, offset_fsb,
|
||||
&imap);
|
||||
if (error)
|
||||
return error;
|
||||
}
|
||||
|
||||
!ISUNWRITTEN(&imap))
|
||||
xfs_map_buffer(inode, bh_result, &imap, offset);
|
||||
if (ISUNWRITTEN(&imap))
|
||||
set_buffer_unwritten(bh_result);
|
||||
/* direct IO needs special help */
|
||||
if (create)
|
||||
xfs_map_direct(inode, bh_result, &imap, offset, is_cow);
|
||||
}
|
||||
|
||||
/*
|
||||
* If this is a realtime file, data may be on a different device.
|
||||
* to that pointed to from the buffer_head b_bdev currently.
|
||||
*/
|
||||
bh_result->b_bdev = xfs_find_bdev_for_inode(inode);
|
||||
|
||||
/*
|
||||
* If we previously allocated a block out beyond eof and we are now
|
||||
* coming back to use it then we will need to flag it as new even if it
|
||||
* has a disk address.
|
||||
*
|
||||
* With sub-block writes into unwritten extents we also need to mark
|
||||
* the buffer as new so that the unwritten parts of the buffer gets
|
||||
* correctly zeroed.
|
||||
*/
|
||||
if (create &&
|
||||
((!buffer_mapped(bh_result) && !buffer_uptodate(bh_result)) ||
|
||||
(offset >= i_size_read(inode)) ||
|
||||
(new || ISUNWRITTEN(&imap))))
|
||||
set_buffer_new(bh_result);
|
||||
|
||||
return 0;
|
||||
|
||||
out_unlock:
|
||||
|
@ -1471,100 +1290,6 @@ out_unlock:
|
|||
return error;
|
||||
}
|
||||
|
||||
int
|
||||
xfs_get_blocks(
|
||||
struct inode *inode,
|
||||
sector_t iblock,
|
||||
struct buffer_head *bh_result,
|
||||
int create)
|
||||
{
|
||||
return __xfs_get_blocks(inode, iblock, bh_result, create, false);
|
||||
}
|
||||
|
||||
int
|
||||
xfs_get_blocks_direct(
|
||||
struct inode *inode,
|
||||
sector_t iblock,
|
||||
struct buffer_head *bh_result,
|
||||
int create)
|
||||
{
|
||||
return __xfs_get_blocks(inode, iblock, bh_result, create, true);
|
||||
}
|
||||
|
||||
/*
|
||||
* Complete a direct I/O write request.
|
||||
*
|
||||
* xfs_map_direct passes us some flags in the private data to tell us what to
|
||||
* do. If no flags are set, then the write IO is an overwrite wholly within
|
||||
* the existing allocated file size and so there is nothing for us to do.
|
||||
*
|
||||
* Note that in this case the completion can be called in interrupt context,
|
||||
* whereas if we have flags set we will always be called in task context
|
||||
* (i.e. from a workqueue).
|
||||
*/
|
||||
int
|
||||
xfs_end_io_direct_write(
|
||||
struct kiocb *iocb,
|
||||
loff_t offset,
|
||||
ssize_t size,
|
||||
void *private)
|
||||
{
|
||||
struct inode *inode = file_inode(iocb->ki_filp);
|
||||
struct xfs_inode *ip = XFS_I(inode);
|
||||
uintptr_t flags = (uintptr_t)private;
|
||||
int error = 0;
|
||||
|
||||
trace_xfs_end_io_direct_write(ip, offset, size);
|
||||
|
||||
if (XFS_FORCED_SHUTDOWN(ip->i_mount))
|
||||
return -EIO;
|
||||
|
||||
if (size <= 0)
|
||||
return size;
|
||||
|
||||
/*
|
||||
* The flags tell us whether we are doing unwritten extent conversions
|
||||
* or an append transaction that updates the on-disk file size. These
|
||||
* cases are the only cases where we should *potentially* be needing
|
||||
* to update the VFS inode size.
|
||||
*/
|
||||
if (flags == 0) {
|
||||
ASSERT(offset + size <= i_size_read(inode));
|
||||
return 0;
|
||||
}
|
||||
|
||||
/*
|
||||
* We need to update the in-core inode size here so that we don't end up
|
||||
* with the on-disk inode size being outside the in-core inode size. We
|
||||
* have no other method of updating EOF for AIO, so always do it here
|
||||
* if necessary.
|
||||
*
|
||||
* We need to lock the test/set EOF update as we can be racing with
|
||||
* other IO completions here to update the EOF. Failing to serialise
|
||||
* here can result in EOF moving backwards and Bad Things Happen when
|
||||
* that occurs.
|
||||
*/
|
||||
spin_lock(&ip->i_flags_lock);
|
||||
if (offset + size > i_size_read(inode))
|
||||
i_size_write(inode, offset + size);
|
||||
spin_unlock(&ip->i_flags_lock);
|
||||
|
||||
if (flags & XFS_DIO_FLAG_COW)
|
||||
error = xfs_reflink_end_cow(ip, offset, size);
|
||||
if (flags & XFS_DIO_FLAG_UNWRITTEN) {
|
||||
trace_xfs_end_io_direct_write_unwritten(ip, offset, size);
|
||||
|
||||
error = xfs_iomap_write_unwritten(ip, offset, size);
|
||||
}
|
||||
if (flags & XFS_DIO_FLAG_APPEND) {
|
||||
trace_xfs_end_io_direct_write_append(ip, offset, size);
|
||||
|
||||
error = xfs_setfilesize(ip, offset, size);
|
||||
}
|
||||
|
||||
return error;
|
||||
}
|
||||
|
||||
STATIC ssize_t
|
||||
xfs_vm_direct_IO(
|
||||
struct kiocb *iocb,
|
||||
|
@ -1585,7 +1310,6 @@ xfs_vm_bmap(
|
|||
struct xfs_inode *ip = XFS_I(inode);
|
||||
|
||||
trace_xfs_vm_bmap(XFS_I(inode));
|
||||
xfs_ilock(ip, XFS_IOLOCK_SHARED);
|
||||
|
||||
/*
|
||||
* The swap code (ab-)uses ->bmap to get a block mapping and then
|
||||
|
@ -1593,12 +1317,10 @@ xfs_vm_bmap(
|
|||
* that on reflinks inodes, so we have to skip out here. And yes,
|
||||
* 0 is the magic code for a bmap error..
|
||||
*/
|
||||
if (xfs_is_reflink_inode(ip)) {
|
||||
xfs_iunlock(ip, XFS_IOLOCK_SHARED);
|
||||
if (xfs_is_reflink_inode(ip))
|
||||
return 0;
|
||||
}
|
||||
|
||||
filemap_write_and_wait(mapping);
|
||||
xfs_iunlock(ip, XFS_IOLOCK_SHARED);
|
||||
return generic_block_bmap(mapping, block, xfs_get_blocks);
|
||||
}
|
||||
|
||||
|
|
|
@ -55,12 +55,6 @@ struct xfs_ioend {
|
|||
|
||||
extern const struct address_space_operations xfs_address_space_operations;
|
||||
|
||||
int xfs_get_blocks(struct inode *inode, sector_t offset,
|
||||
struct buffer_head *map_bh, int create);
|
||||
int xfs_get_blocks_direct(struct inode *inode, sector_t offset,
|
||||
struct buffer_head *map_bh, int create);
|
||||
int xfs_end_io_direct_write(struct kiocb *iocb, loff_t offset,
|
||||
ssize_t size, void *private);
|
||||
int xfs_setfilesize(struct xfs_inode *ip, xfs_off_t offset, size_t size);
|
||||
|
||||
extern void xfs_count_page_state(struct page *, int *, int *);
|
||||
|
|
|
@ -1935,8 +1935,8 @@ xfs_swap_extents(
|
|||
* page cache safely. Once we have done this we can take the ilocks and
|
||||
* do the rest of the checks.
|
||||
*/
|
||||
lock_flags = XFS_IOLOCK_EXCL | XFS_MMAPLOCK_EXCL;
|
||||
xfs_lock_two_inodes(ip, tip, XFS_IOLOCK_EXCL);
|
||||
lock_two_nondirectories(VFS_I(ip), VFS_I(tip));
|
||||
lock_flags = XFS_MMAPLOCK_EXCL;
|
||||
xfs_lock_two_inodes(ip, tip, XFS_MMAPLOCK_EXCL);
|
||||
|
||||
/* Verify that both files have the same format */
|
||||
|
@ -2076,15 +2076,13 @@ xfs_swap_extents(
|
|||
trace_xfs_swap_extent_after(ip, 0);
|
||||
trace_xfs_swap_extent_after(tip, 1);
|
||||
|
||||
out_unlock:
|
||||
xfs_iunlock(ip, lock_flags);
|
||||
xfs_iunlock(tip, lock_flags);
|
||||
unlock_two_nondirectories(VFS_I(ip), VFS_I(tip));
|
||||
return error;
|
||||
|
||||
out_trans_cancel:
|
||||
xfs_trans_cancel(tp);
|
||||
|
||||
out_unlock:
|
||||
xfs_iunlock(ip, lock_flags);
|
||||
xfs_iunlock(tip, lock_flags);
|
||||
return error;
|
||||
goto out_unlock;
|
||||
}
|
||||
|
|
|
@ -677,7 +677,6 @@ xfs_readdir(
|
|||
args.dp = dp;
|
||||
args.geo = dp->i_mount->m_dir_geo;
|
||||
|
||||
xfs_ilock(dp, XFS_IOLOCK_SHARED);
|
||||
if (dp->i_d.di_format == XFS_DINODE_FMT_LOCAL)
|
||||
rval = xfs_dir2_sf_getdents(&args, ctx);
|
||||
else if ((rval = xfs_dir2_isblock(&args, &v)))
|
||||
|
@ -686,7 +685,6 @@ xfs_readdir(
|
|||
rval = xfs_dir2_block_getdents(&args, ctx);
|
||||
else
|
||||
rval = xfs_dir2_leaf_getdents(&args, ctx, bufsize);
|
||||
xfs_iunlock(dp, XFS_IOLOCK_SHARED);
|
||||
|
||||
return rval;
|
||||
}
|
||||
|
|
|
@ -47,40 +47,6 @@
|
|||
|
||||
static const struct vm_operations_struct xfs_file_vm_ops;
|
||||
|
||||
/*
|
||||
* Locking primitives for read and write IO paths to ensure we consistently use
|
||||
* and order the inode->i_mutex, ip->i_lock and ip->i_iolock.
|
||||
*/
|
||||
static inline void
|
||||
xfs_rw_ilock(
|
||||
struct xfs_inode *ip,
|
||||
int type)
|
||||
{
|
||||
if (type & XFS_IOLOCK_EXCL)
|
||||
inode_lock(VFS_I(ip));
|
||||
xfs_ilock(ip, type);
|
||||
}
|
||||
|
||||
static inline void
|
||||
xfs_rw_iunlock(
|
||||
struct xfs_inode *ip,
|
||||
int type)
|
||||
{
|
||||
xfs_iunlock(ip, type);
|
||||
if (type & XFS_IOLOCK_EXCL)
|
||||
inode_unlock(VFS_I(ip));
|
||||
}
|
||||
|
||||
static inline void
|
||||
xfs_rw_ilock_demote(
|
||||
struct xfs_inode *ip,
|
||||
int type)
|
||||
{
|
||||
xfs_ilock_demote(ip, type);
|
||||
if (type & XFS_IOLOCK_EXCL)
|
||||
inode_unlock(VFS_I(ip));
|
||||
}
|
||||
|
||||
/*
|
||||
* Clear the specified ranges to zero through either the pagecache or DAX.
|
||||
* Holes and unwritten extents will be left as-is as they already are zeroed.
|
||||
|
@ -244,62 +210,21 @@ xfs_file_dio_aio_read(
|
|||
struct kiocb *iocb,
|
||||
struct iov_iter *to)
|
||||
{
|
||||
struct address_space *mapping = iocb->ki_filp->f_mapping;
|
||||
struct inode *inode = mapping->host;
|
||||
struct xfs_inode *ip = XFS_I(inode);
|
||||
loff_t isize = i_size_read(inode);
|
||||
struct xfs_inode *ip = XFS_I(file_inode(iocb->ki_filp));
|
||||
size_t count = iov_iter_count(to);
|
||||
loff_t end = iocb->ki_pos + count - 1;
|
||||
struct iov_iter data;
|
||||
struct xfs_buftarg *target;
|
||||
ssize_t ret = 0;
|
||||
ssize_t ret;
|
||||
|
||||
trace_xfs_file_direct_read(ip, count, iocb->ki_pos);
|
||||
|
||||
if (!count)
|
||||
return 0; /* skip atime */
|
||||
|
||||
if (XFS_IS_REALTIME_INODE(ip))
|
||||
target = ip->i_mount->m_rtdev_targp;
|
||||
else
|
||||
target = ip->i_mount->m_ddev_targp;
|
||||
|
||||
/* DIO must be aligned to device logical sector size */
|
||||
if ((iocb->ki_pos | count) & target->bt_logical_sectormask) {
|
||||
if (iocb->ki_pos == isize)
|
||||
return 0;
|
||||
return -EINVAL;
|
||||
}
|
||||
|
||||
file_accessed(iocb->ki_filp);
|
||||
|
||||
xfs_rw_ilock(ip, XFS_IOLOCK_SHARED);
|
||||
if (mapping->nrpages) {
|
||||
ret = filemap_write_and_wait_range(mapping, iocb->ki_pos, end);
|
||||
if (ret)
|
||||
goto out_unlock;
|
||||
xfs_ilock(ip, XFS_IOLOCK_SHARED);
|
||||
ret = iomap_dio_rw(iocb, to, &xfs_iomap_ops, NULL);
|
||||
xfs_iunlock(ip, XFS_IOLOCK_SHARED);
|
||||
|
||||
/*
|
||||
* Invalidate whole pages. This can return an error if we fail
|
||||
* to invalidate a page, but this should never happen on XFS.
|
||||
* Warn if it does fail.
|
||||
*/
|
||||
ret = invalidate_inode_pages2_range(mapping,
|
||||
iocb->ki_pos >> PAGE_SHIFT, end >> PAGE_SHIFT);
|
||||
WARN_ON_ONCE(ret);
|
||||
ret = 0;
|
||||
}
|
||||
|
||||
data = *to;
|
||||
ret = __blockdev_direct_IO(iocb, inode, target->bt_bdev, &data,
|
||||
xfs_get_blocks_direct, NULL, NULL, 0);
|
||||
if (ret >= 0) {
|
||||
iocb->ki_pos += ret;
|
||||
iov_iter_advance(to, ret);
|
||||
}
|
||||
|
||||
out_unlock:
|
||||
xfs_rw_iunlock(ip, XFS_IOLOCK_SHARED);
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
@ -317,9 +242,9 @@ xfs_file_dax_read(
|
|||
if (!count)
|
||||
return 0; /* skip atime */
|
||||
|
||||
xfs_rw_ilock(ip, XFS_IOLOCK_SHARED);
|
||||
xfs_ilock(ip, XFS_IOLOCK_SHARED);
|
||||
ret = dax_iomap_rw(iocb, to, &xfs_iomap_ops);
|
||||
xfs_rw_iunlock(ip, XFS_IOLOCK_SHARED);
|
||||
xfs_iunlock(ip, XFS_IOLOCK_SHARED);
|
||||
|
||||
file_accessed(iocb->ki_filp);
|
||||
return ret;
|
||||
|
@ -335,9 +260,9 @@ xfs_file_buffered_aio_read(
|
|||
|
||||
trace_xfs_file_buffered_read(ip, iov_iter_count(to), iocb->ki_pos);
|
||||
|
||||
xfs_rw_ilock(ip, XFS_IOLOCK_SHARED);
|
||||
xfs_ilock(ip, XFS_IOLOCK_SHARED);
|
||||
ret = generic_file_read_iter(iocb, to);
|
||||
xfs_rw_iunlock(ip, XFS_IOLOCK_SHARED);
|
||||
xfs_iunlock(ip, XFS_IOLOCK_SHARED);
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
@ -418,15 +343,18 @@ restart:
|
|||
if (error <= 0)
|
||||
return error;
|
||||
|
||||
error = xfs_break_layouts(inode, iolock, true);
|
||||
error = xfs_break_layouts(inode, iolock);
|
||||
if (error)
|
||||
return error;
|
||||
|
||||
/* For changing security info in file_remove_privs() we need i_mutex */
|
||||
/*
|
||||
* For changing security info in file_remove_privs() we need i_rwsem
|
||||
* exclusively.
|
||||
*/
|
||||
if (*iolock == XFS_IOLOCK_SHARED && !IS_NOSEC(inode)) {
|
||||
xfs_rw_iunlock(ip, *iolock);
|
||||
xfs_iunlock(ip, *iolock);
|
||||
*iolock = XFS_IOLOCK_EXCL;
|
||||
xfs_rw_ilock(ip, *iolock);
|
||||
xfs_ilock(ip, *iolock);
|
||||
goto restart;
|
||||
}
|
||||
/*
|
||||
|
@ -451,9 +379,9 @@ restart:
|
|||
spin_unlock(&ip->i_flags_lock);
|
||||
if (!drained_dio) {
|
||||
if (*iolock == XFS_IOLOCK_SHARED) {
|
||||
xfs_rw_iunlock(ip, *iolock);
|
||||
xfs_iunlock(ip, *iolock);
|
||||
*iolock = XFS_IOLOCK_EXCL;
|
||||
xfs_rw_ilock(ip, *iolock);
|
||||
xfs_ilock(ip, *iolock);
|
||||
iov_iter_reexpand(from, count);
|
||||
}
|
||||
/*
|
||||
|
@ -496,6 +424,58 @@ restart:
|
|||
return 0;
|
||||
}
|
||||
|
||||
static int
|
||||
xfs_dio_write_end_io(
|
||||
struct kiocb *iocb,
|
||||
ssize_t size,
|
||||
unsigned flags)
|
||||
{
|
||||
struct inode *inode = file_inode(iocb->ki_filp);
|
||||
struct xfs_inode *ip = XFS_I(inode);
|
||||
loff_t offset = iocb->ki_pos;
|
||||
bool update_size = false;
|
||||
int error = 0;
|
||||
|
||||
trace_xfs_end_io_direct_write(ip, offset, size);
|
||||
|
||||
if (XFS_FORCED_SHUTDOWN(ip->i_mount))
|
||||
return -EIO;
|
||||
|
||||
if (size <= 0)
|
||||
return size;
|
||||
|
||||
/*
|
||||
* We need to update the in-core inode size here so that we don't end up
|
||||
* with the on-disk inode size being outside the in-core inode size. We
|
||||
* have no other method of updating EOF for AIO, so always do it here
|
||||
* if necessary.
|
||||
*
|
||||
* We need to lock the test/set EOF update as we can be racing with
|
||||
* other IO completions here to update the EOF. Failing to serialise
|
||||
* here can result in EOF moving backwards and Bad Things Happen when
|
||||
* that occurs.
|
||||
*/
|
||||
spin_lock(&ip->i_flags_lock);
|
||||
if (offset + size > i_size_read(inode)) {
|
||||
i_size_write(inode, offset + size);
|
||||
update_size = true;
|
||||
}
|
||||
spin_unlock(&ip->i_flags_lock);
|
||||
|
||||
if (flags & IOMAP_DIO_COW) {
|
||||
error = xfs_reflink_end_cow(ip, offset, size);
|
||||
if (error)
|
||||
return error;
|
||||
}
|
||||
|
||||
if (flags & IOMAP_DIO_UNWRITTEN)
|
||||
error = xfs_iomap_write_unwritten(ip, offset, size);
|
||||
else if (update_size)
|
||||
error = xfs_setfilesize(ip, offset, size);
|
||||
|
||||
return error;
|
||||
}
|
||||
|
||||
/*
|
||||
* xfs_file_dio_aio_write - handle direct IO writes
|
||||
*
|
||||
|
@ -535,9 +515,7 @@ xfs_file_dio_aio_write(
|
|||
int unaligned_io = 0;
|
||||
int iolock;
|
||||
size_t count = iov_iter_count(from);
|
||||
loff_t end;
|
||||
struct iov_iter data;
|
||||
struct xfs_buftarg *target = XFS_IS_REALTIME_INODE(ip) ?
|
||||
struct xfs_buftarg *target = XFS_IS_REALTIME_INODE(ip) ?
|
||||
mp->m_rtdev_targp : mp->m_ddev_targp;
|
||||
|
||||
/* DIO must be aligned to device logical sector size */
|
||||
|
@ -559,29 +537,12 @@ xfs_file_dio_aio_write(
|
|||
iolock = XFS_IOLOCK_SHARED;
|
||||
}
|
||||
|
||||
xfs_rw_ilock(ip, iolock);
|
||||
xfs_ilock(ip, iolock);
|
||||
|
||||
ret = xfs_file_aio_write_checks(iocb, from, &iolock);
|
||||
if (ret)
|
||||
goto out;
|
||||
count = iov_iter_count(from);
|
||||
end = iocb->ki_pos + count - 1;
|
||||
|
||||
if (mapping->nrpages) {
|
||||
ret = filemap_write_and_wait_range(mapping, iocb->ki_pos, end);
|
||||
if (ret)
|
||||
goto out;
|
||||
|
||||
/*
|
||||
* Invalidate whole pages. This can return an error if we fail
|
||||
* to invalidate a page, but this should never happen on XFS.
|
||||
* Warn if it does fail.
|
||||
*/
|
||||
ret = invalidate_inode_pages2_range(mapping,
|
||||
iocb->ki_pos >> PAGE_SHIFT, end >> PAGE_SHIFT);
|
||||
WARN_ON_ONCE(ret);
|
||||
ret = 0;
|
||||
}
|
||||
|
||||
/*
|
||||
* If we are doing unaligned IO, wait for all other IO to drain,
|
||||
|
@ -591,7 +552,7 @@ xfs_file_dio_aio_write(
|
|||
if (unaligned_io)
|
||||
inode_dio_wait(inode);
|
||||
else if (iolock == XFS_IOLOCK_EXCL) {
|
||||
xfs_rw_ilock_demote(ip, XFS_IOLOCK_EXCL);
|
||||
xfs_ilock_demote(ip, XFS_IOLOCK_EXCL);
|
||||
iolock = XFS_IOLOCK_SHARED;
|
||||
}
|
||||
|
||||
|
@ -604,24 +565,9 @@ xfs_file_dio_aio_write(
|
|||
goto out;
|
||||
}
|
||||
|
||||
data = *from;
|
||||
ret = __blockdev_direct_IO(iocb, inode, target->bt_bdev, &data,
|
||||
xfs_get_blocks_direct, xfs_end_io_direct_write,
|
||||
NULL, DIO_ASYNC_EXTEND);
|
||||
|
||||
/* see generic_file_direct_write() for why this is necessary */
|
||||
if (mapping->nrpages) {
|
||||
invalidate_inode_pages2_range(mapping,
|
||||
iocb->ki_pos >> PAGE_SHIFT,
|
||||
end >> PAGE_SHIFT);
|
||||
}
|
||||
|
||||
if (ret > 0) {
|
||||
iocb->ki_pos += ret;
|
||||
iov_iter_advance(from, ret);
|
||||
}
|
||||
ret = iomap_dio_rw(iocb, from, &xfs_iomap_ops, xfs_dio_write_end_io);
|
||||
out:
|
||||
xfs_rw_iunlock(ip, iolock);
|
||||
xfs_iunlock(ip, iolock);
|
||||
|
||||
/*
|
||||
* No fallback to buffered IO on errors for XFS, direct IO will either
|
||||
|
@ -643,7 +589,7 @@ xfs_file_dax_write(
|
|||
size_t count;
|
||||
loff_t pos;
|
||||
|
||||
xfs_rw_ilock(ip, iolock);
|
||||
xfs_ilock(ip, iolock);
|
||||
ret = xfs_file_aio_write_checks(iocb, from, &iolock);
|
||||
if (ret)
|
||||
goto out;
|
||||
|
@ -652,15 +598,13 @@ xfs_file_dax_write(
|
|||
count = iov_iter_count(from);
|
||||
|
||||
trace_xfs_file_dax_write(ip, count, pos);
|
||||
|
||||
ret = dax_iomap_rw(iocb, from, &xfs_iomap_ops);
|
||||
if (ret > 0 && iocb->ki_pos > i_size_read(inode)) {
|
||||
i_size_write(inode, iocb->ki_pos);
|
||||
error = xfs_setfilesize(ip, pos, ret);
|
||||
}
|
||||
|
||||
out:
|
||||
xfs_rw_iunlock(ip, iolock);
|
||||
xfs_iunlock(ip, iolock);
|
||||
return error ? error : ret;
|
||||
}
|
||||
|
||||
|
@ -677,7 +621,7 @@ xfs_file_buffered_aio_write(
|
|||
int enospc = 0;
|
||||
int iolock = XFS_IOLOCK_EXCL;
|
||||
|
||||
xfs_rw_ilock(ip, iolock);
|
||||
xfs_ilock(ip, iolock);
|
||||
|
||||
ret = xfs_file_aio_write_checks(iocb, from, &iolock);
|
||||
if (ret)
|
||||
|
@ -721,7 +665,7 @@ write_retry:
|
|||
|
||||
current->backing_dev_info = NULL;
|
||||
out:
|
||||
xfs_rw_iunlock(ip, iolock);
|
||||
xfs_iunlock(ip, iolock);
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
@ -797,7 +741,7 @@ xfs_file_fallocate(
|
|||
return -EOPNOTSUPP;
|
||||
|
||||
xfs_ilock(ip, iolock);
|
||||
error = xfs_break_layouts(inode, &iolock, false);
|
||||
error = xfs_break_layouts(inode, &iolock);
|
||||
if (error)
|
||||
goto out_unlock;
|
||||
|
||||
|
@ -1501,15 +1445,9 @@ xfs_filemap_fault(
|
|||
return xfs_filemap_page_mkwrite(vma, vmf);
|
||||
|
||||
xfs_ilock(XFS_I(inode), XFS_MMAPLOCK_SHARED);
|
||||
if (IS_DAX(inode)) {
|
||||
/*
|
||||
* we do not want to trigger unwritten extent conversion on read
|
||||
* faults - that is unnecessary overhead and would also require
|
||||
* changes to xfs_get_blocks_direct() to map unwritten extent
|
||||
* ioend for conversion on read-only mappings.
|
||||
*/
|
||||
if (IS_DAX(inode))
|
||||
ret = dax_iomap_fault(vma, vmf, &xfs_iomap_ops);
|
||||
} else
|
||||
else
|
||||
ret = filemap_fault(vma, vmf);
|
||||
xfs_iunlock(XFS_I(inode), XFS_MMAPLOCK_SHARED);
|
||||
|
||||
|
|
|
@ -70,8 +70,6 @@ xfs_inode_alloc(
|
|||
ASSERT(!xfs_isiflocked(ip));
|
||||
ASSERT(ip->i_ino == 0);
|
||||
|
||||
mrlock_init(&ip->i_iolock, MRLOCK_BARRIER, "xfsio", ip->i_ino);
|
||||
|
||||
/* initialise the xfs inode */
|
||||
ip->i_ino = ino;
|
||||
ip->i_mount = mp;
|
||||
|
@ -394,8 +392,8 @@ xfs_iget_cache_hit(
|
|||
xfs_inode_clear_reclaim_tag(pag, ip->i_ino);
|
||||
inode->i_state = I_NEW;
|
||||
|
||||
ASSERT(!rwsem_is_locked(&ip->i_iolock.mr_lock));
|
||||
mrlock_init(&ip->i_iolock, MRLOCK_BARRIER, "xfsio", ip->i_ino);
|
||||
ASSERT(!rwsem_is_locked(&inode->i_rwsem));
|
||||
init_rwsem(&inode->i_rwsem);
|
||||
|
||||
spin_unlock(&ip->i_flags_lock);
|
||||
spin_unlock(&pag->pag_ici_lock);
|
||||
|
|
|
@ -142,31 +142,31 @@ xfs_ilock_attr_map_shared(
|
|||
}
|
||||
|
||||
/*
|
||||
* The xfs inode contains 3 multi-reader locks: the i_iolock the i_mmap_lock and
|
||||
* the i_lock. This routine allows various combinations of the locks to be
|
||||
* obtained.
|
||||
* In addition to i_rwsem in the VFS inode, the xfs inode contains 2
|
||||
* multi-reader locks: i_mmap_lock and the i_lock. This routine allows
|
||||
* various combinations of the locks to be obtained.
|
||||
*
|
||||
* The 3 locks should always be ordered so that the IO lock is obtained first,
|
||||
* the mmap lock second and the ilock last in order to prevent deadlock.
|
||||
*
|
||||
* Basic locking order:
|
||||
*
|
||||
* i_iolock -> i_mmap_lock -> page_lock -> i_ilock
|
||||
* i_rwsem -> i_mmap_lock -> page_lock -> i_ilock
|
||||
*
|
||||
* mmap_sem locking order:
|
||||
*
|
||||
* i_iolock -> page lock -> mmap_sem
|
||||
* i_rwsem -> page lock -> mmap_sem
|
||||
* mmap_sem -> i_mmap_lock -> page_lock
|
||||
*
|
||||
* The difference in mmap_sem locking order mean that we cannot hold the
|
||||
* i_mmap_lock over syscall based read(2)/write(2) based IO. These IO paths can
|
||||
* fault in pages during copy in/out (for buffered IO) or require the mmap_sem
|
||||
* in get_user_pages() to map the user pages into the kernel address space for
|
||||
* direct IO. Similarly the i_iolock cannot be taken inside a page fault because
|
||||
* direct IO. Similarly the i_rwsem cannot be taken inside a page fault because
|
||||
* page faults already hold the mmap_sem.
|
||||
*
|
||||
* Hence to serialise fully against both syscall and mmap based IO, we need to
|
||||
* take both the i_iolock and the i_mmap_lock. These locks should *only* be both
|
||||
* take both the i_rwsem and the i_mmap_lock. These locks should *only* be both
|
||||
* taken in places where we need to invalidate the page cache in a race
|
||||
* free manner (e.g. truncate, hole punch and other extent manipulation
|
||||
* functions).
|
||||
|
@ -191,10 +191,13 @@ xfs_ilock(
|
|||
(XFS_ILOCK_SHARED | XFS_ILOCK_EXCL));
|
||||
ASSERT((lock_flags & ~(XFS_LOCK_MASK | XFS_LOCK_SUBCLASS_MASK)) == 0);
|
||||
|
||||
if (lock_flags & XFS_IOLOCK_EXCL)
|
||||
mrupdate_nested(&ip->i_iolock, XFS_IOLOCK_DEP(lock_flags));
|
||||
else if (lock_flags & XFS_IOLOCK_SHARED)
|
||||
mraccess_nested(&ip->i_iolock, XFS_IOLOCK_DEP(lock_flags));
|
||||
if (lock_flags & XFS_IOLOCK_EXCL) {
|
||||
down_write_nested(&VFS_I(ip)->i_rwsem,
|
||||
XFS_IOLOCK_DEP(lock_flags));
|
||||
} else if (lock_flags & XFS_IOLOCK_SHARED) {
|
||||
down_read_nested(&VFS_I(ip)->i_rwsem,
|
||||
XFS_IOLOCK_DEP(lock_flags));
|
||||
}
|
||||
|
||||
if (lock_flags & XFS_MMAPLOCK_EXCL)
|
||||
mrupdate_nested(&ip->i_mmaplock, XFS_MMAPLOCK_DEP(lock_flags));
|
||||
|
@ -240,10 +243,10 @@ xfs_ilock_nowait(
|
|||
ASSERT((lock_flags & ~(XFS_LOCK_MASK | XFS_LOCK_SUBCLASS_MASK)) == 0);
|
||||
|
||||
if (lock_flags & XFS_IOLOCK_EXCL) {
|
||||
if (!mrtryupdate(&ip->i_iolock))
|
||||
if (!down_write_trylock(&VFS_I(ip)->i_rwsem))
|
||||
goto out;
|
||||
} else if (lock_flags & XFS_IOLOCK_SHARED) {
|
||||
if (!mrtryaccess(&ip->i_iolock))
|
||||
if (!down_read_trylock(&VFS_I(ip)->i_rwsem))
|
||||
goto out;
|
||||
}
|
||||
|
||||
|
@ -271,9 +274,9 @@ out_undo_mmaplock:
|
|||
mrunlock_shared(&ip->i_mmaplock);
|
||||
out_undo_iolock:
|
||||
if (lock_flags & XFS_IOLOCK_EXCL)
|
||||
mrunlock_excl(&ip->i_iolock);
|
||||
up_write(&VFS_I(ip)->i_rwsem);
|
||||
else if (lock_flags & XFS_IOLOCK_SHARED)
|
||||
mrunlock_shared(&ip->i_iolock);
|
||||
up_read(&VFS_I(ip)->i_rwsem);
|
||||
out:
|
||||
return 0;
|
||||
}
|
||||
|
@ -310,9 +313,9 @@ xfs_iunlock(
|
|||
ASSERT(lock_flags != 0);
|
||||
|
||||
if (lock_flags & XFS_IOLOCK_EXCL)
|
||||
mrunlock_excl(&ip->i_iolock);
|
||||
up_write(&VFS_I(ip)->i_rwsem);
|
||||
else if (lock_flags & XFS_IOLOCK_SHARED)
|
||||
mrunlock_shared(&ip->i_iolock);
|
||||
up_read(&VFS_I(ip)->i_rwsem);
|
||||
|
||||
if (lock_flags & XFS_MMAPLOCK_EXCL)
|
||||
mrunlock_excl(&ip->i_mmaplock);
|
||||
|
@ -345,7 +348,7 @@ xfs_ilock_demote(
|
|||
if (lock_flags & XFS_MMAPLOCK_EXCL)
|
||||
mrdemote(&ip->i_mmaplock);
|
||||
if (lock_flags & XFS_IOLOCK_EXCL)
|
||||
mrdemote(&ip->i_iolock);
|
||||
downgrade_write(&VFS_I(ip)->i_rwsem);
|
||||
|
||||
trace_xfs_ilock_demote(ip, lock_flags, _RET_IP_);
|
||||
}
|
||||
|
@ -370,8 +373,9 @@ xfs_isilocked(
|
|||
|
||||
if (lock_flags & (XFS_IOLOCK_EXCL|XFS_IOLOCK_SHARED)) {
|
||||
if (!(lock_flags & XFS_IOLOCK_SHARED))
|
||||
return !!ip->i_iolock.mr_writer;
|
||||
return rwsem_is_locked(&ip->i_iolock.mr_lock);
|
||||
return !debug_locks ||
|
||||
lockdep_is_held_type(&VFS_I(ip)->i_rwsem, 0);
|
||||
return rwsem_is_locked(&VFS_I(ip)->i_rwsem);
|
||||
}
|
||||
|
||||
ASSERT(0);
|
||||
|
@ -421,11 +425,7 @@ xfs_lock_inumorder(int lock_mode, int subclass)
|
|||
|
||||
if (lock_mode & (XFS_IOLOCK_SHARED|XFS_IOLOCK_EXCL)) {
|
||||
ASSERT(subclass <= XFS_IOLOCK_MAX_SUBCLASS);
|
||||
ASSERT(xfs_lockdep_subclass_ok(subclass +
|
||||
XFS_IOLOCK_PARENT_VAL));
|
||||
class += subclass << XFS_IOLOCK_SHIFT;
|
||||
if (lock_mode & XFS_IOLOCK_PARENT)
|
||||
class += XFS_IOLOCK_PARENT_VAL << XFS_IOLOCK_SHIFT;
|
||||
}
|
||||
|
||||
if (lock_mode & (XFS_MMAPLOCK_SHARED|XFS_MMAPLOCK_EXCL)) {
|
||||
|
@ -477,8 +477,6 @@ xfs_lock_inodes(
|
|||
XFS_ILOCK_EXCL));
|
||||
ASSERT(!(lock_mode & (XFS_IOLOCK_SHARED | XFS_MMAPLOCK_SHARED |
|
||||
XFS_ILOCK_SHARED)));
|
||||
ASSERT(!(lock_mode & XFS_IOLOCK_EXCL) ||
|
||||
inodes <= XFS_IOLOCK_MAX_SUBCLASS + 1);
|
||||
ASSERT(!(lock_mode & XFS_MMAPLOCK_EXCL) ||
|
||||
inodes <= XFS_MMAPLOCK_MAX_SUBCLASS + 1);
|
||||
ASSERT(!(lock_mode & XFS_ILOCK_EXCL) ||
|
||||
|
@ -581,10 +579,8 @@ xfs_lock_two_inodes(
|
|||
int attempts = 0;
|
||||
xfs_log_item_t *lp;
|
||||
|
||||
if (lock_mode & (XFS_IOLOCK_SHARED|XFS_IOLOCK_EXCL)) {
|
||||
ASSERT(!(lock_mode & (XFS_MMAPLOCK_SHARED|XFS_MMAPLOCK_EXCL)));
|
||||
ASSERT(!(lock_mode & (XFS_ILOCK_SHARED|XFS_ILOCK_EXCL)));
|
||||
} else if (lock_mode & (XFS_MMAPLOCK_SHARED|XFS_MMAPLOCK_EXCL))
|
||||
ASSERT(!(lock_mode & (XFS_IOLOCK_SHARED|XFS_IOLOCK_EXCL)));
|
||||
if (lock_mode & (XFS_MMAPLOCK_SHARED|XFS_MMAPLOCK_EXCL))
|
||||
ASSERT(!(lock_mode & (XFS_ILOCK_SHARED|XFS_ILOCK_EXCL)));
|
||||
|
||||
ASSERT(ip0->i_ino != ip1->i_ino);
|
||||
|
@ -715,7 +711,6 @@ xfs_lookup(
|
|||
if (XFS_FORCED_SHUTDOWN(dp->i_mount))
|
||||
return -EIO;
|
||||
|
||||
xfs_ilock(dp, XFS_IOLOCK_SHARED);
|
||||
error = xfs_dir_lookup(NULL, dp, name, &inum, ci_name);
|
||||
if (error)
|
||||
goto out_unlock;
|
||||
|
@ -724,14 +719,12 @@ xfs_lookup(
|
|||
if (error)
|
||||
goto out_free_name;
|
||||
|
||||
xfs_iunlock(dp, XFS_IOLOCK_SHARED);
|
||||
return 0;
|
||||
|
||||
out_free_name:
|
||||
if (ci_name)
|
||||
kmem_free(ci_name->name);
|
||||
out_unlock:
|
||||
xfs_iunlock(dp, XFS_IOLOCK_SHARED);
|
||||
*ipp = NULL;
|
||||
return error;
|
||||
}
|
||||
|
@ -1215,8 +1208,7 @@ xfs_create(
|
|||
if (error)
|
||||
goto out_release_inode;
|
||||
|
||||
xfs_ilock(dp, XFS_IOLOCK_EXCL | XFS_ILOCK_EXCL |
|
||||
XFS_IOLOCK_PARENT | XFS_ILOCK_PARENT);
|
||||
xfs_ilock(dp, XFS_ILOCK_EXCL | XFS_ILOCK_PARENT);
|
||||
unlock_dp_on_error = true;
|
||||
|
||||
xfs_defer_init(&dfops, &first_block);
|
||||
|
@ -1252,7 +1244,7 @@ xfs_create(
|
|||
* the transaction cancel unlocking dp so don't do it explicitly in the
|
||||
* error path.
|
||||
*/
|
||||
xfs_trans_ijoin(tp, dp, XFS_IOLOCK_EXCL | XFS_ILOCK_EXCL);
|
||||
xfs_trans_ijoin(tp, dp, XFS_ILOCK_EXCL);
|
||||
unlock_dp_on_error = false;
|
||||
|
||||
error = xfs_dir_createname(tp, dp, name, ip->i_ino,
|
||||
|
@ -1325,7 +1317,7 @@ xfs_create(
|
|||
xfs_qm_dqrele(pdqp);
|
||||
|
||||
if (unlock_dp_on_error)
|
||||
xfs_iunlock(dp, XFS_IOLOCK_EXCL | XFS_ILOCK_EXCL);
|
||||
xfs_iunlock(dp, XFS_ILOCK_EXCL);
|
||||
return error;
|
||||
}
|
||||
|
||||
|
@ -1466,11 +1458,10 @@ xfs_link(
|
|||
if (error)
|
||||
goto std_return;
|
||||
|
||||
xfs_ilock(tdp, XFS_IOLOCK_EXCL | XFS_IOLOCK_PARENT);
|
||||
xfs_lock_two_inodes(sip, tdp, XFS_ILOCK_EXCL);
|
||||
|
||||
xfs_trans_ijoin(tp, sip, XFS_ILOCK_EXCL);
|
||||
xfs_trans_ijoin(tp, tdp, XFS_IOLOCK_EXCL | XFS_ILOCK_EXCL);
|
||||
xfs_trans_ijoin(tp, tdp, XFS_ILOCK_EXCL);
|
||||
|
||||
/*
|
||||
* If we are using project inheritance, we only allow hard link
|
||||
|
@ -2579,10 +2570,9 @@ xfs_remove(
|
|||
goto std_return;
|
||||
}
|
||||
|
||||
xfs_ilock(dp, XFS_IOLOCK_EXCL | XFS_IOLOCK_PARENT);
|
||||
xfs_lock_two_inodes(dp, ip, XFS_ILOCK_EXCL);
|
||||
|
||||
xfs_trans_ijoin(tp, dp, XFS_IOLOCK_EXCL | XFS_ILOCK_EXCL);
|
||||
xfs_trans_ijoin(tp, dp, XFS_ILOCK_EXCL);
|
||||
xfs_trans_ijoin(tp, ip, XFS_ILOCK_EXCL);
|
||||
|
||||
/*
|
||||
|
@ -2963,12 +2953,6 @@ xfs_rename(
|
|||
* whether the target directory is the same as the source
|
||||
* directory, we can lock from 2 to 4 inodes.
|
||||
*/
|
||||
if (!new_parent)
|
||||
xfs_ilock(src_dp, XFS_IOLOCK_EXCL | XFS_IOLOCK_PARENT);
|
||||
else
|
||||
xfs_lock_two_inodes(src_dp, target_dp,
|
||||
XFS_IOLOCK_EXCL | XFS_IOLOCK_PARENT);
|
||||
|
||||
xfs_lock_inodes(inodes, num_inodes, XFS_ILOCK_EXCL);
|
||||
|
||||
/*
|
||||
|
@ -2976,9 +2960,9 @@ xfs_rename(
|
|||
* we can rely on either trans_commit or trans_cancel to unlock
|
||||
* them.
|
||||
*/
|
||||
xfs_trans_ijoin(tp, src_dp, XFS_IOLOCK_EXCL | XFS_ILOCK_EXCL);
|
||||
xfs_trans_ijoin(tp, src_dp, XFS_ILOCK_EXCL);
|
||||
if (new_parent)
|
||||
xfs_trans_ijoin(tp, target_dp, XFS_IOLOCK_EXCL | XFS_ILOCK_EXCL);
|
||||
xfs_trans_ijoin(tp, target_dp, XFS_ILOCK_EXCL);
|
||||
xfs_trans_ijoin(tp, src_ip, XFS_ILOCK_EXCL);
|
||||
if (target_ip)
|
||||
xfs_trans_ijoin(tp, target_ip, XFS_ILOCK_EXCL);
|
||||
|
|
|
@ -56,7 +56,6 @@ typedef struct xfs_inode {
|
|||
/* Transaction and locking information. */
|
||||
struct xfs_inode_log_item *i_itemp; /* logging information */
|
||||
mrlock_t i_lock; /* inode lock */
|
||||
mrlock_t i_iolock; /* inode IO lock */
|
||||
mrlock_t i_mmaplock; /* inode mmap IO lock */
|
||||
atomic_t i_pincount; /* inode pin count */
|
||||
spinlock_t i_flags_lock; /* inode i_flags lock */
|
||||
|
@ -333,7 +332,7 @@ static inline void xfs_ifunlock(struct xfs_inode *ip)
|
|||
* IOLOCK values
|
||||
*
|
||||
* 0-3 subclass value
|
||||
* 4-7 PARENT subclass values
|
||||
* 4-7 unused
|
||||
*
|
||||
* MMAPLOCK values
|
||||
*
|
||||
|
@ -348,10 +347,8 @@ static inline void xfs_ifunlock(struct xfs_inode *ip)
|
|||
*
|
||||
*/
|
||||
#define XFS_IOLOCK_SHIFT 16
|
||||
#define XFS_IOLOCK_PARENT_VAL 4
|
||||
#define XFS_IOLOCK_MAX_SUBCLASS (XFS_IOLOCK_PARENT_VAL - 1)
|
||||
#define XFS_IOLOCK_MAX_SUBCLASS 3
|
||||
#define XFS_IOLOCK_DEP_MASK 0x000f0000
|
||||
#define XFS_IOLOCK_PARENT (XFS_IOLOCK_PARENT_VAL << XFS_IOLOCK_SHIFT)
|
||||
|
||||
#define XFS_MMAPLOCK_SHIFT 20
|
||||
#define XFS_MMAPLOCK_NUMORDER 0
|
||||
|
|
|
@ -639,7 +639,7 @@ xfs_ioc_space(
|
|||
return error;
|
||||
|
||||
xfs_ilock(ip, iolock);
|
||||
error = xfs_break_layouts(inode, &iolock, false);
|
||||
error = xfs_break_layouts(inode, &iolock);
|
||||
if (error)
|
||||
goto out_unlock;
|
||||
|
||||
|
|
|
@ -950,6 +950,19 @@ static inline bool imap_needs_alloc(struct inode *inode,
|
|||
(IS_DAX(inode) && ISUNWRITTEN(imap));
|
||||
}
|
||||
|
||||
static inline bool need_excl_ilock(struct xfs_inode *ip, unsigned flags)
|
||||
{
|
||||
/*
|
||||
* COW writes will allocate delalloc space, so we need to make sure
|
||||
* to take the lock exclusively here.
|
||||
*/
|
||||
if (xfs_is_reflink_inode(ip) && (flags & (IOMAP_WRITE | IOMAP_ZERO)))
|
||||
return true;
|
||||
if ((flags & IOMAP_DIRECT) && (flags & IOMAP_WRITE))
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
|
||||
static int
|
||||
xfs_file_iomap_begin(
|
||||
struct inode *inode,
|
||||
|
@ -969,18 +982,14 @@ xfs_file_iomap_begin(
|
|||
if (XFS_FORCED_SHUTDOWN(mp))
|
||||
return -EIO;
|
||||
|
||||
if ((flags & IOMAP_WRITE) && !IS_DAX(inode) &&
|
||||
!xfs_get_extsz_hint(ip)) {
|
||||
if (((flags & (IOMAP_WRITE | IOMAP_DIRECT)) == IOMAP_WRITE) &&
|
||||
!IS_DAX(inode) && !xfs_get_extsz_hint(ip)) {
|
||||
/* Reserve delalloc blocks for regular writeback. */
|
||||
return xfs_file_iomap_begin_delay(inode, offset, length, flags,
|
||||
iomap);
|
||||
}
|
||||
|
||||
/*
|
||||
* COW writes will allocate delalloc space, so we need to make sure
|
||||
* to take the lock exclusively here.
|
||||
*/
|
||||
if ((flags & (IOMAP_WRITE | IOMAP_ZERO)) && xfs_is_reflink_inode(ip)) {
|
||||
if (need_excl_ilock(ip, flags)) {
|
||||
lockmode = XFS_ILOCK_EXCL;
|
||||
xfs_ilock(ip, XFS_ILOCK_EXCL);
|
||||
} else {
|
||||
|
@ -993,17 +1002,41 @@ xfs_file_iomap_begin(
|
|||
offset_fsb = XFS_B_TO_FSBT(mp, offset);
|
||||
end_fsb = XFS_B_TO_FSB(mp, offset + length);
|
||||
|
||||
if (xfs_is_reflink_inode(ip) &&
|
||||
(flags & IOMAP_WRITE) && (flags & IOMAP_DIRECT)) {
|
||||
shared = xfs_reflink_find_cow_mapping(ip, offset, &imap);
|
||||
if (shared) {
|
||||
xfs_iunlock(ip, lockmode);
|
||||
goto alloc_done;
|
||||
}
|
||||
ASSERT(!isnullstartblock(imap.br_startblock));
|
||||
}
|
||||
|
||||
error = xfs_bmapi_read(ip, offset_fsb, end_fsb - offset_fsb, &imap,
|
||||
&nimaps, 0);
|
||||
if (error)
|
||||
goto out_unlock;
|
||||
|
||||
if (flags & IOMAP_REPORT) {
|
||||
if ((flags & IOMAP_REPORT) ||
|
||||
(xfs_is_reflink_inode(ip) &&
|
||||
(flags & IOMAP_WRITE) && (flags & IOMAP_DIRECT))) {
|
||||
/* Trim the mapping to the nearest shared extent boundary. */
|
||||
error = xfs_reflink_trim_around_shared(ip, &imap, &shared,
|
||||
&trimmed);
|
||||
if (error)
|
||||
goto out_unlock;
|
||||
|
||||
/*
|
||||
* We're here because we're trying to do a directio write to a
|
||||
* region that isn't aligned to a filesystem block. If the
|
||||
* extent is shared, fall back to buffered mode to handle the
|
||||
* RMW.
|
||||
*/
|
||||
if (!(flags & IOMAP_REPORT) && shared) {
|
||||
trace_xfs_reflink_bounce_dio_write(ip, &imap);
|
||||
error = -EREMCHG;
|
||||
goto out_unlock;
|
||||
}
|
||||
}
|
||||
|
||||
if ((flags & (IOMAP_WRITE | IOMAP_ZERO)) && xfs_is_reflink_inode(ip)) {
|
||||
|
@ -1038,6 +1071,7 @@ xfs_file_iomap_begin(
|
|||
if (error)
|
||||
return error;
|
||||
|
||||
alloc_done:
|
||||
iomap->flags = IOMAP_F_NEW;
|
||||
trace_xfs_iomap_alloc(ip, offset, length, 0, &imap);
|
||||
} else {
|
||||
|
|
|
@ -983,15 +983,13 @@ xfs_vn_setattr(
|
|||
struct xfs_inode *ip = XFS_I(d_inode(dentry));
|
||||
uint iolock = XFS_IOLOCK_EXCL;
|
||||
|
||||
xfs_ilock(ip, iolock);
|
||||
error = xfs_break_layouts(d_inode(dentry), &iolock, true);
|
||||
if (!error) {
|
||||
xfs_ilock(ip, XFS_MMAPLOCK_EXCL);
|
||||
iolock |= XFS_MMAPLOCK_EXCL;
|
||||
error = xfs_break_layouts(d_inode(dentry), &iolock);
|
||||
if (error)
|
||||
return error;
|
||||
|
||||
error = xfs_vn_setattr_size(dentry, iattr);
|
||||
}
|
||||
xfs_iunlock(ip, iolock);
|
||||
xfs_ilock(ip, XFS_MMAPLOCK_EXCL);
|
||||
error = xfs_setattr_size(ip, iattr);
|
||||
xfs_iunlock(ip, XFS_MMAPLOCK_EXCL);
|
||||
} else {
|
||||
error = xfs_vn_setattr_nonsize(dentry, iattr);
|
||||
}
|
||||
|
|
|
@ -32,8 +32,7 @@
|
|||
int
|
||||
xfs_break_layouts(
|
||||
struct inode *inode,
|
||||
uint *iolock,
|
||||
bool with_imutex)
|
||||
uint *iolock)
|
||||
{
|
||||
struct xfs_inode *ip = XFS_I(inode);
|
||||
int error;
|
||||
|
@ -42,12 +41,8 @@ xfs_break_layouts(
|
|||
|
||||
while ((error = break_layout(inode, false) == -EWOULDBLOCK)) {
|
||||
xfs_iunlock(ip, *iolock);
|
||||
if (with_imutex && (*iolock & XFS_IOLOCK_EXCL))
|
||||
inode_unlock(inode);
|
||||
error = break_layout(inode, true);
|
||||
*iolock = XFS_IOLOCK_EXCL;
|
||||
if (with_imutex)
|
||||
inode_lock(inode);
|
||||
xfs_ilock(ip, *iolock);
|
||||
}
|
||||
|
||||
|
|
|
@ -8,10 +8,10 @@ int xfs_fs_map_blocks(struct inode *inode, loff_t offset, u64 length,
|
|||
int xfs_fs_commit_blocks(struct inode *inode, struct iomap *maps, int nr_maps,
|
||||
struct iattr *iattr);
|
||||
|
||||
int xfs_break_layouts(struct inode *inode, uint *iolock, bool with_imutex);
|
||||
int xfs_break_layouts(struct inode *inode, uint *iolock);
|
||||
#else
|
||||
static inline int
|
||||
xfs_break_layouts(struct inode *inode, uint *iolock, bool with_imutex)
|
||||
xfs_break_layouts(struct inode *inode, uint *iolock)
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
|
|
@ -1251,13 +1251,11 @@ xfs_reflink_remap_range(
|
|||
return -EIO;
|
||||
|
||||
/* Lock both files against IO */
|
||||
if (same_inode) {
|
||||
xfs_ilock(src, XFS_IOLOCK_EXCL);
|
||||
lock_two_nondirectories(inode_in, inode_out);
|
||||
if (same_inode)
|
||||
xfs_ilock(src, XFS_MMAPLOCK_EXCL);
|
||||
} else {
|
||||
xfs_lock_two_inodes(src, dest, XFS_IOLOCK_EXCL);
|
||||
else
|
||||
xfs_lock_two_inodes(src, dest, XFS_MMAPLOCK_EXCL);
|
||||
}
|
||||
|
||||
/* Don't touch certain kinds of inodes */
|
||||
ret = -EPERM;
|
||||
|
@ -1402,11 +1400,9 @@ xfs_reflink_remap_range(
|
|||
|
||||
out_unlock:
|
||||
xfs_iunlock(src, XFS_MMAPLOCK_EXCL);
|
||||
xfs_iunlock(src, XFS_IOLOCK_EXCL);
|
||||
if (src->i_ino != dest->i_ino) {
|
||||
if (!same_inode)
|
||||
xfs_iunlock(dest, XFS_MMAPLOCK_EXCL);
|
||||
xfs_iunlock(dest, XFS_IOLOCK_EXCL);
|
||||
}
|
||||
unlock_two_nondirectories(inode_in, inode_out);
|
||||
if (ret)
|
||||
trace_xfs_reflink_remap_range_error(dest, ret, _RET_IP_);
|
||||
return ret;
|
||||
|
|
|
@ -943,7 +943,7 @@ xfs_fs_destroy_inode(
|
|||
|
||||
trace_xfs_destroy_inode(ip);
|
||||
|
||||
ASSERT(!rwsem_is_locked(&ip->i_iolock.mr_lock));
|
||||
ASSERT(!rwsem_is_locked(&inode->i_rwsem));
|
||||
XFS_STATS_INC(ip->i_mount, vn_rele);
|
||||
XFS_STATS_INC(ip->i_mount, vn_remove);
|
||||
|
||||
|
|
|
@ -238,8 +238,7 @@ xfs_symlink(
|
|||
if (error)
|
||||
goto out_release_inode;
|
||||
|
||||
xfs_ilock(dp, XFS_IOLOCK_EXCL | XFS_ILOCK_EXCL |
|
||||
XFS_IOLOCK_PARENT | XFS_ILOCK_PARENT);
|
||||
xfs_ilock(dp, XFS_ILOCK_EXCL | XFS_ILOCK_PARENT);
|
||||
unlock_dp_on_error = true;
|
||||
|
||||
/*
|
||||
|
@ -287,7 +286,7 @@ xfs_symlink(
|
|||
* the transaction cancel unlocking dp so don't do it explicitly in the
|
||||
* error path.
|
||||
*/
|
||||
xfs_trans_ijoin(tp, dp, XFS_IOLOCK_EXCL | XFS_ILOCK_EXCL);
|
||||
xfs_trans_ijoin(tp, dp, XFS_ILOCK_EXCL);
|
||||
unlock_dp_on_error = false;
|
||||
|
||||
/*
|
||||
|
@ -412,7 +411,7 @@ out_release_inode:
|
|||
xfs_qm_dqrele(pdqp);
|
||||
|
||||
if (unlock_dp_on_error)
|
||||
xfs_iunlock(dp, XFS_IOLOCK_EXCL | XFS_ILOCK_EXCL);
|
||||
xfs_iunlock(dp, XFS_ILOCK_EXCL);
|
||||
return error;
|
||||
}
|
||||
|
||||
|
|
|
@ -430,6 +430,7 @@ void bio_chain(struct bio *, struct bio *);
|
|||
extern int bio_add_page(struct bio *, struct page *, unsigned int,unsigned int);
|
||||
extern int bio_add_pc_page(struct request_queue *, struct bio *, struct page *,
|
||||
unsigned int, unsigned int);
|
||||
int bio_iov_iter_get_pages(struct bio *bio, struct iov_iter *iter);
|
||||
struct rq_map_data;
|
||||
extern struct bio *bio_map_user_iov(struct request_queue *,
|
||||
const struct iov_iter *, gfp_t);
|
||||
|
|
|
@ -50,6 +50,7 @@ struct iomap {
|
|||
#define IOMAP_ZERO (1 << 1) /* zeroing operation, may skip holes */
|
||||
#define IOMAP_REPORT (1 << 2) /* report extent status, e.g. FIEMAP */
|
||||
#define IOMAP_FAULT (1 << 3) /* mapping for page fault */
|
||||
#define IOMAP_DIRECT (1 << 4) /* direct I/O */
|
||||
|
||||
struct iomap_ops {
|
||||
/*
|
||||
|
@ -83,4 +84,14 @@ int iomap_page_mkwrite(struct vm_area_struct *vma, struct vm_fault *vmf,
|
|||
int iomap_fiemap(struct inode *inode, struct fiemap_extent_info *fieinfo,
|
||||
loff_t start, loff_t len, struct iomap_ops *ops);
|
||||
|
||||
/*
|
||||
* Flags for direct I/O ->end_io:
|
||||
*/
|
||||
#define IOMAP_DIO_UNWRITTEN (1 << 0) /* covers unwritten extent(s) */
|
||||
#define IOMAP_DIO_COW (1 << 1) /* covers COW extent(s) */
|
||||
typedef int (iomap_dio_end_io_t)(struct kiocb *iocb, ssize_t ret,
|
||||
unsigned flags);
|
||||
ssize_t iomap_dio_rw(struct kiocb *iocb, struct iov_iter *iter,
|
||||
struct iomap_ops *ops, iomap_dio_end_io_t end_io);
|
||||
|
||||
#endif /* LINUX_IOMAP_H */
|
||||
|
|
|
@ -338,9 +338,18 @@ extern void lock_acquire(struct lockdep_map *lock, unsigned int subclass,
|
|||
extern void lock_release(struct lockdep_map *lock, int nested,
|
||||
unsigned long ip);
|
||||
|
||||
#define lockdep_is_held(lock) lock_is_held(&(lock)->dep_map)
|
||||
/*
|
||||
* Same "read" as for lock_acquire(), except -1 means any.
|
||||
*/
|
||||
extern int lock_is_held_type(struct lockdep_map *lock, int read);
|
||||
|
||||
extern int lock_is_held(struct lockdep_map *lock);
|
||||
static inline int lock_is_held(struct lockdep_map *lock)
|
||||
{
|
||||
return lock_is_held_type(lock, -1);
|
||||
}
|
||||
|
||||
#define lockdep_is_held(lock) lock_is_held(&(lock)->dep_map)
|
||||
#define lockdep_is_held_type(lock, r) lock_is_held_type(&(lock)->dep_map, (r))
|
||||
|
||||
extern void lock_set_class(struct lockdep_map *lock, const char *name,
|
||||
struct lock_class_key *key, unsigned int subclass,
|
||||
|
@ -372,6 +381,14 @@ extern void lock_unpin_lock(struct lockdep_map *lock, struct pin_cookie);
|
|||
WARN_ON(debug_locks && !lockdep_is_held(l)); \
|
||||
} while (0)
|
||||
|
||||
#define lockdep_assert_held_exclusive(l) do { \
|
||||
WARN_ON(debug_locks && !lockdep_is_held_type(l, 0)); \
|
||||
} while (0)
|
||||
|
||||
#define lockdep_assert_held_read(l) do { \
|
||||
WARN_ON(debug_locks && !lockdep_is_held_type(l, 1)); \
|
||||
} while (0)
|
||||
|
||||
#define lockdep_assert_held_once(l) do { \
|
||||
WARN_ON_ONCE(debug_locks && !lockdep_is_held(l)); \
|
||||
} while (0)
|
||||
|
@ -428,7 +445,11 @@ struct lock_class_key { };
|
|||
|
||||
#define lockdep_depth(tsk) (0)
|
||||
|
||||
#define lockdep_is_held_type(l, r) (1)
|
||||
|
||||
#define lockdep_assert_held(l) do { (void)(l); } while (0)
|
||||
#define lockdep_assert_held_exclusive(l) do { (void)(l); } while (0)
|
||||
#define lockdep_assert_held_read(l) do { (void)(l); } while (0)
|
||||
#define lockdep_assert_held_once(l) do { (void)(l); } while (0)
|
||||
|
||||
#define lockdep_recursing(tsk) (0)
|
||||
|
|
|
@ -3188,7 +3188,7 @@ print_lock_nested_lock_not_held(struct task_struct *curr,
|
|||
return 0;
|
||||
}
|
||||
|
||||
static int __lock_is_held(struct lockdep_map *lock);
|
||||
static int __lock_is_held(struct lockdep_map *lock, int read);
|
||||
|
||||
/*
|
||||
* This gets called for every mutex_lock*()/spin_lock*() operation.
|
||||
|
@ -3329,7 +3329,7 @@ static int __lock_acquire(struct lockdep_map *lock, unsigned int subclass,
|
|||
}
|
||||
chain_key = iterate_chain_key(chain_key, class_idx);
|
||||
|
||||
if (nest_lock && !__lock_is_held(nest_lock))
|
||||
if (nest_lock && !__lock_is_held(nest_lock, -1))
|
||||
return print_lock_nested_lock_not_held(curr, hlock, ip);
|
||||
|
||||
if (!validate_chain(curr, lock, hlock, chain_head, chain_key))
|
||||
|
@ -3576,7 +3576,7 @@ found_it:
|
|||
return 1;
|
||||
}
|
||||
|
||||
static int __lock_is_held(struct lockdep_map *lock)
|
||||
static int __lock_is_held(struct lockdep_map *lock, int read)
|
||||
{
|
||||
struct task_struct *curr = current;
|
||||
int i;
|
||||
|
@ -3584,8 +3584,12 @@ static int __lock_is_held(struct lockdep_map *lock)
|
|||
for (i = 0; i < curr->lockdep_depth; i++) {
|
||||
struct held_lock *hlock = curr->held_locks + i;
|
||||
|
||||
if (match_held_lock(hlock, lock))
|
||||
return 1;
|
||||
if (match_held_lock(hlock, lock)) {
|
||||
if (read == -1 || hlock->read == read)
|
||||
return 1;
|
||||
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
return 0;
|
||||
|
@ -3769,7 +3773,7 @@ void lock_release(struct lockdep_map *lock, int nested,
|
|||
}
|
||||
EXPORT_SYMBOL_GPL(lock_release);
|
||||
|
||||
int lock_is_held(struct lockdep_map *lock)
|
||||
int lock_is_held_type(struct lockdep_map *lock, int read)
|
||||
{
|
||||
unsigned long flags;
|
||||
int ret = 0;
|
||||
|
@ -3781,13 +3785,13 @@ int lock_is_held(struct lockdep_map *lock)
|
|||
check_flags(flags);
|
||||
|
||||
current->lockdep_recursion = 1;
|
||||
ret = __lock_is_held(lock);
|
||||
ret = __lock_is_held(lock, read);
|
||||
current->lockdep_recursion = 0;
|
||||
raw_local_irq_restore(flags);
|
||||
|
||||
return ret;
|
||||
}
|
||||
EXPORT_SYMBOL_GPL(lock_is_held);
|
||||
EXPORT_SYMBOL_GPL(lock_is_held_type);
|
||||
|
||||
struct pin_cookie lock_pin_lock(struct lockdep_map *lock)
|
||||
{
|
||||
|
|
Loading…
Reference in New Issue