refactor(progress): hide internals behind new writer_progress class

This commit is contained in:
Marcus Holland-Moritz 2024-07-29 10:49:14 +02:00
parent a544ce1e46
commit 61d77b3295
28 changed files with 261 additions and 143 deletions

View File

@ -663,13 +663,14 @@ list(APPEND LIBDWARFS_WRITER_SRC
src/dwarfs/internal/inode_ordering.cpp
src/dwarfs/internal/metadata_freezer.cpp
src/dwarfs/internal/nilsimsa.cpp
src/dwarfs/internal/progress.cpp
src/dwarfs/internal/scanner_progress.cpp
src/dwarfs/internal/similarity.cpp
src/dwarfs/internal/similarity_ordering.cpp
src/dwarfs/progress.cpp
src/dwarfs/scanner.cpp
src/dwarfs/segmenter.cpp
src/dwarfs/segmenter_factory.cpp
src/dwarfs/writer_progress.cpp
)
list(APPEND LIBDWARFS_EXTRACTOR_SRC

View File

@ -32,8 +32,8 @@
namespace dwarfs {
class progress;
class terminal;
class writer_progress;
class console_writer : public stream_logger {
public:
@ -44,7 +44,7 @@ class console_writer : public stream_logger {
progress_mode pg_mode, display_mode mode = NORMAL,
logger_options const& options = {});
void update(progress& p, bool last);
void update(writer_progress& prog, bool last);
private:
void preamble(std::ostream& os) override;

View File

@ -52,12 +52,12 @@ class dir;
class device;
class mmif;
class os_access;
class progress;
namespace internal {
class global_entry_data;
class inode;
class progress;
} // namespace internal
@ -95,7 +95,7 @@ class entry : public entry_interface {
internal::global_entry_data const& data) const;
void update(internal::global_entry_data& data) const;
virtual void accept(entry_visitor& v, bool preorder = false) = 0;
virtual void scan(os_access const& os, progress& prog) = 0;
virtual void scan(os_access const& os, internal::progress& prog) = 0;
file_stat const& status() const { return stat_; }
void set_entry_index(uint32_t index) { entry_index_ = index; }
std::optional<uint32_t> const& entry_index() const { return entry_index_; }
@ -138,11 +138,11 @@ class file : public entry {
void set_inode(std::shared_ptr<internal::inode> ino);
std::shared_ptr<internal::inode> get_inode() const;
void accept(entry_visitor& v, bool preorder) override;
void scan(os_access const& os, progress& prog) override;
void
scan(mmif* mm, progress& prog, std::optional<std::string> const& hash_alg);
void scan(os_access const& os, internal::progress& prog) override;
void scan(mmif* mm, internal::progress& prog,
std::optional<std::string> const& hash_alg);
void create_data();
void hardlink(file* other, progress& prog);
void hardlink(file* other, internal::progress& prog);
uint32_t unique_file_id() const;
void set_inode_num(uint32_t ino) override;
@ -180,9 +180,9 @@ class dir : public entry {
internal::global_entry_data const& data) const;
void pack_entry(thrift::metadata::metadata& mv2,
internal::global_entry_data const& data) const;
void scan(os_access const& os, progress& prog) override;
void scan(os_access const& os, internal::progress& prog) override;
bool empty() const { return entries_.empty(); }
void remove_empty_dirs(progress& prog);
void remove_empty_dirs(internal::progress& prog);
void set_inode_num(uint32_t ino) override { inode_num_ = ino; }
std::optional<uint32_t> const& inode_num() const override {
@ -209,7 +209,7 @@ class link : public entry {
type_t type() const override;
const std::string& linkname() const;
void accept(entry_visitor& v, bool preorder) override;
void scan(os_access const& os, progress& prog) override;
void scan(os_access const& os, internal::progress& prog) override;
void set_inode_num(uint32_t ino) override { inode_num_ = ino; }
std::optional<uint32_t> const& inode_num() const override {
@ -231,7 +231,7 @@ class device : public entry {
type_t type() const override;
void accept(entry_visitor& v, bool preorder) override;
void scan(os_access const& os, progress& prog) override;
void scan(os_access const& os, internal::progress& prog) override;
uint64_t device_id() const;
void set_inode_num(uint32_t ino) override { inode_num_ = ino; }

View File

@ -53,7 +53,7 @@ class logger;
class mmif;
class os_access;
class performance_monitor;
class progress;
class writer_progress;
class filesystem_v2 {
public:
@ -249,7 +249,7 @@ class filesystem_v2 {
return impl_->get_all_gids();
}
void rewrite(progress& prog, filesystem_writer& writer,
void rewrite(writer_progress& prog, filesystem_writer& writer,
category_resolver const& cat_resolver,
rewrite_options const& opts) const {
return impl_->rewrite(prog, writer, cat_resolver, opts);
@ -318,7 +318,7 @@ class filesystem_v2 {
virtual std::vector<std::string> get_all_block_categories() const = 0;
virtual std::vector<file_stat::uid_type> get_all_uids() const = 0;
virtual std::vector<file_stat::gid_type> get_all_gids() const = 0;
virtual void rewrite(progress& prog, filesystem_writer& writer,
virtual void rewrite(writer_progress& prog, filesystem_writer& writer,
category_resolver const& cat_resolver,
rewrite_options const& opts) const = 0;
};

View File

@ -39,7 +39,7 @@ namespace dwarfs {
class block_compressor;
class logger;
class progress;
class writer_progress;
class thread_pool;
namespace internal {
@ -54,7 +54,7 @@ class filesystem_writer {
using physical_block_cb_type = std::function<void(size_t)>;
filesystem_writer(
std::ostream& os, logger& lgr, thread_pool& pool, progress& prog,
std::ostream& os, logger& lgr, thread_pool& pool, writer_progress& prog,
block_compressor const& schema_bc, block_compressor const& metadata_bc,
block_compressor const& history_bc,
filesystem_writer_options const& options = filesystem_writer_options(),

View File

@ -23,7 +23,6 @@
#include <atomic>
#include <chrono>
#include <condition_variable>
#include <cstddef>
#include <cstdint>
#include <functional>
@ -31,7 +30,6 @@
#include <memory>
#include <mutex>
#include <optional>
#include <thread>
#include <vector>
#include <dwarfs/speedometer.h>
@ -41,6 +39,8 @@ namespace dwarfs {
class object;
namespace internal {
class progress {
public:
class context {
@ -62,16 +62,11 @@ class progress {
speedometer<uint64_t> speed{std::chrono::seconds(5)};
};
using update_function_type = std::function<void(progress&, bool)>;
using status_function_type =
std::function<std::string(progress const&, size_t)>;
progress();
explicit progress(update_function_type func);
progress(update_function_type func, std::chrono::microseconds interval);
~progress() noexcept;
~progress();
void set_status_function(status_function_type status_fun);
@ -152,13 +147,11 @@ class progress {
private:
void add_context(std::shared_ptr<context> const& ctx) const;
mutable std::mutex running_mx_;
bool running_{false};
mutable std::mutex mx_;
std::condition_variable cond_;
std::shared_ptr<status_function_type> status_fun_;
std::vector<std::weak_ptr<context>> mutable contexts_;
std::thread thread_;
};
} // namespace internal
} // namespace dwarfs

View File

@ -24,9 +24,10 @@
#include <atomic>
#include <string>
#include <dwarfs/progress.h>
#include <dwarfs/terminal.h>
#include <dwarfs/internal/progress.h>
namespace dwarfs::internal {
class scanner_progress : public progress::context {

View File

@ -36,7 +36,7 @@ class file_access;
class filesystem_writer;
class logger;
class os_access;
class progress;
class writer_progress;
class script;
class segmenter_factory;
class thread_pool;
@ -49,7 +49,8 @@ class scanner {
const scanner_options& options);
void scan(
filesystem_writer& fsw, const std::filesystem::path& path, progress& prog,
filesystem_writer& fsw, const std::filesystem::path& path,
writer_progress& prog,
std::optional<std::span<std::filesystem::path const>> list = std::nullopt,
std::shared_ptr<file_access const> fa = nullptr) {
impl_->scan(fsw, path, prog, list, fa);
@ -61,7 +62,7 @@ class scanner {
virtual void
scan(filesystem_writer& fsw, const std::filesystem::path& path,
progress& prog,
writer_progress& prog,
std::optional<std::span<std::filesystem::path const>> list,
std::shared_ptr<file_access const> fa) = 0;
};

View File

@ -31,7 +31,7 @@ namespace dwarfs {
struct compression_constraints;
class logger;
class progress;
class writer_progress;
namespace internal {
@ -55,7 +55,7 @@ class segmenter {
using block_ready_cb = std::function<void(
std::shared_ptr<internal::block_data>, size_t logical_block_num)>;
segmenter(logger& lgr, progress& prog,
segmenter(logger& lgr, writer_progress& prog,
std::shared_ptr<internal::block_manager> blkmgr, config const& cfg,
compression_constraints const& cc, size_t total_size,
block_ready_cb block_ready);

View File

@ -30,7 +30,7 @@ namespace dwarfs {
class categorizer_manager;
class logger;
class progress;
class writer_progress;
struct compression_constraints;
@ -44,11 +44,11 @@ class segmenter_factory {
unsigned block_size_bits{22};
};
segmenter_factory(logger& lgr, progress& prog,
segmenter_factory(logger& lgr, writer_progress& prog,
std::shared_ptr<categorizer_manager> catmgr,
config const& cfg);
segmenter_factory(logger& lgr, progress& prog, config const& cfg);
segmenter_factory(logger& lgr, writer_progress& prog, config const& cfg);
segmenter create(fragment_category cat, size_t cat_size,
compression_constraints const& cc,

View File

@ -0,0 +1,67 @@
/* 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/>.
*/
#pragma once
#include <chrono>
#include <condition_variable>
#include <functional>
#include <memory>
#include <mutex>
#include <thread>
namespace dwarfs {
namespace internal {
class progress;
} // namespace internal
class writer_progress {
public:
using update_function_type = std::function<void(writer_progress&, bool)>;
writer_progress();
explicit writer_progress(update_function_type func);
writer_progress(update_function_type func,
std::chrono::microseconds interval);
writer_progress(const writer_progress&) = delete;
writer_progress& operator=(const writer_progress&) = delete;
writer_progress(writer_progress&&) = delete;
writer_progress& operator=(writer_progress&&) = delete;
~writer_progress() noexcept;
size_t errors() const;
internal::progress& get_internal() const { return *prog_; }
private:
std::unique_ptr<internal::progress> prog_;
mutable std::mutex running_mx_;
bool running_{false};
std::condition_variable cond_;
std::thread thread_;
};
} // namespace dwarfs

View File

@ -30,9 +30,11 @@
#include <dwarfs/entry_interface.h>
#include <dwarfs/lazy_value.h>
#include <dwarfs/logger.h>
#include <dwarfs/progress.h>
#include <dwarfs/terminal.h>
#include <dwarfs/util.h>
#include <dwarfs/writer_progress.h>
#include <dwarfs/internal/progress.h>
namespace dwarfs {
@ -74,8 +76,8 @@ std::string progress_bar(size_t width, double frac, bool unicode) {
}
void output_context_line(terminal const& term, std::ostream& os,
progress::context& ctx, size_t width, bool unicode_bar,
bool colored) {
internal::progress::context& ctx, size_t width,
bool unicode_bar, bool colored) {
auto st = ctx.get_status();
size_t progress_w = 0;
size_t speed_w = 0;
@ -199,7 +201,7 @@ std::string_view console_writer::get_newline() const {
return pg_mode_ != NONE ? "\x1b[K\n" : "\n";
}
void console_writer::update(progress& p, bool last) {
void console_writer::update(writer_progress& prog, bool last) {
if (pg_mode_ == NONE && !last) {
return;
}
@ -212,7 +214,7 @@ void console_writer::update(progress& p, bool last) {
bool fancy = pg_mode_ == ASCII || pg_mode_ == UNICODE;
auto update_chunk_size = [](progress::scan_progress& sp) {
auto update_chunk_size = [](internal::progress::scan_progress& sp) {
if (auto usec = sp.usec.load(); usec > 10'000) {
auto bytes = sp.bytes.load();
auto bytes_per_second = (bytes << 20) / usec;
@ -223,6 +225,8 @@ void console_writer::update(progress& p, bool last) {
}
};
auto& p = prog.get_internal();
update_chunk_size(p.hash);
update_chunk_size(p.similarity);
update_chunk_size(p.categorize);
@ -340,7 +344,7 @@ void console_writer::update(progress& p, bool last) {
++counter_;
std::vector<std::shared_ptr<progress::context>> ctxs;
std::vector<std::shared_ptr<internal::progress::context>> ctxs;
if (w >= 60) {
ctxs = p.get_active_contexts();

View File

@ -32,11 +32,11 @@
#include <dwarfs/mmif.h>
#include <dwarfs/options.h>
#include <dwarfs/os_access.h>
#include <dwarfs/progress.h>
#include <dwarfs/util.h>
#include <dwarfs/internal/global_entry_data.h>
#include <dwarfs/internal/inode.h>
#include <dwarfs/internal/progress.h>
#include <dwarfs/internal/scanner_progress.h>
#include <dwarfs/gen-cpp2/metadata_types.h>
@ -208,16 +208,16 @@ std::shared_ptr<internal::inode> file::get_inode() const { return inode_; }
void file::accept(entry_visitor& v, bool) { v.visit(this); }
void file::scan(os_access const& /*os*/, progress& /*prog*/) {
void file::scan(os_access const& /*os*/, internal::progress& /*prog*/) {
DWARFS_THROW(runtime_error, "file::scan() without hash_alg is not used");
}
void file::scan(mmif* mm, progress& prog,
void file::scan(mmif* mm, internal::progress& prog,
std::optional<std::string> const& hash_alg) {
size_t s = size();
if (hash_alg) {
progress::scan_updater supd(prog.hash, s);
internal::progress::scan_updater supd(prog.hash, s);
checksum cs(*hash_alg);
if (s > 0) {
@ -271,7 +271,7 @@ void file::create_data() {
data_ = std::make_shared<data>();
}
void file::hardlink(file* other, progress& prog) {
void file::hardlink(file* other, internal::progress& prog) {
assert(!data_);
assert(other->data_);
prog.hardlink_size += size();
@ -327,7 +327,7 @@ void dir::sort() {
});
}
void dir::scan(os_access const&, progress&) {}
void dir::scan(os_access const&, internal::progress&) {}
void dir::pack_entry(thrift::metadata::metadata& mv2,
internal::global_entry_data const& data) const {
@ -360,7 +360,7 @@ void dir::pack(thrift::metadata::metadata& mv2,
}
}
void dir::remove_empty_dirs(progress& prog) {
void dir::remove_empty_dirs(internal::progress& prog) {
auto last = std::remove_if(entries_.begin(), entries_.end(),
[&](std::shared_ptr<entry> const& e) {
if (auto d = dynamic_cast<dir*>(e.get())) {
@ -420,7 +420,7 @@ const std::string& link::linkname() const { return link_; }
void link::accept(entry_visitor& v, bool) { v.visit(this); }
void link::scan(os_access const& os, progress& prog) {
void link::scan(os_access const& os, internal::progress& prog) {
link_ = u8string_to_string(os.read_symlink(fs_path()).u8string());
prog.original_size += size();
prog.symlink_size += size();
@ -438,7 +438,7 @@ entry::type_t device::type() const {
void device::accept(entry_visitor& v, bool) { v.visit(this); }
void device::scan(os_access const&, progress&) {}
void device::scan(os_access const&, internal::progress&) {}
uint64_t device::device_id() const { return status().rdev; }

View File

@ -44,14 +44,15 @@
#include <dwarfs/mmif.h>
#include <dwarfs/options.h>
#include <dwarfs/performance_monitor.h>
#include <dwarfs/progress.h>
#include <dwarfs/util.h>
#include <dwarfs/writer_progress.h>
#include <dwarfs/internal/block_cache.h>
#include <dwarfs/internal/block_data.h>
#include <dwarfs/internal/fs_section.h>
#include <dwarfs/internal/inode_reader_v2.h>
#include <dwarfs/internal/metadata_v2.h>
#include <dwarfs/internal/progress.h>
#include <dwarfs/internal/worker_group.h>
namespace dwarfs {
@ -470,7 +471,7 @@ class filesystem_ final : public filesystem_v2::impl {
std::vector<file_stat::gid_type> get_all_gids() const override {
return meta_.get_all_gids();
}
void rewrite(progress& prog, filesystem_writer& writer,
void rewrite(writer_progress& prog, filesystem_writer& writer,
category_resolver const& cat_resolver,
rewrite_options const& opts) const override;
@ -670,7 +671,7 @@ filesystem_<LoggerPolicy>::filesystem_(
}
template <typename LoggerPolicy>
void filesystem_<LoggerPolicy>::rewrite(progress& prog,
void filesystem_<LoggerPolicy>::rewrite(writer_progress& prog,
filesystem_writer& writer,
category_resolver const& cat_resolver,
rewrite_options const& opts) const {
@ -693,9 +694,11 @@ void filesystem_<LoggerPolicy>::rewrite(progress& prog,
}
}
prog.original_size = mm_->size();
prog.filesystem_size = mm_->size();
prog.block_count = num_blocks();
auto& iprog = prog.get_internal();
iprog.original_size = mm_->size();
iprog.filesystem_size = mm_->size();
iprog.block_count = num_blocks();
if (header_) {
writer.copy_header(*header_);

View File

@ -39,13 +39,14 @@
#include <dwarfs/filesystem_writer.h>
#include <dwarfs/fstypes.h>
#include <dwarfs/logger.h>
#include <dwarfs/progress.h>
#include <dwarfs/thread_pool.h>
#include <dwarfs/util.h>
#include <dwarfs/writer_progress.h>
#include <dwarfs/internal/block_data.h>
#include <dwarfs/internal/fs_section.h>
#include <dwarfs/internal/multi_queue_block_merger.h>
#include <dwarfs/internal/progress.h>
#include <dwarfs/internal/worker_group.h>
namespace dwarfs {
@ -1100,7 +1101,7 @@ void filesystem_writer_<LoggerPolicy>::write_section_index() {
} // namespace internal
filesystem_writer::filesystem_writer(std::ostream& os, logger& lgr,
thread_pool& pool, progress& prog,
thread_pool& pool, writer_progress& prog,
block_compressor const& schema_bc,
block_compressor const& metadata_bc,
block_compressor const& history_bc,
@ -1108,7 +1109,7 @@ filesystem_writer::filesystem_writer(std::ostream& os, logger& lgr,
std::istream* header)
: impl_(make_unique_logging_object<impl, internal::filesystem_writer_,
logger_policies>(
lgr, os, pool.get_worker_group(), prog, schema_bc, metadata_bc,
history_bc, options, header)) {}
lgr, os, pool.get_worker_group(), prog.get_internal(), schema_bc,
metadata_bc, history_bc, options, header)) {}
} // namespace dwarfs

View File

@ -39,12 +39,12 @@
#include <dwarfs/mmif.h>
#include <dwarfs/options.h>
#include <dwarfs/os_access.h>
#include <dwarfs/progress.h>
#include <dwarfs/util.h>
#include <dwarfs/internal/file_scanner.h>
#include <dwarfs/internal/inode.h>
#include <dwarfs/internal/inode_manager.h>
#include <dwarfs/internal/progress.h>
#include <dwarfs/internal/worker_group.h>
namespace dwarfs::internal {

View File

@ -49,13 +49,13 @@
#include <dwarfs/mmif.h>
#include <dwarfs/options.h>
#include <dwarfs/os_access.h>
#include <dwarfs/progress.h>
#include <dwarfs/script.h>
#include <dwarfs/util.h>
#include <dwarfs/internal/inode_manager.h>
#include <dwarfs/internal/inode_ordering.h>
#include <dwarfs/internal/nilsimsa.h>
#include <dwarfs/internal/progress.h>
#include <dwarfs/internal/promise_receiver.h>
#include <dwarfs/internal/scanner_progress.h>
#include <dwarfs/internal/similarity.h>

View File

@ -20,51 +20,14 @@
*/
#include <algorithm>
#include <chrono>
#include <utility>
#include <folly/portability/Windows.h>
#include <folly/system/ThreadName.h>
#include <dwarfs/internal/progress.h>
#include <dwarfs/progress.h>
namespace dwarfs::internal {
namespace dwarfs {
progress::progress() {}
progress::progress(update_function_type func)
: progress(std::move(func), std::chrono::seconds(1)) {}
progress::progress(update_function_type func,
std::chrono::microseconds interval)
: running_(true)
, thread_([this, interval, func = std::move(func)]() mutable {
folly::setThreadName("progress");
#ifdef _WIN32
::SetThreadPriority(::GetCurrentThread(), THREAD_PRIORITY_HIGHEST);
#endif
std::unique_lock lock(running_mx_);
while (running_) {
func(*this, false);
cond_.wait_for(lock, interval);
}
func(*this, true);
}) {
}
progress::~progress() noexcept {
if (running_) {
try {
{
std::lock_guard lock(running_mx_);
running_ = false;
}
cond_.notify_all();
thread_.join();
} catch (...) {
}
}
}
progress::progress() = default;
progress::~progress() = default;
void progress::add_context(std::shared_ptr<context> const& ctx) const {
std::lock_guard lock(mx_);
@ -115,4 +78,4 @@ std::string progress::status(size_t max_len) {
return std::string();
}
} // namespace dwarfs
} // namespace dwarfs::internal

View File

@ -33,8 +33,8 @@
#include <dwarfs/compiler.h>
#include <dwarfs/logger.h>
#include <dwarfs/progress.h>
#include <dwarfs/internal/progress.h>
#include <dwarfs/internal/similarity_ordering.h>
#include <dwarfs/internal/worker_group.h>

View File

@ -48,13 +48,13 @@
#include <dwarfs/mmif.h>
#include <dwarfs/options.h>
#include <dwarfs/os_access.h>
#include <dwarfs/progress.h>
#include <dwarfs/scanner.h>
#include <dwarfs/script.h>
#include <dwarfs/segmenter_factory.h>
#include <dwarfs/thread_pool.h>
#include <dwarfs/util.h>
#include <dwarfs/version.h>
#include <dwarfs/writer_progress.h>
#include <dwarfs/internal/block_data.h>
#include <dwarfs/internal/block_manager.h>
@ -66,6 +66,7 @@
#include <dwarfs/internal/inode_manager.h>
#include <dwarfs/internal/inode_ordering.h>
#include <dwarfs/internal/metadata_freezer.h>
#include <dwarfs/internal/progress.h>
#include <dwarfs/internal/string_table.h>
#include <dwarfs/internal/worker_group.h>
@ -293,7 +294,7 @@ class scanner_ final : public scanner::impl {
const scanner_options& options);
void scan(filesystem_writer& fsw, std::filesystem::path const& path,
progress& prog,
writer_progress& wprog,
std::optional<std::span<std::filesystem::path const>> list,
std::shared_ptr<file_access const> fa) override;
@ -604,9 +605,12 @@ scanner_<LoggerPolicy>::scan_list(std::filesystem::path const& path,
template <typename LoggerPolicy>
void scanner_<LoggerPolicy>::scan(
filesystem_writer& fsw, const std::filesystem::path& path, progress& prog,
filesystem_writer& fsw, const std::filesystem::path& path,
writer_progress& wprog,
std::optional<std::span<std::filesystem::path const>> list,
std::shared_ptr<file_access const> fa) {
auto& prog = wprog.get_internal();
if (!options_.debug_filter_function) {
LOG_INFO << "scanning " << path;
}

View File

@ -46,14 +46,15 @@
#include <dwarfs/entry.h>
#include <dwarfs/error.h>
#include <dwarfs/logger.h>
#include <dwarfs/progress.h>
#include <dwarfs/segmenter.h>
#include <dwarfs/util.h>
#include <dwarfs/writer_progress.h>
#include <dwarfs/internal/block_data.h>
#include <dwarfs/internal/block_manager.h>
#include <dwarfs/internal/chunkable.h>
#include <dwarfs/internal/cyclic_hash.h>
#include <dwarfs/internal/progress.h>
namespace dwarfs {
@ -1353,11 +1354,12 @@ create_segmenter(logger& lgr, progress& prog,
} // namespace internal
segmenter::segmenter(logger& lgr, progress& prog,
segmenter::segmenter(logger& lgr, writer_progress& prog,
std::shared_ptr<internal::block_manager> blkmgr,
config const& cfg, compression_constraints const& cc,
size_t total_size, block_ready_cb block_ready)
: impl_(internal::create_segmenter(lgr, prog, std::move(blkmgr), cfg, cc,
total_size, std::move(block_ready))) {}
: impl_(internal::create_segmenter(lgr, prog.get_internal(),
std::move(blkmgr), cfg, cc, total_size,
std::move(block_ready))) {}
} // namespace dwarfs

View File

@ -28,7 +28,7 @@ namespace internal {
class segmenter_factory_ final : public segmenter_factory::impl {
public:
segmenter_factory_(logger& lgr, progress& prog,
segmenter_factory_(logger& lgr, writer_progress& prog,
std::shared_ptr<categorizer_manager> catmgr,
const segmenter_factory::config& cfg)
: lgr_{lgr}
@ -62,7 +62,7 @@ class segmenter_factory_ final : public segmenter_factory::impl {
private:
logger& lgr_;
progress& prog_;
writer_progress& prog_;
std::shared_ptr<categorizer_manager> catmgr_;
segmenter_factory::config cfg_;
};
@ -70,12 +70,12 @@ class segmenter_factory_ final : public segmenter_factory::impl {
} // namespace internal
segmenter_factory::segmenter_factory(
logger& lgr, progress& prog, std::shared_ptr<categorizer_manager> catmgr,
config const& cfg)
logger& lgr, writer_progress& prog,
std::shared_ptr<categorizer_manager> catmgr, config const& cfg)
: impl_(std::make_unique<internal::segmenter_factory_>(
lgr, prog, std::move(catmgr), cfg)) {}
segmenter_factory::segmenter_factory(logger& lgr, progress& prog,
segmenter_factory::segmenter_factory(logger& lgr, writer_progress& prog,
config const& cfg)
: segmenter_factory(lgr, prog, nullptr, cfg) {}

View File

@ -0,0 +1,71 @@
/* 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 <folly/portability/Windows.h>
#include <folly/system/ThreadName.h>
#include <dwarfs/writer_progress.h>
#include <dwarfs/internal/progress.h>
namespace dwarfs {
writer_progress::writer_progress()
: prog_{std::make_unique<internal::progress>()} {}
writer_progress::writer_progress(update_function_type func)
: writer_progress(std::move(func), std::chrono::seconds(1)) {}
writer_progress::writer_progress(update_function_type func,
std::chrono::microseconds interval)
: prog_{std::make_unique<internal::progress>()}
, running_(true)
, thread_([this, interval, func = std::move(func)]() mutable {
folly::setThreadName("progress");
#ifdef _WIN32
::SetThreadPriority(::GetCurrentThread(), THREAD_PRIORITY_HIGHEST);
#endif
std::unique_lock lock(running_mx_);
while (running_) {
func(*this, false);
cond_.wait_for(lock, interval);
}
func(*this, true);
}) {
}
writer_progress::~writer_progress() noexcept {
if (running_) {
try {
{
std::lock_guard lock(running_mx_);
running_ = false;
}
cond_.notify_all();
thread_.join();
} catch (...) {
}
}
}
size_t writer_progress::errors() const { return prog_->errors; }
} // namespace dwarfs

View File

@ -83,7 +83,6 @@
#include <dwarfs/options.h>
#include <dwarfs/os_access.h>
#include <dwarfs/program_options_helpers.h>
#include <dwarfs/progress.h>
#include <dwarfs/scanner.h>
#include <dwarfs/script.h>
#include <dwarfs/segmenter_factory.h>
@ -91,6 +90,7 @@
#include <dwarfs/thread_pool.h>
#include <dwarfs/tool.h>
#include <dwarfs/util.h>
#include <dwarfs/writer_progress.h>
#include <dwarfs_tool_main.h>
namespace po = boost::program_options;
@ -1092,15 +1092,15 @@ int mkdwarfs_main(int argc, sys_char** argv, iolayer const& iol) {
LOG_PROXY(debug_logger_policy, lgr);
progress::update_function_type updater;
writer_progress::update_function_type updater;
if (options.debug_filter_function) {
updater = [](progress&, bool) {};
updater = [](writer_progress&, bool) {};
} else {
updater = [&](progress& p, bool last) { lgr.update(p, last); };
updater = [&](writer_progress& p, bool last) { lgr.update(p, last); };
}
progress prog(std::move(updater), interval);
writer_progress prog(std::move(updater), interval);
// No more streaming to iol.err after this point as this would
// cause a race with the progress thread.
@ -1393,12 +1393,14 @@ int mkdwarfs_main(int argc, sys_char** argv, iolayer const& iol) {
}
}
auto errors = prog.errors();
if (!options.debug_filter_function) {
std::ostringstream err;
if (prog.errors) {
err << "with " << prog.errors << " error";
if (prog.errors > 1) {
if (errors) {
err << "with " << errors << " error";
if (errors > 1) {
err << "s";
}
} else {
@ -1409,7 +1411,7 @@ int mkdwarfs_main(int argc, sys_char** argv, iolayer const& iol) {
<< err.str();
}
return prog.errors > 0 ? 2 : 0;
return errors > 0 ? 2 : 0;
}
int mkdwarfs_main(int argc, sys_char** argv) {

View File

@ -45,9 +45,9 @@
#include <dwarfs/logger.h>
#include <dwarfs/mmap.h>
#include <dwarfs/options.h>
#include <dwarfs/progress.h>
#include <dwarfs/thread_pool.h>
#include <dwarfs/vfs_stat.h>
#include <dwarfs/writer_progress.h>
#include "mmap_mock.h"
#include "test_helpers.h"
@ -1114,7 +1114,7 @@ TEST_P(rewrite, filesystem_rewrite) {
thread_pool pool(lgr, os, "rewriter", 2);
block_compressor bc("null");
progress prog;
writer_progress prog;
std::ostringstream rewritten, idss;
auto rewrite_fs = [&](auto& fsw, auto const& mm) {

View File

@ -33,11 +33,11 @@
#include <dwarfs/iovec_read_buf.h>
#include <dwarfs/logger.h>
#include <dwarfs/options.h>
#include <dwarfs/progress.h>
#include <dwarfs/scanner.h>
#include <dwarfs/segmenter_factory.h>
#include <dwarfs/thread_pool.h>
#include <dwarfs/vfs_stat.h>
#include <dwarfs/writer_progress.h>
#include <dwarfs/internal/string_table.h>
@ -121,7 +121,7 @@ std::string make_filesystem(::benchmark::State const& state) {
auto os = test::os_access_mock::create_test_instance();
thread_pool pool(lgr, *os, "writer", 4);
progress prog;
writer_progress prog;
auto sf = std::make_shared<segmenter_factory>(lgr, prog, cfg);

View File

@ -45,13 +45,14 @@
#include <dwarfs/logger.h>
#include <dwarfs/mmif.h>
#include <dwarfs/options.h>
#include <dwarfs/progress.h>
#include <dwarfs/scanner.h>
#include <dwarfs/segmenter_factory.h>
#include <dwarfs/thread_pool.h>
#include <dwarfs/vfs_stat.h>
#include <dwarfs/writer_progress.h>
#include <dwarfs/internal/fs_section.h>
#include <dwarfs/internal/progress.h>
#include "filter_test_data.h"
#include "loremipsum.h"
@ -72,15 +73,16 @@ build_dwarfs(logger& lgr, std::shared_ptr<test::os_access_mock> input,
std::string const& compression,
segmenter::config const& cfg = segmenter::config(),
scanner_options const& options = scanner_options(),
progress* prog = nullptr, std::shared_ptr<script> scr = nullptr,
writer_progress* prog = nullptr,
std::shared_ptr<script> scr = nullptr,
std::optional<std::span<std::filesystem::path const>> input_list =
std::nullopt) {
// force multithreading
thread_pool pool(lgr, *input, "worker", 4);
std::unique_ptr<progress> local_prog;
std::unique_ptr<writer_progress> local_prog;
if (!prog) {
local_prog = std::make_unique<progress>();
local_prog = std::make_unique<writer_progress>();
prog = local_prog.get();
}
@ -163,11 +165,12 @@ void basic_end_to_end_test(std::string const& compressor,
input->set_access_fail("/somedir/ipsum.py");
}
progress prog;
writer_progress wprog;
auto scr = std::make_shared<test::script_mock>();
auto fsimage = build_dwarfs(lgr, input, compressor, cfg, options, &prog, scr);
auto fsimage =
build_dwarfs(lgr, input, compressor, cfg, options, &wprog, scr);
EXPECT_EQ(14, scr->filter_calls.size());
EXPECT_EQ(15, scr->transform_calls.size());
@ -180,6 +183,8 @@ void basic_end_to_end_test(std::string const& compressor,
size_t const num_fail_empty = access_fail ? 1 : 0;
auto& prog = wprog.get_internal();
EXPECT_EQ(8, prog.files_found);
EXPECT_EQ(8, prog.files_scanned);
EXPECT_EQ(2, prog.dirs_found);
@ -928,7 +933,7 @@ class filter_test
debug_filter_output(oss, exclude, pe, mode);
};
progress prog;
writer_progress prog;
thread_pool pool(lgr, *input, "worker", 1);
auto sf = std::make_shared<segmenter_factory>(lgr, prog,
segmenter_factory::config{});

View File

@ -25,8 +25,8 @@
#include <folly/Benchmark.h>
#include <dwarfs/compression_constraints.h>
#include <dwarfs/progress.h>
#include <dwarfs/segmenter.h>
#include <dwarfs/writer_progress.h>
#include <dwarfs/internal/block_data.h>
#include <dwarfs/internal/block_manager.h>
@ -140,7 +140,7 @@ void run_segmenter_test(unsigned iters, unsigned granularity,
for (unsigned i = 0; i < iters; ++i) {
dwarfs::test::test_logger lgr;
dwarfs::progress prog;
dwarfs::writer_progress prog;
auto blkmgr = std::make_shared<dwarfs::internal::block_manager>();
std::vector<std::shared_ptr<dwarfs::internal::block_data>> written;