1
0
Fork 0
mirror of https://github.com/borgbackup/borg.git synced 2025-02-24 07:01:59 +00:00

remove LocalCache

Note: this is the default cache implementation in borg 1.x,
it worked well, but there were some issues:

- if the local chunks cache got out of sync with the repository,
  it needed an expensive rebuild from the infos in all archives.
- to optimize that, a local chunks.archive.d cache was used to
  speed that up, but at the price of quite significant space needs.

AdhocCacheWithFiles replaced this with a non-persistent chunks cache,
requesting all chunkids from the repository to initialize a simplified
non-persistent chunks index, that does not do real refcounting and also
initially does not have size information for pre-existing chunks.

We want to move away from precise refcounting, LocalCache needs to die.
This commit is contained in:
Thomas Waldmann 2024-08-13 21:54:35 +02:00
parent 4c052cd65d
commit d6a70f48f2
No known key found for this signature in database
GPG key ID: 243ACFA951F78E01
14 changed files with 36 additions and 745 deletions

View file

@ -3469,7 +3469,7 @@ Other changes:
- archiver tests: add check_cache tool - lints refcounts
- fixed cache sync performance regression from 1.1.0b1 onwards, #1940
- syncing the cache without chunks.archive.d (see :ref:`disable_archive_chunks`)
- syncing the cache without chunks.archive.d
now avoids any merges and is thus faster, #1940
- borg check --verify-data: faster due to linear on-disk-order scan
- borg debug-xxx commands removed, we use "debug xxx" subcommands now, #1627

View file

@ -704,38 +704,6 @@ serialized way in a single script, you need to give them ``--lock-wait N`` (with
being a bit more than the time the server needs to terminate broken down
connections and release the lock).
.. _disable_archive_chunks:
The borg cache eats way too much disk space, what can I do?
-----------------------------------------------------------
This may especially happen if borg needs to rebuild the local "chunks" index -
either because it was removed, or because it was not coherent with the
repository state any more (e.g. because another borg instance changed the
repository).
To optimize this rebuild process, borg caches per-archive information in the
``chunks.archive.d/`` directory. It won't help the first time it happens, but it
will make the subsequent rebuilds faster (because it needs to transfer less data
from the repository). While being faster, the cache needs quite some disk space,
which might be unwanted.
You can disable the cached archive chunk indexes by setting the environment
variable ``BORG_USE_CHUNKS_ARCHIVE`` to ``no``.
This has some pros and cons, though:
- much less disk space needs for ~/.cache/borg.
- chunk cache resyncs will be slower as it will have to transfer chunk usage
metadata for all archives from the repository (which might be slow if your
repo connection is slow) and it will also have to build the hashtables from
that data.
chunk cache resyncs happen e.g. if your repo was written to by another
machine (if you share same backup repo between multiple machines) or if
your local chunks cache was lost somehow.
The long term plan to improve this is called "borgception", see :issue:`474`.
Can I back up my root partition (/) with Borg?
----------------------------------------------

View file

@ -1134,7 +1134,6 @@ The *digests* key contains a mapping of part names to their digests.
Integrity data is generally stored by the upper layers, introduced below. An exception
is the DetachedIntegrityCheckedFile, which automatically writes and reads it from
a ".integrity" file next to the data file.
It is used for archive chunks indexes in chunks.archive.d.
Upper layer
~~~~~~~~~~~
@ -1182,14 +1181,6 @@ easy to tell whether the checksums concern the current state of the cache.
Integrity errors are fatal in these files, terminating the program,
and are not automatically corrected at this time.
.. rubric:: chunks.archive.d
Indices in chunks.archive.d are not transacted and use DetachedIntegrityCheckedFile,
which writes the integrity data to a separate ".integrity" file.
Integrity errors result in deleting the affected index and rebuilding it.
This logs a warning and increases the exit code to WARNING (1).
.. _integrity_repo:
.. rubric:: Repository index and hints

View file

@ -88,9 +88,6 @@ General:
BORG_CACHE_IMPL
Choose the implementation for the clientside cache, choose one of:
- ``local``: uses a persistent chunks cache and keeps it in a perfect state (precise refcounts and
sizes), requiring a potentially resource expensive cache sync in multi-client scenarios.
Also has a persistent files cache.
- ``adhoc``: builds a non-persistent chunks cache by querying the repo. Chunks cache contents
are somewhat sloppy for already existing chunks, concerning their refcount ("infinite") and
size (0). No files cache (slow, will chunk all input files). DEPRECATED.

View file

