bcachefs: Allocator refactoring
authorKent Overstreet <kent.overstreet@gmail.com>
Sun, 18 Apr 2021 00:37:04 +0000 (20:37 -0400)
committerKent Overstreet <kent.overstreet@linux.dev>
Sun, 22 Oct 2023 21:09:01 +0000 (17:09 -0400)
This uses the kthread_wait_freezable() macro to simplify a lot of the
allocator thread code, along with cleaning up bch2_invalidate_bucket2().

Signed-off-by: Kent Overstreet <kent.overstreet@gmail.com>
Signed-off-by: Kent Overstreet <kent.overstreet@linux.dev>
fs/bcachefs/alloc_background.c
fs/bcachefs/alloc_foreground.c
fs/bcachefs/trace.h

index ab60bf259b0c40f9a7ca92cfe7b2fbd7fbe5b50b..2d532fe4d30b0f2cdac3ef5bd089be0b1c1ba280 100644 (file)
@@ -441,50 +441,6 @@ out:
  * commands to the newly free buckets, then puts them on the various freelists.
  */
 
-/**
- * wait_buckets_available - wait on reclaimable buckets
- *
- * If there aren't enough available buckets to fill up free_inc, wait until
- * there are.
- */
-static int wait_buckets_available(struct bch_fs *c, struct bch_dev *ca)
-{
-       unsigned long gc_count = c->gc_count;
-       s64 available;
-       int ret = 0;
-
-       ca->allocator_state = ALLOCATOR_blocked;
-       closure_wake_up(&c->freelist_wait);
-
-       while (1) {
-               set_current_state(TASK_INTERRUPTIBLE);
-               if (kthread_should_stop()) {
-                       ret = 1;
-                       break;
-               }
-
-               if (gc_count != c->gc_count)
-                       ca->inc_gen_really_needs_gc = 0;
-
-               available  = dev_buckets_reclaimable(ca);
-               available -= ca->inc_gen_really_needs_gc;
-
-               available = max(available, 0LL);
-
-               if (available)
-                       break;
-
-               schedule();
-               try_to_freeze();
-       }
-
-       __set_current_state(TASK_RUNNING);
-       ca->allocator_state = ALLOCATOR_running;
-       closure_wake_up(&c->freelist_wait);
-
-       return ret;
-}
-
 static bool bch2_can_invalidate_bucket(struct bch_dev *ca, size_t b,
                                       struct bucket_mark m)
 {
@@ -502,11 +458,8 @@ static bool bch2_can_invalidate_bucket(struct bch_dev *ca, size_t b,
 
        gc_gen = bucket_gc_gen(bucket(ca, b));
 
-       if (gc_gen >= BUCKET_GC_GEN_MAX / 2)
-               ca->inc_gen_needs_gc++;
-
-       if (gc_gen >= BUCKET_GC_GEN_MAX)
-               ca->inc_gen_really_needs_gc++;
+       ca->inc_gen_needs_gc            += gc_gen >= BUCKET_GC_GEN_MAX / 2;
+       ca->inc_gen_really_needs_gc     += gc_gen >= BUCKET_GC_GEN_MAX;
 
        return gc_gen < BUCKET_GC_GEN_MAX;
 }
@@ -583,6 +536,8 @@ static void find_reclaimable_buckets_lru(struct bch_fs *c, struct bch_dev *ca)
                struct bucket_mark m = READ_ONCE(g->mark);
                unsigned key = bucket_sort_key(g, m, now, last_seq_ondisk);
 
+               cond_resched();
+
                if (!bch2_can_invalidate_bucket(ca, b, m))
                        continue;
 
@@ -599,8 +554,6 @@ static void find_reclaimable_buckets_lru(struct bch_fs *c, struct bch_dev *ca)
                                .key    = key,
                        };
                }
-
-               cond_resched();
        }
 
        if (e.nr)
@@ -693,6 +646,7 @@ static size_t find_reclaimable_buckets(struct bch_fs *c, struct bch_dev *ca)
        size_t i, nr = 0;
 
        ca->inc_gen_needs_gc                    = 0;
+       ca->inc_gen_really_needs_gc             = 0;
 
        switch (ca->mi.replacement) {
        case BCH_CACHE_REPLACEMENT_lru:
@@ -714,25 +668,6 @@ static size_t find_reclaimable_buckets(struct bch_fs *c, struct bch_dev *ca)
        return nr;
 }
 
