remove --consider-part-files, related stats code, update docs

we now just treat that one .borg_part file we might have inside
checkpoint archives as a normal file.

people can recognize via the file name it is a partial file.

nobody cares for statistics of checkpoint files and the final
archive now does not contain any partial files any more, thus
no needs to maintain statistics about count and size of part
files.
This commit is contained in:
Thomas Waldmann 2023-01-31 21:05:12 +01:00
parent 0fed44110a
commit b92f4aa487
No known key found for this signature in database
GPG Key ID: 243ACFA951F78E01
19 changed files with 55 additions and 163 deletions

View File

@ -132,10 +132,10 @@ During a backup, a special checkpoint archive named ``<archive-name>.checkpoint`
is saved at every checkpoint interval (the default value for this is 30
minutes) containing all the data backed-up until that point.
This checkpoint archive is a valid archive,
but it is only a partial backup (not all files that you wanted to back up are
contained in it). Having it in the repo until a successful, full backup is
completed is useful because it references all the transmitted chunks up
This checkpoint archive is a valid archive, but it is only a partial backup
(not all files that you wanted to back up are contained in it and the last file
in it might be a partial file). Having it in the repo until a successful, full
backup is completed is useful because it references all the transmitted chunks up
to the checkpoint. This means that in case of an interruption, you only need to
retransfer the data since the last checkpoint.
@ -154,14 +154,12 @@ Once your backup has finished successfully, you can delete all
``<archive-name>.checkpoint`` archives. If you run ``borg prune``, it will
also care for deleting unneeded checkpoints.
Note: the checkpointing mechanism creates hidden, partial files in an archive,
so that checkpoints even work while a big file is being processed.
They are named ``<filename>.borg_part_<N>`` and all operations usually ignore
these files, but you can make them considered by giving the option
``--consider-part-files``. You usually only need that option if you are
really desperate (e.g. if you have no completed backup of that file and you'd
rather get a partial file extracted than nothing). You do **not** want to give
that option under any normal circumstances.
Note: the checkpointing mechanism may create a partial (truncated) last file
in a checkpoint archive named ``<filename>.borg_part``. Such partial files
won't be contained in the final archive.
This is done so that checkpoints work cleanly and promptly while a big
file is being processed.
How can I back up huge file(s) over a unstable connection?
---------------------------------------------------------
@ -171,10 +169,8 @@ Yes. For more details, see :ref:`checkpoints_parts`.
How can I restore huge file(s) over an unstable connection?
-----------------------------------------------------------
If you cannot manage to extract the whole big file in one go, you can extract
all the part files and manually concatenate them together.
For more details, see :ref:`checkpoints_parts`.
Try using ``borg mount`` and ``rsync`` (or a similar tool that supports
resuming a partial file copy from what's already copied).
How can I switch append-only mode on and off?
-----------------------------------------------------------------------------------------------------------------------------------

View File

@ -58,7 +58,6 @@ class Statistics:
self.output_json = output_json
self.iec = iec
self.osize = self.usize = self.nfiles = 0
self.osize_parts = self.usize_parts = self.nfiles_parts = 0
self.last_progress = 0 # timestamp when last progress was shown
self.files_stats = defaultdict(int)
self.chunking_time = 0.0
@ -66,15 +65,10 @@ class Statistics:
self.rx_bytes = 0
self.tx_bytes = 0
def update(self, size, unique, part=False):
if not part:
self.osize += size
if unique:
self.usize += size
else:
self.osize_parts += size
if unique:
self.usize_parts += size
def update(self, size, unique):
self.osize += size
if unique:
self.usize += size
def __add__(self, other):
if not isinstance(other, Statistics):
@ -83,9 +77,6 @@ class Statistics:
stats.osize = self.osize + other.osize
stats.usize = self.usize + other.usize
stats.nfiles = self.nfiles + other.nfiles
stats.osize_parts = self.osize_parts + other.osize_parts
stats.usize_parts = self.usize_parts + other.usize_parts
stats.nfiles_parts = self.nfiles_parts + other.nfiles_parts
stats.chunking_time = self.chunking_time + other.chunking_time
stats.hashing_time = self.hashing_time + other.hashing_time
for key in other.files_stats:
@ -134,20 +125,13 @@ Bytes sent to remote: {stats.tx_bytes}
}
def as_raw_dict(self):
return {
"size": self.osize,
"nfiles": self.nfiles,
"size_parts": self.osize_parts,
"nfiles_parts": self.nfiles_parts,
}
return {"size": self.osize, "nfiles": self.nfiles}
@classmethod
def from_raw_dict(cls, **kw):
self = cls()
self.osize = kw["size"]
self.nfiles = kw["nfiles"]
self.osize_parts = kw["size_parts"]
self.nfiles_parts = kw["nfiles_parts"]
return self
@property
@ -497,7 +481,6 @@ class Archive:
start=None,
start_monotonic=None,
end=None,
consider_part_files=False,
log_json=False,
iec=False,
):
@ -532,7 +515,6 @@ class Archive:
if end is None:
end = archive_ts_now()
self.end = end
self.consider_part_files = consider_part_files
self.pipeline = DownloadPipeline(self.repository, self.repo_objs)
self.create = create
if self.create:
@ -642,9 +624,6 @@ Duration: {0.duration}
return "Archive(%r)" % self.name
def item_filter(self, item, filter=None):
if not self.consider_part_files and "part" in item:
# this is a part(ial) file, we usually don't want to consider it.
return False
return filter(item) if filter else True
def iter_items(self, filter=None, preload=False):
@ -721,14 +700,7 @@ Duration: {0.duration}
# because borg info relies on them. so, either use the given stats (from args)
# or fall back to self.stats if it was not given.
stats = stats or self.stats
metadata.update(
{
"size": stats.osize,
"nfiles": stats.nfiles,
"size_parts": stats.osize_parts,
"nfiles_parts": stats.nfiles_parts,
}
)
metadata.update({"size": stats.osize, "nfiles": stats.nfiles})
metadata.update(additional_metadata or {})
metadata = ArchiveItem(metadata)
data = self.key.pack_and_authenticate_metadata(metadata.as_dict(), context=b"archive")
@ -778,12 +750,9 @@ Duration: {0.duration}
pi.finish()
stats = Statistics(iec=self.iec)
stats.usize = unique_size # the part files use same chunks as the full file
stats.usize = unique_size
stats.nfiles = self.metadata.nfiles
stats.osize = self.metadata.size
if self.consider_part_files:
stats.nfiles += self.metadata.nfiles_parts
stats.osize += self.metadata.size_parts
return stats
@contextmanager
@ -1065,9 +1034,9 @@ Duration: {0.duration}
error = True
return exception_ignored # must not return None here
def chunk_decref(id, stats, part=False):
def chunk_decref(id, stats):
try:
self.cache.chunk_decref(id, stats, wait=False, part=part)
self.cache.chunk_decref(id, stats, wait=False)
except KeyError:
cid = bin_to_hex(id)
raise ChunksIndexError(cid)
@ -1091,9 +1060,8 @@ Duration: {0.duration}
for item in unpacker:
item = Item(internal_dict=item)
if "chunks" in item:
part = not self.consider_part_files and "part" in item
for chunk_id, size in item.chunks:
chunk_decref(chunk_id, stats, part=part)
chunk_decref(chunk_id, stats)
except (TypeError, ValueError):
# if items metadata spans multiple chunks and one chunk got dropped somehow,
# it could be that unpacker yields bad types
@ -1294,7 +1262,6 @@ class ChunksProcessor:
# but we consider only a part of the file here, thus we must recompute the size from the chunks:
item.get_size(memorize=True, from_chunks=True)
item.path += ".borg_part"
item.part = 1 # used to be an increasing number, but now just always 1 IF this is a partial file
self.add_item(item, show_progress=False)
self.write_checkpoint()
@ -1340,8 +1307,6 @@ class ChunksProcessor:
if show_progress:
stats.show_progress(item=item, dt=0.2)
self.maybe_checkpoint(item)
else:
stats.nfiles_parts += 0 # TODO: remove tracking of this
class FilesystemObjectProcessors:

View File

@ -149,7 +149,6 @@ def with_repository(
progress=getattr(args, "progress", False),
lock_wait=self.lock_wait,
cache_mode=getattr(args, "files_cache_mode", FILES_CACHE_MODE_DISABLED),
consider_part_files=getattr(args, "consider_part_files", False),
iec=getattr(args, "iec", False),
) as cache_:
return method(self, args, repository=repository, cache=cache_, **kwargs)
@ -214,7 +213,6 @@ def with_other_repository(manifest=False, cache=False, compatibility=None):
progress=False,
lock_wait=self.lock_wait,
cache_mode=getattr(args, "files_cache_mode", FILES_CACHE_MODE_DISABLED),
consider_part_files=getattr(args, "consider_part_files", False),
iec=getattr(args, "iec", False),
) as cache_:
kwargs["other_cache"] = cache_
@ -240,7 +238,6 @@ def with_archive(method):
noacls=getattr(args, "noacls", False),
noxattrs=getattr(args, "noxattrs", False),
cache=kwargs.get("cache"),
consider_part_files=args.consider_part_files,
log_json=args.log_json,
iec=args.iec,
)
@ -542,12 +539,6 @@ def define_common_options(add_common_option):
type=int,
help="set network upload buffer size in MiB. (default: 0=no buffer)",
)
add_common_option(
"--consider-part-files",
dest="consider_part_files",
action="store_true",
help="treat part files like normal files (e.g. to list/extract them)",
)
add_common_option(
"--debug-profile",
metavar="FILE",

View File

@ -33,7 +33,7 @@ class DebugMixIn:
def do_debug_dump_archive_items(self, args, repository, manifest):
"""dump (decrypted, decompressed) archive items metadata (not: data)"""
repo_objs = manifest.repo_objs
archive = Archive(manifest, args.name, consider_part_files=args.consider_part_files)
archive = Archive(manifest, args.name)
for i, item_id in enumerate(archive.metadata.items):
_, data = repo_objs.parse(item_id, repository.get(item_id))
filename = "%06d_%s.items" % (i, bin_to_hex(item_id))

View File

@ -79,9 +79,7 @@ class DeleteMixIn:
logger_list.info(msg_delete.format(format_archive(archive_info), i, len(archive_names)))
if not dry_run:
archive = Archive(
manifest, archive_name, cache=cache, consider_part_files=args.consider_part_files
)
archive = Archive(manifest, archive_name, cache=cache)
archive.delete(stats, progress=args.progress, forced=args.forced)
checkpointed = self.maybe_checkpoint(
checkpoint_func=checkpoint_func, checkpoint_interval=args.checkpoint_interval

View File

@ -27,7 +27,7 @@ class DiffMixIn:
print_output = print_json_output if args.json_lines else print_text_output
archive1 = archive
archive2 = Archive(manifest, args.other_name, consider_part_files=args.consider_part_files)
archive2 = Archive(manifest, args.other_name)
can_compare_chunk_ids = (
archive1.metadata.get("chunker_params", False) == archive2.metadata.get("chunker_params", True)

View File

@ -24,9 +24,7 @@ class InfoMixIn:
output_data = []
for i, archive_name in enumerate(archive_names, 1):
archive = Archive(
manifest, archive_name, cache=cache, consider_part_files=args.consider_part_files, iec=args.iec
)
archive = Archive(manifest, archive_name, cache=cache, iec=args.iec)
info = archive.info()
if args.json:
output_data.append(info)

View File

@ -27,7 +27,7 @@ class ListMixIn:
format = "{mode} {user:6} {group:6} {size:8} {mtime} {path}{extra}{NL}"
def _list_inner(cache):
archive = Archive(manifest, args.name, cache=cache, consider_part_files=args.consider_part_files)
archive = Archive(manifest, args.name, cache=cache)
formatter = ItemFormatter(archive, format, json_lines=args.json_lines)
for item in archive.iter_items(lambda item: matcher.match(item.path)):

View File

@ -142,7 +142,7 @@ class PruneMixIn:
else:
archives_deleted += 1
log_message = "Pruning archive (%d/%d):" % (archives_deleted, to_delete_len)
archive = Archive(manifest, archive.name, cache, consider_part_files=args.consider_part_files)
archive = Archive(manifest, archive.name, cache)
archive.delete(stats, forced=args.forced)
checkpointed = self.maybe_checkpoint(
checkpoint_func=checkpoint_func, checkpoint_interval=args.checkpoint_interval

View File

@ -404,7 +404,6 @@ class Cache:
lock_wait=None,
permit_adhoc_cache=False,
cache_mode=FILES_CACHE_MODE_DISABLED,
consider_part_files=False,
iec=False,
):
def local():
@ -417,11 +416,10 @@ class Cache:
iec=iec,
lock_wait=lock_wait,
cache_mode=cache_mode,
consider_part_files=consider_part_files,
)
def adhoc():
return AdHocCache(manifest=manifest, lock_wait=lock_wait, iec=iec, consider_part_files=consider_part_files)
return AdHocCache(manifest=manifest, lock_wait=lock_wait, iec=iec)
if not permit_adhoc_cache:
return local()
@ -464,14 +462,11 @@ Total chunks: {0.total_chunks}
# XXX: this should really be moved down to `hashindex.pyx`
total_size, unique_size, total_unique_chunks, total_chunks = self.chunks.summarize()
# the above values have the problem that they do not consider part files,
# thus the total_size might be too high (chunks referenced
# by the part files AND by the complete file).
# since borg 1.2 we have new archive metadata telling the total size per archive,
# so we can just sum up all archives to get the "all archives" stats:
total_size = 0
for archive_name in self.manifest.archives:
archive = Archive(self.manifest, archive_name, consider_part_files=self.consider_part_files)
archive = Archive(self.manifest, archive_name)
stats = archive.calc_stats(self, want_unique=False)
total_size += stats.osize
stats = self.Summary(total_size, unique_size, total_unique_chunks, total_chunks)._asdict()
@ -498,7 +493,6 @@ class LocalCache(CacheStatsMixin):
progress=False,
lock_wait=None,
cache_mode=FILES_CACHE_MODE_DISABLED,
consider_part_files=False,
iec=False,
):
"""
@ -515,7 +509,6 @@ class LocalCache(CacheStatsMixin):
self.repo_objs = manifest.repo_objs
self.progress = progress
self.cache_mode = cache_mode
self.consider_part_files = consider_part_files
self.timestamp = None
self.txn_active = False
@ -971,23 +964,23 @@ class LocalCache(CacheStatsMixin):
)
return refcount
def chunk_incref(self, id, stats, size=None, part=False):
def chunk_incref(self, id, stats, size=None):
if not self.txn_active:
self.begin_txn()
count, _size = self.chunks.incref(id)
stats.update(_size, False, part=part)
stats.update(_size, False)
return ChunkListEntry(id, _size)
def chunk_decref(self, id, stats, wait=True, part=False):
def chunk_decref(self, id, stats, wait=True):
if not self.txn_active:
self.begin_txn()
count, size = self.chunks.decref(id)
if count == 0:
del self.chunks[id]
self.repository.delete(id, wait=wait)
stats.update(-size, True, part=part)
stats.update(-size, True)
else:
stats.update(-size, False, part=part)
stats.update(-size, False)
def file_known_and_unchanged(self, hashed_path, path_hash, st):
"""
@ -1084,14 +1077,13 @@ All archives: unknown unknown unknown
Unique chunks Total chunks
Chunk index: {0.total_unique_chunks:20d} unknown"""
def __init__(self, manifest, warn_if_unencrypted=True, lock_wait=None, consider_part_files=False, iec=False):
def __init__(self, manifest, warn_if_unencrypted=True, lock_wait=None, iec=False):
CacheStatsMixin.__init__(self, iec=iec)
assert isinstance(manifest, Manifest)
self.manifest = manifest
self.repository = manifest.repository
self.key = manifest.key
self.repo_objs = manifest.repo_objs
self.consider_part_files = consider_part_files
self._txn_active = False
self.security_manager = SecurityManager(self.repository)
@ -1145,7 +1137,7 @@ Chunk index: {0.total_unique_chunks:20d} unknown"""
self.chunks[id] = entry._replace(size=size)
return entry.refcount
def chunk_incref(self, id, stats, size=None, part=False):
def chunk_incref(self, id, stats, size=None):
if not self._txn_active:
self.begin_txn()
count, _size = self.chunks.incref(id)
@ -1153,19 +1145,19 @@ Chunk index: {0.total_unique_chunks:20d} unknown"""
# size or add_chunk); we can't add references to those (size=0 is invalid) and generally don't try to.
size = _size or size
assert size
stats.update(size, False, part=part)
stats.update(size, False)
return ChunkListEntry(id, size)
def chunk_decref(self, id, stats, wait=True, part=False):
def chunk_decref(self, id, stats, wait=True):
if not self._txn_active:
self.begin_txn()
count, size = self.chunks.decref(id)
if count == 0:
del self.chunks[id]
self.repository.delete(id, wait=wait)
stats.update(-size, True, part=part)
stats.update(-size, True)
else:
stats.update(-size, False, part=part)
stats.update(-size, False)
def commit(self):
if not self._txn_active:

View File

@ -38,8 +38,6 @@ cache_sync_init(HashIndex *chunks)
unpack_init(&ctx->ctx);
/* needs to be set only once */
ctx->ctx.user.chunks = chunks;
ctx->ctx.user.parts.size = 0;
ctx->ctx.user.parts.num_files = 0;
ctx->ctx.user.totals.size = 0;
ctx->ctx.user.totals.num_files = 0;
ctx->buf = NULL;
@ -71,24 +69,12 @@ cache_sync_num_files_totals(const CacheSyncCtx *ctx)
return ctx->ctx.user.totals.num_files;
}
static uint64_t
cache_sync_num_files_parts(const CacheSyncCtx *ctx)
{
return ctx->ctx.user.parts.num_files;
}
static uint64_t
cache_sync_size_totals(const CacheSyncCtx *ctx)
{
return ctx->ctx.user.totals.size;
}
static uint64_t
cache_sync_size_parts(const CacheSyncCtx *ctx)
{
return ctx->ctx.user.parts.size;
}
/**
* feed data to the cache synchronizer
* 0 = abort, 1 = continue

View File

@ -40,7 +40,7 @@
#endif
typedef struct unpack_user {
/* Item.chunks and Item.part are at the top level; we don't care about anything else,
/* Item.chunks is at the top level; we don't care about anything else,
* only need to track the current level to navigate arbitrary and unknown structure.
* To discern keys from everything else on the top level we use expect_map_item_end.
*/
@ -58,15 +58,12 @@ typedef struct unpack_user {
*/
int inside_chunks;
/* is this item a .part file (created for checkpointing inside files)? */
int part;
/* does this item have a chunks list in it? */
int has_chunks;
enum {
/* the next thing is a map key at the Item root level,
* and it might be the "chunks" or "part" key we're looking for */
* and it might be e.g. the "chunks" key we're looking for */
expect_map_key,
/* blocking state to expect_map_key
@ -114,11 +111,6 @@ typedef struct unpack_user {
uint64_t size, num_files;
} totals;
/* total sizes and files count coming from part files */
struct {
uint64_t size, num_files;
} parts;
} unpack_user;
struct unpack_context;
@ -317,7 +309,6 @@ static inline int unpack_callback_map(unpack_user* u, unsigned int n)
}
/* This begins a new Item */
u->expect = expect_map_key;
u->part = 0;
u->has_chunks = 0;
u->item.size = 0;
}
@ -358,10 +349,6 @@ static inline int unpack_callback_map_end(unpack_user* u)
if(u->level == 0) {
/* This ends processing of an Item */
if(u->has_chunks) {
if(u->part) {
u->parts.num_files += 1;
u->parts.size += u->item.size;
}
u->totals.num_files += 1;
u->totals.size += u->item.size;
}
@ -381,9 +368,6 @@ static inline int unpack_callback_raw(unpack_user* u, const char* b, const char*
u->expect = expect_chunks_begin;
u->inside_chunks = 1;
u->has_chunks = 1;
} else if(length == 4 && !memcmp("part", p, 4)) {
u->expect = expect_map_item_end;
u->part = 1;
} else {
u->expect = expect_map_item_end;
}

View File

@ -18,7 +18,9 @@ ARCHIVE_KEYS = frozenset(['version', 'name', 'hostname', 'username', 'time', 'ti
'command_line', 'recreate_command_line', # v2+ archives
'cmdline', 'recreate_cmdline', # legacy
'recreate_source_id', 'recreate_args', 'recreate_partial_chunks', # used in 1.1.0b1 .. b2
'size', 'nfiles', 'size_parts', 'nfiles_parts'])
'size', 'nfiles',
'size_parts', 'nfiles_parts', # legacy v1 archives
])
# fmt: on
# this is the set of keys that are always present in archives:

View File

@ -147,7 +147,7 @@ class ItemCache:
else:
raise ValueError("Invalid entry type in self.meta")
def iter_archive_items(self, archive_item_ids, filter=None, consider_part_files=False):
def iter_archive_items(self, archive_item_ids, filter=None):
unpacker = msgpack.Unpacker()
# Current offset in the metadata stream, which consists of all metadata chunks glued together
@ -193,7 +193,7 @@ class ItemCache:
break
item = Item(internal_dict=item)
if filter and not filter(item) or not consider_part_files and "part" in item:
if filter and not filter(item):
msgpacked_bytes = b""
continue
@ -330,15 +330,13 @@ class FuseBackend:
"""Build FUSE inode hierarchy from archive metadata"""
self.file_versions = {} # for versions mode: original path -> version
t0 = time.perf_counter()
archive = Archive(self._manifest, archive_name, consider_part_files=self._args.consider_part_files)
archive = Archive(self._manifest, archive_name)
strip_components = self._args.strip_components
matcher = build_matcher(self._args.patterns, self._args.paths)
hlm = HardLinkManager(id_type=bytes, info_type=str) # hlid -> path
filter = build_filter(matcher, strip_components)
for item_inode, item in self.cache.iter_archive_items(
archive.metadata.items, filter=filter, consider_part_files=self._args.consider_part_files
):
for item_inode, item in self.cache.iter_archive_items(archive.metadata.items, filter=filter):
if strip_components:
item.path = os.sep.join(item.path.split(os.sep)[strip_components:])
path = os.fsencode(item.path)

View File

@ -79,11 +79,7 @@ class FuseVersionsIndex(IndexBase):
def __setitem__(self, key: bytes, value: Any) -> None: ...
class CacheSynchronizer:
csize_parts: int
csize_totals: int
num_files_parts: int
num_files_totals: int
size_parts: int
size_totals: int
num_files_totals: int
def __init__(self, chunks_index: Any) -> None: ...
def feed(self, chunk: bytes) -> None: ...

View File

@ -41,9 +41,7 @@ cdef extern from "cache_sync/cache_sync.c":
CacheSyncCtx *cache_sync_init(HashIndex *chunks)
const char *cache_sync_error(const CacheSyncCtx *ctx)
uint64_t cache_sync_num_files_totals(const CacheSyncCtx *ctx)
uint64_t cache_sync_num_files_parts(const CacheSyncCtx *ctx)
uint64_t cache_sync_size_totals(const CacheSyncCtx *ctx)
uint64_t cache_sync_size_parts(const CacheSyncCtx *ctx)
int cache_sync_feed(CacheSyncCtx *ctx, void *data, uint32_t length)
void cache_sync_free(CacheSyncCtx *ctx)
@ -630,14 +628,6 @@ cdef class CacheSynchronizer:
def num_files_totals(self):
return cache_sync_num_files_totals(self.sync)
@property
def num_files_parts(self):
return cache_sync_num_files_parts(self.sync)
@property
def size_totals(self):
return cache_sync_size_totals(self.sync)
@property
def size_parts(self):
return cache_sync_size_parts(self.sync)

View File

@ -91,10 +91,6 @@ class ArchiveItem(PropDict):
@csize.setter
def csize(self, val: int) -> None: ...
@property
def csize_parts(self) -> int: ...
@csize_parts.setter
def csize_parts(self, val: int) -> None: ...
@property
def items(self) -> List: ...
@items.setter
def items(self, val: List) -> None: ...

View File

@ -297,7 +297,7 @@ cdef class Item(PropDict):
deleted = PropDictProperty(bool)
nlink = PropDictProperty(int)
part = PropDictProperty(int)
part = PropDictProperty(int) # legacy only
def get_size(self, *, memorize=False, from_chunks=False, consider_ids=None):
"""
@ -516,8 +516,8 @@ cdef class ArchiveItem(PropDict):
recreate_partial_chunks = PropDictProperty(list) # list of tuples
size = PropDictProperty(int)
nfiles = PropDictProperty(int)
size_parts = PropDictProperty(int)
nfiles_parts = PropDictProperty(int)
size_parts = PropDictProperty(int) # legacy only
nfiles_parts = PropDictProperty(int) # legacy only
def update_internal(self, d):
# legacy support for migration (data from old msgpacks comes in as bytes always, but sometimes we want str)

View File

@ -183,7 +183,7 @@ class ArchiverTestCase(ArchiverTestCaseBase):
# repo looking good overall? checks for rc == 0.
self.cmd(f"--repo={self.repository_location}", "check", "--debug")
# verify that there are no part files in final archive
out = self.cmd(f"--repo={self.repository_location}", "list", "test", "--consider-part-files")
out = self.cmd(f"--repo={self.repository_location}", "list", "test")
assert "stdin.borg_part" not in out
# verify full file
out = self.cmd(f"--repo={self.repository_location}", "extract", "test", "stdin", "--stdout", binary_output=True)