feat: add metadata_builder and --rebuild-metadata option

This commit is contained in:
Marcus Holland-Moritz 2024-11-25 17:16:22 +01:00
parent ca2e5c9e35
commit faea9c3c4d
11 changed files with 616 additions and 23 deletions

View File

@ -499,6 +499,7 @@ if(WITH_TESTS)
test/lazy_value_test.cpp
test/lru_cache_test.cpp
test/metadata_requirements_test.cpp
test/metadata_test.cpp
test/nilsimsa_test.cpp
test/options_test.cpp
test/packed_int_vector_test.cpp

View File

@ -125,6 +125,7 @@ add_library(
src/writer/fragment_category.cpp
src/writer/fragment_order_parser.cpp
src/writer/inode_fragments.cpp
src/writer/metadata_options.cpp
src/writer/rule_based_entry_filter.cpp
src/writer/scanner.cpp
src/writer/segmenter.cpp

View File

@ -239,6 +239,11 @@ Most other options are concerned with compression tuning:
metadata to uncompressed metadata without having to rebuild or recompress
all the other data.
- `--rebuild-metadata`:
Completely rebuild the metadata block. This will upgrade the internal format
of the metadata to the latest version instead of just recompressing the
metadata block.
- `--recompress-categories=`[`!`]*category*[`,`...]:
When `--recompress` is set to `all` or `block`, this option controls
which categories of blocks will be recompressed. Adding a `!` in front

View File

@ -29,12 +29,14 @@
#include <vector>
#include <dwarfs/history_config.h>
#include <dwarfs/writer/metadata_options.h>
namespace dwarfs::utility {
struct rewrite_options {
bool recompress_block{false};
bool recompress_metadata{false};
std::optional<writer::metadata_options> rebuild_metadata;
std::unordered_set<std::string> recompress_categories;
bool recompress_categories_exclude{false};
bool enable_history{true};

View File

@ -24,6 +24,7 @@
#include <cstddef>
#include <cstdint>
#include <functional>
#include <iosfwd>
#include <optional>
#include <string>
#include <vector>
@ -58,4 +59,6 @@ struct metadata_options {
size_t inode_size_cache_min_chunk_count{128};
};
std::ostream& operator<<(std::ostream& os, metadata_options const& opts);
} // namespace dwarfs::writer

View File

@ -31,6 +31,10 @@
#include <dwarfs/reader/internal/filesystem_parser.h>
#include <dwarfs/writer/internal/filesystem_writer_detail.h>
#include <dwarfs/writer/internal/metadata_builder.h>
#include <dwarfs/writer/internal/metadata_freezer.h>
#include <dwarfs/gen-cpp2/metadata_types.h>
namespace dwarfs::utility {
@ -163,6 +167,22 @@ void rewrite_filesystem(logger& lgr, dwarfs::reader::filesystem_v2 const& fs,
case section_type::METADATA_V2_SCHEMA:
case section_type::METADATA_V2:
if (opts.rebuild_metadata) {
DWARFS_CHECK(opts.recompress_metadata,
"rebuild_metadata requires recompress_metadata");
if (s->type() == section_type::METADATA_V2) {
using namespace dwarfs::writer::internal;
auto md = fs.unpacked_metadata();
auto builder = metadata_builder(lgr, std::move(*md),
opts.rebuild_metadata.value());
auto [schema, data] =
metadata_freezer(LOG_GET_LOGGER).freeze(builder.build());
writer.write_metadata_v2_schema(std::move(schema));
writer.write_metadata_v2(std::move(data));
}
} else {
if (opts.recompress_metadata && !from_none_to_none(s)) {
log_recompress(s);
writer.write_section(s->type(), s->compression(),
@ -170,6 +190,7 @@ void rewrite_filesystem(logger& lgr, dwarfs::reader::filesystem_v2 const& fs,
} else {
copy_compressed(s);
}
}
break;
case section_type::HISTORY:

View File

@ -23,12 +23,14 @@
#include <ctime>
#include <filesystem>
#include <thrift/lib/cpp2/protocol/DebugProtocol.h>
#include <dwarfs/logger.h>
#include <dwarfs/version.h>
#include <dwarfs/writer/metadata_options.h>
#include <dwarfs/internal/features.h>
#include <dwarfs/internal/metadata_utils.h>
#include <dwarfs/internal/string_table.h>
#include <dwarfs/writer/internal/block_manager.h>
#include <dwarfs/writer/internal/entry.h>
@ -38,6 +40,8 @@
#include <dwarfs/gen-cpp2/metadata_types.h>
#include <thrift/lib/thrift/gen-cpp2/frozen_types_custom_protocol.h>
namespace dwarfs::writer::internal {
namespace {
@ -55,13 +59,17 @@ class metadata_builder_ final : public metadata_builder::impl {
metadata_options const& options)
: LOG_PROXY_INIT(lgr)
, md_{md}
, options_{options} {}
, options_{options} {
upgrade_metadata();
}
metadata_builder_(logger& lgr, thrift::metadata::metadata&& md,
metadata_options const& options)
: LOG_PROXY_INIT(lgr)
, md_{std::move(md)}
, options_{options} {}
, options_{options} {
upgrade_metadata();
}
void set_devices(std::vector<uint64_t> devices) override {
md_.devices() = std::move(devices);
@ -121,6 +129,8 @@ class metadata_builder_ final : public metadata_builder::impl {
private:
thrift::metadata::inode_size_cache build_inode_size_cache() const;
void upgrade_metadata();
void upgrade_from_pre_v2_2();
LOG_PROXY_DECL(LoggerPolicy);
thrift::metadata::metadata md_;
@ -337,6 +347,8 @@ thrift::metadata::metadata const& metadata_builder_<LoggerPolicy>::build() {
// TODO: don't overwrite all options when upgrading!
md_.options() = fsopts;
md_.features() = features_.get();
// TODO: try and keep metadata upgrade history
md_.dwarfs_version() = std::string("libdwarfs ") + DWARFS_GIT_ID;
if (!options_.no_create_timestamp) {
md_.create_timestamp() = std::time(nullptr);
@ -347,6 +359,209 @@ thrift::metadata::metadata const& metadata_builder_<LoggerPolicy>::build() {
return md_;
}
template <typename LoggerPolicy>
void metadata_builder_<LoggerPolicy>::upgrade_from_pre_v2_2() {
// === v2.2 metadata ===
//
// - `directories` is indexed by directory inode number; this is exactly
// the same as today.
// - `entry_table_v2_2` is indexed by "inode number" and returns an index
// into `inodes`.
// - `inodes` sort of combine the inode data with data from the new
// `dir_entries` array. Inodes are ordered by directory entry index
// (i.e. `first_entry`, `parent_entry` can be used to directly index
// into `inodes`).
// - The format cannot properly represent hardlinks. Rather, it represents
// all shared files as hardlinks, which is not correct.
//
// In order to upgrade to the new format, we need to:
//
// - Keep the `directories` array as is.
// - Rebuild the `inodes` array to be indexed by inode number; the new
// format potentially has *much* more inode numbers than the old format
// because shared files don't share inode numbers anymore, only hardlinks
// do. The order of the new `inodes` array is exactly the same as the
// old `entry_table_v2_2` array, *except* for regular files, where order
// needs to take shared files into account. This means regular file
// inode numbers will change and this needs to be tracked. This also
// means that both the `chunk_table` and `chunks` arrays need to be
// rebuilt.
// - Build the `shared_files_table` array. If multiple entries in `inodes`
// share the same `inode_v2_2`, they are considered shared files.
// - Don't try to perform any hardlink detection, as the old format doesn't
// properly represent hardlinks.
// - Build the `dir_entries` array.
//
// Here's a rough outline of the upgrade process:
//
// - Determine the number of entries that reference the same `inode_v2_2`.
// This will allow us to distinguish between unique and shared files.
LOG_DEBUG << "upgrading entry_table_v2_2 to dir_entries";
auto const lnk_offset = find_inode_rank_offset(md_, inode_rank::INO_LNK);
auto const reg_offset = find_inode_rank_offset(md_, inode_rank::INO_REG);
auto const dev_offset = find_inode_rank_offset(md_, inode_rank::INO_DEV);
LOG_TRACE << "lnk_offset: " << lnk_offset;
LOG_TRACE << "reg_offset: " << reg_offset;
LOG_TRACE << "dev_offset: " << dev_offset;
std::vector<uint32_t> reg_inode_refs(md_.chunk_table()->size() - 1, 0);
for (auto const& inode : md_.inodes().value()) {
auto const inode_v2_2 = inode.inode_v2_2().value();
if (reg_offset <= inode_v2_2 && inode_v2_2 < dev_offset) {
auto const index = inode_v2_2 - reg_offset;
if (index < reg_inode_refs.size()) {
++reg_inode_refs[index];
}
}
}
auto const unique_files =
std::count_if(reg_inode_refs.begin(), reg_inode_refs.end(),
[](auto ref) { return ref == 1; });
auto const num_reg_files =
std::accumulate(reg_inode_refs.begin(), reg_inode_refs.end(), 0,
[](auto sum, auto ref) { return sum + ref; });
LOG_TRACE << "unique_files: " << unique_files;
LOG_TRACE << "num_reg_files: " << num_reg_files;
auto const& entry_table = md_.entry_table_v2_2().value();
thrift::metadata::metadata newmd;
auto& dir_entries = newmd.dir_entries().emplace();
dir_entries.reserve(md_.inodes()->size());
auto& shared_files = newmd.shared_files_table().emplace();
shared_files.reserve(num_reg_files - unique_files);
auto& chunks = newmd.chunks().ensure();
chunks.reserve(md_.chunks()->size());
auto& chunk_table = newmd.chunk_table().ensure();
chunk_table.reserve(md_.chunk_table()->size());
chunk_table.push_back(0);
auto& inodes = newmd.inodes().ensure();
inodes.resize(md_.inodes()->size());
newmd.directories().copy_from(md_.directories());
for (auto& d : newmd.directories().value()) {
d.parent_entry() = entry_table[d.parent_entry().value()];
}
auto& dirs = newmd.directories().value();
uint32_t const shared_offset = reg_offset + unique_files;
uint32_t unique_inode = reg_offset;
uint32_t shared_inode = shared_offset;
uint32_t shared_chunk_index = 0;
std::unordered_map<uint32_t, uint32_t> shared_inode_map;
std::vector<thrift::metadata::chunk> shared_chunks;
std::vector<uint32_t> shared_chunk_table;
shared_chunk_table.push_back(0);
for (auto const& inode : md_.inodes().value()) {
auto const self_index = dir_entries.size();
auto& de = dir_entries.emplace_back();
de.name_index() = inode.name_index_v2_2().value();
auto inode_v2_2 = inode.inode_v2_2().value();
if (inode_v2_2 < reg_offset) {
de.inode_num() = inode_v2_2;
// must reconstruct self_entry for directories
if (inode_v2_2 < lnk_offset) {
dirs.at(inode_v2_2).self_entry() = self_index;
}
} else if (inode_v2_2 < dev_offset) {
auto const index = inode_v2_2 - reg_offset;
auto const refs = reg_inode_refs[index];
auto const chunk_begin = md_.chunk_table()->at(index);
auto const chunk_end = md_.chunk_table()->at(index + 1);
if (refs == 1) {
chunk_table.push_back(chunk_table.back() + chunk_end - chunk_begin);
for (uint32_t i = chunk_begin; i < chunk_end; ++i) {
chunks.push_back(md_.chunks()->at(i));
}
de.inode_num() = unique_inode++;
} else {
auto [it, inserted] =
shared_inode_map.emplace(inode_v2_2, shared_inode);
if (inserted) {
for (uint32_t i = 0; i < refs; ++i) {
shared_files.push_back(shared_chunk_index);
}
++shared_chunk_index;
shared_inode += refs;
shared_chunk_table.push_back(shared_chunk_table.back() + chunk_end -
chunk_begin);
for (uint32_t i = chunk_begin; i < chunk_end; ++i) {
shared_chunks.push_back(md_.chunks()->at(i));
}
}
de.inode_num() = it->second++;
}
} else {
de.inode_num() = (inode_v2_2 - dev_offset) + reg_offset + num_reg_files;
LOG_TRACE << "dev/oth inode: " << inode_v2_2 << " -> "
<< de.inode_num().value();
}
auto& ni = inodes.at(de.inode_num().value());
ni.mode_index() = inode.mode_index().value();
ni.owner_index() = inode.owner_index().value();
ni.group_index() = inode.group_index().value();
ni.atime_offset() = inode.atime_offset().value();
ni.mtime_offset() = inode.mtime_offset().value();
ni.ctime_offset() = inode.ctime_offset().value();
}
std::transform(shared_chunk_table.begin(), shared_chunk_table.end(),
shared_chunk_table.begin(),
[&](auto i) { return i + chunks.size(); });
DWARFS_CHECK(chunk_table.back() == shared_chunk_table.front(),
"inconsistent chunk tables");
chunks.insert(chunks.end(), shared_chunks.begin(), shared_chunks.end());
chunk_table.insert(chunk_table.end(), shared_chunk_table.begin() + 1,
shared_chunk_table.end());
newmd.symlink_table().copy_from(md_.symlink_table());
newmd.uids().copy_from(md_.uids());
newmd.gids().copy_from(md_.gids());
newmd.modes().copy_from(md_.modes());
newmd.names().copy_from(md_.names());
newmd.symlinks().copy_from(md_.symlinks());
newmd.timestamp_base().copy_from(md_.timestamp_base());
newmd.block_size().copy_from(md_.block_size());
newmd.total_fs_size().copy_from(md_.total_fs_size());
newmd.devices().copy_from(md_.devices());
newmd.options().copy_from(md_.options());
md_ = std::move(newmd);
}
template <typename LoggerPolicy>
void metadata_builder_<LoggerPolicy>::upgrade_metadata() {
auto tv = LOG_TIMED_VERBOSE;
// std::cout << apache::thrift::debugString(md_);
if (apache::thrift::is_non_optional_field_set_manually_or_by_serializer(
md_.entry_table_v2_2())) {
DWARFS_CHECK(!md_.dir_entries().has_value(),
"unexpected dir_entries in metadata");
upgrade_from_pre_v2_2();
}
// TODO: update uid, gid, timestamp, mtime_only, time_resolution_sec
tv << "upgrading metadata...";
}
} // namespace
metadata_builder::metadata_builder(logger& lgr, metadata_options const& options)

View File

@ -0,0 +1,81 @@
/* vim:set ts=2 sw=2 sts=2 et: */
/**
* \author Marcus Holland-Moritz (github@mhxnet.de)
* \copyright Copyright (c) Marcus Holland-Moritz
*
* This file is part of dwarfs.
*
* dwarfs is free software: you can redistribute it and/or modify
* it under the terms of the GNU General Public License as published by
* the Free Software Foundation, either version 3 of the License, or
* (at your option) any later version.
*
* dwarfs is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with dwarfs. If not, see <https://www.gnu.org/licenses/>.
*/
#include <ostream>
#include <dwarfs/writer/metadata_options.h>
namespace dwarfs::writer {
std::ostream& operator<<(std::ostream& os, metadata_options const& opts) {
os << "{";
if (opts.uid) {
os << "uid: " << *opts.uid << ", ";
}
if (opts.gid) {
os << "gid: " << *opts.gid << ", ";
}
if (opts.timestamp) {
os << "timestamp: " << *opts.timestamp << ", ";
}
if (opts.keep_all_times) {
os << "keep_all_times, ";
}
os << "time_resolution_sec: " << opts.time_resolution_sec << ", ";
if (opts.pack_chunk_table) {
os << "pack_chunk_table, ";
}
if (opts.pack_directories) {
os << "pack_directories, ";
}
if (opts.pack_shared_files_table) {
os << "pack_shared_files_table, ";
}
if (opts.plain_names_table) {
os << "plain_names_table, ";
}
if (opts.pack_names) {
os << "pack_names, ";
}
if (opts.pack_names_index) {
os << "pack_names_index, ";
}
if (opts.plain_symlinks_table) {
os << "plain_symlinks_table, ";
}
if (opts.pack_symlinks) {
os << "pack_symlinks, ";
}
if (opts.pack_symlinks_index) {
os << "pack_symlinks_index, ";
}
if (opts.force_pack_string_tables) {
os << "force_pack_string_tables, ";
}
if (opts.no_create_timestamp) {
os << "no_create_timestamp, ";
}
os << "inode_size_cache_min_chunk_count: "
<< opts.inode_size_cache_min_chunk_count;
return os;
}
} // namespace dwarfs::writer

View File

@ -39,6 +39,7 @@
#include <folly/String.h>
#include <dwarfs/block_compressor.h>
#include <dwarfs/checksum.h>
#include <dwarfs/config.h>
#include <dwarfs/file_stat.h>
#include <dwarfs/logger.h>
@ -96,6 +97,34 @@ void remove_inode_numbers(nlohmann::json& j) {
});
}
std::map<std::string, std::string> const checksums_xxh3{
{"bench.sh", "f36b0707f57e12b1"},
{"foo/1/2/3/4/5/6/7/8/9/a", "189a846373f241e8"},
{"foo/1/2/3/4/5/6/7/8/9/b", "4aa8385890ef195b"},
{"foo/1/2/3/4/5/6/7/8/9/blubb", "f36b0707f57e12b1"},
{"foo/1/2/3/4/5/6/7/8/9/c", "f743b2af3e60e91f"},
{"foo/1/2/3/4/5/6/7/8/9/d", "fa2a5714cb578dab"},
{"foo/1/2/3/4/5/6/7/8/9/e", "13bc5fbf33dc0141"},
{"foo/1/2/3/4/5/6/7/8/9/f", "ea1aa9ade526f068"},
{"foo/1/2/3/4/5/6/7/8/9/g", "aa6aa985ddff426e"},
{"foo/1/2/3/4/5/6/7/8/9/h", "fdb55c845d7030f0"},
{"foo/1/2/3/4/5/6/7/8/9/i", "a198f272298349aa"},
{"foo/1/2/3/4/5/6/7/8/9/j", "57094636a0f3b971"},
{"foo/1/2/3/4/5/6/7/8/9/k", "f8d0f1f8278733de"},
{"foo/1/2/3/4/5/6/7/8/9/l", "b54e0c6ffb79df4f"},
{"foo/1/2/3/4/5/z", "f36b0707f57e12b1"},
{"foo/1/2/3/4/y", "f36b0707f57e12b1"},
{"foo/1/2/3/copy.sh", "67b2455044ceab01"},
{"foo/1/2/3/x", "f36b0707f57e12b1"},
{"foo/1/2/xxx.sh", "67b2455044ceab01"},
{"foo/1/fmt.sh", "67b2455044ceab01"},
{"foo/bar", "c294d3380580062d"},
{"foo/bla.sh", "f36b0707f57e12b1"},
{"format.sh", "67b2455044ceab01"},
{"perl-exec.sh", "bd546eb6e49fb736"},
{"test.py", "48d97f43e4e9758d"},
};
char const* reference_v0_2 = R"(
{
"root": {
@ -823,6 +852,27 @@ std::vector<std::string> versions{
"0.5.6", "0.6.2", "0.7.5", "0.8.0", "0.9.10",
};
using rebuild_metadata_type =
std::pair<bool, std::optional<writer::metadata_options>>;
[[maybe_unused]] std::ostream&
operator<<(std::ostream& os, rebuild_metadata_type const& rmt) {
os << (rmt.first ? "recompress" : "keep");
if (rmt.second) {
os << ", " << *rmt.second;
}
return os;
}
constexpr std::array<rebuild_metadata_type, 6> rebuild_metadata_options{{
{false, std::nullopt},
{true, std::nullopt},
{true, writer::metadata_options{}},
{true, writer::metadata_options{.pack_chunk_table = true}},
{true, writer::metadata_options{.pack_directories = true}},
{true, writer::metadata_options{.pack_shared_files_table = true}},
}};
std::string format_sh = R"(#!/bin/bash
find test/ src/ include/ -type f -name '*.[ch]*' | xargs -d $'\n' clang-format -i
)";
@ -1179,14 +1229,21 @@ auto get_image_path(std::string const& version) {
class compat_metadata : public testing::TestWithParam<std::string> {};
void check_dynamic(std::string const& version,
reader::filesystem_v2 const& fs) {
void check_dynamic(std::string const& version, reader::filesystem_v2 const& fs,
bool rebuild_metadata [[maybe_unused]] = false) {
auto meta = fs.metadata_as_json();
nlohmann::json ref;
if (version.starts_with("0.2.")) {
ref = nlohmann::json::parse(reference_v0_2);
if (rebuild_metadata) {
ref["statvfs"]["f_files"] = 41;
}
} else {
ref = nlohmann::json::parse(reference_v0_4);
if (rebuild_metadata &&
(version.starts_with("0.3.") or version.starts_with("0.4."))) {
ref["statvfs"]["f_files"] = 44;
}
}
remove_inode_numbers(ref);
@ -1195,6 +1252,22 @@ void check_dynamic(std::string const& version,
EXPECT_EQ(ref, meta) << nlohmann::json::diff(ref, meta).dump(2);
}
void check_checksums(reader::filesystem_v2 const& fs) {
std::map<std::string, std::string> fs_checksums;
fs.walk([&fs, &fs_checksums](auto const& de) {
auto iv = de.inode();
if (iv.is_regular_file()) {
auto content = fs.read_string(iv.inode_num());
checksum cs(checksum::xxh3_64);
cs.update(content.data(), content.size());
fs_checksums[de.unix_path()] = cs.hexdigest();
}
});
EXPECT_EQ(checksums_xxh3, fs_checksums);
}
TEST_P(compat_metadata, backwards_compat) {
auto version = GetParam();
auto filename = get_image_path(version);
@ -1250,11 +1323,12 @@ INSTANTIATE_TEST_SUITE_P(dwarfs_compat, compat_filesystem,
::testing::Combine(::testing::ValuesIn(versions),
::testing::Bool()));
class rewrite
: public testing::TestWithParam<std::tuple<std::string, bool, bool>> {};
class rewrite : public testing::TestWithParam<
std::tuple<std::string, bool, rebuild_metadata_type>> {};
TEST_P(rewrite, filesystem_rewrite) {
auto [version, recompress_block, recompress_metadata] = GetParam();
auto [version, recompress_block, metadata_cfg] = GetParam();
auto [recompress_metadata, rebuild_metadata] = metadata_cfg;
test::test_logger lgr;
test::os_access_mock os;
@ -1263,6 +1337,7 @@ TEST_P(rewrite, filesystem_rewrite) {
utility::rewrite_options opts;
opts.recompress_block = recompress_block;
opts.recompress_metadata = recompress_metadata;
opts.rebuild_metadata = rebuild_metadata;
thread_pool pool(lgr, os, "rewriter", 2);
block_compressor bc("null");
@ -1292,7 +1367,8 @@ TEST_P(rewrite, filesystem_rewrite) {
EXPECT_NO_THROW(reader::filesystem_v2::identify(lgr, os, mm, idss));
EXPECT_FALSE(reader::filesystem_v2::header(mm));
reader::filesystem_v2 fs(lgr, os, mm);
check_dynamic(version, fs);
check_dynamic(version, fs, rebuild_metadata.has_value());
check_checksums(fs);
}
rewritten.str(std::string());
@ -1320,7 +1396,8 @@ TEST_P(rewrite, filesystem_rewrite) {
reader::filesystem_options fsopts;
fsopts.image_offset = reader::filesystem_options::IMAGE_OFFSET_AUTO;
reader::filesystem_v2 fs(lgr, os, mm, fsopts);
check_dynamic(version, fs);
check_dynamic(version, fs, rebuild_metadata.has_value());
check_checksums(fs);
}
std::ostringstream rewritten2;
@ -1376,7 +1453,8 @@ TEST_P(rewrite, filesystem_rewrite) {
EXPECT_FALSE(reader::filesystem_v2::header(mm))
<< folly::hexDump(rewritten4.str().data(), rewritten4.str().size());
reader::filesystem_v2 fs(lgr, os, mm);
check_dynamic(version, fs);
check_dynamic(version, fs, rebuild_metadata.has_value());
check_checksums(fs);
}
std::ostringstream rewritten5;
@ -1395,14 +1473,15 @@ TEST_P(rewrite, filesystem_rewrite) {
EXPECT_FALSE(reader::filesystem_v2::header(mm))
<< folly::hexDump(rewritten5.str().data(), rewritten5.str().size());
reader::filesystem_v2 fs(lgr, os, mm);
check_dynamic(version, fs);
check_dynamic(version, fs, rebuild_metadata.has_value());
check_checksums(fs);
}
}
INSTANTIATE_TEST_SUITE_P(dwarfs_compat, rewrite,
::testing::Combine(::testing::ValuesIn(versions),
::testing::Bool(),
::testing::Bool()));
INSTANTIATE_TEST_SUITE_P(
dwarfs_compat, rewrite,
::testing::Combine(::testing::ValuesIn(versions), ::testing::Bool(),
::testing::ValuesIn(rebuild_metadata_options)));
class set_uidgid_test : public testing::TestWithParam<char const*> {};

173
test/metadata_test.cpp Normal file
View File

@ -0,0 +1,173 @@
/* vim:set ts=2 sw=2 sts=2 et: */
/**
* \author Marcus Holland-Moritz (github@mhxnet.de)
* \copyright Copyright (c) Marcus Holland-Moritz
*
* This file is part of dwarfs.
*
* dwarfs is free software: you can redistribute it and/or modify
* it under the terms of the GNU General Public License as published by
* the Free Software Foundation, either version 3 of the License, or
* (at your option) any later version.
*
* dwarfs is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with dwarfs. If not, see <https://www.gnu.org/licenses/>.
*/
#include <array>
#include <string_view>
#include <gmock/gmock.h>
#include <gtest/gtest.h>
#include <thrift/lib/cpp2/debug_thrift_data_difference/debug.h>
#include <thrift/lib/cpp2/debug_thrift_data_difference/diff.h>
#include <thrift/lib/cpp2/protocol/DebugProtocol.h>
#include <fmt/format.h>
#include <dwarfs/config.h>
#include <dwarfs/block_compressor.h>
#include <dwarfs/mmif.h>
#include <dwarfs/reader/filesystem_v2.h>
#include <dwarfs/reader/fsinfo_options.h>
#include <dwarfs/reader/metadata_options.h>
#include <dwarfs/thread_pool.h>
#include <dwarfs/writer/entry_factory.h>
#include <dwarfs/writer/filesystem_writer.h>
#include <dwarfs/writer/filesystem_writer_options.h>
#include <dwarfs/writer/metadata_options.h>
#include <dwarfs/writer/scanner.h>
#include <dwarfs/writer/scanner_options.h>
#include <dwarfs/writer/segmenter_factory.h>
#include <dwarfs/writer/writer_progress.h>
#include <dwarfs/reader/internal/metadata_v2.h>
#include <dwarfs/writer/internal/metadata_builder.h>
#include <dwarfs/writer/internal/metadata_freezer.h>
// #include <dwarfs/gen-cpp2/metadata_types.h>
#include <dwarfs/gen-cpp2/metadata_types_custom_protocol.h>
#include "loremipsum.h"
#include "mmap_mock.h"
#include "test_helpers.h"
#include "test_logger.h"
using namespace dwarfs;
namespace {
std::string make_fragmented_file(size_t fragment_size, size_t fragment_count) {
std::mt19937_64 rng{0};
auto const fragment = test::create_random_string(fragment_size, rng);
std::string file;
for (size_t i = 0; i < fragment_count; ++i) {
file.append(fragment);
file.append(test::create_random_string(4, rng));
}
return file;
}
auto rebuild_metadata(logger& lgr, thrift::metadata::metadata const& md,
writer::metadata_options const& options) {
using namespace writer::internal;
return metadata_freezer(lgr).freeze(
metadata_builder(lgr, md, options).build());
}
template <typename T>
std::string thrift_diff(T const& t1, T const& t2) {
using namespace ::facebook::thrift;
std::ostringstream oss;
debug_thrift_data_difference(t1, t2, make_diff_output_callback(oss));
return oss.str();
}
} // namespace
class metadata_test : public ::testing::Test {
protected:
void SetUp() override {
os = test::os_access_mock::create_test_instance();
os->add("lib", {333, posix_file_type::directory | 0755, 1, 1000, 100, 0, 0,
100, 200, 300});
auto libc = make_fragmented_file(1024, 130);
os->add("lib/libc.so",
{334, posix_file_type::regular | 0755, 1, 1000, 100,
static_cast<file_stat::off_type>(libc.size()), 0, 100, 200, 300},
libc);
writer::writer_progress prog;
writer::segmenter_factory::config sf_cfg;
sf_cfg.blockhash_window_size.set_default(9);
sf_cfg.window_increment_shift.set_default(1);
sf_cfg.max_active_blocks.set_default(1);
sf_cfg.bloom_filter_size.set_default(4);
sf_cfg.block_size_bits = 12;
writer::segmenter_factory sf(lgr, prog, sf_cfg);
writer::entry_factory ef;
thread_pool pool(lgr, *os, "worker", 4);
writer::scanner_options options;
options.metadata.no_create_timestamp = true;
writer::scanner s(lgr, pool, sf, ef, *os, options);
block_compressor bc("null");
std::ostringstream oss;
writer::filesystem_writer fsw(oss, lgr, pool, prog, {});
fsw.add_default_compressor(bc);
s.scan(fsw, std::filesystem::path("/"), prog);
mm = std::make_shared<test::mmap_mock>(oss.str());
}
void TearDown() override {}
test::test_logger lgr;
std::shared_ptr<test::os_access_mock> os;
std::shared_ptr<mmif> mm;
};
TEST_F(metadata_test, basic) {
reader::filesystem_v2 fs(lgr, *os, mm);
auto thawed1 = *fs.thawed_metadata();
auto unpacked1 = *fs.unpacked_metadata();
// std::cout << ::apache::thrift::debugString(thawed1) << std::endl;
// std::cout << ::apache::thrift::debugString(unpacked1) << std::endl;
{
auto [schema, data] = rebuild_metadata(
lgr, unpacked1,
{.plain_names_table = true, .no_create_timestamp = true});
reader::internal::metadata_v2 mv2(lgr, schema.span(), data.span(), {});
using utils = reader::internal::metadata_v2_utils;
auto thawed2 = *utils(mv2).thaw();
auto unpacked2 = *utils(mv2).unpack();
// std::cout << ::apache::thrift::debugString(unpacked2) << std::endl;
EXPECT_EQ(unpacked1, unpacked2) << thrift_diff(unpacked1, unpacked2);
EXPECT_NE(thawed1, thawed2) << thrift_diff(thawed1, thawed2);
// std::cout << thrift_diff(thawed1, thawed2);
// std::cout << thrift_diff(unpacked1, unpacked2);
}
}

View File

@ -418,7 +418,8 @@ int mkdwarfs_main(int argc, sys_char** argv, iolayer const& iol) {
size_t num_workers, num_scanner_workers, num_segmenter_workers;
bool no_progress = false, remove_header = false, no_section_index = false,
force_overwrite = false, no_history = false,
no_history_timestamps = false, no_history_command_line = false;
no_history_timestamps = false, no_history_command_line = false,
rebuild_metadata = false;
unsigned level;
int compress_niceness;
uint16_t uid, gid;
@ -526,6 +527,9 @@ int mkdwarfs_main(int argc, sys_char** argv, iolayer const& iol) {
("recompress",
po::value<std::string>(&recompress_opts)->implicit_value("all"),
"recompress an existing filesystem (none, block, metadata, all)")
("rebuild-metadata",
po::value<bool>(&rebuild_metadata)->zero_tokens(),
"fully rebuild metadata")
("recompress-categories",
po::value<std::string>(&recompress_categories),
"only recompress blocks of these categories")
@ -885,7 +889,7 @@ int mkdwarfs_main(int argc, sys_char** argv, iolayer const& iol) {
path = iol.os->canonical(path);
bool recompress = vm.contains("recompress");
bool recompress = vm.contains("recompress") || rebuild_metadata;
utility::rewrite_options rw_opts;
if (recompress) {
std::unordered_map<std::string, unsigned> const modes{
@ -894,6 +898,11 @@ int mkdwarfs_main(int argc, sys_char** argv, iolayer const& iol) {
{"block", 1},
{"none", 0},
};
if (recompress_opts.empty() && rebuild_metadata) {
recompress_opts = "metadata";
}
if (auto it = modes.find(recompress_opts); it != modes.end()) {
rw_opts.recompress_block = it->second & 1;
rw_opts.recompress_metadata = it->second & 2;
@ -1392,6 +1401,9 @@ int mkdwarfs_main(int argc, sys_char** argv, iolayer const& iol) {
try {
if (recompress) {
if (rebuild_metadata) {
rw_opts.rebuild_metadata = options.metadata;
}
utility::rewrite_filesystem(lgr, *input_filesystem, *fsw, *cat_resolver,
rw_opts);
} else {