-static inline long next_alloc_bucket(struct bch_dev *ca)
-{
-       struct alloc_heap_entry e, *top = ca->alloc_heap.data;
-
-       while (ca->alloc_heap.used) {
-               if (top->nr) {
-                       size_t b = top->bucket;
-
-                       top->bucket++;
-                       top->nr--;
-                       return b;
-               }
-
-               heap_pop(&ca->alloc_heap, e, bucket_alloc_cmp, NULL);
-       }
-
-       return -1;
-}
-
 /*
  * returns sequence number of most recent journal entry that updated this
  * bucket:
@@ -755,17 +690,56 @@ static u64 bucket_journal_seq(struct bch_fs *c, struct bucket_mark m)
        }
 }
 
-static int bch2_invalidate_one_bucket2(struct btree_trans *trans,
-                                      struct bch_dev *ca,
-                                      struct btree_iter *iter,
-                                      u64 *journal_seq, unsigned flags)
+static int bucket_invalidate_btree(struct btree_trans *trans,
+                                  struct bch_dev *ca, u64 b)
 {
        struct bch_fs *c = trans->c;
-       struct bkey_alloc_buf a;
+       struct bkey_alloc_buf *a;
        struct bkey_alloc_unpacked u;
        struct bucket *g;
        struct bucket_mark m;
-       bool invalidating_cached_data;
+       struct btree_iter *iter =
+               bch2_trans_get_iter(trans, BTREE_ID_alloc,
+                                   POS(ca->dev_idx, b),
+                                   BTREE_ITER_CACHED|
+                                   BTREE_ITER_CACHED_NOFILL|
+                                   BTREE_ITER_INTENT);
+       int ret;
+
+       a = bch2_trans_kmalloc(trans, sizeof(*a));
+       ret = PTR_ERR_OR_ZERO(a);
+       if (ret)
+               goto err;
+
+       ret = bch2_btree_iter_traverse(iter);
+       if (ret)
+               goto err;
+
+       percpu_down_read(&c->mark_lock);
+       g = bucket(ca, b);
+       m = READ_ONCE(g->mark);
+       u = alloc_mem_to_key(iter, g, m);
+       percpu_up_read(&c->mark_lock);
+
+       u.gen++;
+       u.data_type     = 0;
+       u.dirty_sectors = 0;
+       u.cached_sectors = 0;
+       u.read_time     = atomic64_read(&c->io_clock[READ].now);
+       u.write_time    = atomic64_read(&c->io_clock[WRITE].now);
+
+       bch2_alloc_pack(c, a, u);
+       bch2_trans_update(trans, iter, &a->k, BTREE_TRIGGER_BUCKET_INVALIDATE);
+err:
+       bch2_trans_iter_put(trans, iter);
+       return ret;
+}
+
+static int bch2_invalidate_one_bucket(struct bch_fs *c, struct bch_dev *ca,
+                                     u64 *journal_seq, unsigned flags)
+{
+       struct bucket *g;
+       struct bucket_mark m;
        size_t b;
        int ret = 0;
 
@@ -811,48 +785,12 @@ static int bch2_invalidate_one_bucket2(struct btree_trans *trans,
                goto out;
        }
 
-       bch2_btree_iter_set_pos(iter, POS(ca->dev_idx, b));
-retry:
-       ret = bch2_btree_iter_traverse(iter);
-       if (ret)
-               return ret;
-
-       percpu_down_read(&c->mark_lock);
-       g = bucket(ca, iter->pos.offset);
-       m = READ_ONCE(g->mark);
-       u = alloc_mem_to_key(iter, g, m);
-
-       percpu_up_read(&c->mark_lock);
-
-       invalidating_cached_data = u.cached_sectors != 0;
-
-       u.gen++;
-       u.data_type     = 0;
-       u.dirty_sectors = 0;
-       u.cached_sectors = 0;
-       u.read_time     = atomic64_read(&c->io_clock[READ].now);
-       u.write_time    = atomic64_read(&c->io_clock[WRITE].now);
-
-       bch2_alloc_pack(c, &a, u);
-       bch2_trans_update(trans, iter, &a.k,
-                         BTREE_TRIGGER_BUCKET_INVALIDATE);
-
-       /*
-        * XXX:
-        * when using deferred btree updates, we have journal reclaim doing
-        * btree updates and thus requiring the allocator to make forward
-        * progress, and here the allocator is requiring space in the journal -
-        * so we need a journal pre-reservation:
-        */
-       ret = bch2_trans_commit(trans, NULL,
-                               invalidating_cached_data ? journal_seq : NULL,
-                               BTREE_INSERT_NOUNLOCK|
-                               BTREE_INSERT_NOCHECK_RW|
-                               BTREE_INSERT_NOFAIL|
-                               BTREE_INSERT_JOURNAL_RESERVED|
-                               flags);
-       if (ret == -EINTR)
-               goto retry;
+       ret = bch2_trans_do(c, NULL, journal_seq,
+                           BTREE_INSERT_NOCHECK_RW|
+                           BTREE_INSERT_NOFAIL|
+                           BTREE_INSERT_JOURNAL_RESERVED|
+                           flags,
+                           bucket_invalidate_btree(&trans, ca, b));
 out:
        if (!ret) {
                /* remove from alloc_heap: */
@@ -894,28 +832,23 @@ out:
  */
 static int bch2_invalidate_buckets(struct bch_fs *c, struct bch_dev *ca)
 {
-       struct btree_trans trans;
-       struct btree_iter *iter;
        u64 journal_seq = 0;
        int ret = 0;
 
-       bch2_trans_init(&trans, c, 0, 0);
-       iter = bch2_trans_get_iter(&trans, BTREE_ID_alloc,
-                                  POS(ca->dev_idx, 0),
-                                  BTREE_ITER_CACHED|
-                                  BTREE_ITER_CACHED_NOFILL|
-                                  BTREE_ITER_INTENT);
-
        /* Only use nowait if we've already invalidated at least one bucket: */
        while (!ret &&
               !fifo_full(&ca->free_inc) &&
-              ca->alloc_heap.used)
-               ret = bch2_invalidate_one_bucket2(&trans, ca, iter, &journal_seq,
+              ca->alloc_heap.used) {
+               ret = bch2_invalidate_one_bucket(c, ca, &journal_seq,
                                (!fifo_empty(&ca->free_inc)
                                 ? BTREE_INSERT_NOWAIT : 0));
-
-       bch2_trans_iter_put(&trans, iter);
-       bch2_trans_exit(&trans);
+               /*
+                * We only want to batch up invalidates when they're going to
+                * require flushing the journal:
+                */
+               if (!journal_seq)
+                       break;
+       }
 
        /* If we used NOWAIT, don't return the error: */
        if (!fifo_empty(&ca->free_inc))
@@ -935,83 +868,72 @@ static int bch2_invalidate_buckets(struct bch_fs *c, struct bch_dev *ca)
        return 0;
 }
 
-static int push_invalidated_bucket(struct bch_fs *c, struct bch_dev *ca, size_t bucket)
+static void alloc_thread_set_state(struct bch_dev *ca, unsigned new_state)
+{
+       if (ca->allocator_state != new_state) {
+               ca->allocator_state = new_state;
+               closure_wake_up(&ca->fs->freelist_wait);
+       }
+}
+
+static int push_invalidated_bucket(struct bch_fs *c, struct bch_dev *ca, u64 b)
 {
        unsigned i;
        int ret = 0;
 
-       while (1) {
-               set_current_state(TASK_INTERRUPTIBLE);
-
-               spin_lock(&c->freelist_lock);
-               for (i = 0; i < RESERVE_NR; i++) {
-
-                       /*
-                        * Don't strand buckets on the copygc freelist until
-                        * after recovery is finished:
-                        */
-                       if (!test_bit(BCH_FS_STARTED, &c->flags) &&
-                           i == RESERVE_MOVINGGC)
-                               continue;
-
-                       if (fifo_push(&ca->free[i], bucket)) {
-                               fifo_pop(&ca->free_inc, bucket);
-
-                               closure_wake_up(&c->freelist_wait);
-                               ca->allocator_state = ALLOCATOR_running;
-
-                               spin_unlock(&c->freelist_lock);
-                               goto out;
-                       }
-               }
-
-               if (ca->allocator_state != ALLOCATOR_blocked_full) {
-                       ca->allocator_state = ALLOCATOR_blocked_full;
-                       closure_wake_up(&c->freelist_wait);
-               }
-
-               spin_unlock(&c->freelist_lock);
+       spin_lock(&c->freelist_lock);
+       for (i = 0; i < RESERVE_NR; i++) {
+               /*
+                * Don't strand buckets on the copygc freelist until
+                * after recovery is finished:
+                */
+               if (i == RESERVE_MOVINGGC &&
+                   !test_bit(BCH_FS_STARTED, &c->flags))
+                       continue;
 
-               if ((current->flags & PF_KTHREAD) &&
-                   kthread_should_stop()) {
+               if (fifo_push(&ca->free[i], b)) {
+                       fifo_pop(&ca->free_inc, b);
                        ret = 1;
                        break;
                }
-
-               schedule();
-               try_to_freeze();
        }
-out:
-       __set_current_state(TASK_RUNNING);
+       spin_unlock(&c->freelist_lock);
+
+       ca->allocator_state = ret
+               ? ALLOCATOR_running
+               : ALLOCATOR_blocked_full;
+       closure_wake_up(&c->freelist_wait);
        return ret;
 }
 
-/*
- * Pulls buckets off free_inc, discards them (if enabled), then adds them to
- * freelists, waiting until there's room if necessary:
- */
-static int discard_invalidated_buckets(struct bch_fs *c, struct bch_dev *ca)
+static void discard_one_bucket(struct bch_fs *c, struct bch_dev *ca, u64 b)
 {
-       while (!fifo_empty(&ca->free_inc)) {
-               size_t bucket = fifo_peek(&ca->free_inc);
-
-               if (ca->mi.discard &&
-                   bdev_max_discard_sectors(ca->disk_sb.bdev))
-                       blkdev_issue_discard(ca->disk_sb.bdev,
-                                            bucket_to_sector(ca, bucket),
-                                            ca->mi.bucket_size, GFP_NOIO);
-
-               if (push_invalidated_bucket(c, ca, bucket))
-                       return 1;
-       }
+       if (ca->mi.discard &&
+           bdev_max_discard_sectors(ca->disk_sb.bdev))
+               blkdev_issue_discard(ca->disk_sb.bdev, bucket_to_sector(ca, b),
+                                    ca->mi.bucket_size, GFP_NOFS);
+}
 
-       return 0;
+static bool allocator_thread_running(struct bch_dev *ca)
+{
+       unsigned state = ca->mi.state == BCH_MEMBER_STATE_rw &&
+               test_bit(BCH_FS_ALLOCATOR_RUNNING, &ca->fs->flags)
+               ? ALLOCATOR_running
+               : ALLOCATOR_stopped;
+       alloc_thread_set_state(ca, state);
+       return state == ALLOCATOR_running;
 }
 
-static inline bool allocator_thread_running(struct bch_dev *ca)
+static int buckets_available(struct bch_dev *ca, unsigned long gc_count)
 {
-       return ca->mi.state == BCH_MEMBER_STATE_rw &&
-               test_bit(BCH_FS_ALLOCATOR_RUNNING, &ca->fs->flags);
+       s64 available = dev_buckets_reclaimable(ca) -
+               (gc_count == ca->fs->gc_count ? ca->inc_gen_really_needs_gc : 0);
+       bool ret = available > 0;
+
+       alloc_thread_set_state(ca, ret
+                              ? ALLOCATOR_running
+                              : ALLOCATOR_blocked);
+       return ret;
 }
 
 /**
@@ -1026,56 +948,29 @@ static int bch2_allocator_thread(void *arg)
 {
        struct bch_dev *ca = arg;
        struct bch_fs *c = ca->fs;
+       unsigned long gc_count = c->gc_count;
        size_t nr;
        int ret;
 
        set_freezable();
 
        while (1) {
-               if (!allocator_thread_running(ca)) {
-                       ca->allocator_state = ALLOCATOR_stopped;
-                       if (kthread_wait_freezable(allocator_thread_running(ca)))
-                               break;
-               }
-
-               ca->allocator_state = ALLOCATOR_running;
-
-               cond_resched();
-               if (kthread_should_stop())
-                       break;
-
-               pr_debug("discarding %zu invalidated buckets",
-                        fifo_used(&ca->free_inc));
-
-               ret = discard_invalidated_buckets(c, ca);
+               ret = kthread_wait_freezable(allocator_thread_running(ca));
                if (ret)
                        goto stop;
 
-               ret = bch2_invalidate_buckets(c, ca);
-               if (ret)
-                       goto stop;
-
-               if (!fifo_empty(&ca->free_inc))
-                       continue;
-
-               pr_debug("free_inc now empty");
-
-               while (1) {
+               while (!ca->alloc_heap.used) {
                        cond_resched();
-                       /*
-                        * Find some buckets that we can invalidate, either
-                        * they're completely unused, or only contain clean data
-                        * that's been written back to the backing device or
-                        * another cache tier
-                        */
 
-                       pr_debug("scanning for reclaimable buckets");
+                       ret = kthread_wait_freezable(buckets_available(ca, gc_count));
+                       if (ret)
+                               goto stop;
 
+                       gc_count = c->gc_count;
                        nr = find_reclaimable_buckets(c, ca);
 
-                       pr_debug("found %zu buckets", nr);
-
-                       trace_alloc_batch(ca, nr, ca->alloc_heap.size);
+                       trace_alloc_scan(ca, nr, ca->inc_gen_needs_gc,
+                                        ca->inc_gen_really_needs_gc);
 
                        if ((ca->inc_gen_needs_gc >= ALLOC_SCAN_BATCH(ca) ||
                             ca->inc_gen_really_needs_gc) &&
@@ -1083,33 +978,24 @@ static int bch2_allocator_thread(void *arg)
                                atomic_inc(&c->kick_gc);
                                wake_up_process(c->gc_thread);
                        }
+               }
 
-                       if (nr)
-                               break;
+               ret = bch2_invalidate_buckets(c, ca);
+               if (ret)
+                       goto stop;
 
-                       /*
-                        * If we found any buckets, we have to invalidate them
-                        * before we scan for more - but if we didn't find very
-                        * many we may want to wait on more buckets being
-                        * available so we don't spin:
-                        */
-                       ret = wait_buckets_available(c, ca);
+               while (!fifo_empty(&ca->free_inc)) {
+                       u64 b = fifo_peek(&ca->free_inc);
+
+                       discard_one_bucket(c, ca, b);
+
+                       ret = kthread_wait_freezable(push_invalidated_bucket(c, ca, b));
                        if (ret)
                                goto stop;
                }
-
-               pr_debug("%zu buckets to invalidate", nr);
-
-               /*
-                * alloc_heap is now full of newly-invalidated buckets: next,
-                * write out the new bucket gens:
-                */
        }
-
 stop:
-       pr_debug("alloc thread stopping (ret %i)", ret);
-       ca->allocator_state = ALLOCATOR_stopped;
-       closure_wake_up(&c->freelist_wait);
+       alloc_thread_set_state(ca, ALLOCATOR_stopped);
        return 0;
 }
 
