]> git.sesse.net Git - bcachefs-tools-debian/blobdiff - libbcachefs/io.c
Update bcachefs sources to 2272c5f5b7 bcachefs: Mark stripe buckets with correct...
[bcachefs-tools-debian] / libbcachefs / io.c
index f0fca861b90122f7882af40ac9ba86d4b1e9a170..7ec36113b9d62f166e860bf984c6e9ddc05c6a02 100644 (file)
 #include "journal.h"
 #include "keylist.h"
 #include "move.h"
+#include "nocow_locking.h"
 #include "rebalance.h"
 #include "subvolume.h"
 #include "super.h"
 #include "super-io.h"
 
 #include <linux/blkdev.h>
+#include <linux/prefetch.h>
 #include <linux/random.h>
 #include <linux/sched/mm.h>
 
@@ -45,6 +47,8 @@ const char *bch2_blk_status_to_str(blk_status_t status)
        return blk_status_to_str(status);
 }
 
+#ifndef CONFIG_BCACHEFS_NO_LATENCY_ACCT
+
 static bool bch2_target_congested(struct bch_fs *c, u16 target)
 {
        const struct bch_devs_mask *devs;
@@ -133,6 +137,15 @@ void bch2_latency_acct(struct bch_dev *ca, u64 submit_time, int rw)
        __bch2_time_stats_update(&ca->io_latency[rw], submit_time, now);
 }
 
+#else
+
+static bool bch2_target_congested(struct bch_fs *c, u16 target)
+{
+       return false;
+}
+
+#endif
+
 /* Allocate, free from mempool: */
 
 void bch2_bio_free_pages_pool(struct bch_fs *c, struct bio *bio)
@@ -241,6 +254,7 @@ static inline int bch2_extent_update_i_size_sectors(struct btree_trans *trans,
        struct btree_iter iter;
        struct bkey_i *k;
        struct bkey_i_inode_v3 *inode;
+       unsigned inode_update_flags = BTREE_UPDATE_NOJOURNAL;
        int ret;
 
        bch2_trans_iter_init(trans, &iter, BTREE_ID_inodes,
@@ -263,15 +277,24 @@ static inline int bch2_extent_update_i_size_sectors(struct btree_trans *trans,
        inode = bkey_i_to_inode_v3(k);
 
        if (!(le64_to_cpu(inode->v.bi_flags) & BCH_INODE_I_SIZE_DIRTY) &&
-           new_i_size > le64_to_cpu(inode->v.bi_size))
+           new_i_size > le64_to_cpu(inode->v.bi_size)) {
                inode->v.bi_size = cpu_to_le64(new_i_size);
+               inode_update_flags = 0;
+       }
 
-       le64_add_cpu(&inode->v.bi_sectors, i_sectors_delta);
+       if (i_sectors_delta) {
+               le64_add_cpu(&inode->v.bi_sectors, i_sectors_delta);
+               inode_update_flags = 0;
+       }
 
-       inode->k.p.snapshot = iter.snapshot;
+       if (inode->k.p.snapshot != iter.snapshot) {
+               inode->k.p.snapshot = iter.snapshot;
+               inode_update_flags = 0;
+       }
 
        ret = bch2_trans_update(trans, &iter, &inode->k_i,
-                               BTREE_UPDATE_INTERNAL_SNAPSHOT_NODE);
+                               BTREE_UPDATE_INTERNAL_SNAPSHOT_NODE|
+                               inode_update_flags);
 err:
        bch2_trans_iter_exit(trans, &iter);
        return ret;
@@ -427,7 +450,7 @@ retry:
                                opts.data_replicas,
                                opts.data_replicas,
                                RESERVE_none, 0, &cl, &wp);
