Compare commits

..

15 Commits

Author SHA1 Message Date
Zach Brown
92ba4f5662 Add test for parallel restore
Signed-off-by: Zach Brown <zab@versity.com>
2024-04-15 11:06:24 -07:00
Zach Brown
7754feee3f Add parallel restore
Signed-off-by: Zach Brown <zab@versity.com>
2024-04-15 11:06:24 -07:00
Zach Brown
c4b00d059a Add userspace NSEC_PER_SEC
Signed-off-by: Zach Brown <zab@versity.com>
2024-04-15 11:06:24 -07:00
Zach Brown
42a60e31c5 Add bloom filter index calc for userspace utils
Signed-off-by: Zach Brown <zab@versity.com>
2024-04-15 11:06:24 -07:00
Zach Brown
ea7637b915 Add srch_encode_entry() for userspace utils
Signed-off-by: Zach Brown <zab@versity.com>
2024-04-15 11:06:24 -07:00
Zach Brown
1324c39372 Add put_unaligned_leXX() for userspace
Signed-off-by: Zach Brown <zab@versity.com>
2024-04-15 11:06:24 -07:00
Zach Brown
f41bd60ca4 Add fls64() alias for userspace flsll()
Signed-off-by: Zach Brown <zab@versity.com>
2024-04-15 11:06:24 -07:00
Zach Brown
70a932614a Promote userspace btree block initialization
Signed-off-by: Zach Brown <zab@versity.com>
2024-04-15 11:06:24 -07:00
Zach Brown
c6a11fcd6b Add userspace version of our mode to type
Signed-off-by: Zach Brown <zab@versity.com>
2024-04-15 11:06:24 -07:00
Zach Brown
3c67805004 Add userspace version of our dirent name hash
Signed-off-by: Zach Brown <zab@versity.com>
2024-04-15 11:06:24 -07:00
Zach Brown
02fe58b727 Add lk rbtree wrapper
Import the kernel's rbtree implementation with a wrapper so we can use
it from userspace.

Signed-off-by: Zach Brown <zab@versity.com>
2024-04-15 11:06:24 -07:00
Zach Brown
0519830229 Merge pull request #165 from versity/greg/kmod-uninstall-cleanup
More cleanly drive weak-modules on install/uninstall
2024-04-11 14:32:06 -07:00
Greg Cymbalski
4d6e1a14ae More safely install/uninstall with weak-modules
This addresses some minor issues with how we handle driving the
weak-modules infrastructure for handling running on kernels not
explicitly built for.

For one, we now drive weak-modules at install-time more explicitly (it
was adding symlinks for all modules into the right place for the running
kernel, whereas now it only handles that for scoutfs against all
installed kernels).

Also we no longer leave stale modules on the filesystem after an
uninstall/upgrade, similar to what's done for vsm's kmods right now.
RPM's pre/postinstall scriptlets are used to drive weak-modules to clean
things up.

Note that this (intentionally) does not (re)generate initrds of any
kind.

Finally, this was tested on both the native kernel version and on
updates that would need the migrated modules. As a result, installs are
a little quicker, the module still gets migrated successfully, and
uninstalls correctly remove (only) the packaged module.
2024-04-11 13:20:50 -07:00
Greg Cymbalski
fc3e061ea8 Merge pull request #164 from versity/greg/preserve-git-describe
Encode git info into spec to keep git info in final kmod
2024-03-29 13:48:33 -07:00
Greg Cymbalski
a4bc3fb27d Capture git info at spec creation time, pass into make 2024-02-05 15:44:10 -08:00
24 changed files with 4276 additions and 248 deletions

View File

@@ -12,17 +12,22 @@ else
SP = @:
endif
SCOUTFS_GIT_DESCRIBE := \
SCOUTFS_GIT_DESCRIBE ?= \
$(shell git describe --all --abbrev=6 --long 2>/dev/null || \
echo no-git)
ESCAPED_GIT_DESCRIBE := \
$(shell echo $(SCOUTFS_GIT_DESCRIBE) |sed -e 's/\//\\\//g')
RPM_GITHASH ?= $(shell git rev-parse --short HEAD)
SCOUTFS_ARGS := SCOUTFS_GIT_DESCRIBE=$(SCOUTFS_GIT_DESCRIBE) \
RPM_GITHASH=$(RPM_GITHASH) \
CONFIG_SCOUTFS_FS=m -C $(SK_KSRC) M=$(CURDIR)/src \
EXTRA_CFLAGS="-Werror"
# - We use the git describe from tags to set up the RPM versioning
RPM_VERSION := $(shell git describe --long --tags | awk -F '-' '{gsub(/^v/,""); print $$1}')
RPM_GITHASH := $(shell git rev-parse --short HEAD)
TARFILE = scoutfs-kmod-$(RPM_VERSION).tar
@@ -41,7 +46,8 @@ modules_install:
%.spec: %.spec.in .FORCE
sed -e 's/@@VERSION@@/$(RPM_VERSION)/g' \
-e 's/@@GITHASH@@/$(RPM_GITHASH)/g' < $< > $@+
-e 's/@@GITHASH@@/$(RPM_GITHASH)/g' \
-e 's/@@GITDESCRIBE@@/$(ESCAPED_GIT_DESCRIBE)/g' < $< > $@+
mv $@+ $@

View File

@@ -1,6 +1,7 @@
%define kmod_name scoutfs
%define kmod_version @@VERSION@@
%define kmod_git_hash @@GITHASH@@
%define kmod_git_describe @@GITDESCRIBE@@
%define pkg_date %(date +%%Y%%m%%d)
# Disable the building of the debug package(s).
@@ -75,7 +76,7 @@ echo "Building for kernel: %{kernel_version} flavors: '%{flavors_to_build}'"
for flavor in %flavors_to_build; do
rm -rf obj/$flavor
cp -r source obj/$flavor
make SK_KSRC=%{kernel_source $flavor} -C obj/$flavor module
make RPM_GITHASH=%{kmod_git_hash} SCOUTFS_GIT_DESCRIBE=%{kmod_git_describe} SK_KSRC=%{kernel_source $flavor} -C obj/$flavor module
done
%install
@@ -97,10 +98,21 @@ find %{buildroot} -type f -name \*.ko -exec %{__chmod} u+x \{\} \;
/lib/modules
%post
weak-modules --add-kernel --no-initramfs
echo /lib/modules/%{kversion}/%{install_mod_dir}/scoutfs.ko | weak-modules --add-modules --no-initramfs
depmod -a
%endif
%clean
rm -rf %{buildroot}
%preun
# stash our modules for postun cleanup
SCOUTFS_RPM_NAME=$(rpm -q %{name} | grep "%{version}-%{release}")
rpm -ql $SCOUTFS_RPM_NAME | grep '\.ko$' > /var/run/%{name}-modules-%{version}-%{release} || true
%postun
if [ -x /sbin/weak-modules ]; then
cat /var/run/%{name}-modules-%{version}-%{release} | /sbin/weak-modules --remove-modules --no-initramfs
fi
rm /var/run/%{name}-modules-%{version}-%{release} || true

View File