index f2f392eeb54ade3c6a7466a5df370fb7b4c811a3..6bf4140477a04c8b7bce6d2b07c783f0a3523bf6 100644 (file)
@@ -1,57 +1,14 @@
 // SPDX-License-Identifier: GPL-2.0
 /*
- * Primary bucket allocation code
- *
  * Copyright 2012 Google, Inc.
  *
- * Allocation in bcache is done in terms of buckets:
- *
- * Each bucket has associated an 8 bit gen; this gen corresponds to the gen in
- * btree pointers - they must match for the pointer to be considered valid.
- *
- * Thus (assuming a bucket has no dirty data or metadata in it) we can reuse a
- * bucket simply by incrementing its gen.
- *
- * The gens (along with the priorities; it's really the gens are important but
- * the code is named as if it's the priorities) are written in an arbitrary list
- * of buckets on disk, with a pointer to them in the journal header.
- *
- * When we invalidate a bucket, we have to write its new gen to disk and wait
- * for that write to complete before we use it - otherwise after a crash we
- * could have pointers that appeared to be good but pointed to data that had
- * been overwritten.
- *
- * Since the gens and priorities are all stored contiguously on disk, we can
- * batch this up: We fill up the free_inc list with freshly invalidated buckets,
- * call prio_write(), and when prio_write() finishes we pull buckets off the
- * free_inc list and optionally discard them.
- *
- * free_inc isn't the only freelist - if it was, we'd often have to sleep while
- * priorities and gens were being written before we could allocate. c->free is a
- * smaller freelist, and buckets on that list are always ready to be used.
- *
- * If we've got discards enabled, that happens when a bucket moves from the
- * free_inc list to the free list.
- *
- * It's important to ensure that gens don't wrap around - with respect to
- * either the oldest gen in the btree or the gen on disk. This is quite
- * difficult to do in practice, but we explicitly guard against it anyways - if
- * a bucket is in danger of wrapping around we simply skip invalidating it that
- * time around, and we garbage collect or rewrite the priorities sooner than we
- * would have otherwise.
+ * Foreground allocator code: allocate buckets from freelist, and allocate in
+ * sector granularity from writepoints.
  *
  * bch2_bucket_alloc() allocates a single bucket from a specific device.
  *
  * bch2_bucket_alloc_set() allocates one or more buckets from different devices
  * in a given filesystem.
- *
- * invalidate_buckets() drives all the processes described above. It's called
- * from bch2_bucket_alloc() and a few other places that need to make sure free
- * buckets are ready.
- *
- * invalidate_buckets_(lru|fifo)() find buckets that are available to be
- * invalidated, and then invalidate them and stick them on the free_inc list -
- * in either lru or fifo order.
  */
 
 #include "bcachefs.h"
