1
0
mirror of https://github.com/Zygo/bees.git synced 2025-05-17 21:35:45 +02:00

extent scan: introduce SCAN_MODE_EXTENT

The EXTENT scan mode reads the extent tree, splits it into tiers by
extent size, converts each tiers's extents into subvol/inode/offset refs,
then runs the legacy bees dedupe engine on the refs.

The extent scan mode can cheaply compute completion percentage and ETA,
so do that every time a new transid is observed.

Signed-off-by: Zygo Blaxell <bees@furryterror.org>
This commit is contained in:
Zygo Blaxell 2022-12-21 22:42:39 -05:00
parent b99d80b40f
commit 6542917ffa
3 changed files with 749 additions and 24 deletions

View File

@ -182,7 +182,7 @@ BeesContext::home_fd()
}
bool
BeesContext::is_root_ro(uint64_t root)
BeesContext::is_root_ro(uint64_t const root)
{
return roots()->is_root_ro(root);
}

View File

@ -4,6 +4,7 @@
#include "crucible/cache.h"
#include "crucible/ntoa.h"
#include "crucible/string.h"
#include "crucible/table.h"
#include "crucible/task.h"
#include <algorithm>
@ -13,6 +14,43 @@
using namespace crucible;
using namespace std;
static
string
pretty_seconds(uint64_t t)
{
struct unit {
uint64_t m_count;
const char *m_suffix;
};
static const unit units[] = {
{ .m_count = 365 * 24 * 60 * 60, .m_suffix = "y" },
{ .m_count = 7 * 24 * 60 * 60, .m_suffix = "w" },
{ .m_count = 1 * 24 * 60 * 60, .m_suffix = "d" },
{ .m_count = 1 * 1 * 60 * 60, .m_suffix = "h" },
{ .m_count = 1 * 1 * 1 * 60, .m_suffix = "m" },
{ .m_count = 1 * 1 * 1 * 1, .m_suffix = "s" },
};
ostringstream oss;
uint64_t prev_unit_limit = numeric_limits<uint64_t>::max();
size_t precision_count = 0;
for (const auto &i : units) {
const auto this_t = t % prev_unit_limit;
if (this_t >= i.m_count) {
if (precision_count) {
oss << " ";
}
++precision_count;
oss << this_t / i.m_count << i.m_suffix;
}
prev_unit_limit = i.m_count;
if (precision_count > 1) {
break;
}
}
return oss.str();
}
string
format_time(time_t t)
{
@ -50,19 +88,34 @@ BeesCrawlState::operator<(const BeesCrawlState &that) const
< tie(that.m_min_transid, that.m_max_transid, that.m_objectid, that.m_offset, that.m_root);
}
class BeesScanMode {
static
bool
is_subvol_tree(uint64_t objectid)
{
return (objectid == BTRFS_FS_TREE_OBJECTID) || (objectid >= BTRFS_FIRST_FREE_OBJECTID);
}
class BeesScanMode : public enable_shared_from_this<BeesScanMode> {
protected:
shared_ptr<BeesContext> m_ctx;
shared_ptr<BeesRoots> m_roots;
bool crawl_batch(const shared_ptr<BeesCrawl>& crawl);
public:
virtual ~BeesScanMode() {}
BeesScanMode(const shared_ptr<BeesRoots>& roots) : m_roots(roots) {}
BeesScanMode(const shared_ptr<BeesRoots>& roots, const shared_ptr<BeesContext>& ctx);
virtual bool do_scan() = 0;
virtual bool scan() = 0;
using CrawlMap = decltype(BeesRoots::m_root_crawl_map);
virtual void next_transid(const CrawlMap &crawl_map) = 0;
virtual const char *ntoa() const = 0;
};
BeesScanMode::BeesScanMode(const shared_ptr<BeesRoots>& roots, const shared_ptr<BeesContext>& ctx) :
m_ctx(ctx),
m_roots(roots)
{
}
bool
BeesScanMode::crawl_batch(const shared_ptr<BeesCrawl>& crawl)
{
@ -79,6 +132,7 @@ class BeesScanModeLockstep : public BeesScanMode {
public:
using BeesScanMode::BeesScanMode;
~BeesScanModeLockstep() override {}
bool do_scan() override;
bool scan() override;
void next_transid(const CrawlMap &crawl_map) override;
const char *ntoa() const override;
@ -90,6 +144,12 @@ BeesScanModeLockstep::ntoa() const
return "LOCKSTEP";
}
bool
BeesScanModeLockstep::do_scan()
{
return true;
}
bool
BeesScanModeLockstep::scan()
{
@ -124,6 +184,7 @@ BeesScanModeLockstep::next_transid(const CrawlMap &crawl_map)
{
auto new_map = make_shared<Map>();
for (const auto &i : crawl_map) {
if (!is_subvol_tree(i.first)) continue;
const auto this_crawl = i.second;
const auto this_range = this_crawl->peek_front();
if (this_range) {
@ -146,6 +207,7 @@ class BeesScanModeIndependent : public BeesScanMode {
public:
using BeesScanMode::BeesScanMode;
~BeesScanModeIndependent() override {}
bool do_scan() override;
bool scan() override;
void next_transid(const CrawlMap &crawl_map) override;
const char *ntoa() const override;
@ -157,6 +219,12 @@ BeesScanModeIndependent::ntoa() const
return "INDEPENDENT";
}
bool
BeesScanModeIndependent::do_scan()
{
return true;
}
bool
BeesScanModeIndependent::scan()
{
@ -185,6 +253,7 @@ BeesScanModeIndependent::next_transid(const CrawlMap &crawl_map)
{
auto new_subvols = make_shared<List>();
for (const auto &i : crawl_map) {
if (!is_subvol_tree(i.first)) continue;
const auto this_crawl = i.second;
const auto this_range = this_crawl->peek_front();
if (this_range) {
@ -206,6 +275,7 @@ class BeesScanModeSequential : public BeesScanMode {
public:
using BeesScanMode::BeesScanMode;
~BeesScanModeSequential() override {}
bool do_scan() override;
bool scan() override;
void next_transid(const CrawlMap &crawl_map) override;
const char *ntoa() const override;
@ -217,6 +287,12 @@ BeesScanModeSequential::ntoa() const
return "SEQUENTIAL";
}
bool
BeesScanModeSequential::do_scan()
{
return true;
}
bool
BeesScanModeSequential::scan()
{
@ -245,6 +321,7 @@ BeesScanModeSequential::next_transid(const CrawlMap &crawl_map)
{
auto new_map = make_shared<Map>();
for (const auto &i : crawl_map) {
if (!is_subvol_tree(i.first)) continue;
const auto this_crawl = i.second;
const auto this_range = this_crawl->peek_front();
if (this_range) {
@ -273,6 +350,7 @@ class BeesScanModeRecent : public BeesScanMode {
public:
using BeesScanMode::BeesScanMode;
~BeesScanModeRecent() override {}
bool do_scan() override;
bool scan() override;
void next_transid(const CrawlMap &crawl_map) override;
const char *ntoa() const override;
@ -284,6 +362,12 @@ BeesScanModeRecent::ntoa() const
return "RECENT";
}
bool
BeesScanModeRecent::do_scan()
{
return true;
}
bool
BeesScanModeRecent::scan()
{
@ -318,6 +402,7 @@ BeesScanModeRecent::next_transid(const CrawlMap &crawl_map)
auto new_map = make_shared<Map>();
auto &sorted = *new_map;
for (const auto &i : crawl_map) {
if (!is_subvol_tree(i.first)) continue;
const auto this_crawl = i.second;
const auto this_range = this_crawl->peek_front();
if (this_range) {
@ -336,6 +421,607 @@ BeesScanModeRecent::next_transid(const CrawlMap &crawl_map)
swap(m_sorted, new_map);
}
/// Scan the extent tree and submit each extent's references in a single batch.
class BeesScanModeExtent : public BeesScanMode {
mutex m_mutex;
mutex m_insert_root_mutex;
CrawlMap m_crawl_map;
map<uint64_t, Task> m_task_map;
struct MagicCrawl {
uint64_t m_min_size;
uint64_t m_max_size;
};
friend ostream& operator<<(ostream &os, const BeesScanModeExtent::MagicCrawl& magic);
using MagicCrawlMap = map<uint64_t, BeesScanModeExtent::MagicCrawl>;
static MagicCrawlMap s_magic_crawl_map;
struct ExtentRef {
uint64_t m_root;
uint64_t m_inum;
uint64_t m_offset;
uint64_t m_length;
ProgressTracker<BeesCrawlState>::ProgressHolder m_hold;
Timer m_age;
};
friend ostream& operator<<(ostream &os, const BeesScanModeExtent::ExtentRef& todo);
void init_tasks();
void run_tasks();
void map_next_extent(uint64_t subvol);
void crawl_one_extent(const ExtentRef &bior);
void create_extent_map(const uint64_t bytenr, const ProgressTracker<BeesCrawlState>::ProgressHolder& m_hold, uint64_t len);
public:
BeesScanModeExtent(const shared_ptr<BeesRoots>& roots, const shared_ptr<BeesContext>& ctx);
~BeesScanModeExtent() override {}
bool do_scan() override;
bool scan() override;
void next_transid(const CrawlMap &crawl_map) override;
const char *ntoa() const override;
};
ostream &
operator<<(ostream &os, const BeesScanModeExtent::MagicCrawl& magic)
{
return os << "[" << pretty(magic.m_min_size) << ".." << pretty(magic.m_max_size) << "]";
}
ostream &
operator<<(ostream &os, const BeesScanModeExtent::ExtentRef& todo)
{
return os << todo.m_root << ":" << todo.m_inum << " " << to_hex(todo.m_offset) << "+" << pretty(todo.m_length) << " age " << todo.m_age;
}
map<uint64_t, BeesScanModeExtent::MagicCrawl> BeesScanModeExtent::s_magic_crawl_map {
{
BTRFS_FIRST_FREE_OBJECTID - 6,
(BeesScanModeExtent::MagicCrawl) {
.m_min_size = 32 * 1024 * 1024 + 1,
.m_max_size = numeric_limits<uint64_t>::max(),
},
},
{
BTRFS_FIRST_FREE_OBJECTID - 5,
(BeesScanModeExtent::MagicCrawl) {
.m_min_size = 8 * 1024 * 1024 + 1,
.m_max_size = 32 * 1024 * 1024,
},
},
{
BTRFS_FIRST_FREE_OBJECTID - 4,
(BeesScanModeExtent::MagicCrawl) {
.m_min_size = 2 * 1024 * 1024 + 1,
.m_max_size = 8 * 1024 * 1024,
},
},
{
BTRFS_FIRST_FREE_OBJECTID - 3,
(BeesScanModeExtent::MagicCrawl) {
.m_min_size = 512 * 1024 + 1,
.m_max_size = 2 * 1024 * 1024,
},
},
{
BTRFS_FIRST_FREE_OBJECTID - 2,
(BeesScanModeExtent::MagicCrawl) {
.m_min_size = 128 * 1024 + 1,
.m_max_size = 512 * 1024,
},
},
{
BTRFS_FIRST_FREE_OBJECTID - 1,
(BeesScanModeExtent::MagicCrawl) {
.m_min_size = 0,
.m_max_size = 128 * 1024,
},
},
};
BeesScanModeExtent::BeesScanModeExtent(const shared_ptr<BeesRoots>& roots, const shared_ptr<BeesContext>& ctx) :
BeesScanMode(roots, ctx)
{
}
const char *
BeesScanModeExtent::ntoa() const
{
return "EXTENT";
}
static
bool
should_throttle()
{
static bool s_throttled = false;
// If there's too many entries in the queue, stop adding new ones until workers catch up
// If there's not too many entries in the queue, restart the scan task
const auto instance_count = Task::instance_count();
const auto instance_limit = BEES_MAX_EXTENT_REF_COUNT;
const bool queue_empty = s_throttled && instance_count < instance_limit;
const bool queue_full = !s_throttled && instance_count > instance_limit;
if (queue_full) {
BEESLOGDEBUG("Throttling crawl at " << instance_count << " tasks");
s_throttled = true;
BEESCOUNT(crawl_throttled);
}
if (queue_empty) {
BEESLOGDEBUG("Unthrottling crawl at " << instance_count << " tasks");
s_throttled = false;
BEESCOUNT(crawl_unthrottled);
}
return s_throttled;
}
void
BeesScanModeExtent::crawl_one_extent(const BeesScanModeExtent::ExtentRef &bior)
{
auto inode_mutex = m_ctx->get_inode_mutex(bior.m_inum);
auto inode_lock = inode_mutex->try_lock(Task::current_task());
if (!inode_lock) {
BEESCOUNT(extent_deferred_inode);
return;
}
BEESNOTE("scanning root " << bior.m_root << " ino " << bior.m_inum << " offset " << to_hex(bior.m_offset) << " length " << pretty(bior.m_length));
BEESTRACE("scanning root " << bior.m_root << " ino " << bior.m_inum << " offset " << to_hex(bior.m_offset) << " length " << pretty(bior.m_length));
BeesFileRange bfr(
BeesFileId(bior.m_root, bior.m_inum),
bior.m_offset,
bior.m_offset + bior.m_length
);
BEESCOUNT(extent_forward);
m_ctx->scan_forward(bfr);
}
void
BeesScanModeExtent::create_extent_map(const uint64_t bytenr, const ProgressTracker<BeesCrawlState>::ProgressHolder& hold, const uint64_t len)
{
BEESNOTE("Creating extent map for " << to_hex(bytenr) << " with LOGICAL_INO");
BEESTRACE("Creating extent map for " << to_hex(bytenr) << " with LOGICAL_INO");
const auto log_ino_ptr = m_ctx->logical_ino(bytenr, true);
auto &log_ino = *log_ino_ptr;
{
BEESNOTE("waiting to create extent map for " << to_hex(bytenr) << " with LOGICAL_INO");
const auto lock = MultiLocker::get_lock("logical_ino");
BEESNOTE("Resolving bytenr " << to_hex(bytenr) << " refs " << log_ino.m_iors.size());
BEESTOOLONG("Resolving bytenr " << to_hex(bytenr) << " refs " << log_ino.m_iors.size());
// Time how long this takes
Timer resolve_timer;
if (log_ino.do_ioctl_nothrow(m_ctx->root_fd())) {
BEESCOUNT(extent_ok);
} else {
BEESCOUNT(extent_fail);
}
BEESCOUNTADD(extent_ms, resolve_timer.age() * 1000);
}
const size_t rv_count = log_ino.m_iors.size();
// BEESLOGDEBUG("Inserting " << rv_count << " extent refs from " << to_hex(bytenr) << ", " << Task::instance_count() << " tasks");
BEESNOTE("Inserting " << rv_count << " extent refs from " << to_hex(bytenr));
BEESTRACE("Inserting " << rv_count << " extent refs from " << to_hex(bytenr));
// Avoid performance problems - pretend resolve failed if there are too many refs
if (rv_count == 0) {
BEESLOGDEBUG("LOGICAL_INO returned 0 refs for " << len << " bytes (" << pretty(len) << ") at " << to_hex(bytenr));
BEESCOUNT(extent_zero);
return;
} else if (rv_count >= BEES_MAX_EXTENT_REF_COUNT) {
// If we find any duplicates when there are BEES_MAX_EXTENT_REF_COUNT references, then
// we'll end up with some extent with at least BEES_MAX_EXTENT_REF_COUNT + 1 references.
// That's too many, so don't let that happen.
BEESLOGINFO("bytenr " << to_hex(bytenr) << " refs " << rv_count << " overflows configured ref limit " << BEES_MAX_EXTENT_REF_COUNT);
BEESCOUNT(extent_overflow);
return;
}
BtrfsExtentDataFetcher bedf(m_ctx->root_fd());
// Collect extent ref tasks as a series of stand-alone events
// chained after the first task created, then run the first one.
// This prevents other threads from starting to process an
// extent until we have all of its refs in the queue.
Task first_task;
for (const auto &i : log_ino.m_iors) {
catch_all([&](){
BEESTRACE("mapping extent " << to_hex(bytenr) << " ref at root " << i.m_root << " ino " << i.m_inum << " offset " << to_hex(i.m_offset));
BEESNOTE("mapping extent " << to_hex(bytenr) << " ref at root " << i.m_root << " ino " << i.m_inum << " offset " << to_hex(i.m_offset));
bedf.tree(i.m_root);
bedf.objectid(i.m_inum);
const auto bti = bedf.at(i.m_offset);
if (!bti) {
BEESLOGDEBUG("No ref for extent " << to_hex(bytenr) << " at root " << i.m_root << " ino " << i.m_inum << " offset " << to_hex(i.m_offset));
BEESCOUNT(extent_ref_missing);
return;
}
const auto bec = dynamic_pointer_cast<BeesScanModeExtent>(shared_from_this());
const auto length = bti.file_extent_logical_bytes();
const ExtentRef extref = {
.m_root = i.m_root,
.m_inum = i.m_inum,
.m_offset = i.m_offset,
.m_length = length,
.m_hold = hold,
};
ostringstream oss;
oss << "R" << i.m_root << "_" << i.m_inum << "_" << pretty(i.m_offset) << "_" << pretty(length);
Task crawl_one(oss.str(), [bec, extref]() {
bec->crawl_one_extent(extref);
});
if (!first_task) {
first_task = crawl_one;
} else {
first_task.append(crawl_one);
}
BEESCOUNT(extent_ref_ok);
});
}
if (first_task) {
// first_task.append(Task::current_task());
first_task.run();
}
// Go back for more tasks if the current task isn't already appended
Task::current_task().run();
BEESCOUNT(extent_mapped);
}
void
BeesScanModeExtent::init_tasks()
{
// Make sure all the magic crawlers are inserted in m_crawl_map,
// and each one has a Task
unique_lock<mutex> lock_insert_root(m_insert_root_mutex);
unique_lock<mutex> lock(m_mutex);
for (const auto &i : s_magic_crawl_map) {
const auto subvol = i.first;
const auto &magic = i.second;
auto found = m_crawl_map.find(subvol);
if (found == m_crawl_map.end()) {
lock.unlock();
BeesCrawlState new_bcs;
new_bcs.m_root = subvol;
new_bcs.m_min_transid = m_roots->transid_min();
new_bcs.m_max_transid = m_roots->transid_max();
const auto this_crawl = m_roots->insert_root(new_bcs);
lock.lock();
m_crawl_map.insert(make_pair(subvol, this_crawl));
BEESCOUNT(crawl_create);
}
auto task_found = m_task_map.find(subvol);
if (task_found == m_task_map.end()) {
ostringstream oss;
oss << "extent_" << subvol << "_" << pretty(magic.m_min_size & ~BLOCK_MASK_CLONE)
<< "_" << pretty(magic.m_max_size);
const auto bec = dynamic_pointer_cast<BeesScanModeExtent>(shared_from_this());
m_task_map.insert(make_pair(subvol, Task(oss.str(), [bec, subvol]() {
bec->map_next_extent(subvol);
})));
}
}
}
void
BeesScanModeExtent::run_tasks()
{
if (should_throttle()) return;
unique_lock<mutex> lock(m_mutex);
// Good to go, start everything running
for (const auto &i : m_task_map) {
i.second.run();
}
}
void
BeesScanModeExtent::map_next_extent(uint64_t const subvol)
{
BEESTRACE("map_next_extent " << subvol);
if (should_throttle()) return;
size_t discard_count = 0;
size_t gen_low_count = 0;
size_t gen_high_count = 0;
size_t loop_count = 0;
size_t init_s_calls = BtrfsIoctlSearchKey::s_calls;
size_t init_s_loops = BtrfsIoctlSearchKey::s_loops;
Timer crawl_time;
unique_lock<mutex> lock(m_mutex);
auto found = m_crawl_map.find(subvol);
THROW_CHECK0(runtime_error, found != m_crawl_map.end());
CrawlMap::mapped_type this_crawl = found->second;
lock.unlock();
THROW_CHECK0(runtime_error, this_crawl);
BtrfsExtentItemFetcher beif(m_ctx->root_fd());
beif.scale_size(BLOCK_SIZE_SUMS);
// Get the next item from the crawler
while (true) {
++loop_count;
BEESTRACE("get_state_end");
const auto this_state = this_crawl->get_state_end();
BEESNOTE("Crawling extent " << this_state);
BEESTRACE("Crawling extent " << this_state);
const auto this_range = this_crawl->pop_front();
BEESTRACE("this_range check");
// Ran out of data in this subvol, wait for next_transid to refill it
if (!this_range) {
break;
}
// Check extent length against size range
const auto &subvol_magic = s_magic_crawl_map.at(subvol);
const uint64_t lower_size_bound = subvol_magic.m_min_size;
const uint64_t upper_size_bound = subvol_magic.m_max_size;
const uint64_t this_range_size = this_range.size();
// If this extent is out of range, move on to the next
if (this_range_size < lower_size_bound || this_range_size > upper_size_bound) {
// Advance the begin point in case we get into trouble later on
this_crawl->hold_state(this_state);
BEESCOUNT(crawl_discard);
++discard_count;
// Skip the skipping until we get the issues sorted out
continue;
// Skip ahead over any below-min-size extents
BEESTRACE("min_size " << pretty(lower_size_bound) << " > scale_size " << pretty(beif.scale_size()));
const auto lsb_rounded = lower_size_bound & ~(beif.scale_size() - 1);
// Don't bother doing backward searches when skipping less than 128K,
// the search will cost more than reading 32 consecutive extent records
// FIXME: need to make this aware of block group boundaries so it doesn't
// blow 5 CPU seconds scanning metadata
if (lsb_rounded >= 128 * 1024) {
const auto lsb_rounded = lower_size_bound & ~(beif.scale_size() - 1);
const auto objectid = this_range.end() + lsb_rounded - beif.scale_size();
BEESTRACE("objectid = " << this_state.m_objectid << ", adjusted to " << objectid);
BEESTOOLONG("subvol " << subvol << " skipping forward " << pretty(lsb_rounded) << " from " << to_hex(this_state.m_objectid) << " to " << to_hex(objectid));
BEESNOTE("subvol " << subvol << " skipping forward " << pretty(lsb_rounded) << " from " << to_hex(this_state.m_objectid) << " to " << to_hex(objectid));
const auto bti = beif.rlower_bound(objectid);
auto mutable_state = this_state;
mutable_state.m_objectid = bti.objectid();
if (mutable_state.m_objectid <= this_state.m_objectid) {
// BEESLOGDEBUG("skip failed: this_state " << this_state << ", mutable_state " << mutable_state);
// No extent found between end and search position, skip ahead to search position
mutable_state.m_objectid = objectid;
BEESCOUNT(crawl_skip_fail);
} else {
const auto discard_hold = this_crawl->hold_state(mutable_state);
BEESCOUNT(crawl_skip_ok);
}
}
continue;
}
const auto bytenr = this_range.fid().ino();
// Check extent item generation is in range
// FIXME: we already had this in crawl state, and we threw it away
const auto bti = beif.at(bytenr);
const auto gen = bti.extent_generation();
if (gen < this_state.m_min_transid) {
BEESCOUNT(crawl_gen_low);
++gen_low_count;
continue;
}
if (gen > this_state.m_max_transid) {
BEESCOUNT(crawl_gen_high);
++gen_high_count;
continue;
}
// Map this extent here to regulate task creation
create_extent_map(bytenr, this_crawl->hold_state(this_state), bti.offset());
BEESCOUNT(crawl_extent);
const auto search_calls = BtrfsIoctlSearchKey::s_calls - init_s_calls;
const auto search_loops = BtrfsIoctlSearchKey::s_loops - init_s_loops;
if (crawl_time.age() > 1) {
BEESLOGDEBUG("loop_count " << loop_count << " discard_count " << discard_count
<< " gen_low_count " << gen_low_count << " gen_high_count " << gen_high_count
<< " search_calls " << search_calls << " search_loops " << search_loops
<< " time " << crawl_time << " subvol " << subvol);
}
// We did something! Get in line to run again
Task::current_task().run();
return;
}
// All crawls done
BEESCOUNT(crawl_done);
}
bool
BeesScanModeExtent::do_scan()
{
return false;
}
bool
BeesScanModeExtent::scan()
{
// This is now driven directly from next_transid
return false;
}
void
BeesScanModeExtent::next_transid(const CrawlMap &crawl_map)
{
BEESTRACE("Extent next_transid");
// Do the important parts first, the rest can return early or die with an exception
// Can't set this up in the constructor because shared_from_this is a method on a
// virtual base. So we do it here.
init_tasks();
// insert_root does this for non-magic subvols, we have to do it ourselves
for (const auto &i : s_magic_crawl_map) {
const auto subvol = i.first;
const auto found = crawl_map.find(subvol);
if (found != crawl_map.end()) {
found->second->deferred(false);
}
}
// Kick off tasks if they aren't already running
run_tasks();
// Swap in the new crawl map with freshly undeferred crawlers
auto crawl_map_copy = crawl_map;
unique_lock<mutex> lock(m_mutex);
swap(m_crawl_map, crawl_map_copy);
lock.unlock();
// Estimate progress by building a map of where the extent bytenrs are (they are sparse,
// no extents exist between block groups), and report the position within that map.
BtrfsTreeOffsetFetcher btf(m_ctx->root_fd());
btf.tree(BTRFS_CHUNK_TREE_OBJECTID);
btf.objectid(BTRFS_FIRST_CHUNK_TREE_OBJECTID);
btf.type(BTRFS_CHUNK_ITEM_KEY);
uint64_t fs_size = 0;
uint64_t last_bgaddr = 0;
struct bg_info {
uint64_t first_bytenr;
uint64_t first_total;
};
map<uint64_t, bg_info> bg_info_map;
while (true) {
const auto bti = btf.lower_bound(last_bgaddr);
if (!bti) {
break;
}
const auto offset = bti.offset();
const auto chunk_length = bti.chunk_length();
THROW_CHECK0(runtime_error, offset > 0);
THROW_CHECK0(runtime_error, chunk_length > 0);
last_bgaddr = offset + chunk_length;
bg_info_map[last_bgaddr] = (bg_info) {
.first_bytenr = offset,
.first_total = fs_size,
};
if (bti.chunk_type() & (BTRFS_BLOCK_GROUP_METADATA | BTRFS_BLOCK_GROUP_SYSTEM)) {
continue;
}
fs_size += chunk_length;
}
if (!fs_size) {
BEESLOGDEBUG("PROGRESS: no data block groups found in filesystem");
BEESCOUNT(progress_no_data_bg);
return;
}
// Report on progress using extent bytenr map
Table::Table eta;
for (const auto &i : s_magic_crawl_map) {
const auto &subvol = i.first;
const auto &magic = i.second;
const auto found = crawl_map.find(subvol);
if (found == crawl_map.end()) {
// BEESLOGDEBUG("PROGRESS: crawler not yet created for " << magic);
BEESCOUNT(progress_not_created);
continue;
}
const auto this_crawl = found->second;
THROW_CHECK1(runtime_error, subvol, this_crawl);
const auto this_range = this_crawl->peek_front();
if (!this_range) {
BEESLOGDEBUG("PROGRESS: completed crawl " << magic);
BEESCOUNT(progress_complete);
continue;
}
const auto bytenr = this_range.fid().ino();
const auto bg_found = bg_info_map.lower_bound(bytenr);
if (bg_found == bg_info_map.end()) {
BEESLOGDEBUG("PROGRESS: bytenr " << to_hex(bytenr) << " not found in a block group for " << magic);
BEESCOUNT(progress_not_found);
continue;
}
const auto &bi = bg_found->second;
const auto bi_last_bytenr = bg_found->first;
if (bytenr > bi_last_bytenr || bytenr < bi.first_bytenr) {
// This can happen if the crawler happens to be in a metadata block group,
// or if a block group was deleted under us.
BEESLOGDEBUG("PROGRESS: bytenr " << to_hex(bytenr) << " out of range for block group " << to_hex(bi.first_bytenr) << ".." << to_hex(bg_found->first) << " (block group deleted?) for " << magic);
BEESCOUNT(progress_out_of_bg);
}
const auto bytenr_offset = min(bi_last_bytenr, max(bytenr, bi.first_bytenr)) - bi.first_bytenr + bi.first_total;
const auto bytenr_percent = bytenr_offset / (0.01 * fs_size);
const auto this_state = this_crawl->get_state_end();
const auto now = time(NULL);
const auto time_so_far = now - min(now, this_state.m_started);
string eta_stamp = "-";
string eta_pretty = "-";
if (time_so_far > 1 && bytenr_percent > 0) {
const time_t eta_duration = time_so_far / (bytenr_percent / 100);
const time_t eta_time = eta_duration + now;
struct tm ltm = { 0 };
DIE_IF_ZERO(localtime_r(&eta_time, &ltm));
char buf[1024] = { 0 };
DIE_IF_ZERO(strftime(buf, sizeof(buf), "%Y-%m-%d %H:%M:%S", &ltm));
eta_stamp = string(buf);
eta_pretty = pretty_seconds(eta_duration);
}
eta.insert_row(Table::endpos, vector<Table::Content> {
Table::Text(pretty(bytenr_offset)),
Table::Text(pretty(fs_size)),
Table::Text(astringprintf("%.4f%%", bytenr_percent)),
Table::Number(subvol),
Table::Text(pretty(magic.m_min_size & ~BLOCK_MASK_CLONE)),
Table::Text(pretty(magic.m_max_size)),
Table::Number(this_state.m_min_transid),
Table::Number(this_state.m_max_transid),
Table::Text(eta_pretty),
Table::Text(eta_stamp),
});
BEESCOUNT(progress_ok);
}
ostringstream oss;
eta.left("PROGRESS: ");
eta.mid(" ");
eta.right("");
eta.insert_row(0, vector<Table::Content> {
Table::Text("done"),
Table::Text("total"),
Table::Text("%done"),
Table::Text("sub"),
Table::Text("szmn"),
Table::Text("szmx"),
Table::Text("transid"),
Table::Number(m_roots->transid_max()),
Table::Text("remain"),
Table::Text("ETA"),
});
const auto dash_fill = Table::Fill('-');
eta.insert_row(1, vector<Table::Content>(eta.cols().size(), dash_fill));
oss << eta;
BEESLOGDEBUG(oss.str());
}
void
BeesRoots::set_scan_mode(ScanMode mode)
{
@ -343,19 +1029,23 @@ BeesRoots::set_scan_mode(ScanMode mode)
unique_lock<mutex> lock(m_mutex);
switch (mode) {
case SCAN_MODE_LOCKSTEP: {
m_scanner = make_shared<BeesScanModeLockstep>(shared_from_this());
m_scanner = make_shared<BeesScanModeLockstep>(shared_from_this(), m_ctx);
break;
}
case SCAN_MODE_INDEPENDENT: {
m_scanner = make_shared<BeesScanModeIndependent>(shared_from_this());
m_scanner = make_shared<BeesScanModeIndependent>(shared_from_this(), m_ctx);
break;
}
case SCAN_MODE_SEQUENTIAL: {
m_scanner = make_shared<BeesScanModeSequential>(shared_from_this());
m_scanner = make_shared<BeesScanModeSequential>(shared_from_this(), m_ctx);
break;
}
case SCAN_MODE_RECENT: {
m_scanner = make_shared<BeesScanModeRecent>(shared_from_this());
m_scanner = make_shared<BeesScanModeRecent>(shared_from_this(), m_ctx);
break;
}
case SCAN_MODE_EXTENT: {
m_scanner = make_shared<BeesScanModeExtent>(shared_from_this(), m_ctx);
break;
}
case SCAN_MODE_COUNT:
@ -714,6 +1404,10 @@ BeesRoots::crawl_roots()
BEESNOTE("Scanning roots in " << hold_scanner->ntoa() << " mode");
BEESTRACE("scanning roots in " << hold_scanner->ntoa() << " mode");
// Clumsy adapter for legacy scan modes
if (!hold_scanner->do_scan()) {
return false;
}
if (hold_scanner->scan()) {
return true;
}
@ -816,7 +1510,7 @@ BeesRoots::writeback_thread()
}
}
void
shared_ptr<BeesCrawl>
BeesRoots::insert_root(const BeesCrawlState &new_bcs)
{
unique_lock<mutex> lock(m_mutex);
@ -826,9 +1520,10 @@ BeesRoots::insert_root(const BeesCrawlState &new_bcs)
m_root_crawl_map.insert(new_pair);
++m_crawl_dirty;
}
auto found = m_root_crawl_map.find(new_bcs.m_root);
const auto found = m_root_crawl_map.find(new_bcs.m_root);
THROW_CHECK0(runtime_error, found != m_root_crawl_map.end());
found->second->deferred(false);
return found->second;
}
void
@ -845,8 +1540,10 @@ BeesRoots::insert_new_crawl()
unique_lock<mutex> lock(m_mutex);
set<uint64_t> excess_roots;
for (const auto &i : m_root_crawl_map) {
BEESTRACE("excess_roots.insert(" << i.first << ")");
excess_roots.insert(i.first);
if (is_subvol_tree(i.first)) {
BEESTRACE("excess_roots.insert(" << i.first << ")");
excess_roots.insert(i.first);
}
}
lock.unlock();
@ -1104,6 +1801,11 @@ BeesRoots::is_root_ro(uint64_t root)
return false;
}
// If it's not a subvol tree, it's rw
if (!is_subvol_tree(root)) {
return false;
}
BEESTRACE("checking subvol flags on root " << root);
BtrfsRootFetcher root_fetcher(m_ctx->root_fd());
@ -1314,9 +2016,14 @@ BeesCrawl::BeesCrawl(shared_ptr<BeesContext> ctx, BeesCrawlState initial_state)
m_state(initial_state),
m_btof(ctx->root_fd())
{
m_btof.scale_size(1);
m_btof.tree(initial_state.m_root);
m_btof.type(BTRFS_EXTENT_DATA_KEY);
if (is_subvol_tree(initial_state.m_root)) {
m_btof.tree(initial_state.m_root);
m_btof.scale_size(1);
m_btof.type(BTRFS_EXTENT_DATA_KEY);
} else {
m_btof.tree(BTRFS_EXTENT_TREE_OBJECTID);
m_btof.type(BTRFS_EXTENT_ITEM_KEY);
}
}
bool
@ -1410,6 +2117,13 @@ BeesCrawl::fetch_extents()
m_btof.transid(old_state.m_min_transid);
if (catch_all([&]() {
m_next_extent_data = m_btof.lower_bound(old_state.m_objectid);
if (m_btof.tree() == BTRFS_EXTENT_TREE_OBJECTID) {
// Skip over TREE_BLOCK extent items, they don't have files
while (!!m_next_extent_data && (m_next_extent_data.extent_flags() & BTRFS_EXTENT_FLAG_TREE_BLOCK)) {
BEESCOUNT(crawl_tree_block);
m_next_extent_data = m_btof.next(m_next_extent_data.objectid());
}
}
})) {
// Whoops that didn't work. Stop scanning this subvol, move on to the next.
m_deferred = true;
@ -1421,7 +2135,7 @@ BeesCrawl::fetch_extents()
return restart_crawl();
}
auto new_state = old_state;
new_state.m_objectid = max(m_next_extent_data.objectid() + 1, m_next_extent_data.objectid());
new_state.m_objectid = max(m_next_extent_data.objectid() + m_btof.scale_size(), m_next_extent_data.objectid());
new_state.m_offset = 0;
set_state(new_state);
return true;
@ -1446,11 +2160,19 @@ BeesCrawl::bti_to_bfr(const BtrfsTreeItem &bti) const
if (!bti) {
return BeesFileRange();
}
return BeesFileRange(
BeesFileId(get_state_end().m_root, bti.objectid()),
bti.offset(),
bti.offset() + bti.file_extent_logical_bytes()
);
if (bti.type() == BTRFS_EXTENT_ITEM_KEY) {
return BeesFileRange(
BeesFileId(get_state_end().m_root, bti.objectid()),
bti.objectid(),
bti.objectid() + bti.offset()
);
} else {
return BeesFileRange(
BeesFileId(get_state_end().m_root, bti.objectid()),
bti.offset(),
bti.offset() + bti.file_extent_logical_bytes()
);
}
}
BeesFileRange

View File

@ -561,11 +561,8 @@ class BeesRoots : public enable_shared_from_this<BeesRoots> {
bool m_stop_requested = false;
void insert_new_crawl();
void insert_root(const BeesCrawlState &bcs);
Fd open_root_nocache(uint64_t root);
Fd open_root_ino_nocache(uint64_t root, uint64_t ino);
uint64_t transid_min();
uint64_t transid_max();
uint64_t transid_max_nocache();
void state_load();
ostream &state_to_stream(ostream &os);
@ -582,6 +579,9 @@ class BeesRoots : public enable_shared_from_this<BeesRoots> {
bool crawl_batch(shared_ptr<BeesCrawl> crawl);
void clear_caches();
friend class BeesScanModeExtent;
shared_ptr<BeesCrawl> insert_root(const BeesCrawlState &bcs);
friend class BeesCrawl;
friend class BeesFdCache;
friend class BeesScanMode;
@ -600,17 +600,20 @@ public:
Fd open_root_ino(const BeesFileId &bfi) { return open_root_ino(bfi.root(), bfi.ino()); }
bool is_root_ro(uint64_t root);
// TODO: do extent-tree scans instead
enum ScanMode {
SCAN_MODE_LOCKSTEP,
SCAN_MODE_INDEPENDENT,
SCAN_MODE_SEQUENTIAL,
SCAN_MODE_RECENT,
SCAN_MODE_EXTENT,
SCAN_MODE_COUNT, // must be last
};
void set_scan_mode(ScanMode new_mode);
void set_workaround_btrfs_send(bool do_avoid);
uint64_t transid_min();
uint64_t transid_max();
};
struct BeesHash {