summaryrefslogtreecommitdiff
path: root/fs/bcachefs/fs.c
diff options
context:
space:
mode:
Diffstat (limited to 'fs/bcachefs/fs.c')
-rw-r--r--fs/bcachefs/fs.c1491
1 files changed, 1097 insertions, 394 deletions
diff --git a/fs/bcachefs/fs.c b/fs/bcachefs/fs.c
index b5ea9fa1259d..ddfe89d84966 100644
--- a/fs/bcachefs/fs.c
+++ b/fs/bcachefs/fs.c
@@ -11,7 +11,6 @@
#include "errcode.h"
#include "extents.h"
#include "fs.h"
-#include "fs-common.h"
#include "fs-io.h"
#include "fs-ioctl.h"
#include "fs-io-buffered.h"
@@ -22,20 +21,26 @@
#include "io_read.h"
#include "journal.h"
#include "keylist.h"
+#include "namei.h"
#include "quota.h"
+#include "rebalance.h"
#include "snapshot.h"
#include "super.h"
#include "xattr.h"
+#include "trace.h"
#include <linux/aio.h>
#include <linux/backing-dev.h>
#include <linux/exportfs.h>
#include <linux/fiemap.h>
+#include <linux/fileattr.h>
+#include <linux/fs_context.h>
#include <linux/module.h>
#include <linux/pagemap.h>
#include <linux/posix_acl.h>
#include <linux/random.h>
#include <linux/seq_file.h>
+#include <linux/siphash.h>
#include <linux/statfs.h>
#include <linux/string.h>
#include <linux/xattr.h>
@@ -47,6 +52,24 @@ static void bch2_vfs_inode_init(struct btree_trans *, subvol_inum,
struct bch_inode_unpacked *,
struct bch_subvolume *);
+/* Set VFS inode flags from bcachefs inode: */
+static inline void bch2_inode_flags_to_vfs(struct bch_fs *c, struct bch_inode_info *inode)
+{
+ static const __maybe_unused unsigned bch_flags_to_vfs[] = {
+ [__BCH_INODE_sync] = S_SYNC,
+ [__BCH_INODE_immutable] = S_IMMUTABLE,
+ [__BCH_INODE_append] = S_APPEND,
+ [__BCH_INODE_noatime] = S_NOATIME,
+ };
+
+ set_flags(bch_flags_to_vfs, inode->ei_inode.bi_flags, inode->v.i_flags);
+
+ if (bch2_inode_casefold(c, &inode->ei_inode))
+ inode->v.i_flags |= S_CASEFOLD;
+ else
+ inode->v.i_flags &= ~S_CASEFOLD;
+}
+
void bch2_inode_update_after_write(struct btree_trans *trans,
struct bch_inode_info *inode,
struct bch_inode_unpacked *bi,
@@ -56,15 +79,16 @@ void bch2_inode_update_after_write(struct btree_trans *trans,
BUG_ON(bi->bi_inum != inode->v.i_ino);
- bch2_assert_pos_locked(trans, BTREE_ID_inodes,
- POS(0, bi->bi_inum),
- c->opts.inodes_use_key_cache);
+ bch2_assert_pos_locked(trans, BTREE_ID_inodes, POS(0, bi->bi_inum));
set_nlink(&inode->v, bch2_inode_nlink_get(bi));
i_uid_write(&inode->v, bi->bi_uid);
i_gid_write(&inode->v, bi->bi_gid);
inode->v.i_mode = bi->bi_mode;
+ if (fields & ATTR_SIZE)
+ i_size_write(&inode->v, bi->bi_size);
+
if (fields & ATTR_ATIME)
inode_set_atime_to_ts(&inode->v, bch2_time_to_timespec(c, bi->bi_atime));
if (fields & ATTR_MTIME)
@@ -74,7 +98,7 @@ void bch2_inode_update_after_write(struct btree_trans *trans,
inode->ei_inode = *bi;
- bch2_inode_flags_to_vfs(inode);
+ bch2_inode_flags_to_vfs(c, inode);
}
int __must_check bch2_write_inode(struct bch_fs *c,
@@ -83,16 +107,31 @@ int __must_check bch2_write_inode(struct bch_fs *c,
void *p, unsigned fields)
{
struct btree_trans *trans = bch2_trans_get(c);
- struct btree_iter iter = { NULL };
+ struct btree_iter iter = {};
struct bch_inode_unpacked inode_u;
int ret;
retry:
bch2_trans_begin(trans);
- ret = bch2_inode_peek(trans, &iter, &inode_u, inode_inum(inode),
- BTREE_ITER_INTENT) ?:
- (set ? set(trans, inode, &inode_u, p) : 0) ?:
- bch2_inode_write(trans, &iter, &inode_u) ?:
+ ret = bch2_inode_peek(trans, &iter, &inode_u, inode_inum(inode), BTREE_ITER_intent);
+ if (ret)
+ goto err;
+
+ struct bch_extent_rebalance old_r = bch2_inode_rebalance_opts_get(c, &inode_u);
+
+ ret = (set ? set(trans, inode, &inode_u, p) : 0);
+ if (ret)
+ goto err;
+
+ struct bch_extent_rebalance new_r = bch2_inode_rebalance_opts_get(c, &inode_u);
+
+ if (memcmp(&old_r, &new_r, sizeof(new_r))) {
+ ret = bch2_set_rebalance_needs_scan_trans(trans, inode_u.bi_inum);
+ if (ret)
+ goto err;
+ }
+
+ ret = bch2_inode_write(trans, &iter, &inode_u) ?:
bch2_trans_commit(trans, NULL, NULL, BCH_TRANS_COMMIT_no_enospc);
/*
@@ -101,14 +140,14 @@ retry:
*/
if (!ret)
bch2_inode_update_after_write(trans, inode, &inode_u, fields);
-
+err:
bch2_trans_iter_exit(trans, &iter);
if (bch2_err_matches(ret, BCH_ERR_transaction_restart))
goto retry;
bch2_fs_fatal_err_on(bch2_err_matches(ret, ENOENT), c,
- "%s: inode %u:%llu not found when updating",
+ "%s: inode %llu:%llu not found when updating",
bch2_err_str(ret),
inode_inum(inode).subvol,
inode_inum(inode).inum);
@@ -152,56 +191,246 @@ int bch2_fs_quota_transfer(struct bch_fs *c,
return ret;
}
-static int bch2_iget5_test(struct inode *vinode, void *p)
+static u32 bch2_vfs_inode_hash_fn(const void *data, u32 len, u32 seed)
{
- struct bch_inode_info *inode = to_bch_ei(vinode);
- subvol_inum *inum = p;
+ const subvol_inum *inum = data;
+ siphash_key_t k = { .key[0] = seed };
- return inode->ei_subvol == inum->subvol &&
- inode->ei_inode.bi_inum == inum->inum;
+ return siphash_2u64(inum->subvol, inum->inum, &k);
}
-static int bch2_iget5_set(struct inode *vinode, void *p)
+static u32 bch2_vfs_inode_obj_hash_fn(const void *data, u32 len, u32 seed)
{
- struct bch_inode_info *inode = to_bch_ei(vinode);
- subvol_inum *inum = p;
+ const struct bch_inode_info *inode = data;
- inode->v.i_ino = inum->inum;
- inode->ei_subvol = inum->subvol;
- inode->ei_inode.bi_inum = inum->inum;
- return 0;
+ return bch2_vfs_inode_hash_fn(&inode->ei_inum, sizeof(inode->ei_inum), seed);
+}
+
+static int bch2_vfs_inode_cmp_fn(struct rhashtable_compare_arg *arg,
+ const void *obj)
+{
+ const struct bch_inode_info *inode = obj;
+ const subvol_inum *v = arg->key;
+
+ return !subvol_inum_eq(inode->ei_inum, *v);
+}
+
+static const struct rhashtable_params bch2_vfs_inodes_params = {
+ .head_offset = offsetof(struct bch_inode_info, hash),
+ .key_offset = offsetof(struct bch_inode_info, ei_inum),
+ .key_len = sizeof(subvol_inum),
+ .hashfn = bch2_vfs_inode_hash_fn,
+ .obj_hashfn = bch2_vfs_inode_obj_hash_fn,
+ .obj_cmpfn = bch2_vfs_inode_cmp_fn,
+ .automatic_shrinking = true,
+};
+
+static const struct rhashtable_params bch2_vfs_inodes_by_inum_params = {
+ .head_offset = offsetof(struct bch_inode_info, by_inum_hash),
+ .key_offset = offsetof(struct bch_inode_info, ei_inum.inum),
+ .key_len = sizeof(u64),
+ .automatic_shrinking = true,
+};
+
+int bch2_inode_or_descendents_is_open(struct btree_trans *trans, struct bpos p)
+{
+ struct bch_fs *c = trans->c;
+ struct rhltable *ht = &c->vfs_inodes_by_inum_table;
+ u64 inum = p.offset;
+ DARRAY(u32) subvols;
+ int ret = 0;
+
+ if (!test_bit(BCH_FS_started, &c->flags))
+ return false;
+
+ darray_init(&subvols);
+restart_from_top:
+
+ /*
+ * Tweaked version of __rhashtable_lookup(); we need to get a list of
+ * subvolumes in which the given inode number is open.
+ *
+ * For this to work, we don't include the subvolume ID in the key that
+ * we hash - all inodes with the same inode number regardless of
+ * subvolume will hash to the same slot.
+ *
+ * This will be less than ideal if the same file is ever open
+ * simultaneously in many different snapshots:
+ */
+ rcu_read_lock();
+ struct rhash_lock_head __rcu *const *bkt;
+ struct rhash_head *he;
+ unsigned int hash;
+ struct bucket_table *tbl = rht_dereference_rcu(ht->ht.tbl, &ht->ht);
+restart:
+ hash = rht_key_hashfn(&ht->ht, tbl, &inum, bch2_vfs_inodes_by_inum_params);
+ bkt = rht_bucket(tbl, hash);
+ do {
+ struct bch_inode_info *inode;
+
+ rht_for_each_entry_rcu_from(inode, he, rht_ptr_rcu(bkt), tbl, hash, hash) {
+ if (inode->ei_inum.inum == inum) {
+ ret = darray_push_gfp(&subvols, inode->ei_inum.subvol,
+ GFP_NOWAIT|__GFP_NOWARN);
+ if (ret) {
+ rcu_read_unlock();
+ ret = darray_make_room(&subvols, 1);
+ if (ret)
+ goto err;
+ subvols.nr = 0;
+ goto restart_from_top;
+ }
+ }
+ }
+ /* An object might have been moved to a different hash chain,
+ * while we walk along it - better check and retry.
+ */
+ } while (he != RHT_NULLS_MARKER(bkt));
+
+ /* Ensure we see any new tables. */
+ smp_rmb();
+
+ tbl = rht_dereference_rcu(tbl->future_tbl, &ht->ht);
+ if (unlikely(tbl))
+ goto restart;
+ rcu_read_unlock();
+
+ darray_for_each(subvols, i) {
+ u32 snap;
+ ret = bch2_subvolume_get_snapshot(trans, *i, &snap);
+ if (ret)
+ goto err;
+
+ ret = bch2_snapshot_is_ancestor(c, snap, p.snapshot);
+ if (ret)
+ break;
+ }
+err:
+ darray_exit(&subvols);
+ return ret;
}
-static unsigned bch2_inode_hash(subvol_inum inum)
+static struct bch_inode_info *__bch2_inode_hash_find(struct bch_fs *c, subvol_inum inum)
{
- return jhash_3words(inum.subvol, inum.inum >> 32, inum.inum, JHASH_INITVAL);
+ return rhashtable_lookup_fast(&c->vfs_inodes_table, &inum, bch2_vfs_inodes_params);
}
-static struct bch_inode_info *bch2_inode_insert(struct bch_fs *c, struct bch_inode_info *inode)
+static void __wait_on_freeing_inode(struct bch_fs *c,
+ struct bch_inode_info *inode,
+ subvol_inum inum)
{
- subvol_inum inum = inode_inum(inode);
- struct bch_inode_info *old = to_bch_ei(inode_insert5(&inode->v,
- bch2_inode_hash(inum),
- bch2_iget5_test,
- bch2_iget5_set,
- &inum));
- BUG_ON(!old);
+ wait_queue_head_t *wq;
+ struct wait_bit_queue_entry wait;
+
+ wq = inode_bit_waitqueue(&wait, &inode->v, __I_NEW);
+ prepare_to_wait(wq, &wait.wq_entry, TASK_UNINTERRUPTIBLE);
+ spin_unlock(&inode->v.i_lock);
+
+ if (__bch2_inode_hash_find(c, inum) == inode)
+ schedule_timeout(HZ * 10);
+ finish_wait(wq, &wait.wq_entry);
+}
- if (unlikely(old != inode)) {
+static struct bch_inode_info *bch2_inode_hash_find(struct bch_fs *c, struct btree_trans *trans,
+ subvol_inum inum)
+{
+ struct bch_inode_info *inode;
+repeat:
+ inode = __bch2_inode_hash_find(c, inum);
+ if (inode) {
+ spin_lock(&inode->v.i_lock);
+ if (!test_bit(EI_INODE_HASHED, &inode->ei_flags)) {
+ spin_unlock(&inode->v.i_lock);
+ return NULL;
+ }
+ if ((inode->v.i_state & (I_FREEING|I_WILL_FREE))) {
+ if (!trans) {
+ __wait_on_freeing_inode(c, inode, inum);
+ } else {
+ int ret = drop_locks_do(trans,
+ (__wait_on_freeing_inode(c, inode, inum), 0));
+ if (ret)
+ return ERR_PTR(ret);
+ }
+ goto repeat;
+ }
+ __iget(&inode->v);
+ spin_unlock(&inode->v.i_lock);
+ }
+
+ return inode;
+}
+
+static void bch2_inode_hash_remove(struct bch_fs *c, struct bch_inode_info *inode)
+{
+ spin_lock(&inode->v.i_lock);
+ bool remove = test_and_clear_bit(EI_INODE_HASHED, &inode->ei_flags);
+ spin_unlock(&inode->v.i_lock);
+
+ if (remove) {
+ int ret = rhltable_remove(&c->vfs_inodes_by_inum_table,
+ &inode->by_inum_hash, bch2_vfs_inodes_by_inum_params);
+ BUG_ON(ret);
+
+ ret = rhashtable_remove_fast(&c->vfs_inodes_table,
+ &inode->hash, bch2_vfs_inodes_params);
+ BUG_ON(ret);
+ inode->v.i_hash.pprev = NULL;
+ /*
+ * This pairs with the bch2_inode_hash_find() ->
+ * __wait_on_freeing_inode() path
+ */
+ inode_wake_up_bit(&inode->v, __I_NEW);
+ }
+}
+
+static struct bch_inode_info *bch2_inode_hash_insert(struct bch_fs *c,
+ struct btree_trans *trans,
+ struct bch_inode_info *inode)
+{
+ struct bch_inode_info *old = inode;
+
+ set_bit(EI_INODE_HASHED, &inode->ei_flags);
+retry:
+ if (unlikely(rhashtable_lookup_insert_key(&c->vfs_inodes_table,
+ &inode->ei_inum,
+ &inode->hash,
+ bch2_vfs_inodes_params))) {
+ old = bch2_inode_hash_find(c, trans, inode->ei_inum);
+ if (!old)
+ goto retry;
+
+ clear_bit(EI_INODE_HASHED, &inode->ei_flags);
+
+ /*
+ * bcachefs doesn't use I_NEW; we have no use for it since we
+ * only insert fully created inodes in the inode hash table. But
+ * discard_new_inode() expects it to be set...
+ */
+ inode->v.i_state |= I_NEW;
+ /*
+ * We don't want bch2_evict_inode() to delete the inode on disk,
+ * we just raced and had another inode in cache. Normally new
+ * inodes don't have nlink == 0 - except tmpfiles do...
+ */
+ set_nlink(&inode->v, 1);
discard_new_inode(&inode->v);
- inode = old;
+ return old;
} else {
+ int ret = rhltable_insert(&c->vfs_inodes_by_inum_table,
+ &inode->by_inum_hash,
+ bch2_vfs_inodes_by_inum_params);
+ BUG_ON(ret);
+
+ inode_fake_hash(&inode->v);
+
+ inode_sb_list_add(&inode->v);
+
mutex_lock(&c->vfs_inodes_lock);
list_add(&inode->ei_vfs_inode_list, &c->vfs_inodes_list);
mutex_unlock(&c->vfs_inodes_lock);
- /*
- * we really don't want insert_inode_locked2() to be setting
- * I_NEW...
- */
- unlock_new_inode(&inode->v);
+ return inode;
}
-
- return inode;
}
#define memalloc_flags_do(_flags, _do) \
@@ -212,21 +441,47 @@ static struct bch_inode_info *bch2_inode_insert(struct bch_fs *c, struct bch_ino
_ret; \
})
+static struct inode *bch2_alloc_inode(struct super_block *sb)
+{
+ BUG();
+}
+
+static struct bch_inode_info *__bch2_new_inode(struct bch_fs *c, gfp_t gfp)
+{
+ struct bch_inode_info *inode = alloc_inode_sb(c->vfs_sb,
+ bch2_inode_cache, gfp);
+ if (!inode)
+ return NULL;
+
+ inode_init_once(&inode->v);
+ mutex_init(&inode->ei_update_lock);
+ two_state_lock_init(&inode->ei_pagecache_lock);
+ INIT_LIST_HEAD(&inode->ei_vfs_inode_list);
+ inode->ei_flags = 0;
+ mutex_init(&inode->ei_quota_lock);
+ memset(&inode->ei_devs_need_flush, 0, sizeof(inode->ei_devs_need_flush));
+
+ if (unlikely(inode_init_always_gfp(c->vfs_sb, &inode->v, gfp))) {
+ kmem_cache_free(bch2_inode_cache, inode);
+ return NULL;
+ }
+
+ return inode;
+}
+
/*
* Allocate a new inode, dropping/retaking btree locks if necessary:
*/
static struct bch_inode_info *bch2_new_inode(struct btree_trans *trans)
{
- struct bch_fs *c = trans->c;
-
- struct bch_inode_info *inode =
- memalloc_flags_do(PF_MEMALLOC_NORECLAIM|PF_MEMALLOC_NOWARN,
- to_bch_ei(new_inode(c->vfs_sb)));
+ struct bch_inode_info *inode = __bch2_new_inode(trans->c, GFP_NOWAIT);
if (unlikely(!inode)) {
- int ret = drop_locks_do(trans, (inode = to_bch_ei(new_inode(c->vfs_sb))) ? 0 : -ENOMEM);
- if (ret && inode)
- discard_new_inode(&inode->v);
+ int ret = drop_locks_do(trans, (inode = __bch2_new_inode(trans->c, GFP_NOFS)) ? 0 : -ENOMEM);
+ if (ret && inode) {
+ __destroy_inode(&inode->v);
+ kmem_cache_free(bch2_inode_cache, inode);
+ }
if (ret)
return ERR_PTR(ret);
}
@@ -234,13 +489,24 @@ static struct bch_inode_info *bch2_new_inode(struct btree_trans *trans)
return inode;
}
+static struct bch_inode_info *bch2_inode_hash_init_insert(struct btree_trans *trans,
+ subvol_inum inum,
+ struct bch_inode_unpacked *bi,
+ struct bch_subvolume *subvol)
+{
+ struct bch_inode_info *inode = bch2_new_inode(trans);
+ if (IS_ERR(inode))
+ return inode;
+
+ bch2_vfs_inode_init(trans, inum, inode, bi, subvol);
+
+ return bch2_inode_hash_insert(trans->c, trans, inode);
+
+}
+
struct inode *bch2_vfs_inode_get(struct bch_fs *c, subvol_inum inum)
{
- struct bch_inode_info *inode =
- to_bch_ei(ilookup5_nowait(c->vfs_sb,
- bch2_inode_hash(inum),
- bch2_iget5_test,
- &inum));
+ struct bch_inode_info *inode = bch2_inode_hash_find(c, NULL, inum);
if (inode)
return &inode->v;
@@ -249,13 +515,9 @@ struct inode *bch2_vfs_inode_get(struct bch_fs *c, subvol_inum inum)
struct bch_inode_unpacked inode_u;
struct bch_subvolume subvol;
int ret = lockrestart_do(trans,
- bch2_subvolume_get(trans, inum.subvol, true, 0, &subvol) ?:
+ bch2_subvolume_get(trans, inum.subvol, true, &subvol) ?:
bch2_inode_find_by_inum_trans(trans, inum, &inode_u)) ?:
- PTR_ERR_OR_ZERO(inode = bch2_new_inode(trans));
- if (!ret) {
- bch2_vfs_inode_init(trans, inum, inode, &inode_u, &subvol);
- inode = bch2_inode_insert(c, inode);
- }
+ PTR_ERR_OR_ZERO(inode = bch2_inode_hash_init_insert(trans, inum, &inode_u, &subvol));
bch2_trans_put(trans);
return ret ? ERR_PTR(ret) : &inode->v;
@@ -276,6 +538,8 @@ __bch2_create(struct mnt_idmap *idmap,
subvol_inum inum;
struct bch_subvolume subvol;
u64 journal_seq = 0;
+ kuid_t kuid;
+ kgid_t kgid;
int ret;
/*
@@ -287,7 +551,7 @@ __bch2_create(struct mnt_idmap *idmap,
if (ret)
return ERR_PTR(ret);
#endif
- inode = to_bch_ei(new_inode(c->vfs_sb));
+ inode = __bch2_new_inode(c, GFP_NOFS);
if (unlikely(!inode)) {
inode = ERR_PTR(-ENOMEM);
goto err;
@@ -302,13 +566,15 @@ __bch2_create(struct mnt_idmap *idmap,
retry:
bch2_trans_begin(trans);
- ret = bch2_subvol_is_ro_trans(trans, dir->ei_subvol) ?:
+ kuid = mapped_fsuid(idmap, i_user_ns(&dir->v));
+ kgid = mapped_fsgid(idmap, i_user_ns(&dir->v));
+ ret = bch2_subvol_is_ro_trans(trans, dir->ei_inum.subvol) ?:
bch2_create_trans(trans,
inode_inum(dir), &dir_u, &inode_u,
!(flags & BCH_CREATE_TMPFILE)
? &dentry->d_name : NULL,
- from_kuid(i_user_ns(&dir->v), current_fsuid()),
- from_kgid(i_user_ns(&dir->v), current_fsgid()),
+ from_kuid(i_user_ns(&dir->v), kuid),
+ from_kgid(i_user_ns(&dir->v), kgid),
mode, rdev,
default_acl, acl, snapshot_src, flags) ?:
bch2_quota_acct(c, bch_qid(&inode_u), Q_INO, 1,
@@ -316,11 +582,10 @@ retry:
if (unlikely(ret))
goto err_before_quota;
- inum.subvol = inode_u.bi_subvol ?: dir->ei_subvol;
+ inum.subvol = inode_u.bi_subvol ?: dir->ei_inum.subvol;
inum.inum = inode_u.bi_inum;
- ret = bch2_subvolume_get(trans, inum.subvol, true,
- BTREE_ITER_WITH_UPDATES, &subvol) ?:
+ ret = bch2_subvolume_get(trans, inum.subvol, true, &subvol) ?:
bch2_trans_commit(trans, NULL, &journal_seq, 0);
if (unlikely(ret)) {
bch2_quota_acct(c, bch_qid(&inode_u), Q_INO, -1,
@@ -333,7 +598,7 @@ err_before_quota:
if (!(flags & BCH_CREATE_TMPFILE)) {
bch2_inode_update_after_write(trans, dir, &dir_u,
- ATTR_MTIME|ATTR_CTIME);
+ ATTR_MTIME|ATTR_CTIME|ATTR_SIZE);
mutex_unlock(&dir->ei_update_lock);
}
@@ -346,8 +611,16 @@ err_before_quota:
* we must insert the new inode into the inode cache before calling
* bch2_trans_exit() and dropping locks, else we could race with another
* thread pulling the inode in and modifying it:
+ *
+ * also, calling bch2_inode_hash_insert() without passing in the
+ * transaction object is sketchy - if we could ever end up in
+ * __wait_on_freeing_inode(), we'd risk deadlock.
+ *
+ * But that shouldn't be possible, since we still have the inode locked
+ * that we just created, and we _really_ can't take a transaction
+ * restart here.
*/
- inode = bch2_inode_insert(c, inode);
+ inode = bch2_inode_hash_insert(c, NULL, inode);
bch2_trans_put(trans);
err:
posix_acl_release(default_acl);
@@ -371,52 +644,60 @@ static struct bch_inode_info *bch2_lookup_trans(struct btree_trans *trans,
const struct qstr *name)
{
struct bch_fs *c = trans->c;
- struct btree_iter dirent_iter = {};
subvol_inum inum = {};
+ struct printbuf buf = PRINTBUF;
- int ret = bch2_hash_lookup(trans, &dirent_iter, bch2_dirent_hash_desc,
- dir_hash_info, dir, name, 0);
+ struct qstr lookup_name;
+ int ret = bch2_maybe_casefold(trans, dir_hash_info, name, &lookup_name);
if (ret)
return ERR_PTR(ret);
- struct bkey_s_c k = bch2_btree_iter_peek_slot(&dirent_iter);
+ struct btree_iter dirent_iter = {};
+ struct bkey_s_c k = bch2_hash_lookup(trans, &dirent_iter, bch2_dirent_hash_desc,
+ dir_hash_info, dir, &lookup_name, 0);
ret = bkey_err(k);
if (ret)
- goto err;
+ return ERR_PTR(ret);
- ret = bch2_dirent_read_target(trans, dir, bkey_s_c_to_dirent(k), &inum);
+ struct bkey_s_c_dirent d = bkey_s_c_to_dirent(k);
+
+ ret = bch2_dirent_read_target(trans, dir, d, &inum);
if (ret > 0)
ret = -ENOENT;
if (ret)
goto err;
- struct bch_inode_info *inode =
- to_bch_ei(ilookup5_nowait(c->vfs_sb,
- bch2_inode_hash(inum),
- bch2_iget5_test,
- &inum));
+ struct bch_inode_info *inode = bch2_inode_hash_find(c, trans, inum);
if (inode)
goto out;
+ /*
+ * Note: if check/repair needs it, we commit before
+ * bch2_inode_hash_init_insert(), as after that point we can't take a
+ * restart - not in the top level loop with a commit_do(), like we
+ * usually do:
+ */
+
struct bch_subvolume subvol;
struct bch_inode_unpacked inode_u;
- ret = bch2_subvolume_get(trans, inum.subvol, true, 0, &subvol) ?:
+ ret = bch2_subvolume_get(trans, inum.subvol, true, &subvol) ?:
bch2_inode_find_by_inum_nowarn_trans(trans, inum, &inode_u) ?:
- PTR_ERR_OR_ZERO(inode = bch2_new_inode(trans));
- if (bch2_err_matches(ret, ENOENT)) {
- struct printbuf buf = PRINTBUF;
+ bch2_check_dirent_target(trans, &dirent_iter, d, &inode_u, false) ?:
+ bch2_trans_commit(trans, NULL, NULL, BCH_TRANS_COMMIT_no_enospc) ?:
+ PTR_ERR_OR_ZERO(inode = bch2_inode_hash_init_insert(trans, inum, &inode_u, &subvol));
- bch2_bkey_val_to_text(&buf, c, k);
- bch_err(c, "%s points to missing inode", buf.buf);
- printbuf_exit(&buf);
- }
+ /*
+ * don't remove it: check_inodes might find another inode that points
+ * back to this dirent
+ */
+ bch2_fs_inconsistent_on(bch2_err_matches(ret, ENOENT),
+ c, "dirent to missing inode:\n%s",
+ (bch2_bkey_val_to_text(&buf, c, d.s_c), buf.buf));
if (ret)
goto err;
-
- bch2_vfs_inode_init(trans, inum, inode, &inode_u, &subvol);
- inode = bch2_inode_insert(c, inode);
out:
bch2_trans_iter_exit(trans, &dirent_iter);
+ printbuf_exit(&buf);
return inode;
err:
inode = ERR_PTR(ret);
@@ -431,12 +712,29 @@ static struct dentry *bch2_lookup(struct inode *vdir, struct dentry *dentry,
struct bch_hash_info hash = bch2_hash_info_init(c, &dir->ei_inode);
struct bch_inode_info *inode;
- bch2_trans_do(c, NULL, NULL, 0,
+ bch2_trans_do(c,
PTR_ERR_OR_ZERO(inode = bch2_lookup_trans(trans, inode_inum(dir),
&hash, &dentry->d_name)));
if (IS_ERR(inode))
inode = NULL;
+#ifdef CONFIG_UNICODE
+ if (!inode && IS_CASEFOLDED(vdir)) {
+ /*
+ * Do not cache a negative dentry in casefolded directories
+ * as it would need to be invalidated in the following situation:
+ * - Lookup file "blAH" in a casefolded directory
+ * - Creation of file "BLAH" in a casefolded directory
+ * - Lookup file "blAH" in a casefolded directory
+ * which would fail if we had a negative dentry.
+ *
+ * We should come back to this when VFS has a method to handle
+ * this edgecase.
+ */
+ return NULL;
+ }
+#endif
+
return d_splice_alias(&inode->v, dentry);
}
@@ -467,11 +765,11 @@ static int __bch2_link(struct bch_fs *c,
struct bch_inode_info *dir,
struct dentry *dentry)
{
- struct btree_trans *trans = bch2_trans_get(c);
struct bch_inode_unpacked dir_u, inode_u;
int ret;
mutex_lock(&inode->ei_update_lock);
+ struct btree_trans *trans = bch2_trans_get(c);
ret = commit_do(trans, NULL, NULL, 0,
bch2_link_trans(trans,
@@ -481,7 +779,7 @@ static int __bch2_link(struct bch_fs *c,
if (likely(!ret)) {
bch2_inode_update_after_write(trans, dir, &dir_u,
- ATTR_MTIME|ATTR_CTIME);
+ ATTR_MTIME|ATTR_CTIME|ATTR_SIZE);
bch2_inode_update_after_write(trans, inode, &inode_u, ATTR_CTIME);
}
@@ -500,8 +798,8 @@ static int bch2_link(struct dentry *old_dentry, struct inode *vdir,
lockdep_assert_held(&inode->v.i_rwsem);
- ret = bch2_subvol_is_ro(c, dir->ei_subvol) ?:
- bch2_subvol_is_ro(c, inode->ei_subvol) ?:
+ ret = bch2_subvol_is_ro(c, dir->ei_inum.subvol) ?:
+ bch2_subvol_is_ro(c, inode->ei_inum.subvol) ?:
__bch2_link(c, inode, dir, dentry);
if (unlikely(ret))
return bch2_err_class(ret);
@@ -518,11 +816,12 @@ int __bch2_unlink(struct inode *vdir, struct dentry *dentry,
struct bch_inode_info *dir = to_bch_ei(vdir);
struct bch_inode_info *inode = to_bch_ei(dentry->d_inode);
struct bch_inode_unpacked dir_u, inode_u;
- struct btree_trans *trans = bch2_trans_get(c);
int ret;
bch2_lock_inodes(INODE_UPDATE_LOCK, dir, inode);
+ struct btree_trans *trans = bch2_trans_get(c);
+
ret = commit_do(trans, NULL, NULL,
BCH_TRANS_COMMIT_no_enospc,
bch2_unlink_trans(trans,
@@ -533,7 +832,7 @@ int __bch2_unlink(struct inode *vdir, struct dentry *dentry,
goto err;
bch2_inode_update_after_write(trans, dir, &dir_u,
- ATTR_MTIME|ATTR_CTIME);
+ ATTR_MTIME|ATTR_CTIME|ATTR_SIZE);
bch2_inode_update_after_write(trans, inode, &inode_u,
ATTR_MTIME);
@@ -544,9 +843,12 @@ int __bch2_unlink(struct inode *vdir, struct dentry *dentry,
*/
set_nlink(&inode->v, 0);
}
+
+ if (IS_CASEFOLDED(vdir))
+ d_invalidate(dentry);
err:
- bch2_unlock_inodes(INODE_UPDATE_LOCK, dir, inode);
bch2_trans_put(trans);
+ bch2_unlock_inodes(INODE_UPDATE_LOCK, dir, inode);
return ret;
}
@@ -556,7 +858,7 @@ static int bch2_unlink(struct inode *vdir, struct dentry *dentry)
struct bch_inode_info *dir= to_bch_ei(vdir);
struct bch_fs *c = dir->v.i_sb->s_fs_info;
- int ret = bch2_subvol_is_ro(c, dir->ei_subvol) ?:
+ int ret = bch2_subvol_is_ro(c, dir->ei_inum.subvol) ?:
__bch2_unlink(vdir, dentry, false);
return bch2_err_class(ret);
}
@@ -596,10 +898,10 @@ err:
return bch2_err_class(ret);
}
-static int bch2_mkdir(struct mnt_idmap *idmap,
- struct inode *vdir, struct dentry *dentry, umode_t mode)
+static struct dentry *bch2_mkdir(struct mnt_idmap *idmap,
+ struct inode *vdir, struct dentry *dentry, umode_t mode)
{
- return bch2_mknod(idmap, vdir, dentry, mode|S_IFDIR, 0);
+ return ERR_PTR(bch2_mknod(idmap, vdir, dentry, mode|S_IFDIR, 0));
}
static int bch2_rename2(struct mnt_idmap *idmap,
@@ -613,15 +915,16 @@ static int bch2_rename2(struct mnt_idmap *idmap,
struct bch_inode_info *src_inode = to_bch_ei(src_dentry->d_inode);
struct bch_inode_info *dst_inode = to_bch_ei(dst_dentry->d_inode);
struct bch_inode_unpacked dst_dir_u, src_dir_u;
- struct bch_inode_unpacked src_inode_u, dst_inode_u;
+ struct bch_inode_unpacked src_inode_u, dst_inode_u, *whiteout_inode_u;
struct btree_trans *trans;
enum bch_rename_mode mode = flags & RENAME_EXCHANGE
? BCH_RENAME_EXCHANGE
: dst_dentry->d_inode
? BCH_RENAME_OVERWRITE : BCH_RENAME;
+ bool whiteout = !!(flags & RENAME_WHITEOUT);
int ret;
- if (flags & ~(RENAME_NOREPLACE|RENAME_EXCHANGE))
+ if (flags & ~(RENAME_NOREPLACE|RENAME_EXCHANGE|RENAME_WHITEOUT))
return -EINVAL;
if (mode == BCH_RENAME_OVERWRITE) {
@@ -631,18 +934,18 @@ static int bch2_rename2(struct mnt_idmap *idmap,
return ret;
}
- trans = bch2_trans_get(c);
-
bch2_lock_inodes(INODE_UPDATE_LOCK,
src_dir,
dst_dir,
src_inode,
dst_inode);
- ret = bch2_subvol_is_ro_trans(trans, src_dir->ei_subvol) ?:
- bch2_subvol_is_ro_trans(trans, dst_dir->ei_subvol);
+ trans = bch2_trans_get(c);
+
+ ret = bch2_subvol_is_ro_trans(trans, src_dir->ei_inum.subvol) ?:
+ bch2_subvol_is_ro_trans(trans, dst_dir->ei_inum.subvol);
if (ret)
- goto err;
+ goto err_tx_restart;
if (inode_attr_changing(dst_dir, src_inode, Inode_opt_project)) {
ret = bch2_fs_quota_transfer(c, src_inode,
@@ -662,29 +965,59 @@ static int bch2_rename2(struct mnt_idmap *idmap,
if (ret)
goto err;
}
+retry:
+ bch2_trans_begin(trans);
- ret = commit_do(trans, NULL, NULL, 0,
- bch2_rename_trans(trans,
- inode_inum(src_dir), &src_dir_u,
- inode_inum(dst_dir), &dst_dir_u,
- &src_inode_u,
- &dst_inode_u,
- &src_dentry->d_name,
- &dst_dentry->d_name,
- mode));
+ ret = bch2_rename_trans(trans,
+ inode_inum(src_dir), &src_dir_u,
+ inode_inum(dst_dir), &dst_dir_u,
+ &src_inode_u,
+ &dst_inode_u,
+ &src_dentry->d_name,
+ &dst_dentry->d_name,
+ mode);
if (unlikely(ret))
+ goto err_tx_restart;
+
+ if (whiteout) {
+ whiteout_inode_u = bch2_trans_kmalloc_nomemzero(trans, sizeof(*whiteout_inode_u));
+ ret = PTR_ERR_OR_ZERO(whiteout_inode_u);
+ if (unlikely(ret))
+ goto err_tx_restart;
+ bch2_inode_init_early(c, whiteout_inode_u);
+
+ ret = bch2_create_trans(trans,
+ inode_inum(src_dir), &src_dir_u,
+ whiteout_inode_u,
+ &src_dentry->d_name,
+ from_kuid(i_user_ns(&src_dir->v), current_fsuid()),
+ from_kgid(i_user_ns(&src_dir->v), current_fsgid()),
+ S_IFCHR|WHITEOUT_MODE, 0,
+ NULL, NULL, (subvol_inum) { 0 }, 0) ?:
+ bch2_quota_acct(c, bch_qid(whiteout_inode_u), Q_INO, 1,
+ KEY_TYPE_QUOTA_PREALLOC);
+ if (unlikely(ret))
+ goto err_tx_restart;
+ }
+
+ ret = bch2_trans_commit(trans, NULL, NULL, 0);
+ if (unlikely(ret)) {
+err_tx_restart:
+ if (bch2_err_matches(ret, BCH_ERR_transaction_restart))
+ goto retry;
goto err;
+ }
BUG_ON(src_inode->v.i_ino != src_inode_u.bi_inum);
BUG_ON(dst_inode &&
dst_inode->v.i_ino != dst_inode_u.bi_inum);
bch2_inode_update_after_write(trans, src_dir, &src_dir_u,
- ATTR_MTIME|ATTR_CTIME);
+ ATTR_MTIME|ATTR_CTIME|ATTR_SIZE);
if (src_dir != dst_dir)
bch2_inode_update_after_write(trans, dst_dir, &dst_dir_u,
- ATTR_MTIME|ATTR_CTIME);
+ ATTR_MTIME|ATTR_CTIME|ATTR_SIZE);
bch2_inode_update_after_write(trans, src_inode, &src_inode_u,
ATTR_CTIME);
@@ -721,11 +1054,17 @@ static void bch2_setattr_copy(struct mnt_idmap *idmap,
{
struct bch_fs *c = inode->v.i_sb->s_fs_info;
unsigned int ia_valid = attr->ia_valid;
+ kuid_t kuid;
+ kgid_t kgid;
- if (ia_valid & ATTR_UID)
- bi->bi_uid = from_kuid(i_user_ns(&inode->v), attr->ia_uid);
- if (ia_valid & ATTR_GID)
- bi->bi_gid = from_kgid(i_user_ns(&inode->v), attr->ia_gid);
+ if (ia_valid & ATTR_UID) {
+ kuid = from_vfsuid(idmap, i_user_ns(&inode->v), attr->ia_vfsuid);
+ bi->bi_uid = from_kuid(i_user_ns(&inode->v), kuid);
+ }
+ if (ia_valid & ATTR_GID) {
+ kgid = from_vfsgid(idmap, i_user_ns(&inode->v), attr->ia_vfsgid);
+ bi->bi_gid = from_kgid(i_user_ns(&inode->v), kgid);
+ }
if (ia_valid & ATTR_SIZE)
bi->bi_size = attr->ia_size;
@@ -740,11 +1079,11 @@ static void bch2_setattr_copy(struct mnt_idmap *idmap,
if (ia_valid & ATTR_MODE) {
umode_t mode = attr->ia_mode;
kgid_t gid = ia_valid & ATTR_GID
- ? attr->ia_gid
+ ? kgid
: inode->v.i_gid;
- if (!in_group_p(gid) &&
- !capable_wrt_inode_uidgid(idmap, &inode->v, CAP_FSETID))
+ if (!in_group_or_capable(idmap, &inode->v,
+ make_vfsgid(idmap, i_user_ns(&inode->v), gid)))
mode &= ~S_ISGID;
bi->bi_mode = mode;
}
@@ -757,20 +1096,26 @@ int bch2_setattr_nonsize(struct mnt_idmap *idmap,
struct bch_fs *c = inode->v.i_sb->s_fs_info;
struct bch_qid qid;
struct btree_trans *trans;
- struct btree_iter inode_iter = { NULL };
+ struct btree_iter inode_iter = {};
struct bch_inode_unpacked inode_u;
struct posix_acl *acl = NULL;
+ kuid_t kuid;
+ kgid_t kgid;
int ret;
mutex_lock(&inode->ei_update_lock);
qid = inode->ei_qid;
- if (attr->ia_valid & ATTR_UID)
- qid.q[QTYP_USR] = from_kuid(i_user_ns(&inode->v), attr->ia_uid);
+ if (attr->ia_valid & ATTR_UID) {
+ kuid = from_vfsuid(idmap, i_user_ns(&inode->v), attr->ia_vfsuid);
+ qid.q[QTYP_USR] = from_kuid(i_user_ns(&inode->v), kuid);
+ }
- if (attr->ia_valid & ATTR_GID)
- qid.q[QTYP_GRP] = from_kgid(i_user_ns(&inode->v), attr->ia_gid);
+ if (attr->ia_valid & ATTR_GID) {
+ kgid = from_vfsgid(idmap, i_user_ns(&inode->v), attr->ia_vfsgid);
+ qid.q[QTYP_GRP] = from_kgid(i_user_ns(&inode->v), kgid);
+ }
ret = bch2_fs_quota_transfer(c, inode, qid, ~0,
KEY_TYPE_QUOTA_PREALLOC);
@@ -784,7 +1129,7 @@ retry:
acl = NULL;
ret = bch2_inode_peek(trans, &inode_iter, &inode_u, inode_inum(inode),
- BTREE_ITER_INTENT);
+ BTREE_ITER_intent);
if (ret)
goto btree_err;
@@ -826,13 +1171,15 @@ static int bch2_getattr(struct mnt_idmap *idmap,
{
struct bch_inode_info *inode = to_bch_ei(d_inode(path->dentry));
struct bch_fs *c = inode->v.i_sb->s_fs_info;
+ vfsuid_t vfsuid = i_uid_into_vfsuid(idmap, &inode->v);
+ vfsgid_t vfsgid = i_gid_into_vfsgid(idmap, &inode->v);
stat->dev = inode->v.i_sb->s_dev;
stat->ino = inode->v.i_ino;
stat->mode = inode->v.i_mode;
stat->nlink = inode->v.i_nlink;
- stat->uid = inode->v.i_uid;
- stat->gid = inode->v.i_gid;
+ stat->uid = vfsuid_into_kuid(vfsuid);
+ stat->gid = vfsgid_into_kgid(vfsgid);
stat->rdev = inode->v.i_rdev;
stat->size = i_size_read(&inode->v);
stat->atime = inode_get_atime(&inode->v);
@@ -841,6 +1188,19 @@ static int bch2_getattr(struct mnt_idmap *idmap,
stat->blksize = block_bytes(c);
stat->blocks = inode->v.i_blocks;
+ stat->subvol = inode->ei_inum.subvol;
+ stat->result_mask |= STATX_SUBVOL;
+
+ if ((request_mask & STATX_DIOALIGN) && S_ISREG(inode->v.i_mode)) {
+ stat->result_mask |= STATX_DIOALIGN;
+ /*
+ * this is incorrect; we should be tracking this in superblock,
+ * and checking the alignment of open devices
+ */
+ stat->dio_mem_align = SECTOR_SIZE;
+ stat->dio_offset_align = block_bytes(c);
+ }
+
if (request_mask & STATX_BTIME) {
stat->result_mask |= STATX_BTIME;
stat->btime = bch2_time_to_timespec(c, inode->ei_inode.bi_otime);
@@ -870,7 +1230,7 @@ static int bch2_setattr(struct mnt_idmap *idmap,
lockdep_assert_held(&inode->v.i_rwsem);
- ret = bch2_subvol_is_ro(c, inode->ei_subvol) ?:
+ ret = bch2_subvol_is_ro(c, inode->ei_inum.subvol) ?:
setattr_prepare(idmap, dentry, iattr);
if (ret)
return ret;
@@ -896,10 +1256,20 @@ static int bch2_tmpfile(struct mnt_idmap *idmap,
return finish_open_simple(file, 0);
}
+struct bch_fiemap_extent {
+ struct bkey_buf kbuf;
+ unsigned flags;
+};
+
static int bch2_fill_extent(struct bch_fs *c,
struct fiemap_extent_info *info,
- struct bkey_s_c k, unsigned flags)
+ struct bch_fiemap_extent *fe)
{
+ struct bkey_s_c k = bkey_i_to_s_c(fe->kbuf.k);
+ unsigned flags = fe->flags;
+
+ BUG_ON(!k.k->size);
+
if (bkey_extent_is_direct_data(k.k)) {
struct bkey_ptrs_c ptrs = bch2_bkey_ptrs_c(k);
const union bch_extent_entry *entry;
@@ -952,22 +1322,174 @@ static int bch2_fill_extent(struct bch_fs *c,
}
}
+/*
+ * Scan a range of an inode for data in pagecache.
+ *
+ * Intended to be retryable, so don't modify the output params until success is
+ * imminent.
+ */
+static int
+bch2_fiemap_hole_pagecache(struct inode *vinode, u64 *start, u64 *end,
+ bool nonblock)
+{
+ loff_t dstart, dend;
+
+ dstart = bch2_seek_pagecache_data(vinode, *start, *end, 0, nonblock);
+ if (dstart < 0)
+ return dstart;
+
+ if (dstart == *end) {
+ *start = dstart;
+ return 0;
+ }
+
+ dend = bch2_seek_pagecache_hole(vinode, dstart, *end, 0, nonblock);
+ if (dend < 0)
+ return dend;
+
+ /* race */
+ BUG_ON(dstart == dend);
+
+ *start = dstart;
+ *end = dend;
+ return 0;
+}
+
+/*
+ * Scan a range of pagecache that corresponds to a file mapping hole in the
+ * extent btree. If data is found, fake up an extent key so it looks like a
+ * delalloc extent to the rest of the fiemap processing code.
+ */
+static int
+bch2_next_fiemap_pagecache_extent(struct btree_trans *trans, struct bch_inode_info *inode,
+ u64 start, u64 end, struct bch_fiemap_extent *cur)
+{
+ struct bch_fs *c = trans->c;
+ struct bkey_i_extent *delextent;
+ struct bch_extent_ptr ptr = {};
+ loff_t dstart = start << 9, dend = end << 9;
+ int ret;
+
+ /*
+ * We hold btree locks here so we cannot block on folio locks without
+ * dropping trans locks first. Run a nonblocking scan for the common
+ * case of no folios over holes and fall back on failure.
+ *
+ * Note that dropping locks like this is technically racy against
+ * writeback inserting to the extent tree, but a non-sync fiemap scan is
+ * fundamentally racy with writeback anyways. Therefore, just report the
+ * range as delalloc regardless of whether we have to cycle trans locks.
+ */
+ ret = bch2_fiemap_hole_pagecache(&inode->v, &dstart, &dend, true);
+ if (ret == -EAGAIN)
+ ret = drop_locks_do(trans,
+ bch2_fiemap_hole_pagecache(&inode->v, &dstart, &dend, false));
+ if (ret < 0)
+ return ret;
+
+ /*
+ * Create a fake extent key in the buffer. We have to add a dummy extent
+ * pointer for the fill code to add an extent entry. It's explicitly
+ * zeroed to reflect delayed allocation (i.e. phys offset 0).
+ */
+ bch2_bkey_buf_realloc(&cur->kbuf, c, sizeof(*delextent) / sizeof(u64));
+ delextent = bkey_extent_init(cur->kbuf.k);
+ delextent->k.p = POS(inode->ei_inum.inum, dend >> 9);
+ delextent->k.size = (dend - dstart) >> 9;
+ bch2_bkey_append_ptr(&delextent->k_i, ptr);
+
+ cur->flags = FIEMAP_EXTENT_DELALLOC;
+
+ return 0;
+}
+
+static int bch2_next_fiemap_extent(struct btree_trans *trans,
+ struct bch_inode_info *inode,
+ u64 start, u64 end,
+ struct bch_fiemap_extent *cur)
+{
+ u32 snapshot;
+ int ret = bch2_subvolume_get_snapshot(trans, inode->ei_inum.subvol, &snapshot);
+ if (ret)
+ return ret;
+
+ struct btree_iter iter;
+ bch2_trans_iter_init(trans, &iter, BTREE_ID_extents,
+ SPOS(inode->ei_inum.inum, start, snapshot), 0);
+
+ struct bkey_s_c k =
+ bch2_btree_iter_peek_max(trans, &iter, POS(inode->ei_inum.inum, end));
+ ret = bkey_err(k);
+ if (ret)
+ goto err;
+
+ u64 pagecache_end = k.k ? max(start, bkey_start_offset(k.k)) : end;
+
+ ret = bch2_next_fiemap_pagecache_extent(trans, inode, start, pagecache_end, cur);
+ if (ret)
+ goto err;
+
+ struct bpos pagecache_start = bkey_start_pos(&cur->kbuf.k->k);
+
+ /*
+ * Does the pagecache or the btree take precedence?
+ *
+ * It _should_ be the pagecache, so that we correctly report delalloc
+ * extents when dirty in the pagecache (we're COW, after all).
+ *
+ * But we'd have to add per-sector writeback tracking to
+ * bch_folio_state, otherwise we report delalloc extents for clean
+ * cached data in the pagecache.
+ *
+ * We should do this, but even then fiemap won't report stable mappings:
+ * on bcachefs data moves around in the background (copygc, rebalance)
+ * and we don't provide a way for userspace to lock that out.
+ */
+ if (k.k &&
+ bkey_le(bpos_max(iter.pos, bkey_start_pos(k.k)),
+ pagecache_start)) {
+ bch2_bkey_buf_reassemble(&cur->kbuf, trans->c, k);
+ bch2_cut_front(iter.pos, cur->kbuf.k);
+ bch2_cut_back(POS(inode->ei_inum.inum, end), cur->kbuf.k);
+ cur->flags = 0;
+ } else if (k.k) {
+ bch2_cut_back(bkey_start_pos(k.k), cur->kbuf.k);
+ }
+
+ if (cur->kbuf.k->k.type == KEY_TYPE_reflink_p) {
+ unsigned sectors = cur->kbuf.k->k.size;
+ s64 offset_into_extent = 0;
+ enum btree_id data_btree = BTREE_ID_extents;
+ ret = bch2_read_indirect_extent(trans, &data_btree, &offset_into_extent,
+ &cur->kbuf);
+ if (ret)
+ goto err;
+
+ struct bkey_i *k = cur->kbuf.k;
+ sectors = min_t(unsigned, sectors, k->k.size - offset_into_extent);
+
+ bch2_cut_front(POS(k->k.p.inode,
+ bkey_start_offset(&k->k) + offset_into_extent),
+ k);
+ bch2_key_resize(&k->k, sectors);
+ k->k.p = iter.pos;
+ k->k.p.offset += k->k.size;
+ }
+err:
+ bch2_trans_iter_exit(trans, &iter);
+ return ret;
+}
+
static int bch2_fiemap(struct inode *vinode, struct fiemap_extent_info *info,
u64 start, u64 len)
{
struct bch_fs *c = vinode->i_sb->s_fs_info;
struct bch_inode_info *ei = to_bch_ei(vinode);
struct btree_trans *trans;
- struct btree_iter iter;
- struct bkey_s_c k;
- struct bkey_buf cur, prev;
- struct bpos end = POS(ei->v.i_ino, (start + len) >> 9);
- unsigned offset_into_extent, sectors;
- bool have_extent = false;
- u32 snapshot;
+ struct bch_fiemap_extent cur, prev;
int ret = 0;
- ret = fiemap_prep(&ei->v, info, start, &len, FIEMAP_FLAG_SYNC);
+ ret = fiemap_prep(&ei->v, info, start, &len, 0);
if (ret)
return ret;
@@ -975,85 +1497,50 @@ static int bch2_fiemap(struct inode *vinode, struct fiemap_extent_info *info,
return -EINVAL;
start >>= 9;
+ u64 end = (start + len) >> 9;
- bch2_bkey_buf_init(&cur);
- bch2_bkey_buf_init(&prev);
- trans = bch2_trans_get(c);
-retry:
- bch2_trans_begin(trans);
+ bch2_bkey_buf_init(&cur.kbuf);
+ bch2_bkey_buf_init(&prev.kbuf);
+ bkey_init(&prev.kbuf.k->k);
- ret = bch2_subvolume_get_snapshot(trans, ei->ei_subvol, &snapshot);
- if (ret)
- goto err;
-
- bch2_trans_iter_init(trans, &iter, BTREE_ID_extents,
- SPOS(ei->v.i_ino, start, snapshot), 0);
-
- while (!(ret = btree_trans_too_many_iters(trans)) &&
- (k = bch2_btree_iter_peek_upto(&iter, end)).k &&
- !(ret = bkey_err(k))) {
- enum btree_id data_btree = BTREE_ID_extents;
-
- if (!bkey_extent_is_data(k.k) &&
- k.k->type != KEY_TYPE_reservation) {
- bch2_btree_iter_advance(&iter);
- continue;
- }
-
- offset_into_extent = iter.pos.offset -
- bkey_start_offset(k.k);
- sectors = k.k->size - offset_into_extent;
-
- bch2_bkey_buf_reassemble(&cur, c, k);
+ trans = bch2_trans_get(c);
- ret = bch2_read_indirect_extent(trans, &data_btree,
- &offset_into_extent, &cur);
+ while (start < end) {
+ ret = lockrestart_do(trans,
+ bch2_next_fiemap_extent(trans, ei, start, end, &cur));
if (ret)
- break;
+ goto err;
- k = bkey_i_to_s_c(cur.k);
- bch2_bkey_buf_realloc(&prev, c, k.k->u64s);
+ BUG_ON(bkey_start_offset(&cur.kbuf.k->k) < start);
+ BUG_ON(cur.kbuf.k->k.p.offset > end);
- sectors = min(sectors, k.k->size - offset_into_extent);
+ if (bkey_start_offset(&cur.kbuf.k->k) == end)
+ break;
- bch2_cut_front(POS(k.k->p.inode,
- bkey_start_offset(k.k) +
- offset_into_extent),
- cur.k);
- bch2_key_resize(&cur.k->k, sectors);
- cur.k->k.p = iter.pos;
- cur.k->k.p.offset += cur.k->k.size;
+ start = cur.kbuf.k->k.p.offset;
- if (have_extent) {
+ if (!bkey_deleted(&prev.kbuf.k->k)) {
bch2_trans_unlock(trans);
- ret = bch2_fill_extent(c, info,
- bkey_i_to_s_c(prev.k), 0);
+ ret = bch2_fill_extent(c, info, &prev);
if (ret)
- break;
+ goto err;
}
- bkey_copy(prev.k, cur.k);
- have_extent = true;
-
- bch2_btree_iter_set_pos(&iter,
- POS(iter.pos.inode, iter.pos.offset + sectors));
+ bch2_bkey_buf_copy(&prev.kbuf, c, cur.kbuf.k);
+ prev.flags = cur.flags;
}
- start = iter.pos.offset;
- bch2_trans_iter_exit(trans, &iter);
-err:
- if (bch2_err_matches(ret, BCH_ERR_transaction_restart))
- goto retry;
- if (!ret && have_extent) {
+ if (!bkey_deleted(&prev.kbuf.k->k)) {
bch2_trans_unlock(trans);
- ret = bch2_fill_extent(c, info, bkey_i_to_s_c(prev.k),
- FIEMAP_EXTENT_LAST);
+ prev.flags |= FIEMAP_EXTENT_LAST;
+ ret = bch2_fill_extent(c, info, &prev);
}
-
+err:
bch2_trans_put(trans);
- bch2_bkey_buf_exit(&cur, c);
- bch2_bkey_buf_exit(&prev, c);
- return ret < 0 ? ret : 0;
+ bch2_bkey_buf_exit(&cur.kbuf, c);
+ bch2_bkey_buf_exit(&prev.kbuf, c);
+
+ return bch2_err_class(ret < 0 ? ret : 0);
}
static const struct vm_operations_struct bch_vm_ops = {
@@ -1098,20 +1585,158 @@ static int bch2_open(struct inode *vinode, struct file *file)
struct bch_inode_info *inode = to_bch_ei(vinode);
struct bch_fs *c = inode->v.i_sb->s_fs_info;
- int ret = bch2_subvol_is_ro(c, inode->ei_subvol);
+ int ret = bch2_subvol_is_ro(c, inode->ei_inum.subvol);
if (ret)
return ret;
}
+ file->f_mode |= FMODE_CAN_ODIRECT;
+
return generic_file_open(vinode, file);
}
+/* bcachefs inode flags -> FS_IOC_GETFLAGS: */
+static const __maybe_unused unsigned bch_flags_to_uflags[] = {
+ [__BCH_INODE_sync] = FS_SYNC_FL,
+ [__BCH_INODE_immutable] = FS_IMMUTABLE_FL,
+ [__BCH_INODE_append] = FS_APPEND_FL,
+ [__BCH_INODE_nodump] = FS_NODUMP_FL,
+ [__BCH_INODE_noatime] = FS_NOATIME_FL,
+};
+
+/* bcachefs inode flags -> FS_IOC_FSGETXATTR: */
+static const __maybe_unused unsigned bch_flags_to_xflags[] = {
+ [__BCH_INODE_sync] = FS_XFLAG_SYNC,
+ [__BCH_INODE_immutable] = FS_XFLAG_IMMUTABLE,
+ [__BCH_INODE_append] = FS_XFLAG_APPEND,
+ [__BCH_INODE_nodump] = FS_XFLAG_NODUMP,
+ [__BCH_INODE_noatime] = FS_XFLAG_NOATIME,
+};
+
+static int bch2_fileattr_get(struct dentry *dentry,
+ struct fileattr *fa)
+{
+ struct bch_inode_info *inode = to_bch_ei(d_inode(dentry));
+ struct bch_fs *c = inode->v.i_sb->s_fs_info;
+
+ fileattr_fill_xflags(fa, map_flags(bch_flags_to_xflags, inode->ei_inode.bi_flags));
+
+ if (inode->ei_inode.bi_fields_set & (1 << Inode_opt_project))
+ fa->fsx_xflags |= FS_XFLAG_PROJINHERIT;
+
+ if (bch2_inode_casefold(c, &inode->ei_inode))
+ fa->flags |= FS_CASEFOLD_FL;
+
+ fa->fsx_projid = inode->ei_qid.q[QTYP_PRJ];
+ return 0;
+}
+
+struct flags_set {
+ unsigned mask;
+ unsigned flags;
+ unsigned projid;
+ bool set_project;
+ bool set_casefold;
+ bool casefold;
+};
+
+static int fssetxattr_inode_update_fn(struct btree_trans *trans,
+ struct bch_inode_info *inode,
+ struct bch_inode_unpacked *bi,
+ void *p)
+{
+ struct bch_fs *c = trans->c;
+ struct flags_set *s = p;
+
+ /*
+ * We're relying on btree locking here for exclusion with other ioctl
+ * calls - use the flags in the btree (@bi), not inode->i_flags:
+ */
+ if (!S_ISREG(bi->bi_mode) &&
+ !S_ISDIR(bi->bi_mode) &&
+ (s->flags & (BCH_INODE_nodump|BCH_INODE_noatime)) != s->flags)
+ return -EINVAL;
+
+ if (s->casefold != bch2_inode_casefold(c, bi)) {
+ int ret = bch2_inode_set_casefold(trans, inode_inum(inode), bi, s->casefold);
+ if (ret)
+ return ret;
+ }
+
+ if (s->set_project) {
+ bi->bi_project = s->projid;
+ bi->bi_fields_set |= BIT(Inode_opt_project);
+ }
+
+ bi->bi_flags &= ~s->mask;
+ bi->bi_flags |= s->flags;
+
+ bi->bi_ctime = timespec_to_bch2_time(c, current_time(&inode->v));
+ return 0;
+}
+
+static int bch2_fileattr_set(struct mnt_idmap *idmap,
+ struct dentry *dentry,
+ struct fileattr *fa)
+{
+ struct bch_inode_info *inode = to_bch_ei(d_inode(dentry));
+ struct bch_fs *c = inode->v.i_sb->s_fs_info;
+ struct flags_set s = {};
+ int ret;
+
+ if (fa->fsx_valid) {
+ fa->fsx_xflags &= ~FS_XFLAG_PROJINHERIT;
+
+ s.mask = map_defined(bch_flags_to_xflags);
+ s.flags |= map_flags_rev(bch_flags_to_xflags, fa->fsx_xflags);
+ if (fa->fsx_xflags)
+ return -EOPNOTSUPP;
+
+ if (fa->fsx_projid >= U32_MAX)
+ return -EINVAL;
+
+ /*
+ * inode fields accessible via the xattr interface are stored with a +1
+ * bias, so that 0 means unset:
+ */
+ if ((inode->ei_inode.bi_project ||
+ fa->fsx_projid) &&
+ inode->ei_inode.bi_project != fa->fsx_projid + 1) {
+ s.projid = fa->fsx_projid + 1;
+ s.set_project = true;
+ }
+ }
+
+ if (fa->flags_valid) {
+ s.mask = map_defined(bch_flags_to_uflags);
+
+ s.set_casefold = true;
+ s.casefold = (fa->flags & FS_CASEFOLD_FL) != 0;
+ fa->flags &= ~FS_CASEFOLD_FL;
+
+ s.flags |= map_flags_rev(bch_flags_to_uflags, fa->flags);
+ if (fa->flags)
+ return -EOPNOTSUPP;
+ }
+
+ mutex_lock(&inode->ei_update_lock);
+ ret = bch2_subvol_is_ro(c, inode->ei_inum.subvol) ?:
+ (s.set_project
+ ? bch2_set_projid(c, inode, fa->fsx_projid)
+ : 0) ?:
+ bch2_write_inode(c, inode, fssetxattr_inode_update_fn, &s,
+ ATTR_CTIME);
+ mutex_unlock(&inode->ei_update_lock);
+ return ret;
+}
+
static const struct file_operations bch_file_operations = {
.open = bch2_open,
.llseek = bch2_llseek,
.read_iter = bch2_read_iter,
.write_iter = bch2_write_iter,
.mmap = bch2_mmap,
+ .get_unmapped_area = thp_get_unmapped_area,
.fsync = bch2_fsync,
.splice_read = filemap_splice_read,
.splice_write = iter_file_splice_write,
@@ -1129,9 +1754,11 @@ static const struct inode_operations bch_file_inode_operations = {
.fiemap = bch2_fiemap,
.listxattr = bch2_xattr_list,
#ifdef CONFIG_BCACHEFS_POSIX_ACL
- .get_acl = bch2_get_acl,
+ .get_inode_acl = bch2_get_acl,
.set_acl = bch2_set_acl,
#endif
+ .fileattr_get = bch2_fileattr_get,
+ .fileattr_set = bch2_fileattr_set,
};
static const struct inode_operations bch_dir_inode_operations = {
@@ -1149,9 +1776,11 @@ static const struct inode_operations bch_dir_inode_operations = {
.tmpfile = bch2_tmpfile,
.listxattr = bch2_xattr_list,
#ifdef CONFIG_BCACHEFS_POSIX_ACL
- .get_acl = bch2_get_acl,
+ .get_inode_acl = bch2_get_acl,
.set_acl = bch2_set_acl,
#endif
+ .fileattr_get = bch2_fileattr_get,
+ .fileattr_set = bch2_fileattr_set,
};
static const struct file_operations bch_dir_file_operations = {
@@ -1171,9 +1800,11 @@ static const struct inode_operations bch_symlink_inode_operations = {
.setattr = bch2_setattr,
.listxattr = bch2_xattr_list,
#ifdef CONFIG_BCACHEFS_POSIX_ACL
- .get_acl = bch2_get_acl,
+ .get_inode_acl = bch2_get_acl,
.set_acl = bch2_set_acl,
#endif
+ .fileattr_get = bch2_fileattr_get,
+ .fileattr_set = bch2_fileattr_set,
};
static const struct inode_operations bch_special_inode_operations = {
@@ -1181,9 +1812,11 @@ static const struct inode_operations bch_special_inode_operations = {
.setattr = bch2_setattr,
.listxattr = bch2_xattr_list,
#ifdef CONFIG_BCACHEFS_POSIX_ACL
- .get_acl = bch2_get_acl,
+ .get_inode_acl = bch2_get_acl,
.set_acl = bch2_set_acl,
#endif
+ .fileattr_get = bch2_fileattr_get,
+ .fileattr_set = bch2_fileattr_set,
};
static const struct address_space_operations bch_address_space_operations = {
@@ -1195,7 +1828,6 @@ static const struct address_space_operations bch_address_space_operations = {
.write_end = bch2_write_end,
.invalidate_folio = bch2_invalidate_folio,
.release_folio = bch2_release_folio,
- .direct_IO = noop_direct_IO,
#ifdef CONFIG_MIGRATION
.migrate_folio = filemap_migrate_folio,
#endif
@@ -1228,8 +1860,8 @@ static int bcachefs_fid_valid(int fh_len, int fh_type)
static struct bcachefs_fid bch2_inode_to_fid(struct bch_inode_info *inode)
{
return (struct bcachefs_fid) {
- .inum = inode->ei_inode.bi_inum,
- .subvol = inode->ei_subvol,
+ .inum = inode->ei_inum.inum,
+ .subvol = inode->ei_inum.subvol,
.gen = inode->ei_inode.bi_generation,
};
}
@@ -1314,7 +1946,7 @@ static struct dentry *bch2_get_parent(struct dentry *child)
struct bch_fs *c = inode->v.i_sb->s_fs_info;
subvol_inum parent_inum = {
.subvol = inode->ei_inode.bi_parent_subvol ?:
- inode->ei_subvol,
+ inode->ei_inum.subvol,
.inum = inode->ei_inode.bi_dir,
};
@@ -1350,21 +1982,21 @@ static int bch2_get_name(struct dentry *parent, char *name, struct dentry *child
retry:
bch2_trans_begin(trans);
- ret = bch2_subvolume_get_snapshot(trans, dir->ei_subvol, &snapshot);
+ ret = bch2_subvolume_get_snapshot(trans, dir->ei_inum.subvol, &snapshot);
if (ret)
goto err;
- bch2_btree_iter_set_snapshot(&iter1, snapshot);
- bch2_btree_iter_set_snapshot(&iter2, snapshot);
+ bch2_btree_iter_set_snapshot(trans, &iter1, snapshot);
+ bch2_btree_iter_set_snapshot(trans, &iter2, snapshot);
ret = bch2_inode_find_by_inum_trans(trans, inode_inum(inode), &inode_u);
if (ret)
goto err;
if (inode_u.bi_dir == dir->ei_inode.bi_inum) {
- bch2_btree_iter_set_pos(&iter1, POS(inode_u.bi_dir, inode_u.bi_dir_offset));
+ bch2_btree_iter_set_pos(trans, &iter1, POS(inode_u.bi_dir, inode_u.bi_dir_offset));
- k = bch2_btree_iter_peek_slot(&iter1);
+ k = bch2_btree_iter_peek_slot(trans, &iter1);
ret = bkey_err(k);
if (ret)
goto err;
@@ -1381,15 +2013,14 @@ retry:
if (ret)
goto err;
- if (target.subvol == inode->ei_subvol &&
- target.inum == inode->ei_inode.bi_inum)
+ if (subvol_inum_eq(target, inode->ei_inum))
goto found;
} else {
/*
* File with multiple hardlinks and our backref is to the wrong
* directory - linear search:
*/
- for_each_btree_key_continue_norestart(iter2, 0, k, ret) {
+ for_each_btree_key_continue_norestart(trans, iter2, 0, k, ret) {
if (k.k->p.inode > dir->ei_inode.bi_inum)
break;
@@ -1403,8 +2034,7 @@ retry:
if (ret)
continue;
- if (target.subvol == inode->ei_subvol &&
- target.inum == inode->ei_inode.bi_inum)
+ if (subvol_inum_eq(target, inode->ei_inum))
goto found;
}
}
@@ -1436,21 +2066,18 @@ static const struct export_operations bch_export_ops = {
.get_name = bch2_get_name,
};
-static void bch2_vfs_inode_init(struct btree_trans *trans, subvol_inum inum,
+static void bch2_vfs_inode_init(struct btree_trans *trans,
+ subvol_inum inum,
struct bch_inode_info *inode,
struct bch_inode_unpacked *bi,
struct bch_subvolume *subvol)
{
- bch2_iget5_set(&inode->v, &inum);
+ inode->v.i_ino = inum.inum;
+ inode->ei_inum = inum;
+ inode->ei_inode.bi_inum = inum.inum;
bch2_inode_update_after_write(trans, inode, bi, ~0);
- if (BCH_SUBVOLUME_SNAP(subvol))
- set_bit(EI_INODE_SNAPSHOT, &inode->ei_flags);
- else
- clear_bit(EI_INODE_SNAPSHOT, &inode->ei_flags);
-
inode->v.i_blocks = bi->bi_sectors;
- inode->v.i_ino = bi->bi_inum;
inode->v.i_rdev = bi->bi_dev;
inode->v.i_generation = bi->bi_generation;
inode->v.i_size = bi->bi_size;
@@ -1458,7 +2085,9 @@ static void bch2_vfs_inode_init(struct btree_trans *trans, subvol_inum inum,
inode->ei_flags = 0;
inode->ei_quota_reserved = 0;
inode->ei_qid = bch_qid(bi);
- inode->ei_subvol = inum.subvol;
+
+ if (BCH_SUBVOLUME_SNAP(subvol))
+ set_bit(EI_INODE_SNAPSHOT, &inode->ei_flags);
inode->v.i_mapping->a_ops = &bch_address_space_operations;
@@ -1481,37 +2110,13 @@ static void bch2_vfs_inode_init(struct btree_trans *trans, subvol_inum inum,
break;
}
- mapping_set_large_folios(inode->v.i_mapping);
+ mapping_set_folio_min_order(inode->v.i_mapping,
+ get_order(trans->c->opts.block_size));
}
-static struct inode *bch2_alloc_inode(struct super_block *sb)
+static void bch2_free_inode(struct inode *vinode)
{
- struct bch_inode_info *inode;
-
- inode = kmem_cache_alloc(bch2_inode_cache, GFP_NOFS);
- if (!inode)
- return NULL;
-
- inode_init_once(&inode->v);
- mutex_init(&inode->ei_update_lock);
- two_state_lock_init(&inode->ei_pagecache_lock);
- INIT_LIST_HEAD(&inode->ei_vfs_inode_list);
- mutex_init(&inode->ei_quota_lock);
-
- return &inode->v;
-}
-
-static void bch2_i_callback(struct rcu_head *head)
-{
- struct inode *vinode = container_of(head, struct inode, i_rcu);
- struct bch_inode_info *inode = to_bch_ei(vinode);
-
- kmem_cache_free(bch2_inode_cache, inode);
-}
-
-static void bch2_destroy_inode(struct inode *vinode)
-{
- call_rcu(&vinode->i_rcu, bch2_i_callback);
+ kmem_cache_free(bch2_inode_cache, to_bch_ei(vinode));
}
static int inode_update_times_fn(struct btree_trans *trans,
@@ -1547,6 +2152,17 @@ static void bch2_evict_inode(struct inode *vinode)
{
struct bch_fs *c = vinode->i_sb->s_fs_info;
struct bch_inode_info *inode = to_bch_ei(vinode);
+ bool delete = !inode->v.i_nlink && !is_bad_inode(&inode->v);
+
+ /*
+ * evict() has waited for outstanding writeback, we'll do no more IO
+ * through this inode: it's safe to remove from VFS inode hashtable here
+ *
+ * Do that now so that other threads aren't blocked from pulling it back
+ * in, there's no reason for them to be:
+ */
+ if (!delete)
+ bch2_inode_hash_remove(c, inode);
truncate_inode_pages_final(&inode->v.i_data);
@@ -1554,12 +2170,18 @@ static void bch2_evict_inode(struct inode *vinode)
BUG_ON(!is_bad_inode(&inode->v) && inode->ei_quota_reserved);
- if (!inode->v.i_nlink && !is_bad_inode(&inode->v)) {
+ if (delete) {
bch2_quota_acct(c, inode->ei_qid, Q_SPC, -((s64) inode->v.i_blocks),
KEY_TYPE_QUOTA_WARN);
bch2_quota_acct(c, inode->ei_qid, Q_INO, -1,
KEY_TYPE_QUOTA_WARN);
bch2_inode_rm(c, inode_inum(inode));
+
+ /*
+ * If we are deleting, we need it present in the vfs hash table
+ * so that fsck can check if unlinked inodes are still open:
+ */
+ bch2_inode_hash_remove(c, inode);
}
mutex_lock(&c->vfs_inodes_lock);
@@ -1589,7 +2211,7 @@ again:
mutex_lock(&c->vfs_inodes_lock);
list_for_each_entry(inode, &c->vfs_inodes_list, ei_vfs_inode_list) {
- if (!snapshot_list_has_id(s, inode->ei_subvol))
+ if (!snapshot_list_has_id(s, inode->ei_inum.subvol))
continue;
if (!(inode->v.i_state & I_DONTCACHE) &&
@@ -1602,14 +2224,16 @@ again:
break;
}
} else if (clean_pass && this_pass_clean) {
- wait_queue_head_t *wq = bit_waitqueue(&inode->v.i_state, __I_NEW);
- DEFINE_WAIT_BIT(wait, &inode->v.i_state, __I_NEW);
+ struct wait_bit_queue_entry wqe;
+ struct wait_queue_head *wq_head;
- prepare_to_wait(wq, &wait.wq_entry, TASK_UNINTERRUPTIBLE);
+ wq_head = inode_bit_waitqueue(&wqe, &inode->v, __I_NEW);
+ prepare_to_wait_event(wq_head, &wqe.wq_entry,
+ TASK_UNINTERRUPTIBLE);
mutex_unlock(&c->vfs_inodes_lock);
schedule();
- finish_wait(wq, &wait.wq_entry);
+ finish_wait(wq_head, &wqe.wq_entry);
goto again;
}
}
@@ -1663,6 +2287,8 @@ static int bch2_sync_fs(struct super_block *sb, int wait)
struct bch_fs *c = sb->s_fs_info;
int ret;
+ trace_bch2_sync_fs(sb, wait);
+
if (c->opts.journal_flush_disabled)
return 0;
@@ -1691,59 +2317,19 @@ static struct bch_fs *bch2_path_to_fs(const char *path)
return c ?: ERR_PTR(-ENOENT);
}
-static int bch2_remount(struct super_block *sb, int *flags, char *data)
-{
- struct bch_fs *c = sb->s_fs_info;
- struct bch_opts opts = bch2_opts_empty();
- int ret;
-
- ret = bch2_parse_mount_opts(c, &opts, data);
- if (ret)
- goto err;
-
- opt_set(opts, read_only, (*flags & SB_RDONLY) != 0);
-
- if (opts.read_only != c->opts.read_only) {
- down_write(&c->state_lock);
-
- if (opts.read_only) {
- bch2_fs_read_only(c);
-
- sb->s_flags |= SB_RDONLY;
- } else {
- ret = bch2_fs_read_write(c);
- if (ret) {
- bch_err(c, "error going rw: %i", ret);
- up_write(&c->state_lock);
- ret = -EINVAL;
- goto err;
- }
-
- sb->s_flags &= ~SB_RDONLY;
- }
-
- c->opts.read_only = opts.read_only;
-
- up_write(&c->state_lock);
- }
-
- if (opt_defined(opts, errors))
- c->opts.errors = opts.errors;
-err:
- return bch2_err_class(ret);
-}
-
static int bch2_show_devname(struct seq_file *seq, struct dentry *root)
{
struct bch_fs *c = root->d_sb->s_fs_info;
bool first = true;
- for_each_online_member(c, ca) {
+ rcu_read_lock();
+ for_each_online_member_rcu(c, ca) {
if (!first)
seq_putc(seq, ':');
first = false;
seq_puts(seq, ca->disk_sb.sb_name);
}
+ rcu_read_unlock();
return 0;
}
@@ -1751,29 +2337,14 @@ static int bch2_show_devname(struct seq_file *seq, struct dentry *root)
static int bch2_show_options(struct seq_file *seq, struct dentry *root)
{
struct bch_fs *c = root->d_sb->s_fs_info;
- enum bch_opt_id i;
struct printbuf buf = PRINTBUF;
- int ret = 0;
- for (i = 0; i < bch2_opts_nr; i++) {
- const struct bch_option *opt = &bch2_opt_table[i];
- u64 v = bch2_opt_get_by_id(&c->opts, i);
+ bch2_opts_to_text(&buf, c->opts, c, c->disk_sb.sb,
+ OPT_MOUNT, OPT_HIDDEN, OPT_SHOW_MOUNT_STYLE);
+ printbuf_nul_terminate(&buf);
+ seq_printf(seq, ",%s", buf.buf);
- if (!(opt->flags & OPT_MOUNT))
- continue;
-
- if (v == bch2_opt_get_by_id(&bch2_opts_default, i))
- continue;
-
- printbuf_reset(&buf);
- bch2_opt_to_text(&buf, c, c->disk_sb.sb, opt, v,
- OPT_SHOW_MOUNT_STYLE);
- seq_putc(seq, ',');
- seq_puts(seq, buf.buf);
- }
-
- if (buf.allocation_failure)
- ret = -ENOMEM;
+ int ret = buf.allocation_failure ? -ENOMEM : 0;
printbuf_exit(&buf);
return ret;
}
@@ -1819,14 +2390,13 @@ static int bch2_unfreeze(struct super_block *sb)
static const struct super_operations bch_super_operations = {
.alloc_inode = bch2_alloc_inode,
- .destroy_inode = bch2_destroy_inode,
+ .free_inode = bch2_free_inode,
.write_inode = bch2_vfs_write_inode,
.evict_inode = bch2_evict_inode,
.sync_fs = bch2_sync_fs,
.statfs = bch2_statfs,
.show_devname = bch2_show_devname,
.show_options = bch2_show_options,
- .remount_fs = bch2_remount,
.put_super = bch2_put_super,
.freeze_fs = bch2_freeze,
.unfreeze_fs = bch2_unfreeze,
@@ -1859,77 +2429,66 @@ static int bch2_test_super(struct super_block *s, void *data)
return true;
}
-static struct dentry *bch2_mount(struct file_system_type *fs_type,
- int flags, const char *dev_name, void *data)
+static int bch2_fs_get_tree(struct fs_context *fc)
{
struct bch_fs *c;
struct super_block *sb;
struct inode *vinode;
- struct bch_opts opts = bch2_opts_empty();
+ struct bch2_opts_parse *opts_parse = fc->fs_private;
+ struct bch_opts opts = opts_parse->opts;
+ darray_const_str devs;
+ darray_fs devs_to_fs = {};
int ret;
- opt_set(opts, read_only, (flags & SB_RDONLY) != 0);
+ opt_set(opts, read_only, (fc->sb_flags & SB_RDONLY) != 0);
+ opt_set(opts, nostart, true);
- ret = bch2_parse_mount_opts(NULL, &opts, data);
- if (ret) {
- ret = bch2_err_class(ret);
- return ERR_PTR(ret);
- }
-
- if (!dev_name || strlen(dev_name) == 0)
- return ERR_PTR(-EINVAL);
+ if (!fc->source || strlen(fc->source) == 0)
+ return -EINVAL;
- darray_str devs;
- ret = bch2_split_devs(dev_name, &devs);
+ ret = bch2_split_devs(fc->source, &devs);
if (ret)
- return ERR_PTR(ret);
+ return ret;
- darray_fs devs_to_fs = {};
darray_for_each(devs, i) {
ret = darray_push(&devs_to_fs, bch2_path_to_fs(*i));
- if (ret) {
- sb = ERR_PTR(ret);
- goto got_sb;
- }
+ if (ret)
+ goto err;
}
- sb = sget(fs_type, bch2_test_super, bch2_noset_super, flags|SB_NOSEC, &devs_to_fs);
+ sb = sget(fc->fs_type, bch2_test_super, bch2_noset_super, fc->sb_flags|SB_NOSEC, &devs_to_fs);
if (!IS_ERR(sb))
goto got_sb;
- c = bch2_fs_open(devs.data, devs.nr, opts);
- if (IS_ERR(c)) {
- sb = ERR_CAST(c);
- goto got_sb;
- }
+ c = bch2_fs_open(&devs, &opts);
+ ret = PTR_ERR_OR_ZERO(c);
+ if (ret)
+ goto err;
+
+ if (opt_defined(opts, discard))
+ set_bit(BCH_FS_discard_mount_opt_set, &c->flags);
/* Some options can't be parsed until after the fs is started: */
- ret = bch2_parse_mount_opts(c, &opts, data);
- if (ret) {
- bch2_fs_stop(c);
- sb = ERR_PTR(ret);
- goto got_sb;
- }
+ opts = bch2_opts_empty();
+ ret = bch2_parse_mount_opts(c, &opts, NULL, opts_parse->parse_later.buf, false);
+ if (ret)
+ goto err_stop_fs;
bch2_opts_apply(&c->opts, opts);
- sb = sget(fs_type, NULL, bch2_set_super, flags|SB_NOSEC, c);
- if (IS_ERR(sb))
- bch2_fs_stop(c);
-got_sb:
- darray_exit(&devs_to_fs);
- bch2_darray_str_exit(&devs);
-
- if (IS_ERR(sb)) {
- ret = PTR_ERR(sb);
- ret = bch2_err_class(ret);
- return ERR_PTR(ret);
- }
+ ret = bch2_fs_start(c);
+ if (ret)
+ goto err_stop_fs;
+ sb = sget(fc->fs_type, NULL, bch2_set_super, fc->sb_flags|SB_NOSEC, c);
+ ret = PTR_ERR_OR_ZERO(sb);
+ if (ret)
+ goto err_stop_fs;
+got_sb:
c = sb->s_fs_info;
if (sb->s_root) {
- if ((flags ^ sb->s_flags) & SB_RDONLY) {
+ if ((fc->sb_flags ^ sb->s_flags) & SB_RDONLY) {
ret = -EBUSY;
goto err_put_super;
}
@@ -1950,7 +2509,14 @@ got_sb:
sb->s_time_gran = c->sb.nsec_per_time_unit;
sb->s_time_min = div_s64(S64_MIN, c->sb.time_units_per_sec) + 1;
sb->s_time_max = div_s64(S64_MAX, c->sb.time_units_per_sec);
- sb->s_uuid = c->sb.user_uuid;
+ super_set_uuid(sb, c->sb.user_uuid.b, sizeof(c->sb.user_uuid));
+
+ if (c->sb.multi_device)
+ super_set_sysfs_name_uuid(sb);
+ else
+ strscpy(sb->s_sysfs_name, c->name, sizeof(sb->s_sysfs_name));
+
+ sb->s_shrink->seeks = 0;
c->vfs_sb = sb;
strscpy(sb->s_id, c->name, sizeof(sb->s_id));
@@ -1960,15 +2526,16 @@ got_sb:
sb->s_bdi->ra_pages = VM_READAHEAD_PAGES;
- for_each_online_member(c, ca) {
+ rcu_read_lock();
+ for_each_online_member_rcu(c, ca) {
struct block_device *bdev = ca->disk_sb.bdev;
/* XXX: create an anonymous device for multi device filesystems */
sb->s_bdev = bdev;
sb->s_dev = bdev->bd_dev;
- percpu_ref_put(&ca->io_ref);
break;
}
+ rcu_read_unlock();
c->dev = sb->s_dev;
@@ -1979,6 +2546,11 @@ got_sb:
sb->s_shrink->seeks = 0;
+#ifdef CONFIG_UNICODE
+ sb->s_encoding = c->cf_encoding;
+#endif
+ generic_set_sb_d_ops(sb);
+
vinode = bch2_vfs_inode_get(c, BCACHEFS_ROOT_SUBVOL_INUM);
ret = PTR_ERR_OR_ZERO(vinode);
bch_err_msg(c, ret, "mounting: error getting root inode");
@@ -1994,12 +2566,31 @@ got_sb:
sb->s_flags |= SB_ACTIVE;
out:
- return dget(sb->s_root);
+ fc->root = dget(sb->s_root);
+err:
+ darray_exit(&devs_to_fs);
+ bch2_darray_str_exit(&devs);
+ if (ret)
+ pr_err("error: %s", bch2_err_str(ret));
+ /*
+ * On an inconsistency error in recovery we might see an -EROFS derived
+ * errorcode (from the journal), but we don't want to return that to
+ * userspace as that causes util-linux to retry the mount RO - which is
+ * confusing:
+ */
+ if (bch2_err_matches(ret, EROFS) && ret != -EROFS)
+ ret = -EIO;
+ return bch2_err_class(ret);
+
+err_stop_fs:
+ bch2_fs_stop(c);
+ goto err;
err_put_super:
- __bch2_fs_stop(c);
+ if (!sb->s_root)
+ __bch2_fs_stop(c);
deactivate_locked_super(sb);
- return ERR_PTR(bch2_err_class(ret));
+ goto err;
}
static void bch2_kill_sb(struct super_block *sb)
@@ -2010,12 +2601,123 @@ static void bch2_kill_sb(struct super_block *sb)
bch2_fs_free(c);
}
+static void bch2_fs_context_free(struct fs_context *fc)
+{
+ struct bch2_opts_parse *opts = fc->fs_private;
+
+ if (opts) {
+ printbuf_exit(&opts->parse_later);
+ kfree(opts);
+ }
+}
+
+static int bch2_fs_parse_param(struct fs_context *fc,
+ struct fs_parameter *param)
+{
+ /*
+ * the "source" param, i.e., the name of the device(s) to mount,
+ * is handled by the VFS layer.
+ */
+ if (!strcmp(param->key, "source"))
+ return -ENOPARAM;
+
+ struct bch2_opts_parse *opts = fc->fs_private;
+ struct bch_fs *c = NULL;
+
+ /* for reconfigure, we already have a struct bch_fs */
+ if (fc->root)
+ c = fc->root->d_sb->s_fs_info;
+
+ int ret = bch2_parse_one_mount_opt(c, &opts->opts,
+ &opts->parse_later, param->key,
+ param->string);
+ if (ret)
+ pr_err("Error parsing option %s: %s", param->key, bch2_err_str(ret));
+
+ return bch2_err_class(ret);
+}
+
+static int bch2_fs_reconfigure(struct fs_context *fc)
+{
+ struct super_block *sb = fc->root->d_sb;
+ struct bch2_opts_parse *opts = fc->fs_private;
+ struct bch_fs *c = sb->s_fs_info;
+ int ret = 0;
+
+ opt_set(opts->opts, read_only, (fc->sb_flags & SB_RDONLY) != 0);
+
+ if (opts->opts.read_only != c->opts.read_only) {
+ down_write(&c->state_lock);
+
+ if (opts->opts.read_only) {
+ bch2_fs_read_only(c);
+
+ sb->s_flags |= SB_RDONLY;
+ } else {
+ ret = bch2_fs_read_write(c);
+ if (ret) {
+ bch_err(c, "error going rw: %i", ret);
+ up_write(&c->state_lock);
+ ret = -EINVAL;
+ goto err;
+ }
+
+ sb->s_flags &= ~SB_RDONLY;
+ }
+
+ c->opts.read_only = opts->opts.read_only;
+
+ up_write(&c->state_lock);
+ }
+
+ if (opt_defined(opts->opts, errors))
+ c->opts.errors = opts->opts.errors;
+err:
+ return bch2_err_class(ret);
+}
+
+static const struct fs_context_operations bch2_context_ops = {
+ .free = bch2_fs_context_free,
+ .parse_param = bch2_fs_parse_param,
+ .get_tree = bch2_fs_get_tree,
+ .reconfigure = bch2_fs_reconfigure,
+};
+
+static int bch2_init_fs_context(struct fs_context *fc)
+{
+ struct bch2_opts_parse *opts = kzalloc(sizeof(*opts), GFP_KERNEL);
+
+ if (!opts)
+ return -ENOMEM;
+
+ opts->parse_later = PRINTBUF;
+
+ fc->ops = &bch2_context_ops;
+ fc->fs_private = opts;
+
+ return 0;
+}
+
+void bch2_fs_vfs_exit(struct bch_fs *c)
+{
+ if (c->vfs_inodes_by_inum_table.ht.tbl)
+ rhltable_destroy(&c->vfs_inodes_by_inum_table);
+ if (c->vfs_inodes_table.tbl)
+ rhashtable_destroy(&c->vfs_inodes_table);
+}
+
+int bch2_fs_vfs_init(struct bch_fs *c)
+{
+ return rhashtable_init(&c->vfs_inodes_table, &bch2_vfs_inodes_params) ?:
+ rhltable_init(&c->vfs_inodes_by_inum_table, &bch2_vfs_inodes_by_inum_params);
+}
+
static struct file_system_type bcache_fs_type = {
- .owner = THIS_MODULE,
- .name = "bcachefs",
- .mount = bch2_mount,
- .kill_sb = bch2_kill_sb,
- .fs_flags = FS_REQUIRES_DEV,
+ .owner = THIS_MODULE,
+ .name = "bcachefs",
+ .init_fs_context = bch2_init_fs_context,
+ .kill_sb = bch2_kill_sb,
+ .fs_flags = FS_REQUIRES_DEV | FS_ALLOW_IDMAP | FS_LBS,
};
MODULE_ALIAS_FS("bcachefs");
@@ -2030,7 +2732,8 @@ int __init bch2_vfs_init(void)
{
int ret = -ENOMEM;
- bch2_inode_cache = KMEM_CACHE(bch_inode_info, SLAB_RECLAIM_ACCOUNT);
+ bch2_inode_cache = KMEM_CACHE(bch_inode_info, SLAB_RECLAIM_ACCOUNT |
+ SLAB_ACCOUNT);
if (!bch2_inode_cache)
goto err;