-               if (ret == -EAGAIN) {
+               if (bch2_err_matches(ret, BCH_ERR_operation_blocked)) {
                        bch2_trans_unlock(trans);
                        closure_sync(&cl);
                        goto retry;
@@ -512,8 +535,6 @@ int bch2_fpunch_at(struct btree_trans *trans, struct btree_iter *iter,
                if (ret)
                        continue;
 
-               BUG_ON(bkey_ge(iter->pos, end_pos));
-
                bkey_init(&delete.k);
                delete.k.p = iter->pos;
 
@@ -526,8 +547,6 @@ int bch2_fpunch_at(struct btree_trans *trans, struct btree_iter *iter,
                bch2_disk_reservation_put(c, &disk_res);
        }
 
-       BUG_ON(bkey_gt(iter->pos, end_pos));
-
        return ret ?: ret2;
 }
 
@@ -664,6 +683,12 @@ void bch2_submit_wbio_replicas(struct bch_write_bio *wbio, struct bch_fs *c,
                                     bio_sectors(&n->bio));
 
                        bio_set_dev(&n->bio, ca->disk_sb.bdev);
+
+                       if (type != BCH_DATA_btree && unlikely(c->opts.no_data_io)) {
+                               bio_endio(&n->bio);
+                               continue;
+                       }
+
                        submit_bio(&n->bio);
                } else {
                        n->bio.bi_status        = BLK_STS_REMOVED;
@@ -680,11 +705,12 @@ static void bch2_write_done(struct closure *cl)
        struct bch_fs *c = op->c;
 
        bch2_disk_reservation_put(c, &op->res);
-       percpu_ref_put(&c->writes);
+       bch2_write_ref_put(c, BCH_WRITE_REF_write);
        bch2_keylist_free(&op->insert_keys, op->inline_keys);
 
        bch2_time_stats_update(&c->times[BCH_TIME_data_write], op->start_time);
 
+       EBUG_ON(cl->parent);
        closure_debug_destroy(cl);
        if (op->end_io)
                op->end_io(op);
@@ -753,15 +779,17 @@ static void __bch2_write_index(struct bch_write_op *op)
 
                op->written += sectors_start - keylist_sectors(keys);
 
-               if (ret) {
+               if (ret && !bch2_err_matches(ret, EROFS)) {
                        struct bkey_i *k = bch2_keylist_front(&op->insert_keys);
 
                        bch_err_inum_offset_ratelimited(c,
                                k->k.p.inode, k->k.p.offset << 9,
                                "write error while doing btree update: %s",
                                bch2_err_str(ret));
-                       goto err;
                }
+
+               if (ret)
+                       goto err;
        }
 out:
        /* If some a bucket wasn't written, we can't erasure code it: */
@@ -777,6 +805,30 @@ err:
        goto out;
 }
 
+static inline void __wp_update_state(struct write_point *wp, enum write_point_state state)
+{
+       if (state != wp->state) {
+               u64 now = ktime_get_ns();
+
+               if (wp->last_state_change &&
+                   time_after64(now, wp->last_state_change))
+                       wp->time[wp->state] += now - wp->last_state_change;
+               wp->state = state;
+               wp->last_state_change = now;
+       }
+}
+
+static inline void wp_update_state(struct write_point *wp, bool running)
+{
+       enum write_point_state state;
+
+       state = running                  ? WRITE_POINT_running :
+               !list_empty(&wp->writes) ? WRITE_POINT_waiting_io
+                                        : WRITE_POINT_stopped;
+
+       __wp_update_state(wp, state);
+}
+
 static void bch2_write_index(struct closure *cl)
 {
        struct bch_write_op *op = container_of(cl, struct bch_write_op, cl);
@@ -784,10 +836,32 @@ static void bch2_write_index(struct closure *cl)
        struct workqueue_struct *wq = index_update_wq(op);
 
        barrier();
+
+       /*
+        * We're not using wp->writes_lock here, so this is racey: that's ok,
+        * because this is just for diagnostic purposes, and we're running out
+        * of interrupt context here so if we were to take the log we'd have to
+        * switch to spin_lock_irq()/irqsave(), which is not free:
+        */
+       if (wp->state == WRITE_POINT_waiting_io)
+               __wp_update_state(wp, WRITE_POINT_waiting_work);
+
        op->btree_update_ready = true;
        queue_work(wq, &wp->index_update_work);
 }
 
+static inline void bch2_write_queue(struct bch_write_op *op, struct write_point *wp)
+{
+       op->btree_update_ready = false;
+       op->wp = wp;
+
+       spin_lock(&wp->writes_lock);
+       list_add_tail(&op->wp_list, &wp->writes);
+       if (wp->state == WRITE_POINT_stopped)
+               __wp_update_state(wp, WRITE_POINT_waiting_io);
+       spin_unlock(&wp->writes_lock);
+}
+
 void bch2_write_point_do_index_updates(struct work_struct *work)
 {
        struct write_point *wp =
@@ -796,16 +870,21 @@ void bch2_write_point_do_index_updates(struct work_struct *work)
 
        while (1) {
                spin_lock(&wp->writes_lock);
-               op = list_first_entry_or_null(&wp->writes, struct bch_write_op, wp_list);
-               if (op && !op->btree_update_ready)
-                       op = NULL;
-               if (op)
-                       list_del(&op->wp_list);
+               list_for_each_entry(op, &wp->writes, wp_list)
+                       if (op->btree_update_ready) {
+                               list_del(&op->wp_list);
+                               goto unlock;
+                       }
+               op = NULL;
+unlock:
+               wp_update_state(wp, op != NULL);
                spin_unlock(&wp->writes_lock);
 
                if (!op)
                        break;
 
+               op->flags |= BCH_WRITE_IN_WORKER;
+
                __bch2_write_index(op);
 
                if (!(op->flags & BCH_WRITE_DONE))
@@ -847,12 +926,10 @@ static void bch2_write_endio(struct bio *bio)
        if (wbio->put_bio)
                bio_put(bio);
 
-       if (parent) {
+       if (parent)
                bio_endio(&parent->bio);
-               return;
-       }
-
-       closure_put(cl);
+       else
+               closure_put(cl);
 }
 
 static void init_append_extent(struct bch_write_op *op,
@@ -860,7 +937,6 @@ static void init_append_extent(struct bch_write_op *op,
                               struct bversion version,
                               struct bch_extent_crc_unpacked crc)
 {
-       struct bch_fs *c = op->c;
        struct bkey_i_extent *e;
 
        op->pos.offset += crc.uncompressed_size;
@@ -875,7 +951,7 @@ static void init_append_extent(struct bch_write_op *op,
            crc.nonce)
                bch2_extent_crc_append(&e->k_i, crc);
 
-       bch2_alloc_sectors_append_ptrs_inlined(c, wp, &e->k_i, crc.compressed_size,
+       bch2_alloc_sectors_append_ptrs_inlined(op->c, wp, &e->k_i, crc.compressed_size,
                                       op->flags & BCH_WRITE_CACHED);
 
        bch2_keylist_push(&op->insert_keys);
@@ -1357,18 +1433,19 @@ static void bch2_nocow_write_convert_unwritten(struct bch_write_op *op)
                                     bkey_start_pos(&orig->k), orig->k.p,
                                     BTREE_ITER_INTENT, k,
                                     NULL, NULL, BTREE_INSERT_NOFAIL, ({
-                       BUG_ON(bkey_ge(bkey_start_pos(k.k), orig->k.p));
-
                        bch2_nocow_write_convert_one_unwritten(&trans, &iter, orig, k, op->new_i_size);
                }));
 
-               if (ret) {
+               if (ret && !bch2_err_matches(ret, EROFS)) {
                        struct bkey_i *k = bch2_keylist_front(&op->insert_keys);
 
                        bch_err_inum_offset_ratelimited(c,
                                k->k.p.inode, k->k.p.offset << 9,
                                "write error while doing btree update: %s",
                                bch2_err_str(ret));
+               }
+
+               if (ret) {
                        op->error = ret;
                        break;
                }
@@ -1406,7 +1483,7 @@ static void bch2_nocow_write(struct bch_write_op *op)
        struct {
                struct bpos     b;
                unsigned        gen;
-               two_state_lock_t *l;
+               struct nocow_lock_bucket *l;
        } buckets[BCH_REPLICAS_MAX];
        unsigned nr_buckets = 0;
        u32 snapshot;
@@ -1453,7 +1530,8 @@ retry:
                        buckets[nr_buckets].b = PTR_BUCKET_POS(c, ptr);
                        buckets[nr_buckets].gen = ptr->gen;
                        buckets[nr_buckets].l =
-                               bucket_nocow_lock(&c->nocow_locks, buckets[nr_buckets].b);
+                               bucket_nocow_lock(&c->nocow_locks,
+                                                 bucket_to_u64(buckets[nr_buckets].b));
 
                        prefetch(buckets[nr_buckets].l);
                        nr_buckets++;
@@ -1475,11 +1553,12 @@ retry:
 
                for (i = 0; i < nr_buckets; i++) {
                        struct bch_dev *ca = bch_dev_bkey_exists(c, buckets[i].b.inode);
-                       two_state_lock_t *l = buckets[i].l;
+                       struct nocow_lock_bucket *l = buckets[i].l;
                        bool stale;
 
-                       if (!bch2_two_state_trylock(l, BUCKET_NOCOW_LOCK_UPDATE))
-                               __bch2_bucket_nocow_lock(&c->nocow_locks, l, BUCKET_NOCOW_LOCK_UPDATE);
+                       __bch2_bucket_nocow_lock(&c->nocow_locks, l,
+                                                bucket_to_u64(buckets[i].b),
+                                                BUCKET_NOCOW_LOCK_UPDATE);
 
                        rcu_read_lock();
                        stale = gen_after(*bucket_gen(ca, buckets[i].b.offset), buckets[i].gen);
@@ -1583,7 +1662,7 @@ static void __bch2_write(struct bch_write_op *op)
 
        nofs_flags = memalloc_nofs_save();
 
-       if (unlikely(op->opts.nocow)) {
+       if (unlikely(op->opts.nocow && c->opts.nocow_enabled)) {
                bch2_nocow_write(op);
                if (op->flags & BCH_WRITE_DONE)
                        goto out_nofs_restore;
@@ -1627,26 +1706,20 @@ again:
                                              BCH_WRITE_ONLY_SPECIFIED_DEVS))
                                ? NULL : &op->cl, &wp));
                if (unlikely(ret)) {
-                       if (ret == -EAGAIN)
+                       if (bch2_err_matches(ret, BCH_ERR_operation_blocked))
                                break;
 
                        goto err;
                }
 
+               EBUG_ON(!wp);
+
+               bch2_open_bucket_get(c, wp, &op->open_buckets);
                ret = bch2_write_extent(op, wp, &bio);
 
-               if (ret >= 0)
-                       bch2_open_bucket_get(c, wp, &op->open_buckets);
                bch2_alloc_sectors_done_inlined(c, wp);
 err:
                if (ret <= 0) {
-                       if (!(op->flags & BCH_WRITE_SYNC)) {
-                               spin_lock(&wp->writes_lock);
-                               op->wp = wp;
-                               list_add_tail(&op->wp_list, &wp->writes);
-                               spin_unlock(&wp->writes_lock);
-                       }
-
                        op->flags |= BCH_WRITE_DONE;
 
                        if (ret < 0) {
@@ -1675,7 +1748,9 @@ err:
         * synchronously here if we weren't able to submit all of the IO at
         * once, as that signals backpressure to the caller.
         */
-       if ((op->flags & BCH_WRITE_SYNC) || !(op->flags & BCH_WRITE_DONE)) {
+       if ((op->flags & BCH_WRITE_SYNC) ||
+           (!(op->flags & BCH_WRITE_DONE) &&
+            !(op->flags & BCH_WRITE_IN_WORKER))) {
                closure_sync(&op->cl);
                __bch2_write_index(op);
 
@@ -1683,6 +1758,7 @@ err:
                        goto again;
                bch2_write_done(&op->cl);
        } else {
+               bch2_write_queue(op, wp);
                continue_at(&op->cl, bch2_write_index, NULL);
        }
 out_nofs_restore:
@@ -1697,6 +1773,9 @@ static void bch2_write_data_inline(struct bch_write_op *op, unsigned data_len)
        unsigned sectors;
        int ret;
 
+       op->flags |= BCH_WRITE_WROTE_DATA_INLINE;
+       op->flags |= BCH_WRITE_DONE;
+
        bch2_check_set_feature(op->c, BCH_FEATURE_inline_data);
 
        ret = bch2_keylist_realloc(&op->insert_keys, op->inline_keys,
@@ -1724,9 +1803,6 @@ static void bch2_write_data_inline(struct bch_write_op *op, unsigned data_len)
        set_bkey_val_bytes(&id->k, data_len);
        bch2_keylist_push(&op->insert_keys);
 
-       op->flags |= BCH_WRITE_WROTE_DATA_INLINE;
-       op->flags |= BCH_WRITE_DONE;
-
        __bch2_write_index(op);
 err:
        bch2_write_done(&op->cl);
@@ -1774,8 +1850,8 @@ void bch2_write(struct closure *cl)
        }
 
        if (c->opts.nochanges ||
-           !percpu_ref_tryget_live(&c->writes)) {
-               op->error = -EROFS;
+           !bch2_write_ref_tryget(c, BCH_WRITE_REF_write)) {
+               op->error = -BCH_ERR_erofs_no_writes;
                goto err;
        }
 
@@ -1853,10 +1929,12 @@ static void promote_free(struct bch_fs *c, struct promote_op *op)
 {
        int ret;
 
+       bch2_data_update_exit(&op->write);
+
        ret = rhashtable_remove_fast(&c->promote_table, &op->hash,
                                     bch_promote_params);
        BUG_ON(ret);
-       percpu_ref_put(&c->writes);
+       bch2_write_ref_put(c, BCH_WRITE_REF_promote);
        kfree_rcu(op, rcu);
 }
 
@@ -1868,8 +1946,6 @@ static void promote_done(struct bch_write_op *wop)
 
        bch2_time_stats_update(&c->times[BCH_TIME_data_promote],
                               op->start_time);
-
-       bch2_data_update_exit(&op->write);
        promote_free(c, op);
 }
 
@@ -1890,7 +1966,7 @@ static void promote_start(struct promote_op *op, struct bch_read_bio *rbio)
        bch2_data_update_read_done(&op->write, rbio->pick.crc);
 }
 
-static struct promote_op *__promote_alloc(struct bch_fs *c,
+static struct promote_op *__promote_alloc(struct btree_trans *trans,
                                          enum btree_id btree_id,
                                          struct bkey_s_c k,
                                          struct bpos pos,
@@ -1899,12 +1975,13 @@ static struct promote_op *__promote_alloc(struct bch_fs *c,
                                          unsigned sectors,
                                          struct bch_read_bio **rbio)
 {
+       struct bch_fs *c = trans->c;
        struct promote_op *op = NULL;
        struct bio *bio;
        unsigned pages = DIV_ROUND_UP(sectors, PAGE_SECTORS);
        int ret;
 
-       if (!percpu_ref_tryget_live(&c->writes))
+       if (!bch2_write_ref_tryget(c, BCH_WRITE_REF_promote))
                return NULL;
 
        op = kzalloc(sizeof(*op) + sizeof(struct bio_vec) * pages, GFP_NOIO);
@@ -1942,7 +2019,7 @@ static struct promote_op *__promote_alloc(struct bch_fs *c,
        bio = &op->write.op.wbio.bio;
        bio_init(bio, NULL, bio->bi_inline_vecs, pages, 0);
 
-       ret = bch2_data_update_init(c, &op->write,
+       ret = bch2_data_update_init(trans, NULL, &op->write,
                        writepoint_hashed((unsigned long) current),
                        opts,
                        (struct data_update_opts) {
@@ -1951,6 +2028,13 @@ static struct promote_op *__promote_alloc(struct bch_fs *c,
                                .write_flags    = BCH_WRITE_ALLOC_NOWAIT|BCH_WRITE_CACHED,
                        },
                        btree_id, k);
+       if (ret == -BCH_ERR_nocow_lock_blocked) {
+               ret = rhashtable_remove_fast(&c->promote_table, &op->hash,
+                                       bch_promote_params);
+               BUG_ON(ret);
+               goto err;
+       }
+
        BUG_ON(ret);
        op->write.op.end_io = promote_done;
 
@@ -1961,21 +2045,22 @@ err:
        kfree(*rbio);
        *rbio = NULL;
        kfree(op);
-       percpu_ref_put(&c->writes);
+       bch2_write_ref_put(c, BCH_WRITE_REF_promote);
        return NULL;
 }
 
 noinline
-static struct promote_op *promote_alloc(struct bch_fs *c,
-                                              struct bvec_iter iter,
-                                              struct bkey_s_c k,
-                                              struct extent_ptr_decoded *pick,
-                                              struct bch_io_opts opts,
-                                              unsigned flags,
-                                              struct bch_read_bio **rbio,
-                                              bool *bounce,
-                                              bool *read_full)
+static struct promote_op *promote_alloc(struct btree_trans *trans,
+                                       struct bvec_iter iter,
+                                       struct bkey_s_c k,
+                                       struct extent_ptr_decoded *pick,
+                                       struct bch_io_opts opts,
+                                       unsigned flags,
+                                       struct bch_read_bio **rbio,
+                                       bool *bounce,
+                                       bool *read_full)
 {
+       struct bch_fs *c = trans->c;
        bool promote_full = *read_full || READ_ONCE(c->promote_whole_extents);
        /* data might have to be decompressed in the write path: */
        unsigned sectors = promote_full
@@ -1989,7 +2074,7 @@ static struct promote_op *promote_alloc(struct bch_fs *c,
        if (!should_promote(c, k, pos, opts, flags))
                return NULL;
 
-       promote = __promote_alloc(c,
+       promote = __promote_alloc(trans,
                                  k.k->type == KEY_TYPE_reflink_v
                                  ? BTREE_ID_reflink
                                  : BTREE_ID_extents,
@@ -2275,7 +2360,7 @@ static void __bch2_read_endio(struct work_struct *work)
        }
 
        csum = bch2_checksum_bio(c, crc.csum_type, nonce, src);
-       if (bch2_crc_cmp(csum, rbio->pick.crc.csum))
+       if (bch2_crc_cmp(csum, rbio->pick.crc.csum) && !c->opts.no_data_io)
                goto csum_err;
 
        /*
@@ -2596,7 +2681,7 @@ retry_pick:
        }
 
        if (orig->opts.promote_target)
-               promote = promote_alloc(c, iter, k, &pick, orig->opts, flags,
+               promote = promote_alloc(trans, iter, k, &pick, orig->opts, flags,
                                        &rbio, &bounce, &read_full);
 
        if (!read_full) {
@@ -2726,10 +2811,21 @@ get_bio:
                             bio_sectors(&rbio->bio));
                bio_set_dev(&rbio->bio, ca->disk_sb.bdev);
 
-               if (likely(!(flags & BCH_READ_IN_RETRY)))
-                       submit_bio(&rbio->bio);
-               else
-                       submit_bio_wait(&rbio->bio);
+               if (unlikely(c->opts.no_data_io)) {
+                       if (likely(!(flags & BCH_READ_IN_RETRY)))
+                               bio_endio(&rbio->bio);
+               } else {
+                       if (likely(!(flags & BCH_READ_IN_RETRY)))
+                               submit_bio(&rbio->bio);
+                       else
+                               submit_bio_wait(&rbio->bio);
+               }
+
+               /*
+                * We just submitted IO which may block, we expect relock fail
+                * events and shouldn't count them:
+                */
+               trans->notrace_relock_fail = true;
        } else {
                /* Attempting reconstruct read: */
                if (bch2_ec_read_extent(c, rbio)) {
@@ -2905,11 +3001,6 @@ void bch2_fs_io_exit(struct bch_fs *c)
 
 int bch2_fs_io_init(struct bch_fs *c)
 {
-       unsigned i;
-
-       for (i = 0; i < ARRAY_SIZE(c->nocow_locks.l); i++)
-               two_state_lock_init(&c->nocow_locks.l[i]);
-
        if (bioset_init(&c->bio_read, 1, offsetof(struct bch_read_bio, bio),
                        BIOSET_NEED_BVECS) ||
            bioset_init(&c->bio_read_split, 1, offsetof(struct bch_read_bio, bio),