]> git.sesse.net Git - bcachefs-tools-debian/blobdiff - libbcachefs/bcachefs.h
Update bcachefs sources to 24c6361e20 bcachefs: Fix a trans path overflow in bch2_btr...
[bcachefs-tools-debian] / libbcachefs / bcachefs.h
index 505777ba8b54a40e7a724350704473e1d4f77868..ccac2a3fcdf733aa3882dd4bace4b9265abfc551 100644 (file)
  */
 
 #undef pr_fmt
+#ifdef __KERNEL__
 #define pr_fmt(fmt) "bcachefs: %s() " fmt "\n", __func__
+#else
+#define pr_fmt(fmt) "%s() " fmt "\n", __func__
+#endif
 
+#include <linux/backing-dev-defs.h>
 #include <linux/bug.h>
 #include <linux/bio.h>
 #include <linux/closure.h>
 #include <linux/zstd.h>
 
 #include "bcachefs_format.h"
+#include "errcode.h"
 #include "fifo.h"
 #include "opts.h"
 #include "util.h"
 #define dynamic_fault(...)             0
 #define race_fault(...)                        0
 
+#define trace_and_count(_c, _name, ...)                                        \
+do {                                                                   \
+       this_cpu_inc((_c)->counters[BCH_COUNTER_##_name]);              \
+       trace_##_name(__VA_ARGS__);                                     \
+} while (0)
+
 #define bch2_fs_init_fault(name)                                       \
        dynamic_fault("bcachefs:bch_fs_init:" name)
 #define bch2_meta_read_fault(name)                                     \
         dynamic_fault("bcachefs:meta:write:" name)
 
 #ifdef __KERNEL__
-#define bch2_fmt(_c, fmt)              "bcachefs (%s): " fmt "\n", ((_c)->name)
+#define bch2_log_msg(_c, fmt)          "bcachefs (%s): " fmt, ((_c)->name)
+#define bch2_fmt(_c, fmt)              bch2_log_msg(_c, fmt "\n")
 #define bch2_fmt_inum(_c, _inum, fmt)  "bcachefs (%s inum %llu): " fmt "\n", ((_c)->name), (_inum)
 #else
+#define bch2_log_msg(_c, fmt)          fmt
 #define bch2_fmt(_c, fmt)              fmt "\n"
 #define bch2_fmt_inum(_c, _inum, fmt)  "inum %llu: " fmt "\n", (_inum)
 #endif
@@ -259,7 +273,14 @@ do {                                                                       \
        BCH_DEBUG_PARAM(btree_gc_rewrite_disabled,                      \
                "Disables rewriting of btree nodes during mark and sweep")\
        BCH_DEBUG_PARAM(btree_shrinker_disabled,                        \
-               "Disables the shrinker callback for the btree node cache")
+               "Disables the shrinker callback for the btree node cache")\
+       BCH_DEBUG_PARAM(verify_btree_ondisk,                            \
+               "Reread btree nodes at various points to verify the "   \
+               "mergesort in the read path against modifications "     \
+               "done in memory")                                       \
+       BCH_DEBUG_PARAM(verify_all_btree_replicas,                      \
+               "When reading btree nodes, read all replicas and "      \
+               "compare them")
 
 /* Parameters that should only be compiled in in debug mode: */
 #define BCH_DEBUG_PARAMS_DEBUG()                                       \
@@ -268,15 +289,8 @@ do {                                                                       \
                "significantly affect performance")                     \
        BCH_DEBUG_PARAM(debug_check_iterators,                          \
                "Enables extra verification for btree iterators")       \
-       BCH_DEBUG_PARAM(debug_check_bkeys,                              \
-               "Run bkey_debugcheck (primarily checking GC/allocation "\
-               "information) when iterating over keys")                \
        BCH_DEBUG_PARAM(debug_check_btree_accounting,                   \
                "Verify btree accounting for keys within a node")       \
-       BCH_DEBUG_PARAM(verify_btree_ondisk,                            \
-               "Reread btree nodes at various points to verify the "   \
-               "mergesort in the read path against modifications "     \
-               "done in memory")                                       \
        BCH_DEBUG_PARAM(journal_seq_verify,                             \
                "Store the journal sequence number in the version "     \
                "number of every btree key, and verify that btree "     \
@@ -316,17 +330,18 @@ BCH_DEBUG_PARAMS_DEBUG()
 #define BCH_TIME_STATS()                       \
        x(btree_node_mem_alloc)                 \
        x(btree_node_split)                     \
+       x(btree_node_compact)                   \
+       x(btree_node_merge)                     \
        x(btree_node_sort)                      \
        x(btree_node_read)                      \
+       x(btree_interior_update_foreground)     \
+       x(btree_interior_update_total)          \
        x(btree_gc)                             \
-       x(btree_lock_contended_read)            \
-       x(btree_lock_contended_intent)          \
-       x(btree_lock_contended_write)           \
        x(data_write)                           \
        x(data_read)                            \
        x(data_promote)                         \
-       x(journal_write)                        \
-       x(journal_delay)                        \
+       x(journal_flush_write)                  \
+       x(journal_noflush_write)                \
        x(journal_flush_seq)                    \
        x(blocked_journal)                      \
        x(blocked_allocate)                     \
@@ -342,6 +357,7 @@ enum bch_time_stats {
 #include "alloc_types.h"
 #include "btree_types.h"
 #include "buckets_types.h"
+#include "buckets_waiting_for_journal_types.h"
 #include "clock_types.h"
 #include "ec_types.h"
 #include "journal_types.h"
@@ -349,6 +365,7 @@ enum bch_time_stats {
 #include "quota_types.h"
 #include "rebalance_types.h"
 #include "replicas_types.h"
+#include "subvolume_types.h"
 #include "super_types.h"
 
 /* Number of nodes btree coalesce will try to coalesce at once */
@@ -369,17 +386,22 @@ enum gc_phase {
        GC_PHASE_START,
        GC_PHASE_SB,
 
-       GC_PHASE_BTREE_EC,
-       GC_PHASE_BTREE_EXTENTS,
-       GC_PHASE_BTREE_INODES,
-       GC_PHASE_BTREE_DIRENTS,
-       GC_PHASE_BTREE_XATTRS,
-       GC_PHASE_BTREE_ALLOC,
-       GC_PHASE_BTREE_QUOTAS,
-       GC_PHASE_BTREE_REFLINK,
+       GC_PHASE_BTREE_stripes,
+       GC_PHASE_BTREE_extents,
+       GC_PHASE_BTREE_inodes,
+       GC_PHASE_BTREE_dirents,
+       GC_PHASE_BTREE_xattrs,
+       GC_PHASE_BTREE_alloc,
+       GC_PHASE_BTREE_quotas,
+       GC_PHASE_BTREE_reflink,
+       GC_PHASE_BTREE_subvolumes,
+       GC_PHASE_BTREE_snapshots,
+       GC_PHASE_BTREE_lru,
+       GC_PHASE_BTREE_freespace,
+       GC_PHASE_BTREE_need_discard,
+       GC_PHASE_BTREE_backpointers,
 
        GC_PHASE_PENDING_DELETE,
-       GC_PHASE_ALLOC,
 };
 
 struct gc_pos {
@@ -388,6 +410,14 @@ struct gc_pos {
        unsigned                level;
 };
 
+struct reflink_gc {
+       u64             offset;
+       u32             size;
+       u32             refcount;
+};
+
+typedef GENRADIX(struct reflink_gc) reflink_gc_table;
+
 struct io_count {
        u64                     sectors[2][BCH_DATA_NR];
 };
@@ -413,6 +443,7 @@ struct bch_dev {
        struct bch_sb_handle    disk_sb;
        struct bch_sb           *sb_read_scratch;
        int                     sb_write_error;
+       dev_t                   dev;
 
        struct bch_devs_mask    self;
 
@@ -425,54 +456,34 @@ struct bch_dev {
         * gc_lock, for device resize - holding any is sufficient for access:
         * Or rcu_read_lock(), but only for ptr_stale():
         */
-       struct bucket_array __rcu *buckets[2];
+       struct bucket_array __rcu *buckets_gc;
+       struct bucket_gens __rcu *bucket_gens;
+       u8                      *oldest_gen;
        unsigned long           *buckets_nouse;
        struct rw_semaphore     bucket_lock;
 
-       struct bch_dev_usage __percpu *usage[2];
+       struct bch_dev_usage            *usage_base;
+       struct bch_dev_usage __percpu   *usage[JOURNAL_BUF_NR];
+       struct bch_dev_usage __percpu   *usage_gc;
 
        /* Allocator: */
-       struct task_struct __rcu *alloc_thread;
+       u64                     new_fs_bucket_idx;
+       u64                     bucket_alloc_trans_early_cursor;
 
-       /*
-        * free: Buckets that are ready to be used
-        *
-        * free_inc: Incoming buckets - these are buckets that currently have
-        * cached data in them, and we can't reuse them until after we write
-        * their new gen to disk. After prio_write() finishes writing the new
-        * gens/prios, they'll be moved to the free list (and possibly discarded
-        * in the process)
-        */
-       alloc_fifo              free[RESERVE_NR];
-       alloc_fifo              free_inc;
+       unsigned                nr_open_buckets;
+       unsigned                nr_btree_reserve;
 
        open_bucket_idx_t       open_buckets_partial[OPEN_BUCKETS_COUNT];
        open_bucket_idx_t       open_buckets_partial_nr;
 
-       size_t                  fifo_last_bucket;
-
-       /* last calculated minimum prio */
-       u16                     max_last_bucket_io[2];
-
        size_t                  inc_gen_needs_gc;
        size_t                  inc_gen_really_needs_gc;
-
-       /*
-        * XXX: this should be an enum for allocator state, so as to include
-        * error state
-        */
-       enum {
-               ALLOCATOR_STOPPED,
-               ALLOCATOR_RUNNING,
-               ALLOCATOR_BLOCKED,
-               ALLOCATOR_BLOCKED_FULL,
-       }                       allocator_state;
-
-       alloc_heap              alloc_heap;
+       size_t                  buckets_waiting_on_journal;
 
        atomic64_t              rebalance_work;
 
        struct journal_device   journal;
+       u64                     prev_journal_sector;
 
        struct work_struct      io_error_work;
 
@@ -489,37 +500,48 @@ struct bch_dev {
 
 enum {
        /* startup: */
-       BCH_FS_ALLOC_READ_DONE,
-       BCH_FS_ALLOC_CLEAN,
-       BCH_FS_ALLOCATOR_RUNNING,
-       BCH_FS_ALLOCATOR_STOPPING,
-       BCH_FS_INITIAL_GC_DONE,
-       BCH_FS_BTREE_INTERIOR_REPLAY_DONE,
-       BCH_FS_FSCK_DONE,
        BCH_FS_STARTED,
+       BCH_FS_MAY_GO_RW,
        BCH_FS_RW,
+       BCH_FS_WAS_RW,
 
        /* shutdown: */
        BCH_FS_STOPPING,
        BCH_FS_EMERGENCY_RO,
        BCH_FS_WRITE_DISABLE_COMPLETE,
+       BCH_FS_CLEAN_SHUTDOWN,
+
+       /* fsck passes: */
+       BCH_FS_TOPOLOGY_REPAIR_DONE,
+       BCH_FS_INITIAL_GC_DONE,         /* kill when we enumerate fsck passes */
+       BCH_FS_CHECK_LRUS_DONE,
+       BCH_FS_CHECK_BACKPOINTERS_DONE,
+       BCH_FS_CHECK_ALLOC_TO_LRU_REFS_DONE,
+       BCH_FS_FSCK_DONE,
+       BCH_FS_INITIAL_GC_UNFIXED,      /* kill when we enumerate fsck errors */
+       BCH_FS_NEED_ANOTHER_GC,
+
+       BCH_FS_HAVE_DELETED_SNAPSHOTS,
 
        /* errors: */
        BCH_FS_ERROR,
+       BCH_FS_TOPOLOGY_ERROR,
        BCH_FS_ERRORS_FIXED,
-
-       /* misc: */
-       BCH_FS_FIXED_GENS,
-       BCH_FS_NEED_ALLOC_WRITE,
-       BCH_FS_REBUILD_REPLICAS,
-       BCH_FS_HOLD_BTREE_WRITES,
+       BCH_FS_ERRORS_NOT_FIXED,
 };
 
 struct btree_debug {
        unsigned                id;
-       struct dentry           *btree;
-       struct dentry           *btree_format;
-       struct dentry           *failed;
+};
+
+#define BCH_TRANSACTIONS_NR 128
+
+struct btree_transaction_stats {
+       struct mutex            lock;
+       struct time_stats       lock_hold_times;
+       unsigned                nr_max_paths;
+       unsigned                max_mem;
+       char                    *max_paths_text;
 };
 
 struct bch_fs_pcpu {
@@ -537,25 +559,51 @@ struct journal_seq_blacklist_table {
 
 struct journal_keys {
        struct journal_key {
+               u64             journal_seq;
+               u32             journal_offset;
                enum btree_id   btree_id:8;
                unsigned        level:8;
+               bool            allocated;
+               bool            overwritten;
                struct bkey_i   *k;
-               u32             journal_seq;
-               u32             journal_offset;
        }                       *d;
+       /*
+        * Gap buffer: instead of all the empty space in the array being at the
+        * end of the buffer - from @nr to @size - the empty space is at @gap.
+        * This means that sequential insertions are O(n) instead of O(n^2).
+        */
+       size_t                  gap;
        size_t                  nr;
-       u64                     journal_seq_base;
+       size_t                  size;
 };
 
-struct btree_iter_buf {
-       struct btree_iter       *iter;
+struct btree_path_buf {
+       struct btree_path       *path;
 };
 
+#define REPLICAS_DELTA_LIST_MAX        (1U << 16)
+
+struct snapshot_t {
+       u32                     parent;
+       u32                     children[2];
+       u32                     subvol; /* Nonzero only if a subvolume points to this node: */
+       u32                     equiv;
+};
+
+typedef struct {
+       u32             subvol;
+       u64             inum;
+} subvol_inum;
+
+#define BCACHEFS_ROOT_SUBVOL_INUM                                      \
+       ((subvol_inum) { BCACHEFS_ROOT_SUBVOL,  BCACHEFS_ROOT_INO })
+
 struct bch_fs {
        struct closure          cl;
 
        struct list_head        list;
        struct kobject          kobj;
+       struct kobject          counters_kobj;
        struct kobject          internal;
        struct kobject          opts_dir;
        struct kobject          time_stats;
@@ -564,6 +612,7 @@ struct bch_fs {
        int                     minor;
        struct device           *chardev;
        struct super_block      *vfs_sb;
+       dev_t                   dev;
        char                    name[40];
 
        /* ro/rw, add/remove/resize devices: */
@@ -578,8 +627,12 @@ struct bch_fs {
        struct bch_replicas_cpu replicas;
        struct bch_replicas_cpu replicas_gc;
        struct mutex            replicas_gc_lock;
+       mempool_t               replicas_delta_pool;
 
+       struct journal_entry_res btree_root_journal_res;
        struct journal_entry_res replicas_journal_res;
+       struct journal_entry_res clock_journal_res;
+       struct journal_entry_res dev_usage_journal_res;
 
        struct bch_disk_groups_cpu __rcu *disk_groups;
 
@@ -591,7 +644,7 @@ struct bch_fs {
                uuid_le         user_uuid;
 
                u16             version;
-               u16             encoded_extent_max;
+               u16             version_min;
 
                u8              nr_devices;
                u8              clean;
@@ -600,11 +653,13 @@ struct bch_fs {
 
                u64             time_base_lo;
                u32             time_base_hi;
-               u32             time_precision;
+               unsigned        time_units_per_sec;
+               unsigned        nsec_per_time_unit;
                u64             features;
                u64             compat;
        }                       sb;
 
+
        struct bch_sb_handle    disk_sb;
 
        unsigned short          block_bits;     /* ilog2(block_size) */
@@ -614,8 +669,18 @@ struct bch_fs {
        struct closure          sb_write;
        struct mutex            sb_lock;
 
+       /* snapshot.c: */
+       GENRADIX(struct snapshot_t) snapshots;
+       struct bch_snapshot_table __rcu *snapshot_table;
+       struct mutex            snapshot_table_lock;
+       struct work_struct      snapshot_delete_work;
+       struct work_struct      snapshot_wait_for_pagecache_and_delete_work;
+       snapshot_id_list        snapshots_unlinked;
+       struct mutex            snapshots_unlinked_lock;
+
        /* BTREE CACHE */
        struct bio_set          btree_bio;
+       struct workqueue_struct *io_complete_wq;
 
        struct btree_root       btree_roots[BTREE_ID_NR];
        struct mutex            btree_root_lock;
@@ -645,21 +710,22 @@ struct bch_fs {
        /* btree_iter.c: */
        struct mutex            btree_trans_lock;
        struct list_head        btree_trans_list;
-       mempool_t               btree_iters_pool;
-       struct btree_iter_buf  __percpu *btree_iters_bufs;
+       mempool_t               btree_paths_pool;
+       mempool_t               btree_trans_mem_pool;
+       struct btree_path_buf  __percpu *btree_paths_bufs;
 
        struct srcu_struct      btree_trans_barrier;
+       bool                    btree_trans_barrier_initialized;
 
        struct btree_key_cache  btree_key_cache;
+       unsigned                btree_key_cache_btrees;
 
-       struct workqueue_struct *wq;
+       struct workqueue_struct *btree_update_wq;
+       struct workqueue_struct *btree_io_complete_wq;
        /* copygc needs its own workqueue for index updates.. */
        struct workqueue_struct *copygc_wq;
 
        /* ALLOCATION */
-       struct delayed_work     pd_controllers_update;
-       unsigned                pd_controllers_update_seconds;
-
        struct bch_devs_mask    rw_devs[BCH_DATA_NR];
 
        u64                     capacity; /* sectors */
@@ -683,18 +749,11 @@ struct bch_fs {
        struct bch_fs_usage             *usage_base;
        struct bch_fs_usage __percpu    *usage[JOURNAL_BUF_NR];
        struct bch_fs_usage __percpu    *usage_gc;
+       u64 __percpu            *online_reserved;
 
        /* single element mempool: */
        struct mutex            usage_scratch_lock;
-       struct bch_fs_usage     *usage_scratch;
-
-       /*
-        * When we invalidate buckets, we use both the priority and the amount
-        * of good data to determine which buckets to reuse first - to weight
-        * those together consistently we keep track of the smallest nonzero
-        * priority of any bucket.
-        */
-       struct bucket_clock     bucket_clock[2];
+       struct bch_fs_usage_online *usage_scratch;
 
        struct io_clock         io_clock[2];
 
@@ -708,10 +767,12 @@ struct bch_fs {
        struct closure_waitlist freelist_wait;
        u64                     blocked_allocate;
        u64                     blocked_allocate_open_bucket;
+
        open_bucket_idx_t       open_buckets_freelist;
        open_bucket_idx_t       open_buckets_nr_free;
        struct closure_waitlist open_buckets_wait;
        struct open_bucket      open_buckets[OPEN_BUCKETS_COUNT];
+       open_bucket_idx_t       open_buckets_hash[OPEN_BUCKETS_COUNT];
 
        struct write_point      btree_write_point;
        struct write_point      rebalance_write_point;
@@ -721,16 +782,23 @@ struct bch_fs {
        struct mutex            write_points_hash_lock;
        unsigned                write_points_nr;
 
+       struct buckets_waiting_for_journal buckets_waiting_for_journal;
+       struct work_struct      discard_work;
+       struct work_struct      invalidate_work;
+
        /* GARBAGE COLLECTION */
        struct task_struct      *gc_thread;
        atomic_t                kick_gc;
        unsigned long           gc_count;
 
+       enum btree_id           gc_gens_btree;
+       struct bpos             gc_gens_pos;
+
        /*
         * Tracks GC's progress - everything in the range [ZERO_KEY..gc_cur_pos]
         * has been marked by GC.
         *
-        * gc_cur_phase is a superset of btree_ids (BTREE_ID_EXTENTS etc.)
+        * gc_cur_phase is a superset of btree_ids (BTREE_ID_extents etc.)
         *
         * Protected by gc_pos_lock. Only written to by GC thread, so GC thread
         * can read without a lock.
@@ -743,9 +811,9 @@ struct bch_fs {
         * it's not while a gc is in progress.
         */
        struct rw_semaphore     gc_lock;
+       struct mutex            gc_gens_lock;
 
        /* IO PATH */
-       struct semaphore        io_in_flight;
        struct bio_set          bio_read;
        struct bio_set          bio_read_split;
        struct bio_set          bio_write;
@@ -772,12 +840,18 @@ struct bch_fs {
        /* COPYGC */
        struct task_struct      *copygc_thread;
        copygc_heap             copygc_heap;
-       struct bch_pd_controller copygc_pd;
        struct write_point      copygc_write_point;
-       u64                     copygc_threshold;
+       s64                     copygc_wait;
+       bool                    copygc_running;
+       wait_queue_head_t       copygc_running_wq;
+
+       /* DATA PROGRESS STATS */
+       struct list_head        data_progress_list;
+       struct mutex            data_progress_lock;
 
        /* STRIPES: */
-       GENRADIX(struct stripe) stripes[2];
+       GENRADIX(struct stripe) stripes;
+       GENRADIX(struct gc_stripe) gc_stripes;
 
        ec_stripes_heap         ec_stripes_heap;
        spinlock_t              ec_stripes_heap_lock;
@@ -799,14 +873,17 @@ struct bch_fs {
 
        /* REFLINK */
        u64                     reflink_hint;
+       reflink_gc_table        reflink_gc_table;
+       size_t                  reflink_gc_nr;
 
        /* VFS IO PATH - fs-io.c */
        struct bio_set          writepage_bioset;
        struct bio_set          dio_write_bioset;
        struct bio_set          dio_read_bioset;
 
-       struct bio_list         btree_write_error_list;
-       struct work_struct      btree_write_error_work;
+
+       atomic64_t              btree_writes_nr;
+       atomic64_t              btree_writes_sectors;
        spinlock_t              btree_write_error_lock;
 
        /* ERRORS */
@@ -818,13 +895,12 @@ struct bch_fs {
        struct bch_memquota_type quotas[QTYP_NR];
 
        /* DEBUG JUNK */
-       struct dentry           *debug;
+       struct dentry           *fs_debug_dir;
+       struct dentry           *btree_debug_dir;
        struct btree_debug      btree_debug[BTREE_ID_NR];
-#ifdef CONFIG_BCACHEFS_DEBUG
        struct btree            *verify_data;
        struct btree_node       *verify_ondisk;
        struct mutex            verify_lock;
-#endif
 
        u64                     *unused_inode_hints;
        unsigned                inode_shard_bits;
@@ -838,21 +914,24 @@ struct bch_fs {
        mempool_t               btree_bounce_pool;
 
        struct journal          journal;
-       struct list_head        journal_entries;
+       GENRADIX(struct journal_replay *) journal_entries;
+       u64                     journal_entries_base_seq;
        struct journal_keys     journal_keys;
+       struct list_head        journal_iters;
 
        u64                     last_bucket_seq_cleanup;
 
-       /* The rest of this all shows up in sysfs */
-       atomic_long_t           read_realloc_races;
-       atomic_long_t           extent_migrate_done;
-       atomic_long_t           extent_migrate_raced;
+       u64                     counters_on_mount[BCH_COUNTER_NR];
+       u64 __percpu            *counters;
 
        unsigned                btree_gc_periodic:1;
        unsigned                copy_gc_enabled:1;
        bool                    promote_whole_extents;
 
        struct time_stats       times[BCH_TIME_STAT_NR];
+
+       const char              *btree_transaction_fns[BCH_TRANSACTIONS_NR];
+       struct btree_transaction_stats btree_transaction_stats[BCH_TRANSACTIONS_NR];
 };
 
 static inline void bch2_set_ra_pages(struct bch_fs *c, unsigned ra_pages)
@@ -870,25 +949,43 @@ static inline unsigned bucket_bytes(const struct bch_dev *ca)
 
 static inline unsigned block_bytes(const struct bch_fs *c)
 {
-       return c->opts.block_size << 9;
+       return c->opts.block_size;
+}
+
+static inline unsigned block_sectors(const struct bch_fs *c)
+{
+       return c->opts.block_size >> 9;
+}
+
+static inline size_t btree_sectors(const struct bch_fs *c)
+{
+       return c->opts.btree_node_size >> 9;
 }
 
-static inline struct timespec64 bch2_time_to_timespec(struct bch_fs *c, u64 time)
+static inline bool btree_id_cached(const struct bch_fs *c, enum btree_id btree)
 {
-       return ns_to_timespec64(time * c->sb.time_precision + c->sb.time_base_lo);
+       return c->btree_key_cache_btrees & (1U << btree);
 }
 
-static inline s64 timespec_to_bch2_time(struct bch_fs *c, struct timespec64 ts)
+static inline struct timespec64 bch2_time_to_timespec(const struct bch_fs *c, s64 time)
 {
-       s64 ns = timespec64_to_ns(&ts) - c->sb.time_base_lo;
+       struct timespec64 t;
+       s32 rem;
+
+       time += c->sb.time_base_lo;
 
-       if (c->sb.time_precision == 1)
-               return ns;
+       t.tv_sec = div_s64_rem(time, c->sb.time_units_per_sec, &rem);
+       t.tv_nsec = rem * c->sb.nsec_per_time_unit;
+       return t;
+}
 
-       return div_s64(ns, c->sb.time_precision);
+static inline s64 timespec_to_bch2_time(const struct bch_fs *c, struct timespec64 ts)
+{
+       return (ts.tv_sec * c->sb.time_units_per_sec +
+               (int) ts.tv_nsec / c->sb.nsec_per_time_unit) - c->sb.time_base_lo;
 }
 
-static inline s64 bch2_current_time(struct bch_fs *c)
+static inline s64 bch2_current_time(const struct bch_fs *c)
 {
        struct timespec64 now;