index 887c0adddf123203ed20b433512b058710db58cc..c6d98f4c50e7707929f842aa4947ce05cdb7dcf8 100644 (file)
@@ -380,24 +380,27 @@ DEFINE_EVENT(bch_fs, gc_cannot_inc_gens,
 
 /* Allocator */
 
-TRACE_EVENT(alloc_batch,
-       TP_PROTO(struct bch_dev *ca, size_t free, size_t total),
-       TP_ARGS(ca, free, total),
+TRACE_EVENT(alloc_scan,
+       TP_PROTO(struct bch_dev *ca, u64 found, u64 inc_gen, u64 inc_gen_skipped),
+       TP_ARGS(ca, found, inc_gen, inc_gen_skipped),
 
        TP_STRUCT__entry(
-               __array(char,           uuid,   16      )
-               __field(size_t,         free            )
-               __field(size_t,         total           )
+               __field(dev_t,          dev             )
+               __field(u64,            found           )
+               __field(u64,            inc_gen         )
+               __field(u64,            inc_gen_skipped )
        ),
 
        TP_fast_assign(
-               memcpy(__entry->uuid, ca->uuid.b, 16);
-               __entry->free = free;
-               __entry->total = total;
+               __entry->dev            = ca->disk_sb.bdev->bd_dev;
+               __entry->found          = found;
+               __entry->inc_gen        = inc_gen;
+               __entry->inc_gen_skipped = inc_gen_skipped;
        ),
 
-       TP_printk("%pU free %zu total %zu",
-               __entry->uuid, __entry->free, __entry->total)
+       TP_printk("%d,%d found %llu inc_gen %llu inc_gen_skipped %llu",
+                 MAJOR(__entry->dev), MINOR(__entry->dev),
+                 __entry->found, __entry->inc_gen, __entry->inc_gen_skipped)
 );
 
 TRACE_EVENT(invalidate,
@@ -417,8 +420,10 @@ TRACE_EVENT(invalidate,
        ),
 
        TP_printk("invalidated %u sectors at %d,%d sector=%llu",
-                 __entry->sectors, MAJOR(__entry->dev),
-                 MINOR(__entry->dev), __entry->offset)
+                 __entry->sectors,
+                 MAJOR(__entry->dev),
+                 MINOR(__entry->dev),
+                 __entry->offset)
 );
 
 DECLARE_EVENT_CLASS(bucket_alloc,
@@ -426,16 +431,18 @@ DECLARE_EVENT_CLASS(bucket_alloc,
        TP_ARGS(ca, reserve),
 
        TP_STRUCT__entry(
-               __array(char,                   uuid,   16)
-               __field(enum alloc_reserve,     reserve   )
+               __field(dev_t,                  dev     )
+               __field(enum alloc_reserve,     reserve )
        ),
 
        TP_fast_assign(
-               memcpy(__entry->uuid, ca->uuid.b, 16);
-               __entry->reserve = reserve;
+               __entry->dev            = ca->disk_sb.bdev->bd_dev;
+               __entry->reserve        = reserve;
        ),
 
-       TP_printk("%pU reserve %d", __entry->uuid, __entry->reserve)
+       TP_printk("%d,%d reserve %d",
+                 MAJOR(__entry->dev), MINOR(__entry->dev),
+                 __entry->reserve)
 );
 
 DEFINE_EVENT(bucket_alloc, bucket_alloc,