@@ -12,7 +12,8 @@ BIN := src/createmany \
src/find_xattrs \
src/create_xattr_loop \
src/fragmented_data_extents \
src/o_tmpfile_umask
src/o_tmpfile_umask \
src/parallel_restore
DEPS := $(wildcard src/*.d)
@@ -22,8 +23,11 @@ ifneq ($(DEPS),)
-include $(DEPS)
endif
src/parallel_restore_cflags := ../utils/src/scoutfs_parallel_restore.a -lm
$(BIN): %: %.c Makefile
gcc $(CFLAGS) -MD -MP -MF $*.d $< -o $@
gcc $(CFLAGS) -MD -MP -MF $*.d $< -o $@ $($(@)_cflags)
.PHONY: clean
clean:

View File

@@ -0,0 +1,782 @@
#define _GNU_SOURCE /* O_DIRECT */
#include <stdlib.h>
#include <stdio.h>
#include <stdbool.h>
#include <unistd.h>
#include <fcntl.h>
#include <sys/types.h>
#include <sys/xattr.h>
#include <ctype.h>
#include <string.h>
#include <errno.h>
#include <limits.h>
#include <time.h>
#include <sys/prctl.h>
#include <signal.h>
#include <sys/socket.h>
#include "../../utils/src/sparse.h"
#include "../../utils/src/util.h"
#include "../../utils/src/list.h"
#include "../../utils/src/parse.h"
#include "../../kmod/src/format.h"
#include "../../utils/src/parallel_restore.h"
/*
* XXX:
* - add a nice description of what's going on
* - mention allocator contention
* - test child process dying handling
* - root dir entry name length is wrong
*/
#define ERRF " errno %d (%s)"
#define ERRA errno, strerror(errno)
#define error_exit(cond, fmt, args...) \
do { \
if (cond) { \
printf("error: "fmt"\n", ##args); \
exit(1); \
} \
} while (0)
#define dprintf(fmt, args...) \
do { \
if (0) \
printf(fmt, ##args); \
} while (0)
#define REG_MODE (S_IFREG | 0644)
#define DIR_MODE (S_IFDIR | 0755)
struct opts {
unsigned long long buf_size;
unsigned long long write_batch;
unsigned long long low_dirs;
unsigned long long high_dirs;
unsigned long long low_files;
unsigned long long high_files;
char *meta_path;
unsigned long long total_files;
bool read_only;
unsigned long long seed;
unsigned long long nr_writers;
};
static void usage(void)
{
printf("usage:\n"
" -b NR | threads write blocks in batches files (100000)\n"
" -d LOW:HIGH | range of subdirs per directory (5:10)\n"
" -f LOW:HIGH | range of files per directory (10:20)\n"
" -m PATH | path to metadata device\n"
" -n NR | total number of files to create (100)\n"
" -r | read-only, all work except writing, measure cpu cost\n"
" -s NR | randomization seed (random)\n"
" -w NR | number of writing processes to fork (online cpus)\n"
);
}
static size_t write_bufs(struct opts *opts, struct scoutfs_parallel_restore_writer *wri,
void *buf, size_t buf_size, int dev_fd)
{
size_t total = 0;
size_t count;
off_t off;
int ret;
do {
ret = scoutfs_parallel_restore_write_buf(wri, buf, buf_size, &off, &count);
error_exit(ret, "write buf %d", ret);
if (count > 0) {
if (!opts->read_only)
ret = pwrite(dev_fd, buf, count, off);
else
ret = count;
error_exit(ret != count, "pwrite count %zu ret %d", count, ret);
total += ret;
}
} while (count > 0);
return total;
}
struct gen_inode {
struct scoutfs_parallel_restore_inode inode;
struct scoutfs_parallel_restore_xattr **xattrs;
u64 nr_xattrs;
struct scoutfs_parallel_restore_entry **entries;
u64 nr_files;
u64 nr_entries;
};
static void free_gino(struct gen_inode *gino)
{
u64 i;
if (gino) {
if (gino->entries) {
for (i = 0; i < gino->nr_entries; i++)
free(gino->entries[i]);
free(gino->entries);
}
if (gino->xattrs) {
for (i = 0; i < gino->nr_xattrs; i++)
free(gino->xattrs[i]);
free(gino->xattrs);
}
free(gino);
}
}
static struct scoutfs_parallel_restore_xattr *
generate_xattr(struct opts *opts, u64 ino, u64 pos, char *name, int name_len, void *value,
int value_len)
{
struct scoutfs_parallel_restore_xattr *xattr;
xattr = malloc(sizeof(struct scoutfs_parallel_restore_xattr) + name_len + value_len);
error_exit(!xattr, "error allocating generated xattr");
*xattr = (struct scoutfs_parallel_restore_xattr) {
.ino = ino,
.pos = pos,
.name_len = name_len,
.value_len = value_len,
};
xattr->name = (void *)(xattr + 1);
xattr->value = (void *)(xattr->name + name_len);
memcpy(xattr->name, name, name_len);
if (value_len)
memcpy(xattr->value, value, value_len);
return xattr;
}
static struct gen_inode *generate_inode(struct opts *opts, u64 ino, mode_t mode)
{
struct gen_inode *gino;
struct timespec now;
clock_gettime(CLOCK_REALTIME, &now);
gino = calloc(1, sizeof(struct gen_inode));
error_exit(!gino, "failure allocating generated inode");
gino->inode = (struct scoutfs_parallel_restore_inode) {
.ino = ino,
.mode = mode,
.atime = now,
.ctime = now,
.mtime = now,
.crtime = now,
};
/*
* hacky creation of a bunch of xattrs for now.
*/
if ((mode & S_IFMT) == S_IFREG) {
#define NV(n, v) { n, sizeof(n) - 1, v, sizeof(v) - 1, }
struct name_val {
char *name;
int len;
char *value;
int value_len;
} nv[] = {
NV("scoutfs.hide.totl.acct.8314611887310466424.2.0", "1"),
NV("scoutfs.hide.srch.sam_vol_E01001L6_4", ""),
NV("scoutfs.hide.sam_reqcopies", ""),
NV("scoutfs.hide.sam_copy_2", ""),
NV("scoutfs.hide.totl.acct.F01030L6.8314611887310466424.7.30", "1"),
NV("scoutfs.hide.sam_copy_1", ""),
NV("scoutfs.hide.srch.sam_vol_F01030L6_4", ""),
NV("scoutfs.hide.srch.sam_release_cand", ""),
NV("scoutfs.hide.sam_restime", ""),
NV("scoutfs.hide.sam_uuid", ""),
NV("scoutfs.hide.totl.acct.8314611887310466424.3.0", "1"),
NV("scoutfs.hide.srch.sam_vol_F01030L6", ""),
NV("scoutfs.hide.srch.sam_uuid_865939b7-24d6-472f-b85c-7ce7afeb813a", ""),
NV("scoutfs.hide.srch.sam_vol_E01001L6", ""),
NV("scoutfs.hide.totl.acct.E01001L6.8314611887310466424.7.1", "1"),
NV("scoutfs.hide.totl.acct.8314611887310466424.4.0", "1"),
NV("scoutfs.hide.totl.acct.8314611887310466424.11.0", "1"),
NV("scoutfs.hide.totl.acct.8314611887310466424.1.0", "1"),
};
unsigned int nr = array_size(nv);
int i;
gino->xattrs = calloc(nr, sizeof(struct scoutfs_parallel_restore_xattr *));
for (i = 0; i < nr; i++)
gino->xattrs[i] = generate_xattr(opts, ino, i, nv[i].name, nv[i].len,
nv[i].value, nv[i].value_len);
gino->nr_xattrs = nr;
gino->inode.nr_xattrs = nr;
}
return gino;
}
static struct scoutfs_parallel_restore_entry *
generate_entry(struct opts *opts, char *prefix, u64 nr, u64 dir_ino, u64 pos, u64 ino, mode_t mode)
{
struct scoutfs_parallel_restore_entry *entry;
char buf[PATH_MAX];
int bytes;
bytes = snprintf(buf, sizeof(buf), "%s-%llu", prefix, nr);
entry = malloc(sizeof(struct scoutfs_parallel_restore_entry) + bytes);
error_exit(!entry, "error allocating generated entry");
*entry = (struct scoutfs_parallel_restore_entry) {
.dir_ino = dir_ino,
.pos = pos,
.ino = ino,
.mode = mode,
.name = (void *)(entry + 1),
.name_len = bytes,
};
memcpy(entry->name, buf, bytes);
return entry;
}
static u64 random64(void)
{
return ((u64)lrand48() << 32) | lrand48();
}
static u64 random_range(u64 low, u64 high)
{
return low + (random64() % (high - low + 1));
}
static struct gen_inode *generate_dir(struct opts *opts, u64 dir_ino, u64 ino_start, u64 ino_len,
bool no_dirs)
{
struct scoutfs_parallel_restore_entry *entry;
struct gen_inode *gino;
u64 nr_entries;
u64 nr_files;
u64 nr_dirs;
u64 ino;
char *prefix;
mode_t mode;
u64 i;
nr_dirs = no_dirs ? 0 : random_range(opts->low_dirs, opts->high_dirs);
nr_files = random_range(opts->low_files, opts->high_files);
if (1 + nr_dirs + nr_files > ino_len) {
nr_dirs = no_dirs ? 0 : (ino_len - 1) / 2;
nr_files = (ino_len - 1) - nr_dirs;
}
nr_entries = nr_dirs + nr_files;
gino = generate_inode(opts, dir_ino, DIR_MODE);
error_exit(!gino, "error allocating generated inode");
gino->inode.nr_subdirs = nr_dirs;
gino->nr_files = nr_files;
if (nr_entries) {
gino->entries = calloc(nr_entries, sizeof(struct scoutfs_parallel_restore_entry *));
error_exit(!gino->entries, "error allocating generated inode entries");
gino->nr_entries = nr_entries;
}
mode = DIR_MODE;
prefix = "dir";
for (i = 0; i < nr_entries; i++) {
if (i == nr_dirs) {
mode = REG_MODE;
prefix = "file";
}
ino = ino_start + i;
entry = generate_entry(opts, prefix, ino, gino->inode.ino,
SCOUTFS_DIRENT_FIRST_POS + i, ino, mode);
gino->entries[i] = entry;
gino->inode.total_entry_name_bytes += entry->name_len;
}
return gino;
}
/*
* Restore a generated inode. If it's a directory then we also restore
* all its entries. The caller is going to descend into subdir entries and generate
* those dir inodes. We have to generate and restore all non-dir inodes referenced
* by this inode's entries.
*/
static void restore_inode(struct opts *opts, struct scoutfs_parallel_restore_writer *wri,
struct gen_inode *gino)
{
struct gen_inode *nondir;
int ret;
u64 i;
ret = scoutfs_parallel_restore_add_inode(wri, &gino->inode);
error_exit(ret, "thread add root inode %d", ret);
for (i = 0; i < gino->nr_entries; i++) {
ret = scoutfs_parallel_restore_add_entry(wri, gino->entries[i]);
error_exit(ret, "thread add entry %d", ret);
/* caller only needs subdir entries, generate and free others */
if ((gino->entries[i]->mode & S_IFMT) != S_IFDIR) {
nondir = generate_inode(opts, gino->entries[i]->ino,
gino->entries[i]->mode);
restore_inode(opts, wri, nondir);
free_gino(nondir);
free(gino->entries[i]);
if (i != gino->nr_entries - 1)
gino->entries[i] = gino->entries[gino->nr_entries - 1];
gino->nr_entries--;
gino->nr_files--;
i--;
}
}
for (i = 0; i < gino->nr_xattrs; i++) {
ret = scoutfs_parallel_restore_add_xattr(wri, gino->xattrs[i]);
error_exit(ret, "thread add xattr %d", ret);
}
}
struct writer_args {
struct list_head head;
int dev_fd;
int pair_fd;
struct scoutfs_parallel_restore_slice slice;
u64 writer_nr;
u64 dir_height;
u64 ino_start;
u64 ino_len;
};
struct write_result {
struct scoutfs_parallel_restore_progress prog;
struct scoutfs_parallel_restore_slice slice;
__le64 files_created;
__le64 bytes_written;
};
static void write_bufs_and_send(struct opts *opts, struct scoutfs_parallel_restore_writer *wri,
void *buf, size_t buf_size, int dev_fd,
struct write_result *res, bool get_slice, int pair_fd)
{
size_t total;
int ret;
total = write_bufs(opts, wri, buf, buf_size, dev_fd);
le64_add_cpu(&res->bytes_written, total);
ret = scoutfs_parallel_restore_get_progress(wri, &res->prog);
error_exit(ret, "get prog %d", ret);
if (get_slice) {
ret = scoutfs_parallel_restore_get_slice(wri, &res->slice);
error_exit(ret, "thread get slice %d", ret);
}
ret = write(pair_fd, res, sizeof(struct write_result));
error_exit(ret != sizeof(struct write_result), "result send error");
memset(res, 0, sizeof(struct write_result));
}
/*
* Calculate the number of bytes in toplevel "dir-%llu" entry names for the given
* number of writers.
*/
static u64 topdir_entry_bytes(u64 nr_writers)
{
u64 bytes = (3 + 1) * nr_writers;
u64 limit;
u64 done;
u64 wid;
u64 nr;
for (done = 0, wid = 1, limit = 10; done < nr_writers; done += nr, wid++, limit *= 10) {
nr = min(limit - done, nr_writers - done);
bytes += nr * wid;
}
return bytes;
}
struct dir_pos {
struct gen_inode *gino;
u64 pos;
};
static void writer_proc(struct opts *opts, struct writer_args *args)
{
struct scoutfs_parallel_restore_writer *wri = NULL;
struct scoutfs_parallel_restore_entry *entry;
struct dir_pos *dirs = NULL;
struct write_result res;
struct gen_inode *gino;
void *buf = NULL;
u64 level;
u64 ino;
int ret;
memset(&res, 0, sizeof(res));
dirs = calloc(args->dir_height, sizeof(struct dir_pos));
error_exit(errno, "error allocating parent dirs "ERRF, ERRA);
errno = posix_memalign((void **)&buf, 4096, opts->buf_size);
error_exit(errno, "error allocating block buf "ERRF, ERRA);
ret = scoutfs_parallel_restore_create_writer(&wri);
error_exit(ret, "create writer %d", ret);
ret = scoutfs_parallel_restore_add_slice(wri, &args->slice);
error_exit(ret, "add slice %d", ret);
/* writer 0 creates the root dir */
if (args->writer_nr == 0) {
gino = generate_inode(opts, SCOUTFS_ROOT_INO, DIR_MODE);
gino->inode.nr_subdirs = opts->nr_writers;
gino->inode.total_entry_name_bytes = topdir_entry_bytes(opts->nr_writers);
ret = scoutfs_parallel_restore_add_inode(wri, &gino->inode);
error_exit(ret, "thread add root inode %d", ret);
free_gino(gino);
}
/* create root entry for our top level dir */
ino = args->ino_start++;
args->ino_len--;
entry = generate_entry(opts, "top", args->writer_nr,
SCOUTFS_ROOT_INO, SCOUTFS_DIRENT_FIRST_POS + args->writer_nr,
ino, DIR_MODE);
ret = scoutfs_parallel_restore_add_entry(wri, entry);
error_exit(ret, "thread top entry %d", ret);
free(entry);
level = args->dir_height - 1;
while (args->ino_len > 0 && level < args->dir_height) {
gino = dirs[level].gino;
/* generate and restore if we follow entries */
if (!gino) {
gino = generate_dir(opts, ino, args->ino_start, args->ino_len, level == 0);
args->ino_start += gino->nr_entries;
args->ino_len -= gino->nr_entries;
le64_add_cpu(&res.files_created, gino->nr_files);
restore_inode(opts, wri, gino);
dirs[level].gino = gino;
}
if (dirs[level].pos == gino->nr_entries) {
/* ascend if we're done with this dir */
dirs[level].gino = NULL;
dirs[level].pos = 0;
free_gino(gino);
level++;
} else {
/* otherwise descend into subdir entry */
ino = gino->entries[dirs[level].pos]->ino;
dirs[level].pos++;
level--;
}
/* do a partial write at batch intervals when there's still more to do */
if (le64_to_cpu(res.files_created) >= opts->write_batch && args->ino_len > 0)
write_bufs_and_send(opts, wri, buf, opts->buf_size, args->dev_fd,
&res, false, args->pair_fd);
}
write_bufs_and_send(opts, wri, buf, opts->buf_size, args->dev_fd,
&res, true, args->pair_fd);
scoutfs_parallel_restore_destroy_writer(&wri);
free(dirs);
free(buf);
}
static void fork_writer(struct opts *opts, struct writer_args *args)
{
pid_t parent = getpid();
pid_t pid;
int ret;
pid = fork();
error_exit(pid == -1, "fork error");
if (pid != 0)
return;
ret = prctl(PR_SET_PDEATHSIG, SIGHUP);
error_exit(ret < 0, "failed to set parent death sig");
printf("pid %u getpid() %u parent %u getppid() %u\n",
pid, getpid(), parent, getppid());
error_exit(getppid() != parent, "child parent already changed");
writer_proc(opts, args);
exit(0);
}
static int do_restore(struct opts *opts)
{
struct scoutfs_parallel_restore_writer *wri = NULL;
struct scoutfs_parallel_restore_slice *slices = NULL;
struct scoutfs_super_block *super = NULL;
struct write_result res;
struct writer_args *args;
struct timespec begin;
struct timespec end;
LIST_HEAD(writers);
u64 next_ino;
u64 ino_per;
u64 avg_dirs;
u64 avg_files;
u64 dir_height;
u64 tot_files;
u64 tot_bytes;
int pair[2] = {-1, -1};
float secs;
void *buf = NULL;
int dev_fd = -1;
int ret;
int i;
ret = socketpair(PF_LOCAL, SOCK_STREAM, 0, pair);
error_exit(ret, "socketpair error "ERRF, ERRA);
dev_fd = open(opts->meta_path, O_DIRECT | (opts->read_only ? O_RDONLY : (O_RDWR|O_EXCL)));
error_exit(dev_fd < 0, "error opening '%s': "ERRF, opts->meta_path, ERRA);
errno = posix_memalign((void **)&super, 4096, SCOUTFS_BLOCK_SM_SIZE) ?:
posix_memalign((void **)&buf, 4096, opts->buf_size);
error_exit(errno, "error allocating block bufs "ERRF, ERRA);
ret = pread(dev_fd, super, SCOUTFS_BLOCK_SM_SIZE,
SCOUTFS_SUPER_BLKNO << SCOUTFS_BLOCK_SM_SHIFT);
error_exit(ret != SCOUTFS_BLOCK_SM_SIZE, "error reading super, ret %d", ret);
ret = scoutfs_parallel_restore_create_writer(&wri);
error_exit(ret, "create writer %d", ret);
ret = scoutfs_parallel_restore_import_super(wri, super);
error_exit(ret, "import super %d", ret);
slices = calloc(1 + opts->nr_writers, sizeof(struct scoutfs_parallel_restore_slice));
error_exit(!slices, "alloc slices");
scoutfs_parallel_restore_init_slices(wri, slices, 1 + opts->nr_writers);
ret = scoutfs_parallel_restore_add_slice(wri, &slices[0]);
error_exit(ret, "add slices[0] %d", ret);
next_ino = (SCOUTFS_ROOT_INO | SCOUTFS_LOCK_INODE_GROUP_MASK) + 1;
ino_per = opts->total_files / opts->nr_writers;
avg_dirs = (opts->low_dirs + opts->high_dirs) / 2;
avg_files = (opts->low_files + opts->high_files) / 2;
dir_height = 1;
tot_files = avg_files * opts->nr_writers;
while (tot_files < opts->total_files) {
dir_height++;
tot_files *= avg_dirs;
}
dprintf("height %llu tot %llu total %llu\n", dir_height, tot_files, opts->total_files);
clock_gettime(CLOCK_MONOTONIC_RAW, &begin);
/* start each writing process */
for (i = 0; i < opts->nr_writers; i++) {
args = calloc(1, sizeof(struct writer_args));
error_exit(!args, "alloc writer args");
args->dev_fd = dev_fd;
args->pair_fd = pair[1];
args->slice = slices[1 + i];
args->writer_nr = i;
args->dir_height = dir_height;
args->ino_start = next_ino;
args->ino_len = ino_per;
list_add_tail(&args->head, &writers);
next_ino += ino_per;
fork_writer(opts, args);
}
/* read results and watch for writers to finish */
tot_files = 0;
tot_bytes = 0;
i = 0;
while (i < opts->nr_writers) {
ret = read(pair[0], &res, sizeof(struct write_result));
error_exit(ret != sizeof(struct write_result), "result read error");
ret = scoutfs_parallel_restore_add_progress(wri, &res.prog);
error_exit(ret, "add thr prog %d", ret);
if (res.slice.meta_len != 0) {
ret = scoutfs_parallel_restore_add_slice(wri, &res.slice);
error_exit(ret, "add thr slice %d", ret);
i++;
}
tot_files += le64_to_cpu(res.files_created);
tot_bytes += le64_to_cpu(res.bytes_written);
}
tot_bytes += write_bufs(opts, wri, buf, opts->buf_size, dev_fd);
ret = scoutfs_parallel_restore_export_super(wri, super);
error_exit(ret, "update super %d", ret);
if (!opts->read_only) {
ret = pwrite(dev_fd, super, SCOUTFS_BLOCK_SM_SIZE,
SCOUTFS_SUPER_BLKNO << SCOUTFS_BLOCK_SM_SHIFT);
error_exit(ret != SCOUTFS_BLOCK_SM_SIZE, "error writing super, ret %d", ret);
}
clock_gettime(CLOCK_MONOTONIC_RAW, &end);
scoutfs_parallel_restore_destroy_writer(&wri);
secs = ((float)end.tv_sec + ((float)end.tv_nsec/NSEC_PER_SEC)) -
((float)begin.tv_sec + ((float)begin.tv_nsec/NSEC_PER_SEC));
printf("created %llu files in %llu bytes and %f secs => %f bytes/file, %f files/sec\n",
tot_files, tot_bytes, secs,
(float)tot_bytes / tot_files, (float)tot_files / secs);
if (dev_fd >= 0)
close(dev_fd);
if (pair[0] >= 0)
close(pair[0]);
if (pair[1] >= 0)
close(pair[1]);
free(super);
free(slices);
free(buf);
return 0;
}
static int parse_low_high(char *str, u64 *low_ret, u64 *high_ret)
{
char *sep;
int ret = 0;
sep = index(str, ':');
if (sep) {
*sep = '\0';
ret = parse_u64(sep + 1, high_ret);
}
if (ret == 0)
ret = parse_u64(str, low_ret);
if (sep)
*sep = ':';
return ret;
}
int main(int argc, char **argv)
{
struct opts opts = {
.buf_size = (32 * 1024 * 1024),
.write_batch = 1000000,
.low_dirs = 5,
.high_dirs = 10,
.low_files = 10,
.high_files = 20,
.total_files = 100,
};
int ret;
int c;
opts.seed = random64();
opts.nr_writers = sysconf(_SC_NPROCESSORS_ONLN);
while ((c = getopt(argc, argv, "b:d:f:m:n:rs:w:")) != -1) {
switch(c) {
case 'b':
ret = parse_u64(optarg, &opts.write_batch);
error_exit(ret, "error parsing -b '%s'\n", optarg);
error_exit(opts.write_batch == 0, "-b can't be 0");
break;
case 'd':
ret = parse_low_high(optarg, &opts.low_dirs, &opts.high_dirs);
error_exit(ret, "error parsing -d '%s'\n", optarg);
break;
case 'f':
ret = parse_low_high(optarg, &opts.low_files, &opts.high_files);
error_exit(ret, "error parsing -f '%s'\n", optarg);
break;
case 'm':
opts.meta_path = strdup(optarg);
break;
case 'n':
ret = parse_u64(optarg, &opts.total_files);
error_exit(ret, "error parsing -n '%s'\n", optarg);
break;
case 'r':
opts.read_only = true;
break;
case 's':
ret = parse_u64(optarg, &opts.seed);
error_exit(ret, "error parsing -s '%s'\n", optarg);
break;
case 'w':
ret = parse_u64(optarg, &opts.nr_writers);
error_exit(ret, "error parsing -w '%s'\n", optarg);
break;
case '?':
printf("Unknown option '%c'\n", optopt);
usage();
exit(1);
}
}
error_exit(opts.low_dirs > opts.high_dirs, "LOW > HIGH in -d %llu:%llu",
opts.low_dirs, opts.high_dirs);
error_exit(opts.low_files > opts.high_files, "LOW > HIGH in -f %llu:%llu",
opts.low_files, opts.high_files);
error_exit(!opts.meta_path, "must specify metadata device path with -m");
printf("recreate with: -d %llu:%llu -f %llu:%llu -n %llu -s %llu -w %llu\n",
opts.low_dirs, opts.high_dirs, opts.low_files, opts.high_files,
opts.total_files, opts.seed, opts.nr_writers);
ret = do_restore(&opts);
free(opts.meta_path);
return ret == 0 ? 0 : 1;
}

View File

@@ -18,7 +18,9 @@ BIN := src/scoutfs
OBJ := $(patsubst %.c,%.o,$(wildcard src/*.c))
DEPS := $(wildcard */*.d)
all: $(BIN)
AR := src/scoutfs_parallel_restore.a
all: $(BIN) $(AR)
ifneq ($(DEPS),)
-include $(DEPS)
@@ -36,6 +38,10 @@ $(BIN): $(OBJ)
$(QU) [BIN $@]
$(VE)gcc -o $@ $^ -luuid -lm -lcrypto -lblkid
$(AR): $(OBJ)
$(QU) [AR $@]
$(VE)ar rcs $@ $^
%.o %.d: %.c Makefile sparse.sh
$(QU) [CC $<]
$(VE)gcc $(CFLAGS) -MD -MP -MF $*.d -c $< -o $*.o

View File

@@ -54,6 +54,8 @@ cp man/*.8.gz $RPM_BUILD_ROOT%{_mandir}/man8/.
install -m 755 -D src/scoutfs $RPM_BUILD_ROOT%{_sbindir}/scoutfs
install -m 644 -D src/ioctl.h $RPM_BUILD_ROOT%{_includedir}/scoutfs/ioctl.h
install -m 644 -D src/format.h $RPM_BUILD_ROOT%{_includedir}/scoutfs/format.h
install -m 644 -D src/parallel_restore.h $RPM_BUILD_ROOT%{_includedir}/scoutfs/parallel_restore.h
install -m 644 -D src/scoutfs_parallel_restore.a $RPM_BUILD_ROOT%{_libdir}/scoutfs/libscoutfs_parallel_restore.a
install -m 755 -D fenced/scoutfs-fenced $RPM_BUILD_ROOT%{_libexecdir}/scoutfs-fenced/scoutfs-fenced
install -m 644 -D fenced/scoutfs-fenced.service $RPM_BUILD_ROOT%{_unitdir}/scoutfs-fenced.service
install -m 644 -D fenced/scoutfs-fenced.conf.example $RPM_BUILD_ROOT%{_sysconfdir}/scoutfs/scoutfs-fenced.conf.example
@@ -70,6 +72,7 @@ install -m 644 -D fenced/scoutfs-fenced.conf.example $RPM_BUILD_ROOT%{_sysconfdi
%files -n scoutfs-devel
%defattr(644,root,root,755)
%{_includedir}/scoutfs
%{_libdir}/scoutfs
%clean
rm -rf %{buildroot}

20
utils/src/bloom.c Normal file
View File

@@ -0,0 +1,20 @@
#include <errno.h>
#include "sparse.h"
#include "util.h"
#include "format.h"
#include "hash.h"
#include "bloom.h"
void calc_bloom_nrs(struct scoutfs_key *key, unsigned int *nrs)
{
u64 hash;
int i;
hash = scoutfs_hash64(key, sizeof(struct scoutfs_key));
for (i = 0; i < SCOUTFS_FOREST_BLOOM_NRS; i++) {
nrs[i] = (u32)hash % SCOUTFS_FOREST_BLOOM_BITS;
hash >>= SCOUTFS_FOREST_BLOOM_FUNC_BITS;
}
}

6
utils/src/bloom.h Normal file
View File

@@ -0,0 +1,6 @@
#ifndef _BLOOM_H_
#define _BLOOM_H_
void calc_bloom_nrs(struct scoutfs_key *key, unsigned int *nrs);
#endif

View File

@@ -8,7 +8,7 @@
#include "leaf_item_hash.h"
#include "btree.h"
static void init_block(struct scoutfs_btree_block *bt, int level)
void btree_init_block(struct scoutfs_btree_block *bt, int level)
{
int free;
@@ -33,7 +33,7 @@ void btree_init_root_single(struct scoutfs_btree_root *root,
memset(bt, 0, SCOUTFS_BLOCK_LG_SIZE);
init_block(bt, 0);
btree_init_block(bt, 0);
}
static void *alloc_val(struct scoutfs_btree_block *bt, int len)

View File

@@ -1,6 +1,7 @@
#ifndef _BTREE_H_
#define _BTREE_H_
void btree_init_block(struct scoutfs_btree_block *bt, int level);
void btree_init_root_single(struct scoutfs_btree_root *root,
struct scoutfs_btree_block *bt,
u64 seq, u64 blkno);

View File

@@ -0,0 +1,24 @@
#ifndef _LK_RBTREE_WRAPPER_H_
#define _LK_RBTREE_WRAPPER_H_
/*
* We're using this lame hack to build and use the kernel's rbtree in
* userspace. We drop the kernel's rbtree*[ch] implementation in and
* use them with this wrapper. We only have to remove the kernel
* includes from the imported files.
*/
#include <stdbool.h>
#include "util.h"
#define rcu_assign_pointer(a, b) do { a = b; } while (0)
#define READ_ONCE(a) ({ a; })
#define WRITE_ONCE(a, b) do { a = b; } while (0)
#define unlikely(a) ({ a; })
#define EXPORT_SYMBOL(a) /* nop */
#include "rbtree_types.h"
#include "rbtree.h"
#include "rbtree_augmented.h"
#endif

24
utils/src/mode_types.c Normal file
View File

@@ -0,0 +1,24 @@
#include <unistd.h>
#include <sys/stat.h>
#include "sparse.h"
#include "util.h"
#include "format.h"
#include "mode_types.h"
unsigned int mode_to_type(mode_t mode)
{
#define S_SHIFT 12
static unsigned char mode_types[S_IFMT >> S_SHIFT] = {
[S_IFIFO >> S_SHIFT] = SCOUTFS_DT_FIFO,
[S_IFCHR >> S_SHIFT] = SCOUTFS_DT_CHR,
[S_IFDIR >> S_SHIFT] = SCOUTFS_DT_DIR,
[S_IFBLK >> S_SHIFT] = SCOUTFS_DT_BLK,
[S_IFREG >> S_SHIFT] = SCOUTFS_DT_REG,
[S_IFLNK >> S_SHIFT] = SCOUTFS_DT_LNK,
[S_IFSOCK >> S_SHIFT] = SCOUTFS_DT_SOCK,
};
return mode_types[(mode & S_IFMT) >> S_SHIFT];
#undef S_SHIFT
}

6
utils/src/mode_types.h Normal file
View File

@@ -0,0 +1,6 @@
#ifndef _MODE_TYPES_H_
#define _MODE_TYPES_H_
unsigned int mode_to_type(mode_t mode);
#endif

46
utils/src/name_hash.h Normal file
View File

@@ -0,0 +1,46 @@
#ifndef _SCOUTFS_NAME_HASH_H_
#define _SCOUTFS_NAME_HASH_H_
#include "hash.h"
/*
* Test a bit number as though an array of bytes is a large len-bit
* big-endian value. nr 0 is the LSB of the final byte, nr (len - 1) is
* the MSB of the first byte.
*/
static int test_be_bytes_bit(int nr, const char *bytes, int len)
{
return bytes[(len - 1 - nr) >> 3] & (1 << (nr & 7));
}
/*
* Generate a 32bit "fingerprint" of the name by extracting 32 evenly
* distributed bits from the name. The intent is to have the sort order
* of the fingerprints reflect the memcmp() sort order of the names
* while mapping large names down to small fs keys.
*
* Names that are smaller than 32bits are biased towards the high bits
* of the fingerprint so that most significant bits of the fingerprints
* consistently reflect the initial characters of the names.
*/
static inline u32 dirent_name_fingerprint(const char *name, unsigned int name_len)
{
int name_bits = name_len * 8;
int skip = max(name_bits / 32, 1);
u32 fp = 0;
int f;
int n;
for (f = 31, n = name_bits - 1; f >= 0 && n >= 0; f--, n -= skip)
fp |= !!test_be_bytes_bit(n, name, name_bits) << f;
return fp;
}
static inline u64 dirent_name_hash(const char *name, unsigned int name_len)
{
return scoutfs_hash32(name, name_len) |
((u64)dirent_name_fingerprint(name, name_len) << 32);
}
#endif

1871
utils/src/parallel_restore.c Normal file

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,103 @@
#ifndef _SCOUTFS_PARALLEL_RESTORE_H_
#define _SCOUTFS_PARALLEL_RESTORE_H_
#include <errno.h>
struct scoutfs_parallel_restore_progress {
struct scoutfs_btree_root fs_items;
struct scoutfs_btree_root root_items;
struct scoutfs_srch_file sfl;
struct scoutfs_block_ref bloom_ref;
__le64 inode_count;
__le64 max_ino;
};
struct scoutfs_parallel_restore_slice {
__le64 fsid;
__le64 meta_start;
__le64 meta_len;
};
struct scoutfs_parallel_restore_entry {
u64 dir_ino;
u64 pos;
u64 ino;
mode_t mode;
char *name;
unsigned int name_len;
};
struct scoutfs_parallel_restore_xattr {
u64 ino;
u64 pos;
char *name;
unsigned int name_len;
void *value;
unsigned int value_len;
};
struct scoutfs_parallel_restore_inode {
/* all inodes */
u64 ino;
u64 nr_xattrs;
u32 uid;
u32 gid;
u32 mode;
u32 rdev;
struct timespec atime;
struct timespec ctime;
struct timespec mtime;
struct timespec crtime;
/* regular files */
u64 data_version;
u64 size;
bool offline;
/* only used for directories */
u64 nr_subdirs;
u64 total_entry_name_bytes;
/* only used for symlnks */
char *target;
unsigned int target_len; /* not including null terminator */
};
typedef __typeof__(EINVAL) spr_err_t;
struct scoutfs_parallel_restore_writer;
spr_err_t scoutfs_parallel_restore_create_writer(struct scoutfs_parallel_restore_writer **wrip);
void scoutfs_parallel_restore_destroy_writer(struct scoutfs_parallel_restore_writer **wrip);
spr_err_t scoutfs_parallel_restore_init_slices(struct scoutfs_parallel_restore_writer *wri,
struct scoutfs_parallel_restore_slice *slices,
int nr);
spr_err_t scoutfs_parallel_restore_add_slice(struct scoutfs_parallel_restore_writer *wri,
struct scoutfs_parallel_restore_slice *slice);
spr_err_t scoutfs_parallel_restore_get_slice(struct scoutfs_parallel_restore_writer *wri,
struct scoutfs_parallel_restore_slice *slice);
spr_err_t scoutfs_parallel_restore_add_inode(struct scoutfs_parallel_restore_writer *wri,
struct scoutfs_parallel_restore_inode *inode);
spr_err_t scoutfs_parallel_restore_add_entry(struct scoutfs_parallel_restore_writer *wri,
struct scoutfs_parallel_restore_entry *entry);
spr_err_t scoutfs_parallel_restore_add_xattr(struct scoutfs_parallel_restore_writer *wri,
struct scoutfs_parallel_restore_xattr *xattr);
spr_err_t scoutfs_parallel_restore_get_progress(struct scoutfs_parallel_restore_writer *wri,
struct scoutfs_parallel_restore_progress *prog);
spr_err_t scoutfs_parallel_restore_add_progress(struct scoutfs_parallel_restore_writer *wri,
struct scoutfs_parallel_restore_progress *prog);
spr_err_t scoutfs_parallel_restore_write_buf(struct scoutfs_parallel_restore_writer *wri,
void *buf, size_t len, off_t *off_ret,
size_t *count_ret);
spr_err_t scoutfs_parallel_restore_import_super(struct scoutfs_parallel_restore_writer *wri,
struct scoutfs_super_block *super);
spr_err_t scoutfs_parallel_restore_export_super(struct scoutfs_parallel_restore_writer *wri,
struct scoutfs_super_block *super);
#endif

629
utils/src/rbtree.c Normal file
View File

@@ -0,0 +1,629 @@
// SPDX-License-Identifier: GPL-2.0-or-later
/*
Red Black Trees
(C) 1999 Andrea Arcangeli <andrea@suse.de>
(C) 2002 David Woodhouse <dwmw2@infradead.org>
(C) 2012 Michel Lespinasse <walken@google.com>
linux/lib/rbtree.c
*/
#include "lk_rbtree_wrapper.h"
/*
* red-black trees properties: https://en.wikipedia.org/wiki/Rbtree
*
* 1) A node is either red or black
* 2) The root is black
* 3) All leaves (NULL) are black
* 4) Both children of every red node are black
* 5) Every simple path from root to leaves contains the same number
* of black nodes.
*
* 4 and 5 give the O(log n) guarantee, since 4 implies you cannot have two
* consecutive red nodes in a path and every red node is therefore followed by
* a black. So if B is the number of black nodes on every simple path (as per
* 5), then the longest possible path due to 4 is 2B.
*
* We shall indicate color with case, where black nodes are uppercase and red
* nodes will be lowercase. Unknown color nodes shall be drawn as red within
* parentheses and have some accompanying text comment.
*/
/*
* Notes on lockless lookups:
*
* All stores to the tree structure (rb_left and rb_right) must be done using
* WRITE_ONCE(). And we must not inadvertently cause (temporary) loops in the
* tree structure as seen in program order.
*
* These two requirements will allow lockless iteration of the tree -- not
* correct iteration mind you, tree rotations are not atomic so a lookup might
* miss entire subtrees.
*
* But they do guarantee that any such traversal will only see valid elements
* and that it will indeed complete -- does not get stuck in a loop.
*
* It also guarantees that if the lookup returns an element it is the 'correct'
* one. But not returning an element does _NOT_ mean it's not present.
*
* NOTE:
*
* Stores to __rb_parent_color are not important for simple lookups so those
* are left undone as of now. Nor did I check for loops involving parent
* pointers.
*/
static inline void rb_set_black(struct rb_node *rb)
{
rb->__rb_parent_color |= RB_BLACK;
}
static inline struct rb_node *rb_red_parent(struct rb_node *red)
{
return (struct rb_node *)red->__rb_parent_color;
}
/*
* Helper function for rotations:
* - old's parent and color get assigned to new
* - old gets assigned new as a parent and 'color' as a color.
*/
static inline void
__rb_rotate_set_parents(struct rb_node *old, struct rb_node *new,
struct rb_root *root, int color)
{
struct rb_node *parent = rb_parent(old);
new->__rb_parent_color = old->__rb_parent_color;
rb_set_parent_color(old, new, color);
__rb_change_child(old, new, parent, root);
}
static __always_inline void
__rb_insert(struct rb_node *node, struct rb_root *root,
void (*augment_rotate)(struct rb_node *old, struct rb_node *new))
{
struct rb_node *parent = rb_red_parent(node), *gparent, *tmp;
while (true) {
/*
* Loop invariant: node is red.
*/
if (unlikely(!parent)) {
/*
* The inserted node is root. Either this is the
* first node, or we recursed at Case 1 below and
* are no longer violating 4).
*/
rb_set_parent_color(node, NULL, RB_BLACK);
break;
}
/*
* If there is a black parent, we are done.
* Otherwise, take some corrective action as,
* per 4), we don't want a red root or two
* consecutive red nodes.
*/
if(rb_is_black(parent))
break;
gparent = rb_red_parent(parent);
tmp = gparent->rb_right;
if (parent != tmp) { /* parent == gparent->rb_left */
if (tmp && rb_is_red(tmp)) {
/*
* Case 1 - node's uncle is red (color flips).
*
* G g
* / \ / \
* p u --> P U
* / /
* n n
*
* However, since g's parent might be red, and
* 4) does not allow this, we need to recurse
* at g.
*/
rb_set_parent_color(tmp, gparent, RB_BLACK);
rb_set_parent_color(parent, gparent, RB_BLACK);
node = gparent;
parent = rb_parent(node);
rb_set_parent_color(node, parent, RB_RED);
continue;
}
tmp = parent->rb_right;
if (node == tmp) {
/*
* Case 2 - node's uncle is black and node is
* the parent's right child (left rotate at parent).
*
* G G
* / \ / \
* p U --> n U
* \ /
* n p
*
* This still leaves us in violation of 4), the
* continuation into Case 3 will fix that.
*/
tmp = node->rb_left;
WRITE_ONCE(parent->rb_right, tmp);
WRITE_ONCE(node->rb_left, parent);
if (tmp)
rb_set_parent_color(tmp, parent,
RB_BLACK);
rb_set_parent_color(parent, node, RB_RED);
augment_rotate(parent, node);
parent = node;
tmp = node->rb_right;
}
/*
* Case 3 - node's uncle is black and node is
* the parent's left child (right rotate at gparent).
*
* G P
* / \ / \
* p U --> n g
* / \
* n U
*/
WRITE_ONCE(gparent->rb_left, tmp); /* == parent->rb_right */
WRITE_ONCE(parent->rb_right, gparent);
if (tmp)
rb_set_parent_color(tmp, gparent, RB_BLACK);
__rb_rotate_set_parents(gparent, parent, root, RB_RED);
augment_rotate(gparent, parent);
break;
} else {
tmp = gparent->rb_left;
if (tmp && rb_is_red(tmp)) {
/* Case 1 - color flips */
rb_set_parent_color(tmp, gparent, RB_BLACK);
rb_set_parent_color(parent, gparent, RB_BLACK);
node = gparent;
parent = rb_parent(node);
rb_set_parent_color(node, parent, RB_RED);
continue;
}
tmp = parent->rb_left;
if (node == tmp) {
/* Case 2 - right rotate at parent */
tmp = node->rb_right;
WRITE_ONCE(parent->rb_left, tmp);
WRITE_ONCE(node->rb_right, parent);
if (tmp)
rb_set_parent_color(tmp, parent,
RB_BLACK);
rb_set_parent_color(parent, node, RB_RED);
augment_rotate(parent, node);
parent = node;
tmp = node->rb_left;
}
/* Case 3 - left rotate at gparent */
WRITE_ONCE(gparent->rb_right, tmp); /* == parent->rb_left */
WRITE_ONCE(parent->rb_left, gparent);
if (tmp)
rb_set_parent_color(tmp, gparent, RB_BLACK);
__rb_rotate_set_parents(gparent, parent, root, RB_RED);
augment_rotate(gparent, parent);
break;
}
}
}
/*
* Inline version for rb_erase() use - we want to be able to inline
* and eliminate the dummy_rotate callback there
*/
static __always_inline void
____rb_erase_color(struct rb_node *parent, struct rb_root *root,
void (*augment_rotate)(struct rb_node *old, struct rb_node *new))
{
struct rb_node *node = NULL, *sibling, *tmp1, *tmp2;
while (true) {
/*
* Loop invariants:
* - node is black (or NULL on first iteration)
* - node is not the root (parent is not NULL)
* - All leaf paths going through parent and node have a
* black node count that is 1 lower than other leaf paths.
*/
sibling = parent->rb_right;
if (node != sibling) { /* node == parent->rb_left */
if (rb_is_red(sibling)) {
/*
* Case 1 - left rotate at parent
*
* P S
* / \ / \
* N s --> p Sr
* / \ / \
* Sl Sr N Sl
*/
tmp1 = sibling->rb_left;
WRITE_ONCE(parent->rb_right, tmp1);
WRITE_ONCE(sibling->rb_left, parent);
rb_set_parent_color(tmp1, parent, RB_BLACK);
__rb_rotate_set_parents(parent, sibling, root,
RB_RED);
augment_rotate(parent, sibling);
sibling = tmp1;
}
tmp1 = sibling->rb_right;
if (!tmp1 || rb_is_black(tmp1)) {
tmp2 = sibling->rb_left;
if (!tmp2 || rb_is_black(tmp2)) {
/*
* Case 2 - sibling color flip
* (p could be either color here)
*
* (p) (p)
* / \ / \
* N S --> N s
* / \ / \
* Sl Sr Sl Sr
*
* This leaves us violating 5) which
* can be fixed by flipping p to black
* if it was red, or by recursing at p.
* p is red when coming from Case 1.
*/
rb_set_parent_color(sibling, parent,
RB_RED);
if (rb_is_red(parent))
rb_set_black(parent);
else {
node = parent;
parent = rb_parent(node);
if (parent)
continue;
}
break;
}
/*
* Case 3 - right rotate at sibling
* (p could be either color here)
*
* (p) (p)
* / \ / \
* N S --> N sl
* / \ \
* sl Sr S
* \
* Sr
*
* Note: p might be red, and then both
* p and sl are red after rotation(which
* breaks property 4). This is fixed in
* Case 4 (in __rb_rotate_set_parents()
* which set sl the color of p
* and set p RB_BLACK)
*
* (p) (sl)
* / \ / \
* N sl --> P S
* \ / \
* S N Sr
* \
* Sr
*/
tmp1 = tmp2->rb_right;
WRITE_ONCE(sibling->rb_left, tmp1);
WRITE_ONCE(tmp2->rb_right, sibling);
WRITE_ONCE(parent->rb_right, tmp2);
if (tmp1)
rb_set_parent_color(tmp1, sibling,
RB_BLACK);
augment_rotate(sibling, tmp2);
tmp1 = sibling;
sibling = tmp2;
}
/*
* Case 4 - left rotate at parent + color flips
* (p and sl could be either color here.
* After rotation, p becomes black, s acquires
* p's color, and sl keeps its color)
*
* (p) (s)
* / \ / \
* N S --> P Sr
* / \ / \
* (sl) sr N (sl)
*/
tmp2 = sibling->rb_left;
WRITE_ONCE(parent->rb_right, tmp2);
WRITE_ONCE(sibling->rb_left, parent);
rb_set_parent_color(tmp1, sibling, RB_BLACK);
if (tmp2)
rb_set_parent(tmp2, parent);
__rb_rotate_set_parents(parent, sibling, root,
RB_BLACK);
augment_rotate(parent, sibling);
break;
} else {
sibling = parent->rb_left;
if (rb_is_red(sibling)) {
/* Case 1 - right rotate at parent */
tmp1 = sibling->rb_right;
WRITE_ONCE(parent->rb_left, tmp1);
WRITE_ONCE(sibling->rb_right, parent);
rb_set_parent_color(tmp1, parent, RB_BLACK);
__rb_rotate_set_parents(parent, sibling, root,
RB_RED);
augment_rotate(parent, sibling);
sibling = tmp1;
}
tmp1 = sibling->rb_left;
if (!tmp1 || rb_is_black(tmp1)) {
tmp2 = sibling->rb_right;
if (!tmp2 || rb_is_black(tmp2)) {
/* Case 2 - sibling color flip */
rb_set_parent_color(sibling, parent,
RB_RED);
if (rb_is_red(parent))
rb_set_black(parent);
else {
node = parent;
parent = rb_parent(node);
if (parent)
continue;
}
break;
}
/* Case 3 - left rotate at sibling */
tmp1 = tmp2->rb_left;
WRITE_ONCE(sibling->rb_right, tmp1);
WRITE_ONCE(tmp2->rb_left, sibling);
WRITE_ONCE(parent->rb_left, tmp2);
if (tmp1)
rb_set_parent_color(tmp1, sibling,
RB_BLACK);
augment_rotate(sibling, tmp2);
tmp1 = sibling;
sibling = tmp2;
}
/* Case 4 - right rotate at parent + color flips */
tmp2 = sibling->rb_right;
WRITE_ONCE(parent->rb_left, tmp2);
WRITE_ONCE(sibling->rb_right, parent);
rb_set_parent_color(tmp1, sibling, RB_BLACK);
if (tmp2)
rb_set_parent(tmp2, parent);
__rb_rotate_set_parents(parent, sibling, root,
RB_BLACK);
augment_rotate(parent, sibling);
break;
}
}
}
/* Non-inline version for rb_erase_augmented() use */
void __rb_erase_color(struct rb_node *parent, struct rb_root *root,
void (*augment_rotate)(struct rb_node *old, struct rb_node *new))
{
____rb_erase_color(parent, root, augment_rotate);
}
EXPORT_SYMBOL(__rb_erase_color);
/*
* Non-augmented rbtree manipulation functions.
*
* We use dummy augmented callbacks here, and have the compiler optimize them
* out of the rb_insert_color() and rb_erase() function definitions.
*/
static inline void dummy_propagate(struct rb_node *node, struct rb_node *stop) {}
static inline void dummy_copy(struct rb_node *old, struct rb_node *new) {}
static inline void dummy_rotate(struct rb_node *old, struct rb_node *new) {}
static const struct rb_augment_callbacks dummy_callbacks = {
.propagate = dummy_propagate,
.copy = dummy_copy,
.rotate = dummy_rotate
};
void rb_insert_color(struct rb_node *node, struct rb_root *root)
{
__rb_insert(node, root, dummy_rotate);
}
EXPORT_SYMBOL(rb_insert_color);
void rb_erase(struct rb_node *node, struct rb_root *root)
{
struct rb_node *rebalance;
rebalance = __rb_erase_augmented(node, root, &dummy_callbacks);
if (rebalance)
____rb_erase_color(rebalance, root, dummy_rotate);
}
EXPORT_SYMBOL(rb_erase);
/*
* Augmented rbtree manipulation functions.
*
* This instantiates the same __always_inline functions as in the non-augmented
* case, but this time with user-defined callbacks.
*/
void __rb_insert_augmented(struct rb_node *node, struct rb_root *root,
void (*augment_rotate)(struct rb_node *old, struct rb_node *new))
{
__rb_insert(node, root, augment_rotate);
}
EXPORT_SYMBOL(__rb_insert_augmented);
/*
* This function returns the first node (in sort order) of the tree.
*/
struct rb_node *rb_first(const struct rb_root *root)
{
struct rb_node *n;
n = root->rb_node;
if (!n)
return NULL;
while (n->rb_left)
n = n->rb_left;
return n;
}
EXPORT_SYMBOL(rb_first);
struct rb_node *rb_last(const struct rb_root *root)
{
struct rb_node *n;
n = root->rb_node;
if (!n)
return NULL;
while (n->rb_right)
n = n->rb_right;
return n;
}
EXPORT_SYMBOL(rb_last);
struct rb_node *rb_next(const struct rb_node *node)
{
struct rb_node *parent;
if (RB_EMPTY_NODE(node))
return NULL;
/*
* If we have a right-hand child, go down and then left as far
* as we can.
*/
if (node->rb_right) {
node = node->rb_right;
while (node->rb_left)
node = node->rb_left;
return (struct rb_node *)node;
}
/*
* No right-hand children. Everything down and left is smaller than us,
* so any 'next' node must be in the general direction of our parent.
* Go up the tree; any time the ancestor is a right-hand child of its
* parent, keep going up. First time it's a left-hand child of its
* parent, said parent is our 'next' node.
*/
while ((parent = rb_parent(node)) && node == parent->rb_right)
node = parent;
return parent;
}
EXPORT_SYMBOL(rb_next);
struct rb_node *rb_prev(const struct rb_node *node)
{
struct rb_node *parent;
if (RB_EMPTY_NODE(node))
return NULL;
/*
* If we have a left-hand child, go down and then right as far
* as we can.
*/
if (node->rb_left) {
node = node->rb_left;
while (node->rb_right)
node = node->rb_right;
return (struct rb_node *)node;
}
/*
* No left-hand children. Go up till we find an ancestor which
* is a right-hand child of its parent.
*/
while ((parent = rb_parent(node)) && node == parent->rb_left)
node = parent;
return parent;
}
EXPORT_SYMBOL(rb_prev);
void rb_replace_node(struct rb_node *victim, struct rb_node *new,
struct rb_root *root)
{
struct rb_node *parent = rb_parent(victim);
/* Copy the pointers/colour from the victim to the replacement */
*new = *victim;
/* Set the surrounding nodes to point to the replacement */
if (victim->rb_left)
rb_set_parent(victim->rb_left, new);
if (victim->rb_right)
rb_set_parent(victim->rb_right, new);
__rb_change_child(victim, new, parent, root);
}
EXPORT_SYMBOL(rb_replace_node);
void rb_replace_node_rcu(struct rb_node *victim, struct rb_node *new,
struct rb_root *root)
{
struct rb_node *parent = rb_parent(victim);
/* Copy the pointers/colour from the victim to the replacement */
*new = *victim;
/* Set the surrounding nodes to point to the replacement */
if (victim->rb_left)
rb_set_parent(victim->rb_left, new);
if (victim->rb_right)
rb_set_parent(victim->rb_right, new);
/* Set the parent's pointer to the new node last after an RCU barrier
* so that the pointers onwards are seen to be set correctly when doing
* an RCU walk over the tree.
*/
__rb_change_child_rcu(victim, new, parent, root);
}
EXPORT_SYMBOL(rb_replace_node_rcu);
static struct rb_node *rb_left_deepest_node(const struct rb_node *node)
{
for (;;) {
if (node->rb_left)
node = node->rb_left;
else if (node->rb_right)
node = node->rb_right;
else
return (struct rb_node *)node;
}
}
struct rb_node *rb_next_postorder(const struct rb_node *node)
{
const struct rb_node *parent;
if (!node)
return NULL;
parent = rb_parent(node);
/* If we're sitting on node, we've already seen our children */
if (parent && node == parent->rb_left && parent->rb_right) {
/* If we are the parent's left node, go to the parent's right
* node then all the way down to the left */
return rb_left_deepest_node(parent->rb_right);
} else
/* Otherwise we are the parent's right node, and the parent
* should be next */
return (struct rb_node *)parent;
}
EXPORT_SYMBOL(rb_next_postorder);
struct rb_node *rb_first_postorder(const struct rb_root *root)
{
if (!root->rb_node)
return NULL;
return rb_left_deepest_node(root->rb_node);
}
EXPORT_SYMBOL(rb_first_postorder);

328
utils/src/rbtree.h Normal file
View File

@@ -0,0 +1,328 @@
/* SPDX-License-Identifier: GPL-2.0-or-later */
/*
Red Black Trees
(C) 1999 Andrea Arcangeli <andrea@suse.de>
linux/include/linux/rbtree.h
To use rbtrees you'll have to implement your own insert and search cores.
This will avoid us to use callbacks and to drop drammatically performances.
I know it's not the cleaner way, but in C (not in C++) to get
performances and genericity...
See Documentation/core-api/rbtree.rst for documentation and samples.
*/
#ifndef _LINUX_RBTREE_H
#define _LINUX_RBTREE_H
#define rb_parent(r) ((struct rb_node *)((r)->__rb_parent_color & ~3))
#define rb_entry(ptr, type, member) container_of(ptr, type, member)
#define RB_EMPTY_ROOT(root) (READ_ONCE((root)->rb_node) == NULL)
/* 'empty' nodes are nodes that are known not to be inserted in an rbtree */
#define RB_EMPTY_NODE(node) \
((node)->__rb_parent_color == (unsigned long)(node))
#define RB_CLEAR_NODE(node) \
((node)->__rb_parent_color = (unsigned long)(node))
extern void rb_insert_color(struct rb_node *, struct rb_root *);
extern void rb_erase(struct rb_node *, struct rb_root *);
/* Find logical next and previous nodes in a tree */
extern struct rb_node *rb_next(const struct rb_node *);
extern struct rb_node *rb_prev(const struct rb_node *);
extern struct rb_node *rb_first(const struct rb_root *);
extern struct rb_node *rb_last(const struct rb_root *);
/* Postorder iteration - always visit the parent after its children */
extern struct rb_node *rb_first_postorder(const struct rb_root *);
extern struct rb_node *rb_next_postorder(const struct rb_node *);
/* Fast replacement of a single node without remove/rebalance/add/rebalance */
extern void rb_replace_node(struct rb_node *victim, struct rb_node *new,
struct rb_root *root);
extern void rb_replace_node_rcu(struct rb_node *victim, struct rb_node *new,
struct rb_root *root);
static inline void rb_link_node(struct rb_node *node, struct rb_node *parent,
struct rb_node **rb_link)
{
node->__rb_parent_color = (unsigned long)parent;
node->rb_left = node->rb_right = NULL;
*rb_link = node;
}
static inline void rb_link_node_rcu(struct rb_node *node, struct rb_node *parent,
struct rb_node **rb_link)
{
node->__rb_parent_color = (unsigned long)parent;
node->rb_left = node->rb_right = NULL;
rcu_assign_pointer(*rb_link, node);
}
#define rb_entry_safe(ptr, type, member) \
({ typeof(ptr) ____ptr = (ptr); \
____ptr ? rb_entry(____ptr, type, member) : NULL; \
})
/**
* rbtree_postorder_for_each_entry_safe - iterate in post-order over rb_root of
* given type allowing the backing memory of @pos to be invalidated
*
* @pos: the 'type *' to use as a loop cursor.
* @n: another 'type *' to use as temporary storage
* @root: 'rb_root *' of the rbtree.
* @field: the name of the rb_node field within 'type'.
*
* rbtree_postorder_for_each_entry_safe() provides a similar guarantee as
* list_for_each_entry_safe() and allows the iteration to continue independent
* of changes to @pos by the body of the loop.
*
* Note, however, that it cannot handle other modifications that re-order the
* rbtree it is iterating over. This includes calling rb_erase() on @pos, as
* rb_erase() may rebalance the tree, causing us to miss some nodes.
*/
#define rbtree_postorder_for_each_entry_safe(pos, n, root, field) \
for (pos = rb_entry_safe(rb_first_postorder(root), typeof(*pos), field); \
pos && ({ n = rb_entry_safe(rb_next_postorder(&pos->field), \
typeof(*pos), field); 1; }); \
pos = n)
/* Same as rb_first(), but O(1) */
#define rb_first_cached(root) (root)->rb_leftmost
static inline void rb_insert_color_cached(struct rb_node *node,
struct rb_root_cached *root,
bool leftmost)
{
if (leftmost)
root->rb_leftmost = node;
rb_insert_color(node, &root->rb_root);
}
static inline struct rb_node *
rb_erase_cached(struct rb_node *node, struct rb_root_cached *root)
{
struct rb_node *leftmost = NULL;
if (root->rb_leftmost == node)
leftmost = root->rb_leftmost = rb_next(node);
rb_erase(node, &root->rb_root);
return leftmost;
}
static inline void rb_replace_node_cached(struct rb_node *victim,
struct rb_node *new,
struct rb_root_cached *root)
{
if (root->rb_leftmost == victim)
root->rb_leftmost = new;
rb_replace_node(victim, new, &root->rb_root);
}
/*
* The below helper functions use 2 operators with 3 different
* calling conventions. The operators are related like:
*
* comp(a->key,b) < 0 := less(a,b)
* comp(a->key,b) > 0 := less(b,a)
* comp(a->key,b) == 0 := !less(a,b) && !less(b,a)
*
* If these operators define a partial order on the elements we make no
* guarantee on which of the elements matching the key is found. See
* rb_find().
*
* The reason for this is to allow the find() interface without requiring an
* on-stack dummy object, which might not be feasible due to object size.
*/
/**
* rb_add_cached() - insert @node into the leftmost cached tree @tree
* @node: node to insert
* @tree: leftmost cached tree to insert @node into
* @less: operator defining the (partial) node order
*
* Returns @node when it is the new leftmost, or NULL.
*/
static __always_inline struct rb_node *
rb_add_cached(struct rb_node *node, struct rb_root_cached *tree,
bool (*less)(struct rb_node *, const struct rb_node *))
{
struct rb_node **link = &tree->rb_root.rb_node;
struct rb_node *parent = NULL;
bool leftmost = true;
while (*link) {
parent = *link;
if (less(node, parent)) {
link = &parent->rb_left;
} else {
link = &parent->rb_right;
leftmost = false;
}
}
rb_link_node(node, parent, link);
rb_insert_color_cached(node, tree, leftmost);
return leftmost ? node : NULL;
}
/**
* rb_add() - insert @node into @tree
* @node: node to insert
* @tree: tree to insert @node into
* @less: operator defining the (partial) node order
*/
static __always_inline void
rb_add(struct rb_node *node, struct rb_root *tree,
bool (*less)(struct rb_node *, const struct rb_node *))
{
struct rb_node **link = &tree->rb_node;
struct rb_node *parent = NULL;
while (*link) {
parent = *link;
if (less(node, parent))
link = &parent->rb_left;
else
link = &parent->rb_right;
}
rb_link_node(node, parent, link);
rb_insert_color(node, tree);
}
/**
* rb_find_add() - find equivalent @node in @tree, or add @node
* @node: node to look-for / insert
* @tree: tree to search / modify
* @cmp: operator defining the node order
*
* Returns the rb_node matching @node, or NULL when no match is found and @node
* is inserted.
*/
static __always_inline struct rb_node *
rb_find_add(struct rb_node *node, struct rb_root *tree,
int (*cmp)(struct rb_node *, const struct rb_node *))
{
struct rb_node **link = &tree->rb_node;
struct rb_node *parent = NULL;
int c;
while (*link) {
parent = *link;
c = cmp(node, parent);
if (c < 0)
link = &parent->rb_left;
else if (c > 0)
link = &parent->rb_right;
else
return parent;
}
rb_link_node(node, parent, link);
rb_insert_color(node, tree);
return NULL;
}
/**
* rb_find() - find @key in tree @tree
* @key: key to match
* @tree: tree to search
* @cmp: operator defining the node order
*
* Returns the rb_node matching @key or NULL.
*/
static __always_inline struct rb_node *
rb_find(const void *key, const struct rb_root *tree,
int (*cmp)(const void *key, const struct rb_node *))
{
struct rb_node *node = tree->rb_node;
while (node) {
int c = cmp(key, node);
if (c < 0)
node = node->rb_left;
else if (c > 0)
node = node->rb_right;
else
return node;
}
return NULL;
}
/**
* rb_find_first() - find the first @key in @tree
* @key: key to match
* @tree: tree to search
* @cmp: operator defining node order
*
* Returns the leftmost node matching @key, or NULL.
*/
static __always_inline struct rb_node *
rb_find_first(const void *key, const struct rb_root *tree,
int (*cmp)(const void *key, const struct rb_node *))
{
struct rb_node *node = tree->rb_node;
struct rb_node *match = NULL;
while (node) {
int c = cmp(key, node);
if (c <= 0) {
if (!c)
match = node;
node = node->rb_left;
} else if (c > 0) {
node = node->rb_right;
}
}
return match;
}
/**
* rb_next_match() - find the next @key in @tree
* @key: key to match
* @tree: tree to search
* @cmp: operator defining node order
*
* Returns the next node matching @key, or NULL.
*/
static __always_inline struct rb_node *
rb_next_match(const void *key, struct rb_node *node,
int (*cmp)(const void *key, const struct rb_node *))
{
node = rb_next(node);
if (node && cmp(key, node))
node = NULL;
return node;
}
/**
* rb_for_each() - iterates a subtree matching @key
* @node: iterator
* @key: key to match
* @tree: tree to search
* @cmp: operator defining node order
*/
#define rb_for_each(node, key, tree, cmp) \
for ((node) = rb_find_first((key), (tree), (cmp)); \
(node); (node) = rb_next_match((key), (node), (cmp)))
#endif /* _LINUX_RBTREE_H */

View File

@@ -0,0 +1,313 @@
/* SPDX-License-Identifier: GPL-2.0-or-later */
/*
Red Black Trees
(C) 1999 Andrea Arcangeli <andrea@suse.de>
(C) 2002 David Woodhouse <dwmw2@infradead.org>
(C) 2012 Michel Lespinasse <walken@google.com>
linux/include/linux/rbtree_augmented.h
*/
#ifndef _LINUX_RBTREE_AUGMENTED_H
#define _LINUX_RBTREE_AUGMENTED_H
/*
* Please note - only struct rb_augment_callbacks and the prototypes for
* rb_insert_augmented() and rb_erase_augmented() are intended to be public.
* The rest are implementation details you are not expected to depend on.
*
* See Documentation/core-api/rbtree.rst for documentation and samples.
*/
struct rb_augment_callbacks {
void (*propagate)(struct rb_node *node, struct rb_node *stop);
void (*copy)(struct rb_node *old, struct rb_node *new);
void (*rotate)(struct rb_node *old, struct rb_node *new);
};
extern void __rb_insert_augmented(struct rb_node *node, struct rb_root *root,
void (*augment_rotate)(struct rb_node *old, struct rb_node *new));
/*
* Fixup the rbtree and update the augmented information when rebalancing.
*
* On insertion, the user must update the augmented information on the path
* leading to the inserted node, then call rb_link_node() as usual and
* rb_insert_augmented() instead of the usual rb_insert_color() call.
* If rb_insert_augmented() rebalances the rbtree, it will callback into
* a user provided function to update the augmented information on the
* affected subtrees.
*/
static inline void
rb_insert_augmented(struct rb_node *node, struct rb_root *root,
const struct rb_augment_callbacks *augment)
{
__rb_insert_augmented(node, root, augment->rotate);
}
static inline void
rb_insert_augmented_cached(struct rb_node *node,
struct rb_root_cached *root, bool newleft,
const struct rb_augment_callbacks *augment)
{
if (newleft)
root->rb_leftmost = node;
rb_insert_augmented(node, &root->rb_root, augment);
}
/*
* Template for declaring augmented rbtree callbacks (generic case)
*
* RBSTATIC: 'static' or empty
* RBNAME: name of the rb_augment_callbacks structure
* RBSTRUCT: struct type of the tree nodes
* RBFIELD: name of struct rb_node field within RBSTRUCT
* RBAUGMENTED: name of field within RBSTRUCT holding data for subtree
* RBCOMPUTE: name of function that recomputes the RBAUGMENTED data
*/
#define RB_DECLARE_CALLBACKS(RBSTATIC, RBNAME, \
RBSTRUCT, RBFIELD, RBAUGMENTED, RBCOMPUTE) \
static inline void \
RBNAME ## _propagate(struct rb_node *rb, struct rb_node *stop) \
{ \
while (rb != stop) { \
RBSTRUCT *node = rb_entry(rb, RBSTRUCT, RBFIELD); \
if (RBCOMPUTE(node, true)) \
break; \
rb = rb_parent(&node->RBFIELD); \
} \
} \
static inline void \
RBNAME ## _copy(struct rb_node *rb_old, struct rb_node *rb_new) \
{ \
RBSTRUCT *old = rb_entry(rb_old, RBSTRUCT, RBFIELD); \
RBSTRUCT *new = rb_entry(rb_new, RBSTRUCT, RBFIELD); \
new->RBAUGMENTED = old->RBAUGMENTED; \
} \
static void \
RBNAME ## _rotate(struct rb_node *rb_old, struct rb_node *rb_new) \
{ \
RBSTRUCT *old = rb_entry(rb_old, RBSTRUCT, RBFIELD); \
RBSTRUCT *new = rb_entry(rb_new, RBSTRUCT, RBFIELD); \
new->RBAUGMENTED = old->RBAUGMENTED; \
RBCOMPUTE(old, false); \
} \
RBSTATIC const struct rb_augment_callbacks RBNAME = { \
.propagate = RBNAME ## _propagate, \
.copy = RBNAME ## _copy, \
.rotate = RBNAME ## _rotate \
};
/*
* Template for declaring augmented rbtree callbacks,
* computing RBAUGMENTED scalar as max(RBCOMPUTE(node)) for all subtree nodes.
*
* RBSTATIC: 'static' or empty
* RBNAME: name of the rb_augment_callbacks structure
* RBSTRUCT: struct type of the tree nodes
* RBFIELD: name of struct rb_node field within RBSTRUCT
* RBTYPE: type of the RBAUGMENTED field
* RBAUGMENTED: name of RBTYPE field within RBSTRUCT holding data for subtree
* RBCOMPUTE: name of function that returns the per-node RBTYPE scalar
*/
#define RB_DECLARE_CALLBACKS_MAX(RBSTATIC, RBNAME, RBSTRUCT, RBFIELD, \
RBTYPE, RBAUGMENTED, RBCOMPUTE) \
static inline bool RBNAME ## _compute_max(RBSTRUCT *node, bool exit) \
{ \
RBSTRUCT *child; \
RBTYPE max = RBCOMPUTE(node); \
if (node->RBFIELD.rb_left) { \
child = rb_entry(node->RBFIELD.rb_left, RBSTRUCT, RBFIELD); \
if (child->RBAUGMENTED > max) \
max = child->RBAUGMENTED; \
} \
if (node->RBFIELD.rb_right) { \
child = rb_entry(node->RBFIELD.rb_right, RBSTRUCT, RBFIELD); \
if (child->RBAUGMENTED > max) \
max = child->RBAUGMENTED; \
} \
if (exit && node->RBAUGMENTED == max) \
return true; \
node->RBAUGMENTED = max; \
return false; \
} \
RB_DECLARE_CALLBACKS(RBSTATIC, RBNAME, \
RBSTRUCT, RBFIELD, RBAUGMENTED, RBNAME ## _compute_max)
#define RB_RED 0
#define RB_BLACK 1
#define __rb_parent(pc) ((struct rb_node *)(pc & ~3))
#define __rb_color(pc) ((pc) & 1)
#define __rb_is_black(pc) __rb_color(pc)
#define __rb_is_red(pc) (!__rb_color(pc))
#define rb_color(rb) __rb_color((rb)->__rb_parent_color)
#define rb_is_red(rb) __rb_is_red((rb)->__rb_parent_color)
#define rb_is_black(rb) __rb_is_black((rb)->__rb_parent_color)
static inline void rb_set_parent(struct rb_node *rb, struct rb_node *p)
{
rb->__rb_parent_color = rb_color(rb) | (unsigned long)p;
}
static inline void rb_set_parent_color(struct rb_node *rb,
struct rb_node *p, int color)
{
rb->__rb_parent_color = (unsigned long)p | color;
}
static inline void
__rb_change_child(struct rb_node *old, struct rb_node *new,
struct rb_node *parent, struct rb_root *root)
{
if (parent) {
if (parent->rb_left == old)
WRITE_ONCE(parent->rb_left, new);
else
WRITE_ONCE(parent->rb_right, new);
} else
WRITE_ONCE(root->rb_node, new);
}
static inline void
__rb_change_child_rcu(struct rb_node *old, struct rb_node *new,
struct rb_node *parent, struct rb_root *root)
{
if (parent) {
if (parent->rb_left == old)
rcu_assign_pointer(parent->rb_left, new);
else
rcu_assign_pointer(parent->rb_right, new);
} else
rcu_assign_pointer(root->rb_node, new);
}
extern void __rb_erase_color(struct rb_node *parent, struct rb_root *root,
void (*augment_rotate)(struct rb_node *old, struct rb_node *new));
static __always_inline struct rb_node *
__rb_erase_augmented(struct rb_node *node, struct rb_root *root,
const struct rb_augment_callbacks *augment)
{
struct rb_node *child = node->rb_right;
struct rb_node *tmp = node->rb_left;
struct rb_node *parent, *rebalance;
unsigned long pc;
if (!tmp) {
/*
* Case 1: node to erase has no more than 1 child (easy!)
*
* Note that if there is one child it must be red due to 5)
* and node must be black due to 4). We adjust colors locally
* so as to bypass __rb_erase_color() later on.
*/
pc = node->__rb_parent_color;
parent = __rb_parent(pc);
__rb_change_child(node, child, parent, root);
if (child) {
child->__rb_parent_color = pc;
rebalance = NULL;
} else
rebalance = __rb_is_black(pc) ? parent : NULL;
tmp = parent;
} else if (!child) {
/* Still case 1, but this time the child is node->rb_left */
tmp->__rb_parent_color = pc = node->__rb_parent_color;
parent = __rb_parent(pc);
__rb_change_child(node, tmp, parent, root);
rebalance = NULL;
tmp = parent;
} else {
struct rb_node *successor = child, *child2;
tmp = child->rb_left;
if (!tmp) {
/*
* Case 2: node's successor is its right child
*
* (n) (s)
* / \ / \
* (x) (s) -> (x) (c)
* \
* (c)
*/
parent = successor;
child2 = successor->rb_right;
augment->copy(node, successor);
} else {
/*
* Case 3: node's successor is leftmost under
* node's right child subtree
*
* (n) (s)
* / \ / \
* (x) (y) -> (x) (y)
* / /
* (p) (p)
* / /
* (s) (c)
* \
* (c)
*/
do {
parent = successor;
successor = tmp;
tmp = tmp->rb_left;
} while (tmp);
child2 = successor->rb_right;
WRITE_ONCE(parent->rb_left, child2);
WRITE_ONCE(successor->rb_right, child);
rb_set_parent(child, successor);
augment->copy(node, successor);
augment->propagate(parent, successor);
}
tmp = node->rb_left;
WRITE_ONCE(successor->rb_left, tmp);
rb_set_parent(tmp, successor);
pc = node->__rb_parent_color;
tmp = __rb_parent(pc);
__rb_change_child(node, successor, tmp, root);
if (child2) {
rb_set_parent_color(child2, parent, RB_BLACK);
rebalance = NULL;
} else {
rebalance = rb_is_black(successor) ? parent : NULL;
}
successor->__rb_parent_color = pc;
tmp = successor;
}
augment->propagate(tmp, NULL);
return rebalance;
}
static __always_inline void
rb_erase_augmented(struct rb_node *node, struct rb_root *root,
const struct rb_augment_callbacks *augment)
{
struct rb_node *rebalance = __rb_erase_augmented(node, root, augment);
if (rebalance)
__rb_erase_color(rebalance, root, augment->rotate);
}
static __always_inline void
rb_erase_augmented_cached(struct rb_node *node, struct rb_root_cached *root,
const struct rb_augment_callbacks *augment)
{
if (root->rb_leftmost == node)
root->rb_leftmost = rb_next(node);
rb_erase_augmented(node, &root->rb_root, augment);
}
#endif /* _LINUX_RBTREE_AUGMENTED_H */

34
utils/src/rbtree_types.h Normal file
View File

@@ -0,0 +1,34 @@
/* SPDX-License-Identifier: GPL-2.0-or-later */
#ifndef _LINUX_RBTREE_TYPES_H
#define _LINUX_RBTREE_TYPES_H
struct rb_node {
unsigned long __rb_parent_color;
struct rb_node *rb_right;
struct rb_node *rb_left;
} __attribute__((aligned(sizeof(long))));
/* The alignment might seem pointless, but allegedly CRIS needs it */
struct rb_root {
struct rb_node *rb_node;
};
/*
* Leftmost-cached rbtrees.
*
* We do not cache the rightmost node based on footprint
* size vs number of potential users that could benefit
* from O(1) rb_last(). Just not worth it, users that want
* this feature can always implement the logic explicitly.
* Furthermore, users that want to cache both pointers may
* find it a bit asymmetric, but that's ok.
*/
struct rb_root_cached {
struct rb_root rb_root;
struct rb_node *rb_leftmost;
};
#define RB_ROOT (struct rb_root) { NULL, }
#define RB_ROOT_CACHED (struct rb_root_cached) { {NULL, }, NULL }
#endif

View File

@@ -44,3 +44,37 @@ int srch_decode_entry(void *buf, struct scoutfs_srch_entry *sre,
return tot;
}
static int encode_u64(__le64 *buf, u64 val)
{
int bytes;
val = (val << 1) ^ ((s64)val >> 63); /* shift sign extend */
bytes = (fls64(val) + 7) >> 3;
put_unaligned_le64(val, buf);
return bytes;
}
int srch_encode_entry(void *buf, struct scoutfs_srch_entry *sre, struct scoutfs_srch_entry *prev)
{
u64 diffs[] = {
le64_to_cpu(sre->hash) - le64_to_cpu(prev->hash),
le64_to_cpu(sre->ino) - le64_to_cpu(prev->ino),
le64_to_cpu(sre->id) - le64_to_cpu(prev->id),
};
u16 lengths = 0;
int bytes;
int tot = 2;
int i;
for (i = 0; i < array_size(diffs); i++) {
bytes = encode_u64(buf + tot, diffs[i]);
lengths |= bytes << (i << 2);
tot += bytes;
}
put_unaligned_le16(lengths, buf);
return tot;
}

View File

@@ -3,5 +3,6 @@
int srch_decode_entry(void *buf, struct scoutfs_srch_entry *sre,
struct scoutfs_srch_entry *prev);
int srch_encode_entry(void *buf, struct scoutfs_srch_entry *sre, struct scoutfs_srch_entry *prev);
#endif

View File

@@ -69,6 +69,8 @@ do { \
#define container_of(ptr, type, memb) \
((type *)((void *)(ptr) - offsetof(type, memb)))
#define NSEC_PER_SEC 1000000000
#define BITS_PER_LONG (sizeof(long) * 8)
#define U8_MAX ((u8)~0ULL)
#define U16_MAX ((u16)~0ULL)
@@ -81,6 +83,7 @@ do { \
\
(_x == 0 ? 0 : 64 - __builtin_clzll(_x)); \
})
#define fls64(x) flsll(x)
#define ilog2(x) \
({ \
@@ -98,6 +101,16 @@ emit_get_unaligned_le(16)
emit_get_unaligned_le(32)
emit_get_unaligned_le(64)
#define emit_put_unaligned_le(nr) \
static inline void put_unaligned_le##nr(u##nr val, void *buf) \
{ \
__le##nr x = cpu_to_le##nr(val); \
memcpy(buf, &x, sizeof(x)); \
}
emit_put_unaligned_le(16)
emit_put_unaligned_le(32)
emit_put_unaligned_le(64)
/*
* return -1,0,+1 based on the memcmp comparison of the minimum of their
* two lengths. If their min shared bytes are equal but the lengths

View File

@@ -1,238 +0,0 @@
#define _GNU_SOURCE /* O_DIRECT */
#include <unistd.h>
#include <stdbool.h>
#include <stdlib.h>
#include <stdio.h>
#include <string.h>
#include <errno.h>
#include <fcntl.h>
#include <sys/types.h>
#include <sys/stat.h>
#include <unistd.h>
#include <assert.h>
#include <ctype.h>
#include <inttypes.h>
#include <argp.h>
#include "sparse.h"
#include "cmd.h"
#include "util.h"
#include "format.h"
#include "parse.h"
#include "dev.h"
#include "btree.h"
#include "avl.h"
struct wipe_ltdf_args {
char *meta_device;
u64 fsid;
u64 rid;
};
static int do_wipe_ltdf(struct wipe_ltdf_args *args)
{
struct scoutfs_super_block *super = NULL;
struct scoutfs_btree_block *bt = NULL;
struct scoutfs_btree_item *item;
struct scoutfs_avl_node *node;
struct scoutfs_log_trees *lt;
bool found = false;
int meta_fd = -1;
unsigned val_len;
u64 blkno;
int ret;
meta_fd = open(args->meta_device, O_DIRECT | O_RDWR | O_EXCL);
if (meta_fd < 0) {
ret = -errno;
fprintf(stderr, "failed to open '%s': %s (%d)\n",
args->meta_device, strerror(errno), errno);
goto out;
}
ret = read_block_verify(meta_fd, SCOUTFS_BLOCK_MAGIC_SUPER, args->fsid, SCOUTFS_SUPER_BLKNO,
SCOUTFS_BLOCK_SM_SHIFT, (void **)&super);
if (ret) {
ret = -errno;
fprintf(stderr, "failed to read meta super block: %s (%d)\n",
strerror(errno), errno);
goto out;
}
if (!(le64_to_cpu(super->flags) & SCOUTFS_FLAG_IS_META_BDEV)) {
fprintf(stderr, "The super block in device %s doesn't have the META_BDEV flag set, is it a data device?\n",
args->meta_device);
ret = -EIO;
goto out;
}
blkno = le64_to_cpu(super->logs_root.ref.blkno);
if (blkno == 0) {
fprintf(stderr, "The logs_root btree ref is empty, no pending log_trees items\n");
ret = 0;
goto out;
}
if (super->logs_root.height != 1) {
fprintf(stderr, "The logs_root btree is not a single leaf, can't find items\n");
ret = -EINVAL;
goto out;
}
ret = read_block_verify(meta_fd, SCOUTFS_BLOCK_MAGIC_BTREE, args->fsid, blkno,
SCOUTFS_BLOCK_LG_SHIFT, (void **)&bt);
if (ret) {
ret = -errno;
fprintf(stderr, "failed to read meta super block: %s (%d)\n",
strerror(errno), errno);
goto out;
}
if (bt->level != 0) {
fprintf(stderr, "read non-leaf logs_root btree block at blkno %llu with level %u\n",
blkno, bt->level);
ret = -EIO;
goto out;
}
node = avl_first(&bt->item_root);
while (node) {
item = container_of(node, struct scoutfs_btree_item, node);
val_len = le16_to_cpu(item->val_len);
if (val_len != sizeof(struct scoutfs_log_trees)) {
fprintf(stderr, "invalid item length: %u, expected %zu\n",
val_len, sizeof(struct scoutfs_log_trees));
ret = -EIO;
goto out;
}
lt = (void *)bt + le16_to_cpu(item->val_off);
if (le64_to_cpu(lt->rid) == args->rid) {
printf("found rid %016llx\n", args->rid);
found = true;
break;
} else {
printf("skipping mount rid %016llx\n", le64_to_cpu(lt->rid));
}
node = avl_next(&bt->item_root, node);
}
if (!found) {
fprintf(stderr, "couldn't find log_trees item with rid %016llx\n", args->rid);
ret = -ENOENT;
goto out;
}
printf("wiping meta_avail: blkno %llu total_nr %llu\n",
le64_to_cpu(lt->meta_avail.ref.blkno), le64_to_cpu(lt->meta_avail.total_nr));
printf("wiping meta_freed: blkno %llu total_nr %llu\n",
le64_to_cpu(lt->meta_freed.ref.blkno), le64_to_cpu(lt->meta_freed.total_nr));
printf("wiping data_avail: blkno %llu total_len %llu\n",
le64_to_cpu(lt->data_avail.root.ref.blkno), le64_to_cpu(lt->data_avail.total_len));
printf("wiping data_freed: blkno %llu total_len %llu\n",
le64_to_cpu(lt->data_freed.root.ref.blkno), le64_to_cpu(lt->data_freed.total_len));
memset(&lt->meta_avail, 0, sizeof(lt->meta_avail));
memset(&lt->meta_freed, 0, sizeof(lt->meta_freed));
memset(&lt->data_avail, 0, sizeof(lt->data_avail));
memset(&lt->data_freed, 0, sizeof(lt->data_freed));
ret = write_block(meta_fd, SCOUTFS_BLOCK_MAGIC_BTREE, super->hdr.fsid,
le64_to_cpu(bt->hdr.seq), blkno, SCOUTFS_BLOCK_LG_SHIFT,
&bt->hdr);
if (ret)
fprintf(stderr, "Failed to write updated log_trees block.\n");
else
printf("Writing updated log_trees block succeeded.\n");
out:
if (meta_fd >= 0)
close(meta_fd);
if (super)
free(super);
if (bt)
free(bt);
return ret;
}
static int parse_opt(int key, char *arg, struct argp_state *state)
{
struct wipe_ltdf_args *args = state->input;
int ret;
switch (key) {
case 'f': /* fsid */
{
ret = parse_u64(arg, &args->fsid);
if (ret)
return ret;
if (args->fsid == 0)
argp_error(state, "must provide non-zero fsid");
break;
}
case 'r': /* rid */
{
ret = parse_u64(arg, &args->rid);
if (ret)
return ret;
if (args->rid == 0)
argp_error(state, "must provide non-zero rid");
break;
}
case ARGP_KEY_ARG:
if (!args->meta_device)
args->meta_device = strdup_or_error(state, arg);
else
argp_error(state, "more than two metadata device given");
break;
case ARGP_KEY_FINI:
if (!args->fsid)
argp_error(state, "must specify fsid with --fsid|-f");
if (!args->rid)
argp_error(state, "must specify rid with --rid|-r");
if (!args->meta_device)
argp_error(state, "no metadata device argument given");
break;
default:
break;
}
return 0;
}
static struct argp_option options[] = {
{ "fsid", 'f', "FSID", 0, "fsid of volume in which to wipe root item, for validation"},
{ "rid", 'r', "RID", 0, "rid of log_trees item in which to wipe data_freed btree ref"},
{ NULL }
};
static struct argp argp = {
options,
parse_opt,
"META-DEVICE",
"(debug) wipe a data_freed btree root ref for a given mount rid, do not do this"
};
static int wipe_ltdf_cmd(int argc, char *argv[])
{
struct wipe_ltdf_args wipe_ltdf_args = {NULL,};
int ret;
ret = argp_parse(&argp, argc, argv, 0, NULL, &wipe_ltdf_args);
if (ret)
return ret;
return do_wipe_ltdf(&wipe_ltdf_args);
}
static void __attribute__((constructor)) wipe_ltdf_ctor(void)
{
cmd_register_argp("wipe-log-trees-data-freed", &argp, GROUP_DEBUG, wipe_ltdf_cmd);
}