@ -224,8 +224,6 @@ def create_inner(archive, cache, fso):
manifest,
progress=args.progress,
lock_wait=self.lock_wait,
no_cache_sync_permitted=args.no_cache_sync,
no_cache_sync_forced=args.no_cache_sync_forced,
prefer_adhoc_cache=args.prefer_adhoc_cache,
cache_mode=args.files_cache_mode,
iec=args.iec,
@ -799,18 +797,6 @@ def build_parser_create(self, subparsers, common_parser, mid_common_parser):
help="only display items with the given status characters (see description)",
)
subparser.add_argument("--json", action="store_true", help="output stats as JSON. Implies ``--stats``.")
subparser.add_argument(
"--no-cache-sync",
dest="no_cache_sync",
action="store_true",
help="experimental: do not synchronize the chunks cache.",
)
subparser.add_argument(
"--no-cache-sync-forced",
dest="no_cache_sync_forced",
action="store_true",
help="experimental: do not synchronize the chunks cache (forced).",
)
subparser.add_argument(
"--prefer-adhoc-cache",
dest="prefer_adhoc_cache",

View file

@ -12,26 +12,22 @@
files_cache_logger = create_logger("borg.debug.files_cache")
from .constants import CACHE_README, FILES_CACHE_MODE_DISABLED, ROBJ_FILE_STREAM
from .hashindex import ChunkIndex, ChunkIndexEntry, CacheSynchronizer
from .hashindex import ChunkIndex, ChunkIndexEntry
from .helpers import Error
from .helpers import get_cache_dir, get_security_dir
from .helpers import bin_to_hex, hex_to_bin, parse_stringified_list
from .helpers import hex_to_bin, parse_stringified_list
from .helpers import format_file_size
from .helpers import safe_ns
from .helpers import yes
from .helpers import remove_surrogates
from .helpers import ProgressIndicatorPercent, ProgressIndicatorMessage
from .helpers import set_ec, EXIT_WARNING
from .helpers import safe_unlink
from .helpers import ProgressIndicatorMessage
from .helpers import msgpack
from .helpers.msgpack import int_to_timestamp, timestamp_to_int
from .item import ArchiveItem, ChunkListEntry
from .item import ChunkListEntry
from .crypto.key import PlaintextKey
from .crypto.file_integrity import IntegrityCheckedFile, DetachedIntegrityCheckedFile, FileIntegrityError
from .crypto.file_integrity import IntegrityCheckedFile, FileIntegrityError
from .locking import Lock
from .manifest import Manifest
from .platform import SaveFile
from .remote import cache_if_remote
from .remote3 import RemoteRepository3
from .repository3 import LIST_SCAN_LIMIT, Repository3
@ -355,24 +351,10 @@ def __new__(
warn_if_unencrypted=True,
progress=False,
lock_wait=None,
no_cache_sync_permitted=False,
no_cache_sync_forced=False,
prefer_adhoc_cache=False,
cache_mode=FILES_CACHE_MODE_DISABLED,
iec=False,
):
def local():
return LocalCache(
manifest=manifest,
path=path,
sync=sync,
warn_if_unencrypted=warn_if_unencrypted,
progress=progress,
iec=iec,
lock_wait=lock_wait,
cache_mode=cache_mode,
)
def adhocwithfiles():
return AdHocWithFilesCache(
manifest=manifest,
@ -389,38 +371,14 @@ def adhoc():
impl = get_cache_impl()
if impl != "cli":
methods = dict(local=local, adhocwithfiles=adhocwithfiles, adhoc=adhoc)
methods = dict(adhocwithfiles=adhocwithfiles, adhoc=adhoc)
try:
method = methods[impl]
except KeyError:
raise RuntimeError("Unknown BORG_CACHE_IMPL value: %s" % impl)
return method()
if no_cache_sync_forced:
return adhoc() if prefer_adhoc_cache else adhocwithfiles()
if not no_cache_sync_permitted:
return local()
# no cache sync may be permitted, but if the local cache is in sync it'd be stupid to invalidate
# it by needlessly using the AdHocCache or the AdHocWithFilesCache.
# Check if the local cache exists and is in sync.
cache_config = CacheConfig(repository, path, lock_wait)
if cache_config.exists():
with cache_config:
cache_in_sync = cache_config.manifest_id == manifest.id
# Don't nest cache locks
if cache_in_sync:
# Local cache is in sync, use it
logger.debug("Cache: choosing local cache (in sync)")
return local()
if prefer_adhoc_cache: # adhoc cache, without files cache
logger.debug("Cache: choosing AdHocCache (local cache does not exist or is not in sync)")
return adhoc()
else:
logger.debug("Cache: choosing AdHocWithFilesCache (local cache does not exist or is not in sync)")
return adhocwithfiles()
return adhoc() if prefer_adhoc_cache else adhocwithfiles()
class CacheStatsMixin:
@ -671,15 +629,7 @@ def seen_chunk(self, id, size=None):
entry = self.chunks.get(id, ChunkIndexEntry(0, None))
if entry.refcount and size is not None:
assert isinstance(entry.size, int)
if entry.size:
# LocalCache: has existing size information and uses *size* to make an effort at detecting collisions.
if size != entry.size:
# we already have a chunk with that id, but different size.
# this is either a hash collision (unlikely) or corruption or a bug.
raise Exception(
"chunk has same id [%r], but different size (stored: %d new: %d)!" % (id, entry.size, size)
)
else:
if not entry.size:
# AdHocWithFilesCache / AdHocCache:
# Here *size* is used to update the chunk's size information, which will be zero for existing chunks.
self.chunks[id] = entry._replace(size=size)
@ -737,7 +687,7 @@ def _load_chunks_from_repo(self):
for id_ in result:
num_chunks += 1
chunks[id_] = init_entry
# LocalCache does not contain the manifest, either.
# Cache does not contain the manifest.
if not isinstance(self.repository, (Repository3, RemoteRepository3)):
del chunks[self.manifest.MANIFEST_ID]
duration = perf_counter() - t0 or 0.01
@ -753,413 +703,6 @@ def _load_chunks_from_repo(self):
return chunks
class LocalCache(CacheStatsMixin, FilesCacheMixin, ChunksMixin):
"""
Persistent, local (client-side) cache.
"""
def __init__(
self,
manifest,
path=None,
sync=True,
warn_if_unencrypted=True,
progress=False,
lock_wait=None,
cache_mode=FILES_CACHE_MODE_DISABLED,
iec=False,
):
"""
:param warn_if_unencrypted: print warning if accessing unknown unencrypted repository
:param lock_wait: timeout for lock acquisition (int [s] or None [wait forever])
:param sync: do :meth:`.sync`
:param cache_mode: what shall be compared in the file stat infos vs. cached stat infos comparison
"""
CacheStatsMixin.__init__(self, iec=iec)
FilesCacheMixin.__init__(self, cache_mode)
assert isinstance(manifest, Manifest)
self.manifest = manifest
self.repository = manifest.repository
self.key = manifest.key
self.repo_objs = manifest.repo_objs
self.progress = progress
self._txn_active = False
self.do_cache = os.environ.get("BORG_USE_CHUNKS_ARCHIVE", "yes").lower() in ["yes", "1", "true"]
self.path = cache_dir(self.repository, path)
self.security_manager = SecurityManager(self.repository)
self.cache_config = CacheConfig(self.repository, self.path, lock_wait)
# Warn user before sending data to a never seen before unencrypted repository
if not os.path.exists(self.path):
self.security_manager.assert_access_unknown(warn_if_unencrypted, manifest, self.key)
self.create()
try:
self.open()
except (FileNotFoundError, FileIntegrityError):
self.wipe_cache()
self.open()
try:
self.security_manager.assert_secure(manifest, self.key)
if not self.check_cache_compatibility():
self.wipe_cache()
self.update_compatibility()
if sync and self.manifest.id != self.cache_config.manifest_id:
self.sync()
self.commit()
except: # noqa
self.close()
raise
def __enter__(self):
return self
def __exit__(self, exc_type, exc_val, exc_tb):
self.close()
def create(self):
"""Create a new empty cache at `self.path`"""
os.makedirs(self.path)
with open(os.path.join(self.path, "README"), "w") as fd:
fd.write(CACHE_README)
self.cache_config.create()
ChunkIndex().write(os.path.join(self.path, "chunks"))
os.makedirs(os.path.join(self.path, "chunks.archive.d"))
self._create_empty_files_cache(self.path)
def _do_open(self):
self.cache_config.load()
with IntegrityCheckedFile(
path=os.path.join(self.path, "chunks"),
write=False,
integrity_data=self.cache_config.integrity.get("chunks"),
) as fd:
self.chunks = ChunkIndex.read(fd)
self._read_files_cache()
def open(self):
if not os.path.isdir(self.path):
raise Exception("%s Does not look like a Borg cache" % self.path)
self.cache_config.open()
self.rollback()
def close(self):
if self.cache_config is not None:
self.cache_config.close()
self.cache_config = None
def begin_txn(self):
# Initialize transaction snapshot
pi = ProgressIndicatorMessage(msgid="cache.begin_transaction")
txn_dir = os.path.join(self.path, "txn.tmp")
os.mkdir(txn_dir)
pi.output("Initializing cache transaction: Reading config")
shutil.copy(os.path.join(self.path, "config"), txn_dir)
pi.output("Initializing cache transaction: Reading chunks")
shutil.copy(os.path.join(self.path, "chunks"), txn_dir)
pi.output("Initializing cache transaction: Reading files")
try:
shutil.copy(os.path.join(self.path, self.files_cache_name()), txn_dir)
except FileNotFoundError:
self._create_empty_files_cache(txn_dir)
os.replace(txn_dir, os.path.join(self.path, "txn.active"))
self._txn_active = True
pi.finish()
def commit(self):
"""Commit transaction"""
if not self._txn_active:
return
self.security_manager.save(self.manifest, self.key)
pi = ProgressIndicatorMessage(msgid="cache.commit")
if self.files is not None:
pi.output("Saving files cache")
integrity_data = self._write_files_cache()
self.cache_config.integrity[self.files_cache_name()] = integrity_data
pi.output("Saving chunks cache")
with IntegrityCheckedFile(path=os.path.join(self.path, "chunks"), write=True) as fd:
self.chunks.write(fd)
self.cache_config.integrity["chunks"] = fd.integrity_data
pi.output("Saving cache config")
self.cache_config.save(self.manifest)
os.replace(os.path.join(self.path, "txn.active"), os.path.join(self.path, "txn.tmp"))
shutil.rmtree(os.path.join(self.path, "txn.tmp"))
self._txn_active = False
pi.finish()
def rollback(self):
"""Roll back partial and aborted transactions"""
# Remove partial transaction
if os.path.exists(os.path.join(self.path, "txn.tmp")):
shutil.rmtree(os.path.join(self.path, "txn.tmp"))
# Roll back active transaction
txn_dir = os.path.join(self.path, "txn.active")
if os.path.exists(txn_dir):
shutil.copy(os.path.join(txn_dir, "config"), self.path)
shutil.copy(os.path.join(txn_dir, "chunks"), self.path)
shutil.copy(os.path.join(txn_dir, self.discover_files_cache_name(txn_dir)), self.path)
txn_tmp = os.path.join(self.path, "txn.tmp")
os.replace(txn_dir, txn_tmp)
if os.path.exists(txn_tmp):
shutil.rmtree(txn_tmp)
self._txn_active = False
self._do_open()
def sync(self):
"""Re-synchronize chunks cache with repository.
Maintains a directory with known backup archive indexes, so it only
needs to fetch infos from repo and build a chunk index once per backup
archive.
If out of sync, missing archive indexes get added, outdated indexes
get removed and a new master chunks index is built by merging all
archive indexes.
"""
archive_path = os.path.join(self.path, "chunks.archive.d")
# Instrumentation
processed_item_metadata_bytes = 0
processed_item_metadata_chunks = 0
compact_chunks_archive_saved_space = 0
def mkpath(id, suffix=""):
id_hex = bin_to_hex(id)
path = os.path.join(archive_path, id_hex + suffix)
return path
def cached_archives():
if self.do_cache:
fns = os.listdir(archive_path)
# filenames with 64 hex digits == 256bit,
# or compact indices which are 64 hex digits + ".compact"
return {hex_to_bin(fn) for fn in fns if len(fn) == 64} | {
hex_to_bin(fn[:64]) for fn in fns if len(fn) == 72 and fn.endswith(".compact")
}
else:
return set()
def repo_archives():
return {info.id for info in self.manifest.archives.list()}
def cleanup_outdated(ids):
for id in ids:
cleanup_cached_archive(id)
def cleanup_cached_archive(id, cleanup_compact=True):
try:
os.unlink(mkpath(id))
os.unlink(mkpath(id) + ".integrity")
except FileNotFoundError:
pass
if not cleanup_compact:
return
try:
os.unlink(mkpath(id, suffix=".compact"))
os.unlink(mkpath(id, suffix=".compact") + ".integrity")
except FileNotFoundError:
pass
def fetch_and_build_idx(archive_id, decrypted_repository, chunk_idx):
nonlocal processed_item_metadata_bytes
nonlocal processed_item_metadata_chunks
csize, data = decrypted_repository.get(archive_id)
chunk_idx.add(archive_id, 1, len(data))
archive = self.key.unpack_archive(data)
archive = ArchiveItem(internal_dict=archive)
if archive.version not in (1, 2): # legacy
raise Exception("Unknown archive metadata version")
if archive.version == 1:
items = archive.items
elif archive.version == 2:
items = []
for chunk_id, (csize, data) in zip(archive.item_ptrs, decrypted_repository.get_many(archive.item_ptrs)):
chunk_idx.add(chunk_id, 1, len(data))
ids = msgpack.unpackb(data)
items.extend(ids)
sync = CacheSynchronizer(chunk_idx)
for item_id, (csize, data) in zip(items, decrypted_repository.get_many(items)):
chunk_idx.add(item_id, 1, len(data))
processed_item_metadata_bytes += len(data)
processed_item_metadata_chunks += 1
sync.feed(data)
if self.do_cache:
write_archive_index(archive_id, chunk_idx)
def write_archive_index(archive_id, chunk_idx):
nonlocal compact_chunks_archive_saved_space
compact_chunks_archive_saved_space += chunk_idx.compact()
fn = mkpath(archive_id, suffix=".compact")
fn_tmp = mkpath(archive_id, suffix=".tmp")
try:
with DetachedIntegrityCheckedFile(
path=fn_tmp, write=True, filename=bin_to_hex(archive_id) + ".compact"
) as fd:
chunk_idx.write(fd)
except Exception:
safe_unlink(fn_tmp)
else:
os.replace(fn_tmp, fn)
def read_archive_index(archive_id, archive_name):
archive_chunk_idx_path = mkpath(archive_id)
logger.info("Reading cached archive chunk index for %s", archive_name)
try:
try:
# Attempt to load compact index first
with DetachedIntegrityCheckedFile(path=archive_chunk_idx_path + ".compact", write=False) as fd:
archive_chunk_idx = ChunkIndex.read(fd, permit_compact=True)
# In case a non-compact index exists, delete it.
cleanup_cached_archive(archive_id, cleanup_compact=False)
# Compact index read - return index, no conversion necessary (below).
return archive_chunk_idx
except FileNotFoundError:
# No compact index found, load non-compact index, and convert below.
with DetachedIntegrityCheckedFile(path=archive_chunk_idx_path, write=False) as fd:
archive_chunk_idx = ChunkIndex.read(fd)
except FileIntegrityError as fie:
logger.error("Cached archive chunk index of %s is corrupted: %s", archive_name, fie)
# Delete corrupted index, set warning. A new index must be build.
cleanup_cached_archive(archive_id)
set_ec(EXIT_WARNING)
return None
# Convert to compact index. Delete the existing index first.
logger.debug("Found non-compact index for %s, converting to compact.", archive_name)
cleanup_cached_archive(archive_id)
write_archive_index(archive_id, archive_chunk_idx)
return archive_chunk_idx
def get_archive_ids_to_names(archive_ids):
# Pass once over all archives and build a mapping from ids to names.
# The easier approach, doing a similar loop for each archive, has
# square complexity and does about a dozen million functions calls
# with 1100 archives (which takes 30s CPU seconds _alone_).
archive_names = {}
for info in self.manifest.archives.list():
if info.id in archive_ids:
archive_names[info.id] = info.name
assert len(archive_names) == len(archive_ids)
return archive_names
def create_master_idx(chunk_idx):
logger.debug("Synchronizing chunks index...")
cached_ids = cached_archives()
archive_ids = repo_archives()
logger.info(
"Cached archive chunk indexes: %d fresh, %d stale, %d need fetching.",
len(archive_ids & cached_ids),
len(cached_ids - archive_ids),
len(archive_ids - cached_ids),
)
# deallocates old hashindex, creates empty hashindex:
chunk_idx.clear()
cleanup_outdated(cached_ids - archive_ids)
# Explicitly set the usable initial hash table capacity to avoid performance issues
# due to hash table "resonance".
master_index_capacity = len(self.repository)
if archive_ids:
chunk_idx = None if not self.do_cache else ChunkIndex(usable=master_index_capacity)
pi = ProgressIndicatorPercent(
total=len(archive_ids),
step=0.1,
msg="%3.0f%% Syncing chunks index. Processing archive %s.",
msgid="cache.sync",
)
archive_ids_to_names = get_archive_ids_to_names(archive_ids)
for archive_id, archive_name in archive_ids_to_names.items():
pi.show(info=[remove_surrogates(archive_name)]) # legacy. borg2 always has pure unicode arch names.
if self.do_cache:
if archive_id in cached_ids:
archive_chunk_idx = read_archive_index(archive_id, archive_name)
if archive_chunk_idx is None:
cached_ids.remove(archive_id)
if archive_id not in cached_ids:
# Do not make this an else branch; the FileIntegrityError exception handler
# above can remove *archive_id* from *cached_ids*.
logger.info("Fetching and building archive index for %s.", archive_name)
archive_chunk_idx = ChunkIndex()
fetch_and_build_idx(archive_id, decrypted_repository, archive_chunk_idx)
logger.debug("Merging into master chunks index.")
chunk_idx.merge(archive_chunk_idx)
else:
chunk_idx = chunk_idx or ChunkIndex(usable=master_index_capacity)
logger.info("Fetching archive index for %s.", archive_name)
fetch_and_build_idx(archive_id, decrypted_repository, chunk_idx)
pi.finish()
logger.debug(
"Chunks index sync: processed %s (%d chunks) of metadata.",
format_file_size(processed_item_metadata_bytes),
processed_item_metadata_chunks,
)
logger.debug(
"Chunks index sync: compact chunks.archive.d storage saved %s bytes.",
format_file_size(compact_chunks_archive_saved_space),
)
logger.debug("Chunks index sync done.")
return chunk_idx
# The cache can be used by a command that e.g. only checks against Manifest.Operation.WRITE,
# which does not have to include all flags from Manifest.Operation.READ.
# Since the sync will attempt to read archives, check compatibility with Manifest.Operation.READ.
self.manifest.check_repository_compatibility((Manifest.Operation.READ,))
self.begin_txn()
with cache_if_remote(self.repository, decrypted_cache=self.repo_objs) as decrypted_repository:
self.chunks = create_master_idx(self.chunks)
def check_cache_compatibility(self):
my_features = Manifest.SUPPORTED_REPO_FEATURES
if self.cache_config.ignored_features & my_features:
# The cache might not contain references of chunks that need a feature that is mandatory for some operation
# and which this version supports. To avoid corruption while executing that operation force rebuild.
return False
if not self.cache_config.mandatory_features <= my_features:
# The cache was build with consideration to at least one feature that this version does not understand.
# This client might misinterpret the cache. Thus force a rebuild.
return False
return True
def wipe_cache(self):
logger.warning("Discarding incompatible or corrupted cache and forcing a cache rebuild")
archive_path = os.path.join(self.path, "chunks.archive.d")
if os.path.isdir(archive_path):
shutil.rmtree(os.path.join(self.path, "chunks.archive.d"))
os.makedirs(os.path.join(self.path, "chunks.archive.d"))
self.chunks = ChunkIndex()
with IntegrityCheckedFile(path=os.path.join(self.path, "chunks"), write=True) as fd:
self.chunks.write(fd)
self.cache_config.integrity["chunks"] = fd.integrity_data
integrity_data = self._create_empty_files_cache(self.path)
self.cache_config.integrity[self.files_cache_name()] = integrity_data
self.cache_config.manifest_id = ""
self.cache_config._config.set("cache", "manifest", "")
if not self.cache_config._config.has_section("integrity"):
self.cache_config._config.add_section("integrity")
for file, integrity_data in self.cache_config.integrity.items():
self.cache_config._config.set("integrity", file, integrity_data)
# This is needed to pass the integrity check later on inside CacheConfig.load()
self.cache_config._config.set("integrity", "manifest", "")
self.cache_config.ignored_features = set()
self.cache_config.mandatory_features = set()
with SaveFile(self.cache_config.config_path) as fd:
self.cache_config._config.write(fd)
def update_compatibility(self):
operation_to_features_map = self.manifest.get_all_mandatory_features()
my_features = Manifest.SUPPORTED_REPO_FEATURES
repo_features = set()
for operation, features in operation_to_features_map.items():
repo_features.update(features)
self.cache_config.ignored_features.update(repo_features - my_features)
self.cache_config.mandatory_features.update(repo_features & my_features)
class AdHocWithFilesCache(CacheStatsMixin, FilesCacheMixin, ChunksMixin):
"""
Like AdHocCache, but with a files cache.
@ -1326,10 +869,10 @@ class AdHocCache(CacheStatsMixin, ChunksMixin):
"""
Ad-hoc, non-persistent cache.
Compared to the standard LocalCache the AdHocCache does not maintain accurate reference count,
nor does it provide a files cache (which would require persistence). Chunks that were not added
during the current AdHocCache lifetime won't have correct size set (0 bytes) and will
have an infinite reference count (MAX_VALUE).
The AdHocCache does not maintain accurate reference count, nor does it provide a files cache
(which would require persistence).
Chunks that were not added during the current AdHocCache lifetime won't have correct size set
(0 bytes) and will have an infinite reference count (MAX_VALUE).
"""
str_format = """\

View file

@ -1186,13 +1186,13 @@ def default(self, o):
from ..remote import RemoteRepository
from ..remote3 import RemoteRepository3
from ..archive import Archive
from ..cache import LocalCache, AdHocCache, AdHocWithFilesCache
from ..cache import AdHocCache, AdHocWithFilesCache
if isinstance(o, (Repository, RemoteRepository)) or isinstance(o, (Repository3, RemoteRepository3)):
return {"id": bin_to_hex(o.id), "location": o._location.canonical_path()}
if isinstance(o, Archive):
return o.info()
if isinstance(o, (LocalCache, AdHocWithFilesCache)):
if isinstance(o, (AdHocWithFilesCache, )):
return {"path": o.path, "stats": o.stats()}
if isinstance(o, AdHocCache):
return {"stats": o.stats()}

View file

@ -8,7 +8,6 @@
import sys
import tempfile
import time
from configparser import ConfigParser
from contextlib import contextmanager
from datetime import datetime
from io import BytesIO, StringIO
@ -18,11 +17,9 @@
from ... import xattr, platform
from ...archive import Archive
from ...archiver import Archiver, PURE_PYTHON_MSGPACK_WARNING
from ...cache import Cache, LocalCache
from ...constants import * # NOQA
from ...helpers import Location, umount
from ...helpers import EXIT_SUCCESS
from ...helpers import bin_to_hex
from ...helpers import init_ec_warnings
from ...logger import flush_logging
from ...manifest import Manifest
@ -344,34 +341,6 @@ def _assert_test_keep_tagged(archiver):
assert sorted(os.listdir("output/input/taggedall")), [".NOBACKUP1", ".NOBACKUP2", CACHE_TAG_NAME]
def check_cache(archiver):
# First run a regular borg check
cmd(archiver, "check")
# Then check that the cache on disk matches exactly what's in the repo.
with open_repository(archiver) as repository:
manifest = Manifest.load(repository, Manifest.NO_OPERATION_CHECK)
with Cache(repository, manifest, sync=False) as cache:
original_chunks = cache.chunks
# the LocalCache implementation has an on-disk chunks cache,
# but AdHocWithFilesCache and AdHocCache don't have persistent chunks cache.
persistent = isinstance(cache, LocalCache)
Cache.destroy(repository)
with Cache(repository, manifest) as cache:
correct_chunks = cache.chunks
if not persistent:
# there is no point in doing the checks
return
assert original_chunks is not correct_chunks
seen = set()
for id, (refcount, size) in correct_chunks.iteritems():
o_refcount, o_size = original_chunks[id]
assert refcount == o_refcount
assert size == o_size
seen.add(id)
for id, (refcount, size) in original_chunks.iteritems():
assert id in seen
@contextmanager
def assert_creates_file(path):
assert not os.path.exists(path), f"{path} should not exist"

View file

@ -4,7 +4,7 @@
import pytest
from ...cache import Cache, LocalCache, get_cache_impl
from ...cache import Cache
from ...constants import * # NOQA
from ...helpers import Location, get_security_dir, bin_to_hex
from ...helpers import EXIT_ERROR
@ -13,7 +13,7 @@
from ...repository3 import Repository3
from .. import llfuse
from .. import changedir
from . import cmd, _extract_repository_id, open_repository, check_cache, create_test_files
from . import cmd, _extract_repository_id, create_test_files
from . import _set_repository_id, create_regular_file, assert_creates_file, generate_archiver_tests, RK_ENCRYPTION
pytest_generate_tests = lambda metafunc: generate_archiver_tests(metafunc, kinds="local,remote") # NOQA
@ -204,17 +204,6 @@ def test_unknown_feature_on_create(archivers, request):
cmd_raises_unknown_feature(archiver, ["create", "test", "input"])
@pytest.mark.skipif(get_cache_impl() in ("adhocwithfiles", "adhoc"), reason="only works with LocalCache")
def test_unknown_feature_on_cache_sync(archivers, request):
# LocalCache.sync checks repo compat
archiver = request.getfixturevalue(archivers)
cmd(archiver, "rcreate", RK_ENCRYPTION)
# delete the cache to trigger a cache sync later in borg create
cmd(archiver, "rdelete", "--cache-only")
add_unknown_feature(archiver.repository_path, Manifest.Operation.READ)
cmd_raises_unknown_feature(archiver, ["create", "test", "input"])
def test_unknown_feature_on_change_passphrase(archivers, request):
archiver = request.getfixturevalue(archivers)
print(cmd(archiver, "rcreate", RK_ENCRYPTION))
@ -266,7 +255,6 @@ def test_unknown_feature_on_mount(archivers, request):
cmd_raises_unknown_feature(archiver, ["mount", mountpoint])
@pytest.mark.allow_cache_wipe
def test_unknown_mandatory_feature_in_cache(archivers, request):
archiver = request.getfixturevalue(archivers)
remote_repo = archiver.get_kind() == "remote"
@ -277,27 +265,12 @@ def test_unknown_mandatory_feature_in_cache(archivers, request):
repository._location = Location(archiver.repository_location)
manifest = Manifest.load(repository, Manifest.NO_OPERATION_CHECK)
with Cache(repository, manifest) as cache:
is_localcache = isinstance(cache, LocalCache)
cache.begin_txn()
cache.cache_config.mandatory_features = {"unknown-feature"}
cache.commit()
if archiver.FORK_DEFAULT:
cmd(archiver, "create", "test", "input")
else:
called = False
wipe_cache_safe = LocalCache.wipe_cache
def wipe_wrapper(*args):
nonlocal called
called = True
wipe_cache_safe(*args)
with patch.object(LocalCache, "wipe_cache", wipe_wrapper):
cmd(archiver, "create", "test", "input")
if is_localcache:
assert called
with Repository3(archiver.repository_path, exclusive=True) as repository:
if remote_repo:
@ -307,41 +280,6 @@ def wipe_wrapper(*args):
assert cache.cache_config.mandatory_features == set()
def test_check_cache(archivers, request):
archiver = request.getfixturevalue(archivers)
cmd(archiver, "rcreate", RK_ENCRYPTION)
cmd(archiver, "create", "test", "input")
with open_repository(archiver) as repository:
manifest = Manifest.load(repository, Manifest.NO_OPERATION_CHECK)
with Cache(repository, manifest, sync=False) as cache:
cache.begin_txn()
cache.chunks.incref(list(cache.chunks.iteritems())[0][0])
cache.commit()
persistent = isinstance(cache, LocalCache)
if not persistent:
pytest.skip("check_cache is pointless if we do not have a persistent chunks cache")
with pytest.raises(AssertionError):
check_cache(archiver)
@pytest.mark.skipif(get_cache_impl() in ("adhocwithfiles", "adhoc"), reason="only works with LocalCache")
def test_env_use_chunks_archive(archivers, request, monkeypatch):
archiver = request.getfixturevalue(archivers)
create_test_files(archiver.input_path)
monkeypatch.setenv("BORG_USE_CHUNKS_ARCHIVE", "no")
cmd(archiver, "rcreate", RK_ENCRYPTION)
repository_id = bin_to_hex(_extract_repository_id(archiver.repository_path))
cache_path = os.path.join(archiver.cache_path, repository_id)
cmd(archiver, "create", "test", "input")
assert os.path.exists(cache_path)
assert os.path.exists(os.path.join(cache_path, "chunks.archive.d"))
assert len(os.listdir(os.path.join(cache_path, "chunks.archive.d"))) == 0
cmd(archiver, "rdelete", "--cache-only")
monkeypatch.setenv("BORG_USE_CHUNKS_ARCHIVE", "yes")
cmd(archiver, "create", "test2", "input")
assert len(os.listdir(os.path.join(cache_path, "chunks.archive.d"))) > 0
# Begin Remote Tests
def test_remote_repo_restrict_to_path(remote_archiver):
original_location, repo_path = remote_archiver.repository_location, remote_archiver.repository_path

View file

@ -2,15 +2,12 @@
import json
import os
from configparser import ConfigParser
from unittest.mock import patch
import pytest
from ...constants import * # NOQA
from ...helpers import bin_to_hex, Error
from . import cmd, create_src_archive, create_test_files, RK_ENCRYPTION
from ...hashindex import ChunkIndex
from ...cache import LocalCache
from ...helpers import bin_to_hex
from . import cmd, create_test_files, RK_ENCRYPTION
def corrupt_archiver(archiver):
@ -27,40 +24,6 @@ def corrupt(file, amount=1):
fd.write(corrupted)
@pytest.mark.allow_cache_wipe
def test_cache_chunks(archiver):
corrupt_archiver(archiver)
if archiver.cache_path is None:
pytest.skip("no cache path for this kind of Cache implementation")
create_src_archive(archiver, "test")
chunks_path = os.path.join(archiver.cache_path, "chunks")
if not os.path.exists(chunks_path):
pytest.skip("no persistent chunks index for this kind of Cache implementation")
chunks_before_corruption = set(ChunkIndex(path=chunks_path).iteritems())
corrupt(chunks_path)
assert not archiver.FORK_DEFAULT # test does not support forking
chunks_in_memory = None
sync_chunks = LocalCache.sync
def sync_wrapper(cache):
nonlocal chunks_in_memory
sync_chunks(cache)
chunks_in_memory = set(cache.chunks.iteritems())
with patch.object(LocalCache, "sync", sync_wrapper):
out = cmd(archiver, "rinfo")
assert chunks_in_memory == chunks_before_corruption
assert "forcing a cache rebuild" in out
chunks_after_repair = set(ChunkIndex(path=chunks_path).iteritems())
assert chunks_after_repair == chunks_before_corruption
def test_cache_files(archiver):
corrupt_archiver(archiver)
if archiver.cache_path is None:
@ -73,42 +36,6 @@ def test_cache_files(archiver):
assert "files cache is corrupted" in out
def test_chunks_archive(archiver):
corrupt_archiver(archiver)
if archiver.cache_path is None:
pytest.skip("no cache path for this kind of Cache implementation")
cmd(archiver, "create", "test1", "input")
# Find ID of test1, so we can corrupt it later :)
target_id = cmd(archiver, "rlist", "--format={id}{NL}").strip()
cmd(archiver, "create", "test2", "input")
# Force cache sync, creating archive chunks of test1 and test2 in chunks.archive.d
cmd(archiver, "rdelete", "--cache-only")
cmd(archiver, "rinfo", "--json")
chunks_archive = os.path.join(archiver.cache_path, "chunks.archive.d")
if not os.path.exists(chunks_archive):
pytest.skip("Only LocalCache has a per-archive chunks index cache.")
assert len(os.listdir(chunks_archive)) == 4 # two archives, one chunks cache and one .integrity file each
corrupt(os.path.join(chunks_archive, target_id + ".compact"))
# Trigger cache sync by changing the manifest ID in the cache config
config_path = os.path.join(archiver.cache_path, "config")
config = ConfigParser(interpolation=None)
config.read(config_path)
config.set("cache", "manifest", bin_to_hex(bytes(32)))
with open(config_path, "w") as fd:
config.write(fd)
# Cache sync notices corrupted archive chunks, but automatically recovers.
out = cmd(archiver, "create", "-v", "test3", "input", exit_code=1)
assert "Reading cached archive chunk index for test1" in out
assert "Cached archive chunk index of test1 is corrupted" in out
assert "Fetching and building archive index for test1" in out
def test_old_version_interfered(archiver):
corrupt_archiver(archiver)
if archiver.cache_path is None:

View file

@ -550,7 +550,7 @@ def test_create_pattern_intermediate_folders_first(archivers, request):
assert out_list.index("d x/b") < out_list.index("- x/b/foo_b")
@pytest.mark.skipif(get_cache_impl() in ("adhocwithfiles", "local"), reason="only works with AdHocCache")
@pytest.mark.skipif(get_cache_impl() != "adhoc", reason="only works with AdHocCache")
def test_create_no_cache_sync_adhoc(archivers, request): # TODO: add test for AdHocWithFilesCache
archiver = request.getfixturevalue(archivers)
create_test_files(archiver.input_path)

View file

@ -168,12 +168,8 @@ def test_debug_refcount_obj(archivers, request):
create_json = json.loads(cmd(archiver, "create", "--json", "test", "input"))
archive_id = create_json["archive"]["id"]
output = cmd(archiver, "debug", "refcount-obj", archive_id).strip()
# LocalCache does precise refcounting, so we'll get 1 reference for the archive.
# AdHocCache or AdHocWithFilesCache doesn't, we'll get ChunkIndex.MAX_VALUE as refcount.
assert (
output == f"object {archive_id} has 1 referrers [info from chunks cache]."
or output == f"object {archive_id} has 4294966271 referrers [info from chunks cache]."
)
# AdHocCache or AdHocWithFilesCache don't do precise refcounting, we'll get ChunkIndex.MAX_VALUE as refcount.
assert output == f"object {archive_id} has 4294966271 referrers [info from chunks cache]."
# Invalid IDs do not abort or return an error
output = cmd(archiver, "debug", "refcount-obj", "124", "xyza").strip()

View file

@ -16,7 +16,6 @@
_assert_test_keep_tagged,
_extract_hardlinks_setup,
generate_archiver_tests,
check_cache,
cmd,
create_regular_file,
create_test_files,
@ -96,12 +95,12 @@ def test_recreate_target(archivers, request):
archiver = request.getfixturevalue(archivers)
create_test_files(archiver.input_path)
cmd(archiver, "rcreate", RK_ENCRYPTION)
check_cache(archiver)
cmd(archiver, "check")
cmd(archiver, "create", "test0", "input")
check_cache(archiver)
cmd(archiver, "check")
original_archive = cmd(archiver, "rlist")
cmd(archiver, "recreate", "test0", "input/dir2", "-e", "input/dir2/file3", "--target=new-archive")
check_cache(archiver)
cmd(archiver, "check")
archives = cmd(archiver, "rlist")
assert original_archive in archives
@ -120,7 +119,7 @@ def test_recreate_basic(archivers, request):
cmd(archiver, "rcreate", RK_ENCRYPTION)
cmd(archiver, "create", "test0", "input")
cmd(archiver, "recreate", "test0", "input/dir2", "-e", "input/dir2/file3")
check_cache(archiver)
cmd(archiver, "check")
listing = cmd(archiver, "list", "test0", "--short")
assert "file1" not in listing
assert "dir2/file2" in listing
@ -134,7 +133,7 @@ def test_recreate_subtree_hardlinks(archivers, request):
_extract_hardlinks_setup(archiver)
cmd(archiver, "create", "test2", "input")
cmd(archiver, "recreate", "-a", "test", "input/dir1")
check_cache(archiver)
cmd(archiver, "check")
with changedir("output"):
cmd(archiver, "extract", "test")
assert os.stat("input/dir1/hardlink").st_nlink == 2
@ -159,7 +158,7 @@ def test_recreate_rechunkify(archivers, request):
# right now, the file is chunked differently
assert num_chunks1 != num_chunks2
cmd(archiver, "recreate", "--chunker-params", "default")
check_cache(archiver)
cmd(archiver, "check")
num_chunks1 = int(cmd(archiver, "list", "test1", "input/large_file", "--format", "{num_chunks}"))
num_chunks2 = int(cmd(archiver, "list", "test2", "input/large_file", "--format", "{num_chunks}"))
# now the files are chunked in the same way
@ -220,7 +219,7 @@ def test_recreate_dry_run(archivers, request):
cmd(archiver, "create", "test", "input")
archives_before = cmd(archiver, "list", "test")
cmd(archiver, "recreate", "-n", "-e", "input/compressible")
check_cache(archiver)
cmd(archiver, "check")
archives_after = cmd(archiver, "list", "test")
assert archives_after == archives_before
@ -232,7 +231,7 @@ def test_recreate_skips_nothing_to_do(archivers, request):
cmd(archiver, "create", "test", "input")
info_before = cmd(archiver, "info", "-a", "test")
cmd(archiver, "recreate", "--chunker-params", "default")
check_cache(archiver)
cmd(archiver, "check")
info_after = cmd(archiver, "info", "-a", "test")
assert info_before == info_after # includes archive ID
@ -248,22 +247,22 @@ def test_recreate_list_output(archivers, request):
cmd(archiver, "create", "test", "input")
output = cmd(archiver, "recreate", "-a", "test", "--list", "--info", "-e", "input/file2")
check_cache(archiver)
cmd(archiver, "check")
assert "input/file1" in output
assert "- input/file2" in output
output = cmd(archiver, "recreate", "-a", "test", "--list", "-e", "input/file3")
check_cache(archiver)
cmd(archiver, "check")
assert "input/file1" in output
assert "- input/file3" in output
output = cmd(archiver, "recreate", "-a", "test", "-e", "input/file4")
check_cache(archiver)
cmd(archiver, "check")
assert "input/file1" not in output
assert "- input/file4" not in output
output = cmd(archiver, "recreate", "-a", "test", "--info", "-e", "input/file5")
check_cache(archiver)
cmd(archiver, "check")
assert "input/file1" not in output
assert "- input/file5" not in output

View file

@ -9,8 +9,7 @@
pytest.register_assert_rewrite("borg.testsuite")
import borg.cache # noqa: E402
from borg.archiver import Archiver
from borg.archiver import Archiver # noqa: E402
from borg.logger import setup_logging # noqa: E402
# Ensure that the loggers exist for all tests
@ -56,28 +55,6 @@ def pytest_report_header(config, start_path):
return output
class DefaultPatches:
def __init__(self, request):
self.org_cache_wipe_cache = borg.cache.LocalCache.wipe_cache
def wipe_should_not_be_called(*a, **kw):
raise AssertionError(
"Cache wipe was triggered, if this is part of the test add " "@pytest.mark.allow_cache_wipe"
)
if "allow_cache_wipe" not in request.keywords:
borg.cache.LocalCache.wipe_cache = wipe_should_not_be_called
request.addfinalizer(self.undo)
def undo(self):
borg.cache.LocalCache.wipe_cache = self.org_cache_wipe_cache
@pytest.fixture(autouse=True)
def default_patches(request):
return DefaultPatches(request)
@pytest.fixture()
def set_env_variables():
os.environ["BORG_CHECK_I_KNOW_WHAT_I_AM_DOING"] = "YES"