Переглянути джерело

refactor: rename repository/locking classes/modules

Repository -> LegacyRepository
RemoteRepository -> LegacyRemoteRepository
borg.repository -> borg.legacyrepository
borg.remote -> borg.legacyremote

Repository3 -> Repository
RemoteRepository3 -> RemoteRepository
borg.repository3 -> borg.repository
borg.remote3 -> borg.remote

borg.locking -> borg.fslocking
borg.locking3 -> borg.storelocking
Thomas Waldmann 9 місяців тому
батько
коміт
05739aaa65
36 змінених файлів з 3603 додано та 3082 видалено
  1. 6 7
      src/borg/archive.py
  2. 3 3
      src/borg/archiver/__init__.py
  3. 4 4
      src/borg/archiver/_common.py
  4. 3 3
      src/borg/archiver/compact_cmd.py
  5. 2 3
      src/borg/archiver/debug_cmd.py
  6. 3 3
      src/borg/archiver/rcompress_cmd.py
  7. 1 1
      src/borg/archiver/serve_cmd.py
  8. 2 2
      src/borg/archiver/version_cmd.py
  9. 4 4
      src/borg/cache.py
  10. 0 0
      src/borg/fslocking.py
  11. 2 2
      src/borg/fuse.py
  12. 1 1
      src/borg/helpers/fs.py
  13. 3 3
      src/borg/helpers/parseformat.py
  14. 22 373
      src/borg/legacyremote.py
  15. 1824 0
      src/borg/legacyrepository.py
  16. 6 6
      src/borg/manifest.py
  17. 361 10
      src/borg/remote.py
  18. 188 1072
      src/borg/repository.py
  19. 0 418
      src/borg/repository3.py
  20. 0 0
      src/borg/storelocking.py
  21. 7 7
      src/borg/testsuite/archiver/__init__.py
  22. 8 8
      src/borg/testsuite/archiver/check_cmd.py
  23. 12 12
      src/borg/testsuite/archiver/checks.py
  24. 2 2
      src/borg/testsuite/archiver/create_cmd.py
  25. 7 7
      src/borg/testsuite/archiver/key_cmds.py
  26. 1 1
      src/borg/testsuite/archiver/mount_cmds.py
  27. 2 2
      src/borg/testsuite/archiver/rcompress_cmd.py
  28. 2 2
      src/borg/testsuite/archiver/rename_cmd.py
  29. 3 3
      src/borg/testsuite/cache.py
  30. 1 1
      src/borg/testsuite/fslocking.py
  31. 1115 0
      src/borg/testsuite/legacyrepository.py
  32. 1 1
      src/borg/testsuite/platform.py
  33. 2 2
      src/borg/testsuite/repoobj.py
  34. 4 841
      src/borg/testsuite/repository.py
  35. 0 277
      src/borg/testsuite/repository3.py
  36. 1 1
      src/borg/testsuite/storelocking.py

+ 6 - 7
src/borg/archive.py

@@ -49,9 +49,8 @@ from .manifest import Manifest
 from .patterns import PathPrefixPattern, FnmatchPattern, IECommand
 from .item import Item, ArchiveItem, ItemDiff
 from .platform import acl_get, acl_set, set_flags, get_flags, swidth, hostname
-from .remote import cache_if_remote
-from .remote3 import RemoteRepository3
-from .repository3 import Repository3, LIST_SCAN_LIMIT, NoManifestError
+from .remote import RemoteRepository, cache_if_remote
+from .repository import Repository, LIST_SCAN_LIMIT, NoManifestError
 from .repoobj import RepoObj
 
 has_link = hasattr(os, "link")
@@ -1655,7 +1654,7 @@ class ArchiveChecker:
         self.repair = repair
         self.repository = repository
         self.init_chunks()
-        if not isinstance(repository, (Repository3, RemoteRepository3)) and not self.chunks:
+        if not isinstance(repository, (Repository, RemoteRepository)) and not self.chunks:
             logger.error("Repository contains no apparent data at all, cannot continue check/repair.")
             return False
         self.key = self.make_key(repository)
@@ -1673,7 +1672,7 @@ class ArchiveChecker:
             except IntegrityErrorBase as exc:
                 logger.error("Repository manifest is corrupted: %s", exc)
                 self.error_found = True
-                if not isinstance(repository, (Repository3, RemoteRepository3)):
+                if not isinstance(repository, (Repository, RemoteRepository)):
                     del self.chunks[Manifest.MANIFEST_ID]
                 self.manifest = self.rebuild_manifest()
         self.rebuild_refcounts(
@@ -1758,7 +1757,7 @@ class ArchiveChecker:
                 chunk_id = chunk_ids_revd.pop(-1)  # better efficiency
                 try:
                     encrypted_data = next(chunk_data_iter)
-                except (Repository3.ObjectNotFound, IntegrityErrorBase) as err:
+                except (Repository.ObjectNotFound, IntegrityErrorBase) as err:
                     self.error_found = True
                     errors += 1
                     logger.error("chunk %s: %s", bin_to_hex(chunk_id), err)
@@ -1889,7 +1888,7 @@ class ArchiveChecker:
         Missing and/or incorrect data is repaired when detected
         """
         # Exclude the manifest from chunks (manifest entry might be already deleted from self.chunks)
-        if not isinstance(self.repository, (Repository3, RemoteRepository3)):
+        if not isinstance(self.repository, (Repository, RemoteRepository)):
             self.chunks.pop(Manifest.MANIFEST_ID, None)
 
         def mark_as_possibly_superseded(id_):

+ 3 - 3
src/borg/archiver/__init__.py

@@ -35,7 +35,7 @@ try:
     from ..helpers import ErrorIgnoringTextIOWrapper
     from ..helpers import msgpack
     from ..helpers import sig_int
-    from ..remote3 import RemoteRepository3
+    from ..remote import RemoteRepository
     from ..selftest import selftest
 except BaseException:
     # an unhandled exception in the try-block would cause the borg cli command to exit with rc 1 due to python's
@@ -546,7 +546,7 @@ def sig_trace_handler(sig_no, stack):  # pragma: no cover
 
 def format_tb(exc):
     qualname = type(exc).__qualname__
-    remote = isinstance(exc, RemoteRepository3.RPCError)
+    remote = isinstance(exc, RemoteRepository.RPCError)
     if remote:
         prefix = "Borg server: "
         trace_back = "\n".join(prefix + line for line in exc.exception_full.splitlines())
@@ -624,7 +624,7 @@ def main():  # pragma: no cover
             tb_log_level = logging.ERROR if e.traceback else logging.DEBUG
             tb = format_tb(e)
             exit_code = e.exit_code
-        except RemoteRepository3.RPCError as e:
+        except RemoteRepository.RPCError as e:
             important = e.traceback
             msg = e.exception_full if important else e.get_message()
             msgid = e.exception_class

+ 4 - 4
src/borg/archiver/_common.py

@@ -12,10 +12,10 @@ from ..helpers import Highlander
 from ..helpers.nanorst import rst_to_terminal
 from ..manifest import Manifest, AI_HUMAN_SORT_KEYS
 from ..patterns import PatternMatcher
+from ..legacyremote import LegacyRemoteRepository
 from ..remote import RemoteRepository
-from ..remote3 import RemoteRepository3
+from ..legacyrepository import LegacyRepository
 from ..repository import Repository
-from ..repository3 import Repository3
 from ..repoobj import RepoObj, RepoObj1
 from ..patterns import (
     ArgparsePatternAction,
@@ -34,7 +34,7 @@ def get_repository(
     location, *, create, exclusive, lock_wait, lock, append_only, make_parent_dirs, storage_quota, args, v1_or_v2
 ):
     if location.proto in ("ssh", "socket"):
-        RemoteRepoCls = RemoteRepository if v1_or_v2 else RemoteRepository3
+        RemoteRepoCls = LegacyRemoteRepository if v1_or_v2 else RemoteRepository
         repository = RemoteRepoCls(
             location,
             create=create,
@@ -47,7 +47,7 @@ def get_repository(
         )
 
     else:
-        RepoCls = Repository if v1_or_v2 else Repository3
+        RepoCls = LegacyRepository if v1_or_v2 else Repository
         repository = RepoCls(
             location.path,
             create=create,

+ 3 - 3
src/borg/archiver/compact_cmd.py

@@ -7,8 +7,8 @@ from ..constants import *  # NOQA
 from ..helpers import set_ec, EXIT_WARNING, EXIT_ERROR, format_file_size
 from ..helpers import ProgressIndicatorPercent
 from ..manifest import Manifest
-from ..remote3 import RemoteRepository3
-from ..repository3 import Repository3
+from ..remote import RemoteRepository
+from ..repository import Repository
 
 from ..logger import create_logger
 
@@ -18,7 +18,7 @@ logger = create_logger()
 class ArchiveGarbageCollector:
     def __init__(self, repository, manifest):
         self.repository = repository
-        assert isinstance(repository, (Repository3, RemoteRepository3))
+        assert isinstance(repository, (Repository, RemoteRepository))
         self.manifest = manifest
         self.repository_chunks = None  # what we have in the repository
         self.used_chunks = None  # what archives currently reference

+ 2 - 3
src/borg/archiver/debug_cmd.py

@@ -15,8 +15,7 @@ from ..helpers import archivename_validator
 from ..helpers import CommandError, RTError
 from ..manifest import Manifest
 from ..platform import get_process_id
-from ..repository import Repository
-from ..repository3 import Repository3, LIST_SCAN_LIMIT
+from ..repository import Repository, LIST_SCAN_LIMIT
 from ..repoobj import RepoObj
 
 from ._common import with_repository, Highlander
@@ -306,7 +305,7 @@ class DebugMixIn:
                 try:
                     repository.delete(id)
                     print("object %s deleted." % hex_id)
-                except Repository3.ObjectNotFound:
+                except Repository.ObjectNotFound:
                     print("object %s not found." % hex_id)
         print("Done.")
 

+ 3 - 3
src/borg/archiver/rcompress_cmd.py

@@ -5,8 +5,8 @@ from ._common import with_repository, Highlander
 from ..constants import *  # NOQA
 from ..compress import CompressionSpec, ObfuscateSize, Auto, COMPRESSOR_TABLE
 from ..helpers import sig_int, ProgressIndicatorPercent, Error
-from ..repository3 import Repository3
-from ..remote3 import RemoteRepository3
+from ..repository import Repository
+from ..remote import RemoteRepository
 from ..manifest import Manifest
 
 from ..logger import create_logger
@@ -111,7 +111,7 @@ class RCompressMixIn:
         recompress_candidate_count = len(recompress_ids)
         chunks_limit = min(1000, max(100, recompress_candidate_count // 1000))
 
-        if not isinstance(repository, (Repository3, RemoteRepository3)):
+        if not isinstance(repository, (Repository, RemoteRepository)):
             # start a new transaction
             data = repository.get_manifest()
             repository.put_manifest(data)

+ 1 - 1
src/borg/archiver/serve_cmd.py

@@ -3,7 +3,7 @@ import argparse
 from ._common import Highlander
 from ..constants import *  # NOQA
 from ..helpers import parse_storage_quota
-from ..remote3 import RepositoryServer
+from ..remote import RepositoryServer
 
 from ..logger import create_logger
 

+ 2 - 2
src/borg/archiver/version_cmd.py

@@ -2,7 +2,7 @@ import argparse
 
 from .. import __version__
 from ..constants import *  # NOQA
-from ..remote3 import RemoteRepository3
+from ..remote import RemoteRepository
 
 from ..logger import create_logger
 
@@ -16,7 +16,7 @@ class VersionMixIn:
 
         client_version = parse_version(__version__)
         if args.location.proto in ("ssh", "socket"):
-            with RemoteRepository3(args.location, lock=False, args=args) as repository:
+            with RemoteRepository(args.location, lock=False, args=args) as repository:
                 server_version = repository.server_version
         else:
             server_version = client_version

+ 4 - 4
src/borg/cache.py

@@ -25,11 +25,11 @@ from .helpers.msgpack import int_to_timestamp, timestamp_to_int
 from .item import ChunkListEntry
 from .crypto.key import PlaintextKey
 from .crypto.file_integrity import IntegrityCheckedFile, FileIntegrityError
-from .locking import Lock
+from .fslocking import Lock
 from .manifest import Manifest
 from .platform import SaveFile
-from .remote3 import RemoteRepository3
-from .repository3 import LIST_SCAN_LIMIT, Repository3
+from .remote import RemoteRepository
+from .repository import LIST_SCAN_LIMIT, Repository
 
 # note: cmtime might be either a ctime or a mtime timestamp, chunks is a list of ChunkListEntry
 FileCacheEntry = namedtuple("FileCacheEntry", "age inode size cmtime chunks")
@@ -648,7 +648,7 @@ class ChunksMixin:
                 num_chunks += 1
                 chunks[id_] = init_entry
         # Cache does not contain the manifest.
-        if not isinstance(self.repository, (Repository3, RemoteRepository3)):
+        if not isinstance(self.repository, (Repository, RemoteRepository)):
             del chunks[self.manifest.MANIFEST_ID]
         duration = perf_counter() - t0 or 0.01
         logger.debug(

+ 0 - 0
src/borg/locking.py → src/borg/fslocking.py


+ 2 - 2
src/borg/fuse.py

@@ -46,7 +46,7 @@ from .helpers.lrucache import LRUCache
 from .item import Item
 from .platform import uid2user, gid2group
 from .platformflags import is_darwin
-from .remote3 import RemoteRepository3
+from .remote import RemoteRepository
 
 
 def fuse_main():
@@ -546,7 +546,7 @@ class FuseOperations(llfuse.Operations, FuseBackend):
         self._create_filesystem()
         llfuse.init(self, mountpoint, options)
         if not foreground:
-            if isinstance(self.repository_uncached, RemoteRepository3):
+            if isinstance(self.repository_uncached, RemoteRepository):
                 daemonize()
             else:
                 with daemonizing() as (old_id, new_id):

+ 1 - 1
src/borg/helpers/fs.py

@@ -416,7 +416,7 @@ def safe_unlink(path):
     Use this when deleting potentially large files when recovering
     from a VFS error such as ENOSPC. It can help a full file system
     recover. Refer to the "File system interaction" section
-    in repository.py for further explanations.
+    in legacyrepository.py for further explanations.
     """
     try:
         os.unlink(path)

+ 3 - 3
src/borg/helpers/parseformat.py

@@ -1163,14 +1163,14 @@ def ellipsis_truncate(msg, space):
 
 class BorgJsonEncoder(json.JSONEncoder):
     def default(self, o):
+        from ..legacyrepository import LegacyRepository
         from ..repository import Repository
-        from ..repository3 import Repository3
+        from ..legacyremote import LegacyRemoteRepository
         from ..remote import RemoteRepository
-        from ..remote3 import RemoteRepository3
         from ..archive import Archive
         from ..cache import AdHocCache, AdHocWithFilesCache
 
-        if isinstance(o, (Repository, RemoteRepository)) or isinstance(o, (Repository3, RemoteRepository3)):
+        if isinstance(o, (LegacyRepository, LegacyRemoteRepository)) or isinstance(o, (Repository, RemoteRepository)):
             return {"id": bin_to_hex(o.id), "location": o._location.canonical_path()}
         if isinstance(o, Archive):
             return o.info()

+ 22 - 373
src/borg/remote3.py → src/borg/legacyremote.py

@@ -1,10 +1,8 @@
-import atexit
 import errno
 import functools
 import inspect
 import logging
 import os
-import queue
 import select
 import shlex
 import shutil
@@ -14,10 +12,8 @@ import sys
 import tempfile
 import textwrap
 import time
-import traceback
 from subprocess import Popen, PIPE
 
-import borg.logger
 from . import __version__
 from .compress import Compressor
 from .constants import *  # NOQA
@@ -25,17 +21,14 @@ from .helpers import Error, ErrorWithTraceback, IntegrityError
 from .helpers import bin_to_hex
 from .helpers import get_limited_unpacker
 from .helpers import replace_placeholders
-from .helpers import sysinfo
 from .helpers import format_file_size
 from .helpers import safe_unlink
 from .helpers import prepare_subprocess_env, ignore_sigint
 from .helpers import get_socket_filename
-from .locking import LockTimeout, NotLocked, NotMyLock, LockFailed
-from .logger import create_logger, borg_serve_log_queue
-from .manifest import NoManifestError
+from .fslocking import LockTimeout, NotLocked, NotMyLock, LockFailed
+from .logger import create_logger
 from .helpers import msgpack
-from .repository import Repository
-from .repository3 import Repository3
+from .legacyrepository import LegacyRepository
 from .version import parse_version, format_version
 from .checksums import xxh64
 from .helpers.datastruct import EfficientCollectionQueue
@@ -50,25 +43,6 @@ MAX_INFLIGHT = 100
 RATELIMIT_PERIOD = 0.1
 
 
-def os_write(fd, data):
-    """os.write wrapper so we do not lose data for partial writes."""
-    # TODO: this issue is fixed in cygwin since at least 2.8.0, remove this
-    #       wrapper / workaround when this version is considered ancient.
-    # This is happening frequently on cygwin due to its small pipe buffer size of only 64kiB
-    # and also due to its different blocking pipe behaviour compared to Linux/*BSD.
-    # Neither Linux nor *BSD ever do partial writes on blocking pipes, unless interrupted by a
-    # signal, in which case serve() would terminate.
-    amount = remaining = len(data)
-    while remaining:
-        count = os.write(fd, data)
-        remaining -= count
-        if not remaining:
-            break
-        data = data[count:]
-        time.sleep(count * 1e-09)
-    return amount
-
-
 class ConnectionClosed(Error):
     """Connection closed by remote host"""
 
@@ -127,7 +101,7 @@ class ConnectionBrokenWithHint(Error):
 # For the client the return of the negotiate method is a dict which includes the server version.
 #
 # All method calls on the remote repository object must be allowlisted in RepositoryServer.rpc_methods and have api
-# stubs in RemoteRepository*. The @api decorator on these stubs is used to set server version requirements.
+# stubs in LegacyRemoteRepository. The @api decorator on these stubs is used to set server version requirements.
 #
 # Method parameters are identified only by name and never by position. Unknown parameters are ignored by the server.
 # If a new parameter is important and may not be ignored, on the client a parameter specific version requirement needs
@@ -136,317 +110,6 @@ class ConnectionBrokenWithHint(Error):
 # servers still get compatible input.
 
 
-class RepositoryServer:  # pragma: no cover
-    _rpc_methods = (
-        "__len__",
-        "check",
-        "commit",
-        "delete",
-        "destroy",
-        "get",
-        "list",
-        "negotiate",
-        "open",
-        "close",
-        "info",
-        "put",
-        "rollback",
-        "save_key",
-        "load_key",
-        "break_lock",
-        "inject_exception",
-    )
-
-    _rpc_methods3 = (
-        "__len__",
-        "check",
-        "delete",
-        "destroy",
-        "get",
-        "list",
-        "negotiate",
-        "open",
-        "close",
-        "info",
-        "put",
-        "save_key",
-        "load_key",
-        "break_lock",
-        "inject_exception",
-        "get_manifest",
-        "put_manifest",
-        "store_list",
-        "store_load",
-        "store_store",
-        "store_delete",
-    )
-
-    def __init__(self, restrict_to_paths, restrict_to_repositories, append_only, storage_quota, use_socket):
-        self.repository = None
-        self.RepoCls = None
-        self.rpc_methods = ("open", "close", "negotiate")
-        self.restrict_to_paths = restrict_to_paths
-        self.restrict_to_repositories = restrict_to_repositories
-        # This flag is parsed from the serve command line via Archiver.do_serve,
-        # i.e. it reflects local system policy and generally ranks higher than
-        # whatever the client wants, except when initializing a new repository
-        # (see RepositoryServer.open below).
-        self.append_only = append_only
-        self.storage_quota = storage_quota
-        self.client_version = None  # we update this after client sends version information
-        if use_socket is False:
-            self.socket_path = None
-        elif use_socket is True:  # --socket
-            self.socket_path = get_socket_filename()
-        else:  # --socket=/some/path
-            self.socket_path = use_socket
-
-    def filter_args(self, f, kwargs):
-        """Remove unknown named parameters from call, because client did (implicitly) say it's ok."""
-        known = set(inspect.signature(f).parameters)
-        return {name: kwargs[name] for name in kwargs if name in known}
-
-    def send_queued_log(self):
-        while True:
-            try:
-                # lr_dict contents see BorgQueueHandler
-                lr_dict = borg_serve_log_queue.get_nowait()
-            except queue.Empty:
-                break
-            else:
-                msg = msgpack.packb({LOG: lr_dict})
-                os_write(self.stdout_fd, msg)
-
-    def serve(self):
-        def inner_serve():
-            os.set_blocking(self.stdin_fd, False)
-            assert not os.get_blocking(self.stdin_fd)
-            os.set_blocking(self.stdout_fd, True)
-            assert os.get_blocking(self.stdout_fd)
-
-            unpacker = get_limited_unpacker("server")
-            shutdown_serve = False
-            while True:
-                # before processing any new RPCs, send out all pending log output
-                self.send_queued_log()
-
-                if shutdown_serve:
-                    # shutdown wanted! get out of here after sending all log output.
-                    assert self.repository is None
-                    return
-
-                # process new RPCs
-                r, w, es = select.select([self.stdin_fd], [], [], 10)
-                if r:
-                    data = os.read(self.stdin_fd, BUFSIZE)
-                    if not data:
-                        shutdown_serve = True
-                        continue
-                    unpacker.feed(data)
-                    for unpacked in unpacker:
-                        if isinstance(unpacked, dict):
-                            msgid = unpacked[MSGID]
-                            method = unpacked[MSG]
-                            args = unpacked[ARGS]
-                        else:
-                            if self.repository is not None:
-                                self.repository.close()
-                            raise UnexpectedRPCDataFormatFromClient(__version__)
-                        try:
-                            # logger.debug(f"{type(self)} method: {type(self.repository)}.{method}")
-                            if method not in self.rpc_methods:
-                                raise InvalidRPCMethod(method)
-                            try:
-                                f = getattr(self, method)
-                            except AttributeError:
-                                f = getattr(self.repository, method)
-                            args = self.filter_args(f, args)
-                            res = f(**args)
-                        except BaseException as e:
-                            # logger.exception(e)
-                            ex_short = traceback.format_exception_only(e.__class__, e)
-                            ex_full = traceback.format_exception(*sys.exc_info())
-                            ex_trace = True
-                            if isinstance(e, Error):
-                                ex_short = [e.get_message()]
-                                ex_trace = e.traceback
-                            if isinstance(e, (self.RepoCls.DoesNotExist, self.RepoCls.AlreadyExists, PathNotAllowed)):
-                                # These exceptions are reconstructed on the client end in RemoteRepository*.call_many(),
-                                # and will be handled just like locally raised exceptions. Suppress the remote traceback
-                                # for these, except ErrorWithTraceback, which should always display a traceback.
-                                pass
-                            else:
-                                logging.debug("\n".join(ex_full))
-
-                            sys_info = sysinfo()
-                            try:
-                                msg = msgpack.packb(
-                                    {
-                                        MSGID: msgid,
-                                        "exception_class": e.__class__.__name__,
-                                        "exception_args": e.args,
-                                        "exception_full": ex_full,
-                                        "exception_short": ex_short,
-                                        "exception_trace": ex_trace,
-                                        "sysinfo": sys_info,
-                                    }
-                                )
-                            except TypeError:
-                                msg = msgpack.packb(
-                                    {
-                                        MSGID: msgid,
-                                        "exception_class": e.__class__.__name__,
-                                        "exception_args": [
-                                            x if isinstance(x, (str, bytes, int)) else None for x in e.args
-                                        ],
-                                        "exception_full": ex_full,
-                                        "exception_short": ex_short,
-                                        "exception_trace": ex_trace,
-                                        "sysinfo": sys_info,
-                                    }
-                                )
-                            os_write(self.stdout_fd, msg)
-                        else:
-                            os_write(self.stdout_fd, msgpack.packb({MSGID: msgid, RESULT: res}))
-                if es:
-                    shutdown_serve = True
-                    continue
-
-        if self.socket_path:  # server for socket:// connections
-            try:
-                # remove any left-over socket file
-                os.unlink(self.socket_path)
-            except OSError:
-                if os.path.exists(self.socket_path):
-                    raise
-            sock_dir = os.path.dirname(self.socket_path)
-            os.makedirs(sock_dir, exist_ok=True)
-            if self.socket_path.endswith(".sock"):
-                pid_file = self.socket_path.replace(".sock", ".pid")
-            else:
-                pid_file = self.socket_path + ".pid"
-            pid = os.getpid()
-            with open(pid_file, "w") as f:
-                f.write(str(pid))
-            atexit.register(functools.partial(os.remove, pid_file))
-            atexit.register(functools.partial(os.remove, self.socket_path))
-            sock = socket.socket(family=socket.AF_UNIX, type=socket.SOCK_STREAM)
-            sock.bind(self.socket_path)  # this creates the socket file in the fs
-            sock.listen(0)  # no backlog
-            os.chmod(self.socket_path, mode=0o0770)  # group members may use the socket, too.
-            print(f"borg serve: PID {pid}, listening on socket {self.socket_path} ...", file=sys.stderr)
-
-            while True:
-                connection, client_address = sock.accept()
-                print(f"Accepted a connection on socket {self.socket_path} ...", file=sys.stderr)
-                self.stdin_fd = connection.makefile("rb").fileno()
-                self.stdout_fd = connection.makefile("wb").fileno()
-                inner_serve()
-                print(f"Finished with connection on socket {self.socket_path} .", file=sys.stderr)
-        else:  # server for one ssh:// connection
-            self.stdin_fd = sys.stdin.fileno()
-            self.stdout_fd = sys.stdout.fileno()
-            inner_serve()
-
-    def negotiate(self, client_data):
-        if isinstance(client_data, dict):
-            self.client_version = client_data["client_version"]
-        else:
-            self.client_version = BORG_VERSION  # seems to be newer than current version (no known old format)
-
-        # not a known old format, send newest negotiate this version knows
-        return {"server_version": BORG_VERSION}
-
-    def _resolve_path(self, path):
-        if isinstance(path, bytes):
-            path = os.fsdecode(path)
-        if path.startswith("/~/"):  # /~/x = path x relative to own home dir
-            home_dir = os.environ.get("HOME") or os.path.expanduser("~%s" % os.environ.get("USER", ""))
-            path = os.path.join(home_dir, path[3:])
-        elif path.startswith("/./"):  # /./x = path x relative to cwd
-            path = path[3:]
-        return os.path.realpath(path)
-
-    def open(
-        self,
-        path,
-        create=False,
-        lock_wait=None,
-        lock=True,
-        exclusive=None,
-        append_only=False,
-        make_parent_dirs=False,
-        v1_or_v2=False,
-    ):
-        self.RepoCls = Repository if v1_or_v2 else Repository3
-        self.rpc_methods = self._rpc_methods if v1_or_v2 else self._rpc_methods3
-        logging.debug("Resolving repository path %r", path)
-        path = self._resolve_path(path)
-        logging.debug("Resolved repository path to %r", path)
-        path_with_sep = os.path.join(path, "")  # make sure there is a trailing slash (os.sep)
-        if self.restrict_to_paths:
-            # if --restrict-to-path P is given, we make sure that we only operate in/below path P.
-            # for the prefix check, it is important that the compared paths both have trailing slashes,
-            # so that a path /foobar will NOT be accepted with --restrict-to-path /foo option.
-            for restrict_to_path in self.restrict_to_paths:
-                restrict_to_path_with_sep = os.path.join(os.path.realpath(restrict_to_path), "")  # trailing slash
-                if path_with_sep.startswith(restrict_to_path_with_sep):
-                    break
-            else:
-                raise PathNotAllowed(path)
-        if self.restrict_to_repositories:
-            for restrict_to_repository in self.restrict_to_repositories:
-                restrict_to_repository_with_sep = os.path.join(os.path.realpath(restrict_to_repository), "")
-                if restrict_to_repository_with_sep == path_with_sep:
-                    break
-            else:
-                raise PathNotAllowed(path)
-        # "borg init" on "borg serve --append-only" (=self.append_only) does not create an append only repo,
-        # while "borg init --append-only" (=append_only) does, regardless of the --append-only (self.append_only)
-        # flag for serve.
-        append_only = (not create and self.append_only) or append_only
-        self.repository = self.RepoCls(
-            path,
-            create,
-            lock_wait=lock_wait,
-            lock=lock,
-            append_only=append_only,
-            storage_quota=self.storage_quota,
-            exclusive=exclusive,
-            make_parent_dirs=make_parent_dirs,
-            send_log_cb=self.send_queued_log,
-        )
-        self.repository.__enter__()  # clean exit handled by serve() method
-        return self.repository.id
-
-    def close(self):
-        if self.repository is not None:
-            self.repository.__exit__(None, None, None)
-            self.repository = None
-        borg.logger.flush_logging()
-        self.send_queued_log()
-
-    def inject_exception(self, kind):
-        s1 = "test string"
-        s2 = "test string2"
-        if kind == "DoesNotExist":
-            raise self.RepoCls.DoesNotExist(s1)
-        elif kind == "AlreadyExists":
-            raise self.RepoCls.AlreadyExists(s1)
-        elif kind == "CheckNeeded":
-            raise self.RepoCls.CheckNeeded(s1)
-        elif kind == "IntegrityError":
-            raise IntegrityError(s1)
-        elif kind == "PathNotAllowed":
-            raise PathNotAllowed("foo")
-        elif kind == "ObjectNotFound":
-            raise self.RepoCls.ObjectNotFound(s1, s2)
-        elif kind == "InvalidRPCMethod":
-            raise InvalidRPCMethod(s1)
-        elif kind == "divide":
-            0 // 0
-
-
 class SleepingBandwidthLimiter:
     def __init__(self, limit):
         if limit:
@@ -542,7 +205,7 @@ def api(*, since, **kwargs_decorator):
     return decorator
 
 
-class RemoteRepository3:
+class LegacyRemoteRepository:
     extra_test_args = []  # type: ignore
 
     class RPCError(Exception):
@@ -587,7 +250,7 @@ class RemoteRepository3:
         location,
         create=False,
         exclusive=False,
-        lock_wait=1.0,
+        lock_wait=None,
         lock=True,
         append_only=False,
         make_parent_dirs=False,
@@ -676,6 +339,7 @@ class RemoteRepository3:
                 exclusive=exclusive,
                 append_only=append_only,
                 make_parent_dirs=make_parent_dirs,
+                v1_or_v2=True,  # make remote use LegacyRepository
             )
             info = self.info()
             self.version = info["version"]
@@ -687,10 +351,10 @@ class RemoteRepository3:
 
     def __del__(self):
         if len(self.responses):
-            logging.debug("still %d cached responses left in RemoteRepository3" % (len(self.responses),))
+            logging.debug("still %d cached responses left in LegacyRemoteRepository" % (len(self.responses),))
         if self.p or self.sock:
             self.close()
-            assert False, "cleanup happened in Repository3.__del__"
+            assert False, "cleanup happened in LegacyRemoteRepository.__del__"
 
     def __repr__(self):
         return f"<{self.__class__.__name__} {self.location.canonical_path()}>"
@@ -702,10 +366,13 @@ class RemoteRepository3:
         try:
             if exc_type is not None:
                 self.shutdown_time = time.monotonic() + 30
+                self.rollback()
         finally:
-            # in any case, we want to close the repo cleanly.
+            # in any case, we want to close the repo cleanly, even if the
+            # rollback can not succeed (e.g. because the connection was
+            # already closed) and raised another exception:
             logger.debug(
-                "RemoteRepository3: %s bytes sent, %s bytes received, %d messages sent",
+                "LegacyRemoteRepository: %s bytes sent, %s bytes received, %d messages sent",
                 format_file_size(self.tx_bytes),
                 format_file_size(self.rx_bytes),
                 self.msgid,
@@ -813,21 +480,21 @@ class RemoteRepository3:
             elif error == "ErrorWithTraceback":
                 raise ErrorWithTraceback(args[0])
             elif error == "DoesNotExist":
-                raise Repository3.DoesNotExist(self.location.processed)
+                raise LegacyRepository.DoesNotExist(self.location.processed)
             elif error == "AlreadyExists":
-                raise Repository3.AlreadyExists(self.location.processed)
+                raise LegacyRepository.AlreadyExists(self.location.processed)
             elif error == "CheckNeeded":
-                raise Repository3.CheckNeeded(self.location.processed)
+                raise LegacyRepository.CheckNeeded(self.location.processed)
             elif error == "IntegrityError":
                 raise IntegrityError(args[0])
             elif error == "PathNotAllowed":
                 raise PathNotAllowed(args[0])
             elif error == "PathPermissionDenied":
-                raise Repository3.PathPermissionDenied(args[0])
+                raise LegacyRepository.PathPermissionDenied(args[0])
             elif error == "ParentPathDoesNotExist":
-                raise Repository3.ParentPathDoesNotExist(args[0])
+                raise LegacyRepository.ParentPathDoesNotExist(args[0])
             elif error == "ObjectNotFound":
-                raise Repository3.ObjectNotFound(args[0], self.location.processed)
+                raise LegacyRepository.ObjectNotFound(args[0], self.location.processed)
             elif error == "InvalidRPCMethod":
                 raise InvalidRPCMethod(args[0])
             elif error == "LockTimeout":
@@ -838,8 +505,6 @@ class RemoteRepository3:
                 raise NotLocked(args[0])
             elif error == "NotMyLock":
                 raise NotMyLock(args[0])
-            elif error == "NoManifestError":
-                raise NoManifestError
             else:
                 raise self.RPCError(unpacked)
 
@@ -849,7 +514,7 @@ class RemoteRepository3:
         send_buffer()  # Try to send data, as some cases (async_response) will never try to send data otherwise.
         while wait or calls:
             if self.shutdown_time and time.monotonic() > self.shutdown_time:
-                # we are shutting this RemoteRepository3 down already, make sure we do not waste
+                # we are shutting this LegacyRemoteRepository down already, make sure we do not waste
                 # a lot of time in case a lot of async stuff is coming in or remote is gone or slow.
                 logger.debug(
                     "shutdown_time reached, shutting down with %d waiting_for and %d async_responses.",
@@ -1080,22 +745,6 @@ class RemoteRepository3:
     def put_manifest(self, data):
         """actual remoting is done via self.call in the @api decorator"""
 
-    @api(since=parse_version("2.0.0b8"))
-    def store_list(self, name):
-        """actual remoting is done via self.call in the @api decorator"""
-
-    @api(since=parse_version("2.0.0b8"))
-    def store_load(self, name):
-        """actual remoting is done via self.call in the @api decorator"""
-
-    @api(since=parse_version("2.0.0b8"))
-    def store_store(self, name, value):
-        """actual remoting is done via self.call in the @api decorator"""
-
-    @api(since=parse_version("2.0.0b8"))
-    def store_delete(self, name):
-        """actual remoting is done via self.call in the @api decorator"""
-
 
 class RepositoryNoCache:
     """A not caching Repository wrapper, passes through to repository.
@@ -1298,7 +947,7 @@ def cache_if_remote(repository, *, decrypted_cache=False, pack=None, unpack=None
             csize = meta.get("csize", len(data))
             return csize, decrypted
 
-    if isinstance(repository, RemoteRepository3) or force_cache:
+    if isinstance(repository, LegacyRemoteRepository) or force_cache:
         return RepositoryCache(repository, pack, unpack, transform)
     else:
         return RepositoryNoCache(repository, transform)

+ 1824 - 0
src/borg/legacyrepository.py

@@ -0,0 +1,1824 @@
+import errno
+import mmap
+import os
+import shutil
+import stat
+import struct
+import time
+from collections import defaultdict
+from configparser import ConfigParser
+from datetime import datetime, timezone
+from functools import partial
+from itertools import islice
+from typing import Callable, DefaultDict
+
+from .constants import *  # NOQA
+from .hashindex import NSIndexEntry, NSIndex, NSIndex1, hashindex_variant
+from .helpers import Error, ErrorWithTraceback, IntegrityError, format_file_size, parse_file_size
+from .helpers import Location
+from .helpers import ProgressIndicatorPercent
+from .helpers import bin_to_hex, hex_to_bin
+from .helpers import secure_erase, safe_unlink
+from .helpers import msgpack
+from .helpers.lrucache import LRUCache
+from .fslocking import Lock, LockError, LockErrorT
+from .logger import create_logger
+from .manifest import Manifest, NoManifestError
+from .platform import SaveFile, SyncFile, sync_dir, safe_fadvise
+from .repoobj import RepoObj
+from .checksums import crc32, StreamingXXH64
+from .crypto.file_integrity import IntegrityCheckedFile, FileIntegrityError
+
+logger = create_logger(__name__)
+
+MAGIC = b"BORG_SEG"
+MAGIC_LEN = len(MAGIC)
+
+TAG_PUT = 0
+TAG_DELETE = 1
+TAG_COMMIT = 2
+TAG_PUT2 = 3
+
+# Highest ID usable as TAG_* value
+#
+# Code may expect not to find any tags exceeding this value. In particular,
+# in order to speed up `borg check --repair`, any tag greater than MAX_TAG_ID
+# is assumed to be corrupted. When increasing this value, in order to add more
+# tags, keep in mind that old versions of Borg accessing a new repository
+# may not be able to handle the new tags.
+MAX_TAG_ID = 15
+
+FreeSpace: Callable[[], DefaultDict] = partial(defaultdict, int)
+
+
+def header_size(tag):
+    if tag == TAG_PUT2:
+        size = LoggedIO.HEADER_ID_SIZE + LoggedIO.ENTRY_HASH_SIZE
+    elif tag == TAG_PUT or tag == TAG_DELETE:
+        size = LoggedIO.HEADER_ID_SIZE
+    elif tag == TAG_COMMIT:
+        size = LoggedIO.header_fmt.size
+    else:
+        raise ValueError(f"unsupported tag: {tag!r}")
+    return size
+
+
+class LegacyRepository:
+    """
+    Filesystem based transactional key value store
+
+    Transactionality is achieved by using a log (aka journal) to record changes. The log is a series of numbered files
+    called segments. Each segment is a series of log entries. The segment number together with the offset of each
+    entry relative to its segment start establishes an ordering of the log entries. This is the "definition" of
+    time for the purposes of the log.
+
+    Log entries are either PUT, DELETE or COMMIT.
+
+    A COMMIT is always the final log entry in a segment and marks all data from the beginning of the log until the
+    segment ending with the COMMIT as committed and consistent. The segment number of a segment ending with a COMMIT
+    is called the transaction ID of that commit, and a segment ending with a COMMIT is called committed.
+
+    When reading from a repository it is first checked whether the last segment is committed. If it is not, then
+    all segments after the last committed segment are deleted; they contain log entries whose consistency is not
+    established by a COMMIT.
+
+    Note that the COMMIT can't establish consistency by itself, but only manages to do so with proper support from
+    the platform (including the hardware). See platform.base.SyncFile for details.
+
+    A PUT inserts a key-value pair. The value is stored in the log entry, hence the repository implements
+    full data logging, meaning that all data is consistent, not just metadata (which is common in file systems).
+
+    A DELETE marks a key as deleted.
+
+    For a given key only the last entry regarding the key, which is called current (all other entries are called
+    superseded), is relevant: If there is no entry or the last entry is a DELETE then the key does not exist.
+    Otherwise the last PUT defines the value of the key.
+
+    By superseding a PUT (with either another PUT or a DELETE) the log entry becomes obsolete. A segment containing
+    such obsolete entries is called sparse, while a segment containing no such entries is called compact.
+
+    Sparse segments can be compacted and thereby disk space freed. This destroys the transaction for which the
+    superseded entries where current.
+
+    On disk layout:
+
+    dir/README
+    dir/config
+    dir/data/<X // SEGMENTS_PER_DIR>/<X>
+    dir/index.X
+    dir/hints.X
+
+    File system interaction
+    -----------------------
+
+    LoggedIO generally tries to rely on common behaviours across transactional file systems.
+
+    Segments that are deleted are truncated first, which avoids problems if the FS needs to
+    allocate space to delete the dirent of the segment. This mostly affects CoW file systems,
+    traditional journaling file systems have a fairly good grip on this problem.
+
+    Note that deletion, i.e. unlink(2), is atomic on every file system that uses inode reference
+    counts, which includes pretty much all of them. To remove a dirent the inodes refcount has
+    to be decreased, but you can't decrease the refcount before removing the dirent nor can you
+    decrease the refcount after removing the dirent. File systems solve this with a lock,
+    and by ensuring it all stays within the same FS transaction.
+
+    Truncation is generally not atomic in itself, and combining truncate(2) and unlink(2) is of
+    course never guaranteed to be atomic. Truncation in a classic extent-based FS is done in
+    roughly two phases, first the extents are removed then the inode is updated. (In practice
+    this is of course way more complex).
+
+    LoggedIO gracefully handles truncate/unlink splits as long as the truncate resulted in
+    a zero length file. Zero length segments are considered not to exist, while LoggedIO.cleanup()
+    will still get rid of them.
+    """
+
+    class AlreadyExists(Error):
+        """A repository already exists at {}."""
+
+        exit_mcode = 10
+
+    class CheckNeeded(ErrorWithTraceback):
+        """Inconsistency detected. Please run "borg check {}"."""
+
+        exit_mcode = 12
+
+    class DoesNotExist(Error):
+        """Repository {} does not exist."""
+
+        exit_mcode = 13
+
+    class InsufficientFreeSpaceError(Error):
+        """Insufficient free space to complete transaction (required: {}, available: {})."""
+
+        exit_mcode = 14
+
+    class InvalidRepository(Error):
+        """{} is not a valid repository. Check repo config."""
+
+        exit_mcode = 15
+
+    class InvalidRepositoryConfig(Error):
+        """{} does not have a valid configuration. Check repo config [{}]."""
+
+        exit_mcode = 16
+
+    class ObjectNotFound(ErrorWithTraceback):
+        """Object with key {} not found in repository {}."""
+
+        exit_mcode = 17
+
+        def __init__(self, id, repo):
+            if isinstance(id, bytes):
+                id = bin_to_hex(id)
+            super().__init__(id, repo)
+
+    class ParentPathDoesNotExist(Error):
+        """The parent path of the repo directory [{}] does not exist."""
+
+        exit_mcode = 18
+
+    class PathAlreadyExists(Error):
+        """There is already something at {}."""
+
+        exit_mcode = 19
+
+    class StorageQuotaExceeded(Error):
+        """The storage quota ({}) has been exceeded ({}). Try deleting some archives."""
+
+        exit_mcode = 20
+
+    class PathPermissionDenied(Error):
+        """Permission denied to {}."""
+
+        exit_mcode = 21
+
+    def __init__(
+        self,
+        path,
+        create=False,
+        exclusive=False,
+        lock_wait=None,
+        lock=True,
+        append_only=False,
+        storage_quota=None,
+        make_parent_dirs=False,
+        send_log_cb=None,
+    ):
+        self.path = os.path.abspath(path)
+        self._location = Location("file://%s" % self.path)
+        self.version = None
+        # long-running repository methods which emit log or progress output are responsible for calling
+        # the ._send_log method periodically to get log and progress output transferred to the borg client
+        # in a timely manner, in case we have a LegacyRemoteRepository.
+        # for local repositories ._send_log can be called also (it will just do nothing in that case).
+        self._send_log = send_log_cb or (lambda: None)
+        self.io = None  # type: LoggedIO
+        self.lock = None
+        self.index = None
+        # This is an index of shadowed log entries during this transaction. Consider the following sequence:
+        # segment_n PUT A, segment_x DELETE A
+        # After the "DELETE A" in segment_x the shadow index will contain "A -> [n]".
+        # .delete() is updating this index, it is persisted into "hints" file and is later used by .compact_segments().
+        # We need the entries in the shadow_index to not accidentally drop the "DELETE A" when we compact segment_x
+        # only (and we do not compact segment_n), because DELETE A is still needed then because PUT A will be still
+        # there. Otherwise chunk A would reappear although it was previously deleted.
+        self.shadow_index = {}
+        self._active_txn = False
+        self.lock_wait = lock_wait
+        self.do_lock = lock
+        self.do_create = create
+        self.created = False
+        self.exclusive = exclusive
+        self.append_only = append_only
+        self.storage_quota = storage_quota
+        self.storage_quota_use = 0
+        self.transaction_doomed = None
+        self.make_parent_dirs = make_parent_dirs
+        # v2 is the default repo version for borg 2.0
+        # v1 repos must only be used in a read-only way, e.g. for
+        # --other-repo=V1_REPO with borg init and borg transfer!
+        self.acceptable_repo_versions = (1, 2)
+
+    def __del__(self):
+        if self.lock:
+            self.close()
+            assert False, "cleanup happened in Repository.__del__"
+
+    def __repr__(self):
+        return f"<{self.__class__.__name__} {self.path}>"
+
+    def __enter__(self):
+        if self.do_create:
+            self.do_create = False
+            self.create(self.path)
+            self.created = True
+        self.open(self.path, bool(self.exclusive), lock_wait=self.lock_wait, lock=self.do_lock)
+        return self
+
+    def __exit__(self, exc_type, exc_val, exc_tb):
+        if exc_type is not None:
+            no_space_left_on_device = exc_type is OSError and exc_val.errno == errno.ENOSPC
+            # The ENOSPC could have originated somewhere else besides the Repository. The cleanup is always safe, unless
+            # EIO or FS corruption ensues, which is why we specifically check for ENOSPC.
+            if self._active_txn and no_space_left_on_device:
+                logger.warning("No space left on device, cleaning up partial transaction to free space.")
+                cleanup = True
+            else:
+                cleanup = False
+            self._rollback(cleanup=cleanup)
+        self.close()
+
+    @property
+    def id_str(self):
+        return bin_to_hex(self.id)
+
+    @staticmethod
+    def is_repository(path):
+        """Check whether there is already a Borg repository at *path*."""
+        try:
+            # Use binary mode to avoid troubles if a README contains some stuff not in our locale
+            with open(os.path.join(path, "README"), "rb") as fd:
+                # Read only the first ~100 bytes (if any), in case some README file we stumble upon is large.
+                readme_head = fd.read(100)
+                # The first comparison captures our current variant (REPOSITORY_README), the second comparison
+                # is an older variant of the README file (used by 1.0.x).
+                return b"Borg Backup repository" in readme_head or b"Borg repository" in readme_head
+        except OSError:
+            # Ignore FileNotFound, PermissionError, ...
+            return False
+
+    def check_can_create_repository(self, path):
+        """
+        Raise an exception if a repository already exists at *path* or any parent directory.
+
+        Checking parent directories is done for two reasons:
+        (1) It's just a weird thing to do, and usually not intended. A Borg using the "parent" repository
+            may be confused, or we may accidentally put stuff into the "data/" or "data/<n>/" directories.
+        (2) When implementing repository quotas (which we currently don't), it's important to prohibit
+            folks from creating quota-free repositories. Since no one can create a repository within another
+            repository, user's can only use the quota'd repository, when their --restrict-to-path points
+            at the user's repository.
+        """
+        try:
+            st = os.stat(path)
+        except FileNotFoundError:
+            pass  # nothing there!
+        except PermissionError:
+            raise self.PathPermissionDenied(path) from None
+        else:
+            # there is something already there!
+            if self.is_repository(path):
+                raise self.AlreadyExists(path)
+            if not stat.S_ISDIR(st.st_mode):
+                raise self.PathAlreadyExists(path)
+            try:
+                files = os.listdir(path)
+            except PermissionError:
+                raise self.PathPermissionDenied(path) from None
+            else:
+                if files:  # a dir, but not empty
+                    raise self.PathAlreadyExists(path)
+                else:  # an empty directory is acceptable for us.
+                    pass
+
+        while True:
+            # Check all parent directories for Borg's repository README
+            previous_path = path
+            # Thus, path = previous_path/..
+            path = os.path.abspath(os.path.join(previous_path, os.pardir))
+            if path == previous_path:
+                # We reached the root of the directory hierarchy (/.. = / and C:\.. = C:\).
+                break
+            if self.is_repository(path):
+                raise self.AlreadyExists(path)
+
+    def create(self, path):
+        """Create a new empty repository at `path`"""
+        self.check_can_create_repository(path)
+        if self.make_parent_dirs:
+            parent_path = os.path.join(path, os.pardir)
+            os.makedirs(parent_path, exist_ok=True)
+        if not os.path.exists(path):
+            try:
+                os.mkdir(path)
+            except FileNotFoundError as err:
+                raise self.ParentPathDoesNotExist(path) from err
+        with open(os.path.join(path, "README"), "w") as fd:
+            fd.write(REPOSITORY_README)
+        os.mkdir(os.path.join(path, "data"))
+        config = ConfigParser(interpolation=None)
+        config.add_section("repository")
+        self.version = 2
+        config.set("repository", "version", str(self.version))
+        config.set("repository", "segments_per_dir", str(DEFAULT_SEGMENTS_PER_DIR))
+        config.set("repository", "max_segment_size", str(DEFAULT_MAX_SEGMENT_SIZE))
+        config.set("repository", "append_only", str(int(self.append_only)))
+        if self.storage_quota:
+            config.set("repository", "storage_quota", str(self.storage_quota))
+        else:
+            config.set("repository", "storage_quota", "0")
+        config.set("repository", "additional_free_space", "0")
+        config.set("repository", "id", bin_to_hex(os.urandom(32)))
+        self.save_config(path, config)
+
+    def save_config(self, path, config):
+        config_path = os.path.join(path, "config")
+        old_config_path = os.path.join(path, "config.old")
+
+        if os.path.isfile(old_config_path):
+            logger.warning("Old config file not securely erased on previous config update")
+            secure_erase(old_config_path, avoid_collateral_damage=True)
+
+        if os.path.isfile(config_path):
+            link_error_msg = (
+                "Failed to erase old repository config file securely (hardlinks not supported). "
+                "Old repokey data, if any, might persist on physical storage."
+            )
+            try:
+                os.link(config_path, old_config_path)
+            except OSError as e:
+                if e.errno in (errno.EMLINK, errno.ENOSYS, errno.EPERM, errno.EACCES, errno.ENOTSUP, errno.EIO):
+                    logger.warning(link_error_msg)
+                else:
+                    raise
+            except AttributeError:
+                # some python ports have no os.link, see #4901
+                logger.warning(link_error_msg)
+
+        try:
+            with SaveFile(config_path) as fd:
+                config.write(fd)
+        except PermissionError as e:
+            # error is only a problem if we even had a lock
+            if self.do_lock:
+                raise
+            logger.warning(
+                "%s: Failed writing to '%s'. This is expected when working on "
+                "read-only repositories." % (e.strerror, e.filename)
+            )
+
+        if os.path.isfile(old_config_path):
+            secure_erase(old_config_path, avoid_collateral_damage=True)
+
+    def save_key(self, keydata):
+        assert self.config
+        keydata = keydata.decode("utf-8")  # remote repo: msgpack issue #99, getting bytes
+        # note: saving an empty key means that there is no repokey any more
+        self.config.set("repository", "key", keydata)
+        self.save_config(self.path, self.config)
+
+    def load_key(self):
+        keydata = self.config.get("repository", "key", fallback="").strip()
+        # note: if we return an empty string, it means there is no repo key
+        return keydata.encode("utf-8")  # remote repo: msgpack issue #99, returning bytes
+
+    def destroy(self):
+        """Destroy the repository at `self.path`"""
+        if self.append_only:
+            raise ValueError(self.path + " is in append-only mode")
+        self.close()
+        os.remove(os.path.join(self.path, "config"))  # kill config first
+        shutil.rmtree(self.path)
+
+    def get_index_transaction_id(self):
+        indices = sorted(
+            int(fn[6:])
+            for fn in os.listdir(self.path)
+            if fn.startswith("index.") and fn[6:].isdigit() and os.stat(os.path.join(self.path, fn)).st_size != 0
+        )
+        if indices:
+            return indices[-1]
+        else:
+            return None
+
+    def check_transaction(self):
+        index_transaction_id = self.get_index_transaction_id()
+        segments_transaction_id = self.io.get_segments_transaction_id()
+        if index_transaction_id is not None and segments_transaction_id is None:
+            # we have a transaction id from the index, but we did not find *any*
+            # commit in the segment files (thus no segments transaction id).
+            # this can happen if a lot of segment files are lost, e.g. due to a
+            # filesystem or hardware malfunction. it means we have no identifiable
+            # valid (committed) state of the repo which we could use.
+            msg = '%s" - although likely this is "beyond repair' % self.path  # dirty hack
+            raise self.CheckNeeded(msg)
+        # Attempt to rebuild index automatically if we crashed between commit
+        # tag write and index save.
+        if index_transaction_id != segments_transaction_id:
+            if index_transaction_id is not None and index_transaction_id > segments_transaction_id:
+                replay_from = None
+            else:
+                replay_from = index_transaction_id
+            self.replay_segments(replay_from, segments_transaction_id)
+
+    def get_transaction_id(self):
+        self.check_transaction()
+        return self.get_index_transaction_id()
+
+    def break_lock(self):
+        Lock(os.path.join(self.path, "lock")).break_lock()
+
+    def migrate_lock(self, old_id, new_id):
+        # note: only needed for local repos
+        if self.lock is not None:
+            self.lock.migrate_lock(old_id, new_id)
+
+    def open(self, path, exclusive, lock_wait=None, lock=True):
+        self.path = path
+        try:
+            st = os.stat(path)
+        except FileNotFoundError:
+            raise self.DoesNotExist(path)
+        if not stat.S_ISDIR(st.st_mode):
+            raise self.InvalidRepository(path)
+        if lock:
+            self.lock = Lock(os.path.join(path, "lock"), exclusive, timeout=lock_wait).acquire()
+        else:
+            self.lock = None
+        self.config = ConfigParser(interpolation=None)
+        try:
+            with open(os.path.join(self.path, "config")) as fd:
+                self.config.read_file(fd)
+        except FileNotFoundError:
+            self.close()
+            raise self.InvalidRepository(self.path)
+        if "repository" not in self.config.sections():
+            self.close()
+            raise self.InvalidRepositoryConfig(path, "no repository section found")
+        self.version = self.config.getint("repository", "version")
+        if self.version not in self.acceptable_repo_versions:
+            self.close()
+            raise self.InvalidRepositoryConfig(
+                path, "repository version %d is not supported by this borg version" % self.version
+            )
+        self.max_segment_size = parse_file_size(self.config.get("repository", "max_segment_size"))
+        if self.max_segment_size >= MAX_SEGMENT_SIZE_LIMIT:
+            self.close()
+            raise self.InvalidRepositoryConfig(path, "max_segment_size >= %d" % MAX_SEGMENT_SIZE_LIMIT)  # issue 3592
+        self.segments_per_dir = self.config.getint("repository", "segments_per_dir")
+        self.additional_free_space = parse_file_size(self.config.get("repository", "additional_free_space", fallback=0))
+        # append_only can be set in the constructor
+        # it shouldn't be overridden (True -> False) here
+        self.append_only = self.append_only or self.config.getboolean("repository", "append_only", fallback=False)
+        if self.storage_quota is None:
+            # self.storage_quota is None => no explicit storage_quota was specified, use repository setting.
+            self.storage_quota = parse_file_size(self.config.get("repository", "storage_quota", fallback=0))
+        self.id = hex_to_bin(self.config.get("repository", "id").strip(), length=32)
+        self.io = LoggedIO(self.path, self.max_segment_size, self.segments_per_dir)
+
+    def _load_hints(self):
+        if (transaction_id := self.get_transaction_id()) is None:
+            # self is a fresh repo, so transaction_id is None and there is no hints file
+            return
+        hints = self._unpack_hints(transaction_id)
+        self.version = hints["version"]
+        self.storage_quota_use = hints["storage_quota_use"]
+        self.shadow_index = hints["shadow_index"]
+
+    def info(self):
+        """return some infos about the repo (must be opened first)"""
+        info = dict(id=self.id, version=self.version, append_only=self.append_only)
+        self._load_hints()
+        info["storage_quota"] = self.storage_quota
+        info["storage_quota_use"] = self.storage_quota_use
+        return info
+
+    def close(self):
+        if self.lock:
+            if self.io:
+                self.io.close()
+            self.io = None
+            self.lock.release()
+            self.lock = None
+
+    def commit(self, compact=True, threshold=0.1):
+        """Commit transaction"""
+        if self.transaction_doomed:
+            exception = self.transaction_doomed
+            self.rollback()
+            raise exception
+        self.check_free_space()
+        segment = self.io.write_commit()
+        self.segments.setdefault(segment, 0)
+        self.compact[segment] += LoggedIO.header_fmt.size
+        if compact and not self.append_only:
+            self.compact_segments(threshold)
+        self.write_index()
+        self.rollback()
+
+    def _read_integrity(self, transaction_id, key):
+        integrity_file = "integrity.%d" % transaction_id
+        integrity_path = os.path.join(self.path, integrity_file)
+        try:
+            with open(integrity_path, "rb") as fd:
+                integrity = msgpack.unpack(fd)
+        except FileNotFoundError:
+            return
+        if integrity.get("version") != 2:
+            logger.warning("Unknown integrity data version %r in %s", integrity.get("version"), integrity_file)
+            return
+        return integrity[key]
+
+    def open_index(self, transaction_id, auto_recover=True):
+        if transaction_id is None:
+            return NSIndex()
+        index_path = os.path.join(self.path, "index.%d" % transaction_id)
+        variant = hashindex_variant(index_path)
+        integrity_data = self._read_integrity(transaction_id, "index")
+        try:
+            with IntegrityCheckedFile(index_path, write=False, integrity_data=integrity_data) as fd:
+                if variant == 2:
+                    return NSIndex.read(fd)
+                if variant == 1:  # legacy
+                    return NSIndex1.read(fd)
+        except (ValueError, OSError, FileIntegrityError) as exc:
+            logger.warning("Repository index missing or corrupted, trying to recover from: %s", exc)
+            os.unlink(index_path)
+            if not auto_recover:
+                raise
+            self.prepare_txn(self.get_transaction_id())
+            # don't leave an open transaction around
+            self.commit(compact=False)
+            return self.open_index(self.get_transaction_id())
+
+    def _unpack_hints(self, transaction_id):
+        hints_path = os.path.join(self.path, "hints.%d" % transaction_id)
+        integrity_data = self._read_integrity(transaction_id, "hints")
+        with IntegrityCheckedFile(hints_path, write=False, integrity_data=integrity_data) as fd:
+            return msgpack.unpack(fd)
+
+    def prepare_txn(self, transaction_id, do_cleanup=True):
+        self._active_txn = True
+        if self.do_lock and not self.lock.got_exclusive_lock():
+            if self.exclusive is not None:
+                # self.exclusive is either True or False, thus a new client is active here.
+                # if it is False and we get here, the caller did not use exclusive=True although
+                # it is needed for a write operation. if it is True and we get here, something else
+                # went very wrong, because we should have an exclusive lock, but we don't.
+                raise AssertionError("bug in code, exclusive lock should exist here")
+            # if we are here, this is an old client talking to a new server (expecting lock upgrade).
+            # or we are replaying segments and might need a lock upgrade for that.
+            try:
+                self.lock.upgrade()
+            except (LockError, LockErrorT):
+                # if upgrading the lock to exclusive fails, we do not have an
+                # active transaction. this is important for "serve" mode, where
+                # the repository instance lives on - even if exceptions happened.
+                self._active_txn = False
+                raise
+        if not self.index or transaction_id is None:
+            try:
+                self.index = self.open_index(transaction_id, auto_recover=False)
+            except (ValueError, OSError, FileIntegrityError) as exc:
+                logger.warning("Checking repository transaction due to previous error: %s", exc)
+                self.check_transaction()
+                self.index = self.open_index(transaction_id, auto_recover=False)
+        if transaction_id is None:
+            self.segments = {}  # XXX bad name: usage_count_of_segment_x = self.segments[x]
+            self.compact = FreeSpace()  # XXX bad name: freeable_space_of_segment_x = self.compact[x]
+            self.storage_quota_use = 0
+            self.shadow_index.clear()
+        else:
+            if do_cleanup:
+                self.io.cleanup(transaction_id)
+            hints_path = os.path.join(self.path, "hints.%d" % transaction_id)
+            index_path = os.path.join(self.path, "index.%d" % transaction_id)
+            try:
+                hints = self._unpack_hints(transaction_id)
+            except (msgpack.UnpackException, FileNotFoundError, FileIntegrityError) as e:
+                logger.warning("Repository hints file missing or corrupted, trying to recover: %s", e)
+                if not isinstance(e, FileNotFoundError):
+                    os.unlink(hints_path)
+                # index must exist at this point
+                os.unlink(index_path)
+                self.check_transaction()
+                self.prepare_txn(transaction_id)
+                return
+            if hints["version"] == 1:
+                logger.debug("Upgrading from v1 hints.%d", transaction_id)
+                self.segments = hints["segments"]
+                self.compact = FreeSpace()
+                self.storage_quota_use = 0
+                self.shadow_index = {}
+                for segment in sorted(hints["compact"]):
+                    logger.debug("Rebuilding sparse info for segment %d", segment)
+                    self._rebuild_sparse(segment)
+                logger.debug("Upgrade to v2 hints complete")
+            elif hints["version"] != 2:
+                raise ValueError("Unknown hints file version: %d" % hints["version"])
+            else:
+                self.segments = hints["segments"]
+                self.compact = FreeSpace(hints["compact"])
+                self.storage_quota_use = hints.get("storage_quota_use", 0)
+                self.shadow_index = hints.get("shadow_index", {})
+            # Drop uncommitted segments in the shadow index
+            for key, shadowed_segments in self.shadow_index.items():
+                for segment in list(shadowed_segments):
+                    if segment > transaction_id:
+                        shadowed_segments.remove(segment)
+
+    def write_index(self):
+        def flush_and_sync(fd):
+            fd.flush()
+            os.fsync(fd.fileno())
+
+        def rename_tmp(file):
+            os.replace(file + ".tmp", file)
+
+        hints = {
+            "version": 2,
+            "segments": self.segments,
+            "compact": self.compact,
+            "storage_quota_use": self.storage_quota_use,
+            "shadow_index": self.shadow_index,
+        }
+        integrity = {
+            # Integrity version started at 2, the current hints version.
+            # Thus, integrity version == hints version, for now.
+            "version": 2
+        }
+        transaction_id = self.io.get_segments_transaction_id()
+        assert transaction_id is not None
+
+        # Log transaction in append-only mode
+        if self.append_only:
+            with open(os.path.join(self.path, "transactions"), "a") as log:
+                print(
+                    "transaction %d, UTC time %s"
+                    % (transaction_id, datetime.now(tz=timezone.utc).isoformat(timespec="microseconds")),
+                    file=log,
+                )
+
+        # Write hints file
+        hints_name = "hints.%d" % transaction_id
+        hints_file = os.path.join(self.path, hints_name)
+        with IntegrityCheckedFile(hints_file + ".tmp", filename=hints_name, write=True) as fd:
+            msgpack.pack(hints, fd)
+            flush_and_sync(fd)
+        integrity["hints"] = fd.integrity_data
+
+        # Write repository index
+        index_name = "index.%d" % transaction_id
+        index_file = os.path.join(self.path, index_name)
+        with IntegrityCheckedFile(index_file + ".tmp", filename=index_name, write=True) as fd:
+            # XXX: Consider using SyncFile for index write-outs.
+            self.index.write(fd)
+            flush_and_sync(fd)
+        integrity["index"] = fd.integrity_data
+
+        # Write integrity file, containing checksums of the hints and index files
+        integrity_name = "integrity.%d" % transaction_id
+        integrity_file = os.path.join(self.path, integrity_name)
+        with open(integrity_file + ".tmp", "wb") as fd:
+            msgpack.pack(integrity, fd)
+            flush_and_sync(fd)
+
+        # Rename the integrity file first
+        rename_tmp(integrity_file)
+        sync_dir(self.path)
+        # Rename the others after the integrity file is hypothetically on disk
+        rename_tmp(hints_file)
+        rename_tmp(index_file)
+        sync_dir(self.path)
+
+        # Remove old auxiliary files
+        current = ".%d" % transaction_id
+        for name in os.listdir(self.path):
+            if not name.startswith(("index.", "hints.", "integrity.")):
+                continue
+            if name.endswith(current):
+                continue
+            os.unlink(os.path.join(self.path, name))
+        self.index = None
+
+    def check_free_space(self):
+        """Pre-commit check for sufficient free space necessary to perform the commit."""
+        # As a baseline we take four times the current (on-disk) index size.
+        # At this point the index may only be updated by compaction, which won't resize it.
+        # We still apply a factor of four so that a later, separate invocation can free space
+        # (journaling all deletes for all chunks is one index size) or still make minor additions
+        # (which may grow the index up to twice its current size).
+        # Note that in a subsequent operation the committed index is still on-disk, therefore we
+        # arrive at index_size * (1 + 2 + 1).
+        # In that order: journaled deletes (1), hashtable growth (2), persisted index (1).
+        required_free_space = self.index.size() * 4
+
+        # Conservatively estimate hints file size:
+        # 10 bytes for each segment-refcount pair, 10 bytes for each segment-space pair
+        # Assume maximum of 5 bytes per integer. Segment numbers will usually be packed more densely (1-3 bytes),
+        # as will refcounts and free space integers. For 5 MiB segments this estimate is good to ~20 PB repo size.
+        # Add a generous 4K to account for constant format overhead.
+        hints_size = len(self.segments) * 10 + len(self.compact) * 10 + 4096
+        required_free_space += hints_size
+
+        required_free_space += self.additional_free_space
+        if not self.append_only:
+            full_segment_size = self.max_segment_size + MAX_OBJECT_SIZE
+            if len(self.compact) < 10:
+                # This is mostly for the test suite to avoid overestimated free space needs. This can be annoying
+                # if TMP is a small-ish tmpfs.
+                compact_working_space = 0
+                for segment, free in self.compact.items():
+                    try:
+                        compact_working_space += self.io.segment_size(segment) - free
+                    except FileNotFoundError:
+                        # looks like self.compact is referring to a nonexistent segment file, ignore it.
+                        pass
+                logger.debug("check_free_space: Few segments, not requiring a full free segment")
+                compact_working_space = min(compact_working_space, full_segment_size)
+                logger.debug(
+                    "check_free_space: Calculated working space for compact as %d bytes", compact_working_space
+                )
+                required_free_space += compact_working_space
+            else:
+                # Keep one full worst-case segment free in non-append-only mode
+                required_free_space += full_segment_size
+
+        try:
+            free_space = shutil.disk_usage(self.path).free
+        except OSError as os_error:
+            logger.warning("Failed to check free space before committing: " + str(os_error))
+            return
+        logger.debug(f"check_free_space: Required bytes {required_free_space}, free bytes {free_space}")
+        if free_space < required_free_space:
+            if self.created:
+                logger.error("Not enough free space to initialize repository at this location.")
+                self.destroy()
+            else:
+                self._rollback(cleanup=True)
+            formatted_required = format_file_size(required_free_space)
+            formatted_free = format_file_size(free_space)
+            raise self.InsufficientFreeSpaceError(formatted_required, formatted_free)
+
+    def compact_segments(self, threshold):
+        """Compact sparse segments by copying data into new segments"""
+        if not self.compact:
+            logger.debug("Nothing to do: compact empty")
+            return
+        quota_use_before = self.storage_quota_use
+        index_transaction_id = self.get_index_transaction_id()
+        segments = self.segments
+        unused = []  # list of segments, that are not used anymore
+
+        def complete_xfer(intermediate=True):
+            # complete the current transfer (when some target segment is full)
+            nonlocal unused
+            # commit the new, compact, used segments
+            segment = self.io.write_commit(intermediate=intermediate)
+            self.segments.setdefault(segment, 0)
+            self.compact[segment] += LoggedIO.header_fmt.size
+            logger.debug(
+                "complete_xfer: Wrote %scommit at segment %d", "intermediate " if intermediate else "", segment
+            )
+            # get rid of the old, sparse, unused segments. free space.
+            for segment in unused:
+                logger.debug("complete_xfer: Deleting unused segment %d", segment)
+                count = self.segments.pop(segment)
+                assert count == 0, "Corrupted segment reference count - corrupted index or hints"
+                self.io.delete_segment(segment)
+                del self.compact[segment]
+            unused = []
+
+        logger.debug("Compaction started (threshold is %i%%).", threshold * 100)
+        pi = ProgressIndicatorPercent(
+            total=len(self.compact), msg="Compacting segments %3.0f%%", step=1, msgid="repository.compact_segments"
+        )
+        for segment, freeable_space in sorted(self.compact.items()):
+            if not self.io.segment_exists(segment):
+                logger.warning("Segment %d not found, but listed in compaction data", segment)
+                del self.compact[segment]
+                pi.show()
+                self._send_log()
+                continue
+            segment_size = self.io.segment_size(segment)
+            freeable_ratio = 1.0 * freeable_space / segment_size
+            # we want to compact if:
+            # - we can free a considerable relative amount of space (freeable_ratio over some threshold)
+            if not (freeable_ratio > threshold):
+                logger.debug(
+                    "Not compacting segment %d (maybe freeable: %2.2f%% [%d bytes])",
+                    segment,
+                    freeable_ratio * 100.0,
+                    freeable_space,
+                )
+                pi.show()
+                self._send_log()
+                continue
+            segments.setdefault(segment, 0)
+            logger.debug(
+                "Compacting segment %d with usage count %d (maybe freeable: %2.2f%% [%d bytes])",
+                segment,
+                segments[segment],
+                freeable_ratio * 100.0,
+                freeable_space,
+            )
+            for tag, key, offset, _, data in self.io.iter_objects(segment):
+                if tag == TAG_COMMIT:
+                    continue
+                in_index = self.index.get(key)
+                is_index_object = in_index and (in_index.segment, in_index.offset) == (segment, offset)
+                if tag in (TAG_PUT2, TAG_PUT) and is_index_object:
+                    try:
+                        new_segment, offset = self.io.write_put(key, data, raise_full=True)
+                    except LoggedIO.SegmentFull:
+                        complete_xfer()
+                        new_segment, offset = self.io.write_put(key, data)
+                    self.index[key] = NSIndexEntry(new_segment, offset, len(data))
+                    segments.setdefault(new_segment, 0)
+                    segments[new_segment] += 1
+                    segments[segment] -= 1
+                    if tag == TAG_PUT:
+                        # old tag is PUT, but new will be PUT2 and use a bit more storage
+                        self.storage_quota_use += self.io.ENTRY_HASH_SIZE
+                elif tag in (TAG_PUT2, TAG_PUT) and not is_index_object:
+                    # If this is a PUT shadowed by a later tag, then it will be gone when this segment is deleted after
+                    # this loop. Therefore it is removed from the shadow index.
+                    try:
+                        self.shadow_index[key].remove(segment)
+                    except (KeyError, ValueError):
+                        # do not remove entry with empty shadowed_segments list here,
+                        # it is needed for shadowed_put_exists code (see below)!
+                        pass
+                    self.storage_quota_use -= header_size(tag) + len(data)
+                elif tag == TAG_DELETE and not in_index:
+                    # If the shadow index doesn't contain this key, then we can't say if there's a shadowed older tag,
+                    # therefore we do not drop the delete, but write it to a current segment.
+                    key_not_in_shadow_index = key not in self.shadow_index
+                    # If the key is in the shadow index and there is any segment with an older PUT of this
+                    # key, we have a shadowed put.
+                    shadowed_put_exists = key_not_in_shadow_index or any(
+                        shadowed < segment for shadowed in self.shadow_index[key]
+                    )
+                    delete_is_not_stable = index_transaction_id is None or segment > index_transaction_id
+
+                    if shadowed_put_exists or delete_is_not_stable:
+                        # (introduced in 6425d16aa84be1eaaf88)
+                        # This is needed to avoid object un-deletion if we crash between the commit and the deletion
+                        # of old segments in complete_xfer().
+                        #
+                        # However, this only happens if the crash also affects the FS to the effect that file deletions
+                        # did not materialize consistently after journal recovery. If they always materialize in-order
+                        # then this is not a problem, because the old segment containing a deleted object would be
+                        # deleted before the segment containing the delete.
+                        #
+                        # Consider the following series of operations if we would not do this, i.e. this entire if:
+                        # would be removed.
+                        # Columns are segments, lines are different keys (line 1 = some key, line 2 = some other key)
+                        # Legend: P=TAG_PUT/TAG_PUT2, D=TAG_DELETE, c=commit, i=index is written for latest commit
+                        #
+                        # Segment | 1     | 2   | 3
+                        # --------+-------+-----+------
+                        # Key 1   | P     | D   |
+                        # Key 2   | P     |     | P
+                        # commits |   c i |   c |   c i
+                        # --------+-------+-----+------
+                        #                       ^- compact_segments starts
+                        #                           ^- complete_xfer commits, after that complete_xfer deletes
+                        #                              segments 1 and 2 (and then the index would be written).
+                        #
+                        # Now we crash. But only segment 2 gets deleted, while segment 1 is still around. Now key 1
+                        # is suddenly undeleted (because the delete in segment 2 is now missing).
+                        # Again, note the requirement here. We delete these in the correct order that this doesn't
+                        # happen, and only if the FS materialization of these deletes is reordered or parts dropped
+                        # this can happen.
+                        # In this case it doesn't cause outright corruption, 'just' an index count mismatch, which
+                        # will be fixed by borg-check --repair.
+                        #
+                        # Note that in this check the index state is the proxy for a "most definitely settled"
+                        # repository state, i.e. the assumption is that *all* operations on segments <= index state
+                        # are completed and stable.
+                        try:
+                            new_segment, size = self.io.write_delete(key, raise_full=True)
+                        except LoggedIO.SegmentFull:
+                            complete_xfer()
+                            new_segment, size = self.io.write_delete(key)
+                        self.compact[new_segment] += size
+                        segments.setdefault(new_segment, 0)
+                    else:
+                        logger.debug(
+                            "Dropping DEL for id %s - seg %d, iti %r, knisi %r, spe %r, dins %r, si %r",
+                            bin_to_hex(key),
+                            segment,
+                            index_transaction_id,
+                            key_not_in_shadow_index,
+                            shadowed_put_exists,
+                            delete_is_not_stable,
+                            self.shadow_index.get(key),
+                        )
+                        # we did not keep the delete tag for key (see if-branch)
+                        if not self.shadow_index[key]:
+                            # shadowed segments list is empty -> remove it
+                            del self.shadow_index[key]
+            assert segments[segment] == 0, "Corrupted segment reference count - corrupted index or hints"
+            unused.append(segment)
+            pi.show()
+            self._send_log()
+        pi.finish()
+        self._send_log()
+        complete_xfer(intermediate=False)
+        self.io.clear_empty_dirs()
+        quota_use_after = self.storage_quota_use
+        logger.info("Compaction freed about %s repository space.", format_file_size(quota_use_before - quota_use_after))
+        logger.debug("Compaction completed.")
+
+    def replay_segments(self, index_transaction_id, segments_transaction_id):
+        # fake an old client, so that in case we do not have an exclusive lock yet, prepare_txn will upgrade the lock:
+        remember_exclusive = self.exclusive
+        self.exclusive = None
+        self.prepare_txn(index_transaction_id, do_cleanup=False)
+        try:
+            segment_count = sum(1 for _ in self.io.segment_iterator())
+            pi = ProgressIndicatorPercent(
+                total=segment_count, msg="Replaying segments %3.0f%%", msgid="repository.replay_segments"
+            )
+            for i, (segment, filename) in enumerate(self.io.segment_iterator()):
+                pi.show(i)
+                self._send_log()
+                if index_transaction_id is not None and segment <= index_transaction_id:
+                    continue
+                if segment > segments_transaction_id:
+                    break
+                objects = self.io.iter_objects(segment)
+                self._update_index(segment, objects)
+            pi.finish()
+            self._send_log()
+            self.write_index()
+        finally:
+            self.exclusive = remember_exclusive
+            self.rollback()
+
+    def _update_index(self, segment, objects, report=None):
+        """some code shared between replay_segments and check"""
+        self.segments[segment] = 0
+        for tag, key, offset, size, _ in objects:
+            if tag in (TAG_PUT2, TAG_PUT):
+                try:
+                    # If this PUT supersedes an older PUT, mark the old segment for compaction and count the free space
+                    in_index = self.index[key]
+                    self.compact[in_index.segment] += header_size(tag) + size
+                    self.segments[in_index.segment] -= 1
+                    self.shadow_index.setdefault(key, []).append(in_index.segment)
+                except KeyError:
+                    pass
+                self.index[key] = NSIndexEntry(segment, offset, size)
+                self.segments[segment] += 1
+                self.storage_quota_use += header_size(tag) + size
+            elif tag == TAG_DELETE:
+                try:
+                    # if the deleted PUT is not in the index, there is nothing to clean up
+                    in_index = self.index.pop(key)
+                except KeyError:
+                    pass
+                else:
+                    if self.io.segment_exists(in_index.segment):
+                        # the old index is not necessarily valid for this transaction (e.g. compaction); if the segment
+                        # is already gone, then it was already compacted.
+                        self.segments[in_index.segment] -= 1
+                        self.compact[in_index.segment] += header_size(tag) + in_index.size
+                        self.shadow_index.setdefault(key, []).append(in_index.segment)
+            elif tag == TAG_COMMIT:
+                continue
+            else:
+                msg = f"Unexpected tag {tag} in segment {segment}"
+                if report is None:
+                    raise self.CheckNeeded(msg)
+                else:
+                    report(msg)
+        if self.segments[segment] == 0:
+            self.compact[segment] = self.io.segment_size(segment)
+
+    def _rebuild_sparse(self, segment):
+        """Rebuild sparse bytes count for a single segment relative to the current index."""
+        try:
+            segment_size = self.io.segment_size(segment)
+        except FileNotFoundError:
+            # segment does not exist any more, remove it from the mappings.
+            # note: no need to self.compact.pop(segment), as we start from empty mapping.
+            self.segments.pop(segment)
+            return
+
+        if self.segments[segment] == 0:
+            self.compact[segment] = segment_size
+            return
+
+        self.compact[segment] = 0
+        for tag, key, offset, size, _ in self.io.iter_objects(segment, read_data=False):
+            if tag in (TAG_PUT2, TAG_PUT):
+                in_index = self.index.get(key)
+                if not in_index or (in_index.segment, in_index.offset) != (segment, offset):
+                    # This PUT is superseded later.
+                    self.compact[segment] += header_size(tag) + size
+            elif tag == TAG_DELETE:
+                # The outcome of the DELETE has been recorded in the PUT branch already.
+                self.compact[segment] += header_size(tag) + size
+
+    def check(self, repair=False, max_duration=0):
+        """Check repository consistency
+
+        This method verifies all segment checksums and makes sure
+        the index is consistent with the data stored in the segments.
+        """
+        if self.append_only and repair:
+            raise ValueError(self.path + " is in append-only mode")
+        error_found = False
+
+        def report_error(msg, *args):
+            nonlocal error_found
+            error_found = True
+            logger.error(msg, *args)
+
+        logger.info("Starting repository check")
+        assert not self._active_txn
+        try:
+            transaction_id = self.get_transaction_id()
+            current_index = self.open_index(transaction_id)
+            logger.debug("Read committed index of transaction %d", transaction_id)
+        except Exception as exc:
+            transaction_id = self.io.get_segments_transaction_id()
+            current_index = None
+            logger.debug("Failed to read committed index (%s)", exc)
+        if transaction_id is None:
+            logger.debug("No segments transaction found")
+            transaction_id = self.get_index_transaction_id()
+        if transaction_id is None:
+            logger.debug("No index transaction found, trying latest segment")
+            transaction_id = self.io.get_latest_segment()
+        if transaction_id is None:
+            report_error("This repository contains no valid data.")
+            return False
+        if repair:
+            self.io.cleanup(transaction_id)
+        segments_transaction_id = self.io.get_segments_transaction_id()
+        logger.debug("Segment transaction is    %s", segments_transaction_id)
+        logger.debug("Determined transaction is %s", transaction_id)
+        self.prepare_txn(None)  # self.index, self.compact, self.segments, self.shadow_index all empty now!
+        segment_count = sum(1 for _ in self.io.segment_iterator())
+        logger.debug("Found %d segments", segment_count)
+
+        partial = bool(max_duration)
+        assert not (repair and partial)
+        mode = "partial" if partial else "full"
+        if partial:
+            # continue a past partial check (if any) or start one from beginning
+            last_segment_checked = self.config.getint("repository", "last_segment_checked", fallback=-1)
+            logger.info("Skipping to segments >= %d", last_segment_checked + 1)
+        else:
+            # start from the beginning and also forget about any potential past partial checks
+            last_segment_checked = -1
+            self.config.remove_option("repository", "last_segment_checked")
+            self.save_config(self.path, self.config)
+        t_start = time.monotonic()
+        pi = ProgressIndicatorPercent(
+            total=segment_count, msg="Checking segments %3.1f%%", step=0.1, msgid="repository.check"
+        )
+        segment = -1  # avoid uninitialized variable if there are no segment files at all
+        for i, (segment, filename) in enumerate(self.io.segment_iterator()):
+            pi.show(i)
+            self._send_log()
+            if segment <= last_segment_checked:
+                continue
+            if segment > transaction_id:
+                continue
+            logger.debug("Checking segment file %s...", filename)
+            try:
+                objects = list(self.io.iter_objects(segment))
+            except IntegrityError as err:
+                report_error(str(err))
+                objects = []
+                if repair:
+                    self.io.recover_segment(segment, filename)
+                    objects = list(self.io.iter_objects(segment))
+            if not partial:
+                self._update_index(segment, objects, report_error)
+            if partial and time.monotonic() > t_start + max_duration:
+                logger.info("Finished partial segment check, last segment checked is %d", segment)
+                self.config.set("repository", "last_segment_checked", str(segment))
+                self.save_config(self.path, self.config)
+                break
+        else:
+            logger.info("Finished segment check at segment %d", segment)
+            self.config.remove_option("repository", "last_segment_checked")
+            self.save_config(self.path, self.config)
+
+        pi.finish()
+        self._send_log()
+        # self.index, self.segments, self.compact now reflect the state of the segment files up to <transaction_id>.
+        # We might need to add a commit tag if no committed segment is found.
+        if repair and segments_transaction_id is None:
+            report_error(f"Adding commit tag to segment {transaction_id}")
+            self.io.segment = transaction_id + 1
+            self.io.write_commit()
+        if not partial:
+            logger.info("Starting repository index check")
+            if current_index and not repair:
+                # current_index = "as found on disk"
+                # self.index = "as rebuilt in-memory from segments"
+                if len(current_index) != len(self.index):
+                    report_error("Index object count mismatch.")
+                    report_error("committed index: %d objects", len(current_index))
+                    report_error("rebuilt index:   %d objects", len(self.index))
+                else:
+                    logger.info("Index object count match.")
+                line_format = "ID: %-64s rebuilt index: %-16s committed index: %-16s"
+                not_found = "<not found>"
+                for key, value in self.index.iteritems():
+                    current_value = current_index.get(key, not_found)
+                    if current_value != value:
+                        report_error(line_format, bin_to_hex(key), value, current_value)
+                self._send_log()
+                for key, current_value in current_index.iteritems():
+                    if key in self.index:
+                        continue
+                    value = self.index.get(key, not_found)
+                    if current_value != value:
+                        report_error(line_format, bin_to_hex(key), value, current_value)
+                self._send_log()
+            if repair:
+                self.write_index()
+        self.rollback()
+        if error_found:
+            if repair:
+                logger.info("Finished %s repository check, errors found and repaired.", mode)
+            else:
+                logger.error("Finished %s repository check, errors found.", mode)
+        else:
+            logger.info("Finished %s repository check, no problems found.", mode)
+        return not error_found or repair
+
+    def _rollback(self, *, cleanup):
+        if cleanup:
+            self.io.cleanup(self.io.get_segments_transaction_id())
+        self.index = None
+        self._active_txn = False
+        self.transaction_doomed = None
+
+    def rollback(self):
+        # note: when used in remote mode, this is time limited, see LegacyRemoteRepository.shutdown_time.
+        self._rollback(cleanup=False)
+
+    def __len__(self):
+        if not self.index:
+            self.index = self.open_index(self.get_transaction_id())
+        return len(self.index)
+
+    def __contains__(self, id):
+        if not self.index:
+            self.index = self.open_index(self.get_transaction_id())
+        return id in self.index
+
+    def list(self, limit=None, marker=None):
+        """
+        list <limit> IDs starting from after id <marker> - in index (pseudo-random) order.
+        """
+        if not self.index:
+            self.index = self.open_index(self.get_transaction_id())
+        return [id_ for id_, _ in islice(self.index.iteritems(marker=marker), limit)]
+
+    def get(self, id, read_data=True):
+        if not self.index:
+            self.index = self.open_index(self.get_transaction_id())
+        try:
+            in_index = NSIndexEntry(*((self.index[id] + (None,))[:3]))  # legacy: index entries have no size element
+            return self.io.read(in_index.segment, in_index.offset, id, expected_size=in_index.size, read_data=read_data)
+        except KeyError:
+            raise self.ObjectNotFound(id, self.path) from None
+
+    def get_many(self, ids, read_data=True, is_preloaded=False):
+        for id_ in ids:
+            yield self.get(id_, read_data=read_data)
+
+    def put(self, id, data, wait=True):
+        """put a repo object
+
+        Note: when doing calls with wait=False this gets async and caller must
+              deal with async results / exceptions later.
+        """
+        if not self._active_txn:
+            self.prepare_txn(self.get_transaction_id())
+        try:
+            in_index = self.index[id]
+        except KeyError:
+            pass
+        else:
+            # this put call supersedes a previous put to same id.
+            # it is essential to do a delete first to get correct quota bookkeeping
+            # and also a correctly updated shadow_index, so that the compaction code
+            # does not wrongly resurrect an old PUT by dropping a DEL that is still needed.
+            self._delete(id, in_index.segment, in_index.offset, in_index.size)
+        segment, offset = self.io.write_put(id, data)
+        self.storage_quota_use += header_size(TAG_PUT2) + len(data)
+        self.segments.setdefault(segment, 0)
+        self.segments[segment] += 1
+        self.index[id] = NSIndexEntry(segment, offset, len(data))
+        if self.storage_quota and self.storage_quota_use > self.storage_quota:
+            self.transaction_doomed = self.StorageQuotaExceeded(
+                format_file_size(self.storage_quota), format_file_size(self.storage_quota_use)
+            )
+            raise self.transaction_doomed
+
+    def delete(self, id, wait=True):
+        """delete a repo object
+
+        Note: when doing calls with wait=False this gets async and caller must
+              deal with async results / exceptions later.
+        """
+        if not self._active_txn:
+            self.prepare_txn(self.get_transaction_id())
+        try:
+            in_index = self.index.pop(id)
+        except KeyError:
+            raise self.ObjectNotFound(id, self.path) from None
+        self._delete(id, in_index.segment, in_index.offset, in_index.size)
+
+    def _delete(self, id, segment, offset, size):
+        # common code used by put and delete
+        # because we'll write a DEL tag to the repository, we must update the shadow index.
+        # this is always true, no matter whether we are called from put() or delete().
+        # the compaction code needs this to not drop DEL tags if they are still required
+        # to keep a PUT in an earlier segment in the "effectively deleted" state.
+        self.shadow_index.setdefault(id, []).append(segment)
+        self.segments[segment] -= 1
+        self.compact[segment] += header_size(TAG_PUT2) + size
+        segment, size = self.io.write_delete(id)
+        self.compact[segment] += size
+        self.segments.setdefault(segment, 0)
+
+    def async_response(self, wait=True):
+        """Get one async result (only applies to remote repositories).
+
+        async commands (== calls with wait=False, e.g. delete and put) have no results,
+        but may raise exceptions. These async exceptions must get collected later via
+        async_response() calls. Repeat the call until it returns None.
+        The previous calls might either return one (non-None) result or raise an exception.
+        If wait=True is given and there are outstanding responses, it will wait for them
+        to arrive. With wait=False, it will only return already received responses.
+        """
+
+    def preload(self, ids):
+        """Preload objects (only applies to remote repositories)"""
+
+    def get_manifest(self):
+        try:
+            return self.get(Manifest.MANIFEST_ID)
+        except self.ObjectNotFound:
+            raise NoManifestError
+
+    def put_manifest(self, data):
+        return self.put(Manifest.MANIFEST_ID, data)
+
+
+class LoggedIO:
+    class SegmentFull(Exception):
+        """raised when a segment is full, before opening next"""
+
+    header_fmt = struct.Struct("<IIB")
+    assert header_fmt.size == 9
+    header_no_crc_fmt = struct.Struct("<IB")
+    assert header_no_crc_fmt.size == 5
+    crc_fmt = struct.Struct("<I")
+    assert crc_fmt.size == 4
+
+    _commit = header_no_crc_fmt.pack(9, TAG_COMMIT)
+    COMMIT = crc_fmt.pack(crc32(_commit)) + _commit
+
+    HEADER_ID_SIZE = header_fmt.size + 32
+    ENTRY_HASH_SIZE = 8
+
+    def __init__(self, path, limit, segments_per_dir, capacity=90):
+        self.path = path
+        self.fds = LRUCache(capacity, dispose=self._close_fd)
+        self.segment = 0
+        self.limit = limit
+        self.segments_per_dir = segments_per_dir
+        self.offset = 0
+        self._write_fd = None
+        self._fds_cleaned = 0
+
+    def close(self):
+        self.close_segment()
+        self.fds.clear()
+        self.fds = None  # Just to make sure we're disabled
+
+    def _close_fd(self, ts_fd):
+        ts, fd = ts_fd
+        safe_fadvise(fd.fileno(), 0, 0, "DONTNEED")
+        fd.close()
+
+    def get_segment_dirs(self, data_dir, start_index=MIN_SEGMENT_DIR_INDEX, end_index=MAX_SEGMENT_DIR_INDEX):
+        """Returns generator yielding required segment dirs in data_dir as `os.DirEntry` objects.
+        Start and end are inclusive.
+        """
+        segment_dirs = (
+            f
+            for f in os.scandir(data_dir)
+            if f.is_dir() and f.name.isdigit() and start_index <= int(f.name) <= end_index
+        )
+        return segment_dirs
+
+    def get_segment_files(self, segment_dir, start_index=MIN_SEGMENT_INDEX, end_index=MAX_SEGMENT_INDEX):
+        """Returns generator yielding required segment files in segment_dir as `os.DirEntry` objects.
+        Start and end are inclusive.
+        """
+        segment_files = (
+            f
+            for f in os.scandir(segment_dir)
+            if f.is_file() and f.name.isdigit() and start_index <= int(f.name) <= end_index
+        )
+        return segment_files
+
+    def segment_iterator(self, start_segment=None, end_segment=None, reverse=False):
+        if start_segment is None:
+            start_segment = MIN_SEGMENT_INDEX if not reverse else MAX_SEGMENT_INDEX
+        if end_segment is None:
+            end_segment = MAX_SEGMENT_INDEX if not reverse else MIN_SEGMENT_INDEX
+        data_path = os.path.join(self.path, "data")
+        start_segment_dir = start_segment // self.segments_per_dir
+        end_segment_dir = end_segment // self.segments_per_dir
+        if not reverse:
+            dirs = self.get_segment_dirs(data_path, start_index=start_segment_dir, end_index=end_segment_dir)
+        else:
+            dirs = self.get_segment_dirs(data_path, start_index=end_segment_dir, end_index=start_segment_dir)
+        dirs = sorted(dirs, key=lambda dir: int(dir.name), reverse=reverse)
+        for dir in dirs:
+            if not reverse:
+                files = self.get_segment_files(dir, start_index=start_segment, end_index=end_segment)
+            else:
+                files = self.get_segment_files(dir, start_index=end_segment, end_index=start_segment)
+            files = sorted(files, key=lambda file: int(file.name), reverse=reverse)
+            for file in files:
+                # Note: Do not filter out logically deleted segments  (see "File system interaction" above),
+                # since this is used by cleanup and txn state detection as well.
+                yield int(file.name), file.path
+
+    def get_latest_segment(self):
+        for segment, filename in self.segment_iterator(reverse=True):
+            return segment
+        return None
+
+    def get_segments_transaction_id(self):
+        """Return the last committed segment."""
+        for segment, filename in self.segment_iterator(reverse=True):
+            if self.is_committed_segment(segment):
+                return segment
+        return None
+
+    def cleanup(self, transaction_id):
+        """Delete segment files left by aborted transactions"""
+        self.close_segment()
+        self.segment = transaction_id + 1
+        count = 0
+        for segment, filename in self.segment_iterator(reverse=True):
+            if segment > transaction_id:
+                self.delete_segment(segment)
+                count += 1
+            else:
+                break
+        logger.debug("Cleaned up %d uncommitted segment files (== everything after segment %d).", count, transaction_id)
+
+    def is_committed_segment(self, segment):
+        """Check if segment ends with a COMMIT_TAG tag"""
+        try:
+            iterator = self.iter_objects(segment)
+        except IntegrityError:
+            return False
+        with open(self.segment_filename(segment), "rb") as fd:
+            try:
+                fd.seek(-self.header_fmt.size, os.SEEK_END)
+            except OSError as e:
+                # return False if segment file is empty or too small
+                if e.errno == errno.EINVAL:
+                    return False
+                raise e
+            if fd.read(self.header_fmt.size) != self.COMMIT:
+                return False
+        seen_commit = False
+        while True:
+            try:
+                tag, key, offset, _, _ = next(iterator)
+            except IntegrityError:
+                return False
+            except StopIteration:
+                break
+            if tag == TAG_COMMIT:
+                seen_commit = True
+                continue
+            if seen_commit:
+                return False
+        return seen_commit
+
+    def segment_filename(self, segment):
+        return os.path.join(self.path, "data", str(segment // self.segments_per_dir), str(segment))
+
+    def get_write_fd(self, no_new=False, want_new=False, raise_full=False):
+        if not no_new and (want_new or self.offset and self.offset > self.limit):
+            if raise_full:
+                raise self.SegmentFull
+            self.close_segment()
+        if not self._write_fd:
+            if self.segment % self.segments_per_dir == 0:
+                dirname = os.path.join(self.path, "data", str(self.segment // self.segments_per_dir))
+                if not os.path.exists(dirname):
+                    os.mkdir(dirname)
+                    sync_dir(os.path.join(self.path, "data"))
+            self._write_fd = SyncFile(self.segment_filename(self.segment), binary=True)
+            self._write_fd.write(MAGIC)
+            self.offset = MAGIC_LEN
+            if self.segment in self.fds:
+                # we may have a cached fd for a segment file we already deleted and
+                # we are writing now a new segment file to same file name. get rid of
+                # the cached fd that still refers to the old file, so it will later
+                # get repopulated (on demand) with a fd that refers to the new file.
+                del self.fds[self.segment]
+        return self._write_fd
+
+    def get_fd(self, segment):
+        # note: get_fd() returns a fd with undefined file pointer position,
+        # so callers must always seek() to desired position afterwards.
+        now = time.monotonic()
+
+        def open_fd():
+            fd = open(self.segment_filename(segment), "rb")
+            self.fds[segment] = (now, fd)
+            return fd
+
+        def clean_old():
+            # we regularly get rid of all old FDs here:
+            if now - self._fds_cleaned > FD_MAX_AGE // 8:
+                self._fds_cleaned = now
+                for k, ts_fd in list(self.fds.items()):
+                    ts, fd = ts_fd
+                    if now - ts > FD_MAX_AGE:
+                        # we do not want to touch long-unused file handles to
+                        # avoid ESTALE issues (e.g. on network filesystems).
+                        del self.fds[k]
+
+        clean_old()
+        if self._write_fd is not None:
+            # without this, we have a test failure now
+            self._write_fd.sync()
+        try:
+            ts, fd = self.fds[segment]
+        except KeyError:
+            fd = open_fd()
+        else:
+            # we only have fresh enough stuff here.
+            # update the timestamp of the lru cache entry.
+            self.fds.replace(segment, (now, fd))
+        return fd
+
+    def close_segment(self):
+        # set self._write_fd to None early to guard against reentry from error handling code paths:
+        fd, self._write_fd = self._write_fd, None
+        if fd is not None:
+            self.segment += 1
+            self.offset = 0
+            fd.close()
+
+    def delete_segment(self, segment):
+        if segment in self.fds:
+            del self.fds[segment]
+        try:
+            safe_unlink(self.segment_filename(segment))
+        except FileNotFoundError:
+            pass
+
+    def clear_empty_dirs(self):
+        """Delete empty segment dirs, i.e those with no segment files."""
+        data_dir = os.path.join(self.path, "data")
+        segment_dirs = self.get_segment_dirs(data_dir)
+        for segment_dir in segment_dirs:
+            try:
+                # os.rmdir will only delete the directory if it is empty
+                # so we don't need to explicitly check for emptiness first.
+                os.rmdir(segment_dir)
+            except OSError:
+                # OSError is raised by os.rmdir if directory is not empty. This is expected.
+                # Its subclass FileNotFoundError may be raised if the directory already does not exist. Ignorable.
+                pass
+        sync_dir(data_dir)
+
+    def segment_exists(self, segment):
+        filename = self.segment_filename(segment)
+        # When deleting segments, they are first truncated. If truncate(2) and unlink(2) are split
+        # across FS transactions, then logically deleted segments will show up as truncated.
+        return os.path.exists(filename) and os.path.getsize(filename)
+
+    def segment_size(self, segment):
+        return os.path.getsize(self.segment_filename(segment))
+
+    def get_segment_magic(self, segment):
+        fd = self.get_fd(segment)
+        fd.seek(0)
+        return fd.read(MAGIC_LEN)
+
+    def iter_objects(self, segment, read_data=True):
+        """
+        Return object iterator for *segment*.
+
+        See the _read() docstring about confidence in the returned data.
+
+        The iterator returns five-tuples of (tag, key, offset, size, data).
+        """
+        fd = self.get_fd(segment)
+        offset = 0
+        fd.seek(offset)
+        if fd.read(MAGIC_LEN) != MAGIC:
+            raise IntegrityError(f"Invalid segment magic [segment {segment}, offset {offset}]")
+        offset = MAGIC_LEN
+        header = fd.read(self.header_fmt.size)
+        while header:
+            size, tag, key, data = self._read(
+                fd, header, segment, offset, (TAG_PUT2, TAG_DELETE, TAG_COMMIT, TAG_PUT), read_data=read_data
+            )
+            # tuple[3]: corresponds to len(data) == length of the full chunk payload (meta_len+enc_meta+enc_data)
+            # tuple[4]: data will be None if read_data is False.
+            yield tag, key, offset, size - header_size(tag), data
+            assert size >= 0
+            offset += size
+            # we must get the fd via get_fd() here again as we yielded to our caller and it might
+            # have triggered closing of the fd we had before (e.g. by calling io.read() for
+            # different segment(s)).
+            # by calling get_fd() here again we also make our fd "recently used" so it likely
+            # does not get kicked out of self.fds LRUcache.
+            fd = self.get_fd(segment)
+            fd.seek(offset)
+            header = fd.read(self.header_fmt.size)
+
+    def recover_segment(self, segment, filename):
+        logger.info("Attempting to recover " + filename)
+        if segment in self.fds:
+            del self.fds[segment]
+        if os.path.getsize(filename) < MAGIC_LEN + self.header_fmt.size:
+            # this is either a zero-byte file (which would crash mmap() below) or otherwise
+            # just too small to be a valid non-empty segment file, so do a shortcut here:
+            with SaveFile(filename, binary=True) as fd:
+                fd.write(MAGIC)
+            return
+        with SaveFile(filename, binary=True) as dst_fd:
+            with open(filename, "rb") as src_fd:
+                # note: file must not be 0 size or mmap() will crash.
+                with mmap.mmap(src_fd.fileno(), 0, access=mmap.ACCESS_READ) as mm:
+                    # memoryview context manager is problematic, see https://bugs.python.org/issue35686
+                    data = memoryview(mm)
+                    d = data
+                    try:
+                        dst_fd.write(MAGIC)
+                        while len(d) >= self.header_fmt.size:
+                            crc, size, tag = self.header_fmt.unpack(d[: self.header_fmt.size])
+                            size_invalid = size > MAX_OBJECT_SIZE or size < self.header_fmt.size or size > len(d)
+                            if size_invalid or tag > MAX_TAG_ID:
+                                d = d[1:]
+                                continue
+                            if tag == TAG_PUT2:
+                                c_offset = self.HEADER_ID_SIZE + self.ENTRY_HASH_SIZE
+                                # skip if header is invalid
+                                if crc32(d[4:c_offset]) & 0xFFFFFFFF != crc:
+                                    d = d[1:]
+                                    continue
+                                # skip if content is invalid
+                                if (
+                                    self.entry_hash(d[4 : self.HEADER_ID_SIZE], d[c_offset:size])
+                                    != d[self.HEADER_ID_SIZE : c_offset]
+                                ):
+                                    d = d[1:]
+                                    continue
+                            elif tag in (TAG_DELETE, TAG_COMMIT, TAG_PUT):
+                                if crc32(d[4:size]) & 0xFFFFFFFF != crc:
+                                    d = d[1:]
+                                    continue
+                            else:  # tag unknown
+                                d = d[1:]
+                                continue
+                            dst_fd.write(d[:size])
+                            d = d[size:]
+                    finally:
+                        del d
+                        data.release()
+
+    def entry_hash(self, *data):
+        h = StreamingXXH64()
+        for d in data:
+            h.update(d)
+        return h.digest()
+
+    def read(self, segment, offset, id, *, read_data=True, expected_size=None):
+        """
+        Read entry from *segment* at *offset* with *id*.
+
+        See the _read() docstring about confidence in the returned data.
+        """
+        if segment == self.segment and self._write_fd:
+            self._write_fd.sync()
+        fd = self.get_fd(segment)
+        fd.seek(offset)
+        header = fd.read(self.header_fmt.size)
+        size, tag, key, data = self._read(fd, header, segment, offset, (TAG_PUT2, TAG_PUT), read_data=read_data)
+        if id != key:
+            raise IntegrityError(
+                f"Invalid segment entry header, is not for wanted id [segment {segment}, offset {offset}]"
+            )
+        data_size_from_header = size - header_size(tag)
+        if expected_size is not None and expected_size != data_size_from_header:
+            raise IntegrityError(
+                f"size from repository index: {expected_size} != " f"size from entry header: {data_size_from_header}"
+            )
+        return data
+
+    def _read(self, fd, header, segment, offset, acceptable_tags, read_data=True):
+        """
+        Code shared by read() and iter_objects().
+
+        Confidence in returned data:
+        PUT2 tags, read_data == True: crc32 check (header) plus digest check (header+data)
+        PUT2 tags, read_data == False: crc32 check (header)
+        PUT tags, read_data == True: crc32 check (header+data)
+        PUT tags, read_data == False: crc32 check can not be done, all data obtained must be considered informational
+
+        read_data == False behaviour:
+        PUT2 tags: return enough of the chunk so that the client is able to decrypt the metadata,
+                   do not read, but just seek over the data.
+        PUT tags:  return None and just seek over the data.
+        """
+
+        def check_crc32(wanted, header, *data):
+            result = crc32(memoryview(header)[4:])  # skip first 32 bits of the header, they contain the crc.
+            for d in data:
+                result = crc32(d, result)
+            if result & 0xFFFFFFFF != wanted:
+                raise IntegrityError(f"Segment entry header checksum mismatch [segment {segment}, offset {offset}]")
+
+        # See comment on MAX_TAG_ID for details
+        assert max(acceptable_tags) <= MAX_TAG_ID, "Exceeding MAX_TAG_ID will break backwards compatibility"
+        key = data = None
+        fmt = self.header_fmt
+        try:
+            hdr_tuple = fmt.unpack(header)
+        except struct.error as err:
+            raise IntegrityError(f"Invalid segment entry header [segment {segment}, offset {offset}]: {err}") from None
+        crc, size, tag = hdr_tuple
+        length = size - fmt.size  # we already read the header
+        if size > MAX_OBJECT_SIZE:
+            # if you get this on an archive made with borg < 1.0.7 and millions of files and
+            # you need to restore it, you can disable this check by using "if False:" above.
+            raise IntegrityError(f"Invalid segment entry size {size} - too big [segment {segment}, offset {offset}]")
+        if size < fmt.size:
+            raise IntegrityError(f"Invalid segment entry size {size} - too small [segment {segment}, offset {offset}]")
+        if tag not in (TAG_PUT2, TAG_DELETE, TAG_COMMIT, TAG_PUT):
+            raise IntegrityError(
+                f"Invalid segment entry header, did not get a known tag " f"[segment {segment}, offset {offset}]"
+            )
+        if tag not in acceptable_tags:
+            raise IntegrityError(
+                f"Invalid segment entry header, did not get acceptable tag " f"[segment {segment}, offset {offset}]"
+            )
+        if tag == TAG_COMMIT:
+            check_crc32(crc, header)
+            # that's all for COMMITs.
+        else:
+            # all other tags (TAG_PUT2, TAG_DELETE, TAG_PUT) have a key
+            key = fd.read(32)
+            length -= 32
+            if len(key) != 32:
+                raise IntegrityError(
+                    f"Segment entry key short read [segment {segment}, offset {offset}]: "
+                    f"expected {32}, got {len(key)} bytes"
+                )
+            if tag == TAG_DELETE:
+                check_crc32(crc, header, key)
+                # that's all for DELETEs.
+            else:
+                # TAG_PUT: we can not do a crc32 header check here, because the crc32 is computed over header+data!
+                #          for the check, see code below when read_data is True.
+                if tag == TAG_PUT2:
+                    entry_hash = fd.read(self.ENTRY_HASH_SIZE)
+                    length -= self.ENTRY_HASH_SIZE
+                    if len(entry_hash) != self.ENTRY_HASH_SIZE:
+                        raise IntegrityError(
+                            f"Segment entry hash short read [segment {segment}, offset {offset}]: "
+                            f"expected {self.ENTRY_HASH_SIZE}, got {len(entry_hash)} bytes"
+                        )
+                    check_crc32(crc, header, key, entry_hash)
+                if not read_data:
+                    if tag == TAG_PUT2:
+                        # PUT2 is only used in new repos and they also have different RepoObj layout,
+                        # supporting separately encrypted metadata and data.
+                        # In this case, we return enough bytes so the client can decrypt the metadata
+                        # and seek over the rest (over the encrypted data).
+                        hdr_size = RepoObj.obj_header.size
+                        hdr = fd.read(hdr_size)
+                        length -= hdr_size
+                        if len(hdr) != hdr_size:
+                            raise IntegrityError(
+                                f"Segment entry meta length short read [segment {segment}, offset {offset}]: "
+                                f"expected {hdr_size}, got {len(hdr)} bytes"
+                            )
+                        meta_size = RepoObj.obj_header.unpack(hdr)[0]
+                        meta = fd.read(meta_size)
+                        length -= meta_size
+                        if len(meta) != meta_size:
+                            raise IntegrityError(
+                                f"Segment entry meta short read [segment {segment}, offset {offset}]: "
+                                f"expected {meta_size}, got {len(meta)} bytes"
+                            )
+                        data = hdr + meta  # shortened chunk - enough so the client can decrypt the metadata
+                    # in any case, we seek over the remainder of the chunk
+                    oldpos = fd.tell()
+                    seeked = fd.seek(length, os.SEEK_CUR) - oldpos
+                    if seeked != length:
+                        raise IntegrityError(
+                            f"Segment entry data short seek [segment {segment}, offset {offset}]: "
+                            f"expected {length}, got {seeked} bytes"
+                        )
+                else:  # read data!
+                    data = fd.read(length)
+                    if len(data) != length:
+                        raise IntegrityError(
+                            f"Segment entry data short read [segment {segment}, offset {offset}]: "
+                            f"expected {length}, got {len(data)} bytes"
+                        )
+                    if tag == TAG_PUT2:
+                        if self.entry_hash(memoryview(header)[4:], key, data) != entry_hash:
+                            raise IntegrityError(f"Segment entry hash mismatch [segment {segment}, offset {offset}]")
+                    elif tag == TAG_PUT:
+                        check_crc32(crc, header, key, data)
+        return size, tag, key, data
+
+    def write_put(self, id, data, raise_full=False):
+        data_size = len(data)
+        if data_size > MAX_DATA_SIZE:
+            # this would push the segment entry size beyond MAX_OBJECT_SIZE.
+            raise IntegrityError(f"More than allowed put data [{data_size} > {MAX_DATA_SIZE}]")
+        fd = self.get_write_fd(want_new=(id == Manifest.MANIFEST_ID), raise_full=raise_full)
+        size = data_size + self.HEADER_ID_SIZE + self.ENTRY_HASH_SIZE
+        offset = self.offset
+        header = self.header_no_crc_fmt.pack(size, TAG_PUT2)
+        entry_hash = self.entry_hash(header, id, data)
+        crc = self.crc_fmt.pack(crc32(entry_hash, crc32(id, crc32(header))) & 0xFFFFFFFF)
+        fd.write(b"".join((crc, header, id, entry_hash)))
+        fd.write(data)
+        self.offset += size
+        return self.segment, offset
+
+    def write_delete(self, id, raise_full=False):
+        fd = self.get_write_fd(want_new=(id == Manifest.MANIFEST_ID), raise_full=raise_full)
+        header = self.header_no_crc_fmt.pack(self.HEADER_ID_SIZE, TAG_DELETE)
+        crc = self.crc_fmt.pack(crc32(id, crc32(header)) & 0xFFFFFFFF)
+        fd.write(b"".join((crc, header, id)))
+        self.offset += self.HEADER_ID_SIZE
+        return self.segment, self.HEADER_ID_SIZE
+
+    def write_commit(self, intermediate=False):
+        # Intermediate commits go directly into the current segment - this makes checking their validity more
+        # expensive, but is faster and reduces clobber. Final commits go into a new segment.
+        fd = self.get_write_fd(want_new=not intermediate, no_new=intermediate)
+        if intermediate:
+            fd.sync()
+        header = self.header_no_crc_fmt.pack(self.header_fmt.size, TAG_COMMIT)
+        crc = self.crc_fmt.pack(crc32(header) & 0xFFFFFFFF)
+        fd.write(b"".join((crc, header)))
+        self.close_segment()
+        return self.segment - 1  # close_segment() increments it
+
+
+assert LoggedIO.HEADER_ID_SIZE + LoggedIO.ENTRY_HASH_SIZE == 41 + 8  # see constants.MAX_OBJECT_SIZE

+ 6 - 6
src/borg/manifest.py

@@ -242,8 +242,8 @@ class Manifest:
     def load(cls, repository, operations, key=None, *, ro_cls=RepoObj):
         from .item import ManifestItem
         from .crypto.key import key_factory
-        from .remote3 import RemoteRepository3
-        from .repository3 import Repository3
+        from .remote import RemoteRepository
+        from .repository import Repository
 
         cdata = repository.get_manifest()
         if not key:
@@ -256,7 +256,7 @@ class Manifest:
         if m.get("version") not in (1, 2):
             raise ValueError("Invalid manifest version")
 
-        if isinstance(repository, (Repository3, RemoteRepository3)):
+        if isinstance(repository, (Repository, RemoteRepository)):
             from .helpers import msgpack
 
             archives = {}
@@ -310,8 +310,8 @@ class Manifest:
 
     def write(self):
         from .item import ManifestItem
-        from .remote3 import RemoteRepository3
-        from .repository3 import Repository3
+        from .remote import RemoteRepository
+        from .repository import Repository
 
         # self.timestamp needs to be strictly monotonically increasing. Clocks often are not set correctly
         if self.timestamp is None:
@@ -327,7 +327,7 @@ class Manifest:
         assert len(self.item_keys) <= 100
         self.config["item_keys"] = tuple(sorted(self.item_keys))
 
-        if isinstance(self.repository, (Repository3, RemoteRepository3)):
+        if isinstance(self.repository, (Repository, RemoteRepository)):
             valid_keys = set()
             for name, info in self.archives.get_raw_dict().items():
                 archive = dict(name=name, id=info["id"], time=info["time"])

+ 361 - 10
src/borg/remote.py

@@ -1,8 +1,10 @@
+import atexit
 import errno
 import functools
 import inspect
 import logging
 import os
+import queue
 import select
 import shlex
 import shutil
@@ -12,8 +14,10 @@ import sys
 import tempfile
 import textwrap
 import time
+import traceback
 from subprocess import Popen, PIPE
 
+import borg.logger
 from . import __version__
 from .compress import Compressor
 from .constants import *  # NOQA
@@ -21,13 +25,16 @@ from .helpers import Error, ErrorWithTraceback, IntegrityError
 from .helpers import bin_to_hex
 from .helpers import get_limited_unpacker
 from .helpers import replace_placeholders
+from .helpers import sysinfo
 from .helpers import format_file_size
 from .helpers import safe_unlink
 from .helpers import prepare_subprocess_env, ignore_sigint
 from .helpers import get_socket_filename
-from .locking import LockTimeout, NotLocked, NotMyLock, LockFailed
-from .logger import create_logger
+from .fslocking import LockTimeout, NotLocked, NotMyLock, LockFailed
+from .logger import create_logger, borg_serve_log_queue
+from .manifest import NoManifestError
 from .helpers import msgpack
+from .legacyrepository import LegacyRepository
 from .repository import Repository
 from .version import parse_version, format_version
 from .checksums import xxh64
@@ -43,6 +50,25 @@ MAX_INFLIGHT = 100
 RATELIMIT_PERIOD = 0.1
 
 
+def os_write(fd, data):
+    """os.write wrapper so we do not lose data for partial writes."""
+    # TODO: this issue is fixed in cygwin since at least 2.8.0, remove this
+    #       wrapper / workaround when this version is considered ancient.
+    # This is happening frequently on cygwin due to its small pipe buffer size of only 64kiB
+    # and also due to its different blocking pipe behaviour compared to Linux/*BSD.
+    # Neither Linux nor *BSD ever do partial writes on blocking pipes, unless interrupted by a
+    # signal, in which case serve() would terminate.
+    amount = remaining = len(data)
+    while remaining:
+        count = os.write(fd, data)
+        remaining -= count
+        if not remaining:
+            break
+        data = data[count:]
+        time.sleep(count * 1e-09)
+    return amount
+
+
 class ConnectionClosed(Error):
     """Connection closed by remote host"""
 
@@ -101,7 +127,7 @@ class ConnectionBrokenWithHint(Error):
 # For the client the return of the negotiate method is a dict which includes the server version.
 #
 # All method calls on the remote repository object must be allowlisted in RepositoryServer.rpc_methods and have api
-# stubs in RemoteRepository. The @api decorator on these stubs is used to set server version requirements.
+# stubs in RemoteRepository*. The @api decorator on these stubs is used to set server version requirements.
 #
 # Method parameters are identified only by name and never by position. Unknown parameters are ignored by the server.
 # If a new parameter is important and may not be ignored, on the client a parameter specific version requirement needs
@@ -110,6 +136,317 @@ class ConnectionBrokenWithHint(Error):
 # servers still get compatible input.
 
 
+class RepositoryServer:  # pragma: no cover
+    _legacy_rpc_methods = (  # LegacyRepository
+        "__len__",
+        "check",
+        "commit",
+        "delete",
+        "destroy",
+        "get",
+        "list",
+        "negotiate",
+        "open",
+        "close",
+        "info",
+        "put",
+        "rollback",
+        "save_key",
+        "load_key",
+        "break_lock",
+        "inject_exception",
+    )
+
+    _rpc_methods = (  # Repository
+        "__len__",
+        "check",
+        "delete",
+        "destroy",
+        "get",
+        "list",
+        "negotiate",
+        "open",
+        "close",
+        "info",
+        "put",
+        "save_key",
+        "load_key",
+        "break_lock",
+        "inject_exception",
+        "get_manifest",
+        "put_manifest",
+        "store_list",
+        "store_load",
+        "store_store",
+        "store_delete",
+    )
+
+    def __init__(self, restrict_to_paths, restrict_to_repositories, append_only, storage_quota, use_socket):
+        self.repository = None
+        self.RepoCls = None
+        self.rpc_methods = ("open", "close", "negotiate")
+        self.restrict_to_paths = restrict_to_paths
+        self.restrict_to_repositories = restrict_to_repositories
+        # This flag is parsed from the serve command line via Archiver.do_serve,
+        # i.e. it reflects local system policy and generally ranks higher than
+        # whatever the client wants, except when initializing a new repository
+        # (see RepositoryServer.open below).
+        self.append_only = append_only
+        self.storage_quota = storage_quota
+        self.client_version = None  # we update this after client sends version information
+        if use_socket is False:
+            self.socket_path = None
+        elif use_socket is True:  # --socket
+            self.socket_path = get_socket_filename()
+        else:  # --socket=/some/path
+            self.socket_path = use_socket
+
+    def filter_args(self, f, kwargs):
+        """Remove unknown named parameters from call, because client did (implicitly) say it's ok."""
+        known = set(inspect.signature(f).parameters)
+        return {name: kwargs[name] for name in kwargs if name in known}
+
+    def send_queued_log(self):
+        while True:
+            try:
+                # lr_dict contents see BorgQueueHandler
+                lr_dict = borg_serve_log_queue.get_nowait()
+            except queue.Empty:
+                break
+            else:
+                msg = msgpack.packb({LOG: lr_dict})
+                os_write(self.stdout_fd, msg)
+
+    def serve(self):
+        def inner_serve():
+            os.set_blocking(self.stdin_fd, False)
+            assert not os.get_blocking(self.stdin_fd)
+            os.set_blocking(self.stdout_fd, True)
+            assert os.get_blocking(self.stdout_fd)
+
+            unpacker = get_limited_unpacker("server")
+            shutdown_serve = False
+            while True:
+                # before processing any new RPCs, send out all pending log output
+                self.send_queued_log()
+
+                if shutdown_serve:
+                    # shutdown wanted! get out of here after sending all log output.
+                    assert self.repository is None
+                    return
+
+                # process new RPCs
+                r, w, es = select.select([self.stdin_fd], [], [], 10)
+                if r:
+                    data = os.read(self.stdin_fd, BUFSIZE)
+                    if not data:
+                        shutdown_serve = True
+                        continue
+                    unpacker.feed(data)
+                    for unpacked in unpacker:
+                        if isinstance(unpacked, dict):
+                            msgid = unpacked[MSGID]
+                            method = unpacked[MSG]
+                            args = unpacked[ARGS]
+                        else:
+                            if self.repository is not None:
+                                self.repository.close()
+                            raise UnexpectedRPCDataFormatFromClient(__version__)
+                        try:
+                            # logger.debug(f"{type(self)} method: {type(self.repository)}.{method}")
+                            if method not in self.rpc_methods:
+                                raise InvalidRPCMethod(method)
+                            try:
+                                f = getattr(self, method)
+                            except AttributeError:
+                                f = getattr(self.repository, method)
+                            args = self.filter_args(f, args)
+                            res = f(**args)
+                        except BaseException as e:
+                            # logger.exception(e)
+                            ex_short = traceback.format_exception_only(e.__class__, e)
+                            ex_full = traceback.format_exception(*sys.exc_info())
+                            ex_trace = True
+                            if isinstance(e, Error):
+                                ex_short = [e.get_message()]
+                                ex_trace = e.traceback
+                            if isinstance(e, (self.RepoCls.DoesNotExist, self.RepoCls.AlreadyExists, PathNotAllowed)):
+                                # These exceptions are reconstructed on the client end in RemoteRepository*.call_many(),
+                                # and will be handled just like locally raised exceptions. Suppress the remote traceback
+                                # for these, except ErrorWithTraceback, which should always display a traceback.
+                                pass
+                            else:
+                                logging.debug("\n".join(ex_full))
+
+                            sys_info = sysinfo()
+                            try:
+                                msg = msgpack.packb(
+                                    {
+                                        MSGID: msgid,
+                                        "exception_class": e.__class__.__name__,
+                                        "exception_args": e.args,
+                                        "exception_full": ex_full,
+                                        "exception_short": ex_short,
+                                        "exception_trace": ex_trace,
+                                        "sysinfo": sys_info,
+                                    }
+                                )
+                            except TypeError:
+                                msg = msgpack.packb(
+                                    {
+                                        MSGID: msgid,
+                                        "exception_class": e.__class__.__name__,
+                                        "exception_args": [
+                                            x if isinstance(x, (str, bytes, int)) else None for x in e.args
+                                        ],
+                                        "exception_full": ex_full,
+                                        "exception_short": ex_short,
+                                        "exception_trace": ex_trace,
+                                        "sysinfo": sys_info,
+                                    }
+                                )
+                            os_write(self.stdout_fd, msg)
+                        else:
+                            os_write(self.stdout_fd, msgpack.packb({MSGID: msgid, RESULT: res}))
+                if es:
+                    shutdown_serve = True
+                    continue
+
+        if self.socket_path:  # server for socket:// connections
+            try:
+                # remove any left-over socket file
+                os.unlink(self.socket_path)
+            except OSError:
+                if os.path.exists(self.socket_path):
+                    raise
+            sock_dir = os.path.dirname(self.socket_path)
+            os.makedirs(sock_dir, exist_ok=True)
+            if self.socket_path.endswith(".sock"):
+                pid_file = self.socket_path.replace(".sock", ".pid")
+            else:
+                pid_file = self.socket_path + ".pid"
+            pid = os.getpid()
+            with open(pid_file, "w") as f:
+                f.write(str(pid))
+            atexit.register(functools.partial(os.remove, pid_file))
+            atexit.register(functools.partial(os.remove, self.socket_path))
+            sock = socket.socket(family=socket.AF_UNIX, type=socket.SOCK_STREAM)
+            sock.bind(self.socket_path)  # this creates the socket file in the fs
+            sock.listen(0)  # no backlog
+            os.chmod(self.socket_path, mode=0o0770)  # group members may use the socket, too.
+            print(f"borg serve: PID {pid}, listening on socket {self.socket_path} ...", file=sys.stderr)
+
+            while True:
+                connection, client_address = sock.accept()
+                print(f"Accepted a connection on socket {self.socket_path} ...", file=sys.stderr)
+                self.stdin_fd = connection.makefile("rb").fileno()
+                self.stdout_fd = connection.makefile("wb").fileno()
+                inner_serve()
+                print(f"Finished with connection on socket {self.socket_path} .", file=sys.stderr)
+        else:  # server for one ssh:// connection
+            self.stdin_fd = sys.stdin.fileno()
+            self.stdout_fd = sys.stdout.fileno()
+            inner_serve()
+
+    def negotiate(self, client_data):
+        if isinstance(client_data, dict):
+            self.client_version = client_data["client_version"]
+        else:
+            self.client_version = BORG_VERSION  # seems to be newer than current version (no known old format)
+
+        # not a known old format, send newest negotiate this version knows
+        return {"server_version": BORG_VERSION}
+
+    def _resolve_path(self, path):
+        if isinstance(path, bytes):
+            path = os.fsdecode(path)
+        if path.startswith("/~/"):  # /~/x = path x relative to own home dir
+            home_dir = os.environ.get("HOME") or os.path.expanduser("~%s" % os.environ.get("USER", ""))
+            path = os.path.join(home_dir, path[3:])
+        elif path.startswith("/./"):  # /./x = path x relative to cwd
+            path = path[3:]
+        return os.path.realpath(path)
+
+    def open(
+        self,
+        path,
+        create=False,
+        lock_wait=None,
+        lock=True,
+        exclusive=None,
+        append_only=False,
+        make_parent_dirs=False,
+        v1_or_v2=False,
+    ):
+        self.RepoCls = LegacyRepository if v1_or_v2 else Repository
+        self.rpc_methods = self._legacy_rpc_methods if v1_or_v2 else self._rpc_methods
+        logging.debug("Resolving repository path %r", path)
+        path = self._resolve_path(path)
+        logging.debug("Resolved repository path to %r", path)
+        path_with_sep = os.path.join(path, "")  # make sure there is a trailing slash (os.sep)
+        if self.restrict_to_paths:
+            # if --restrict-to-path P is given, we make sure that we only operate in/below path P.
+            # for the prefix check, it is important that the compared paths both have trailing slashes,
+            # so that a path /foobar will NOT be accepted with --restrict-to-path /foo option.
+            for restrict_to_path in self.restrict_to_paths:
+                restrict_to_path_with_sep = os.path.join(os.path.realpath(restrict_to_path), "")  # trailing slash
+                if path_with_sep.startswith(restrict_to_path_with_sep):
+                    break
+            else:
+                raise PathNotAllowed(path)
+        if self.restrict_to_repositories:
+            for restrict_to_repository in self.restrict_to_repositories:
+                restrict_to_repository_with_sep = os.path.join(os.path.realpath(restrict_to_repository), "")
+                if restrict_to_repository_with_sep == path_with_sep:
+                    break
+            else:
+                raise PathNotAllowed(path)
+        # "borg init" on "borg serve --append-only" (=self.append_only) does not create an append only repo,
+        # while "borg init --append-only" (=append_only) does, regardless of the --append-only (self.append_only)
+        # flag for serve.
+        append_only = (not create and self.append_only) or append_only
+        self.repository = self.RepoCls(
+            path,
+            create,
+            lock_wait=lock_wait,
+            lock=lock,
+            append_only=append_only,
+            storage_quota=self.storage_quota,
+            exclusive=exclusive,
+            make_parent_dirs=make_parent_dirs,
+            send_log_cb=self.send_queued_log,
+        )
+        self.repository.__enter__()  # clean exit handled by serve() method
+        return self.repository.id
+
+    def close(self):
+        if self.repository is not None:
+            self.repository.__exit__(None, None, None)
+            self.repository = None
+        borg.logger.flush_logging()
+        self.send_queued_log()
+
+    def inject_exception(self, kind):
+        s1 = "test string"
+        s2 = "test string2"
+        if kind == "DoesNotExist":
+            raise self.RepoCls.DoesNotExist(s1)
+        elif kind == "AlreadyExists":
+            raise self.RepoCls.AlreadyExists(s1)
+        elif kind == "CheckNeeded":
+            raise self.RepoCls.CheckNeeded(s1)
+        elif kind == "IntegrityError":
+            raise IntegrityError(s1)
+        elif kind == "PathNotAllowed":
+            raise PathNotAllowed("foo")
+        elif kind == "ObjectNotFound":
+            raise self.RepoCls.ObjectNotFound(s1, s2)
+        elif kind == "InvalidRPCMethod":
+            raise InvalidRPCMethod(s1)
+        elif kind == "divide":
+            0 // 0
+
+
 class SleepingBandwidthLimiter:
     def __init__(self, limit):
         if limit:
@@ -250,7 +587,7 @@ class RemoteRepository:
         location,
         create=False,
         exclusive=False,
-        lock_wait=None,
+        lock_wait=1.0,
         lock=True,
         append_only=False,
         make_parent_dirs=False,
@@ -339,7 +676,6 @@ class RemoteRepository:
                 exclusive=exclusive,
                 append_only=append_only,
                 make_parent_dirs=make_parent_dirs,
-                v1_or_v2=True,  # make remote use Repository, not Repository3
             )
             info = self.info()
             self.version = info["version"]
@@ -354,7 +690,7 @@ class RemoteRepository:
             logging.debug("still %d cached responses left in RemoteRepository" % (len(self.responses),))
         if self.p or self.sock:
             self.close()
-            assert False, "cleanup happened in Repository.__del__"
+            assert False, "cleanup happened in RemoteRepository.__del__"
 
     def __repr__(self):
         return f"<{self.__class__.__name__} {self.location.canonical_path()}>"
@@ -366,11 +702,8 @@ class RemoteRepository:
         try:
             if exc_type is not None:
                 self.shutdown_time = time.monotonic() + 30
-                self.rollback()
         finally:
-            # in any case, we want to close the repo cleanly, even if the
-            # rollback can not succeed (e.g. because the connection was
-            # already closed) and raised another exception:
+            # in any case, we want to close the repo cleanly.
             logger.debug(
                 "RemoteRepository: %s bytes sent, %s bytes received, %d messages sent",
                 format_file_size(self.tx_bytes),
@@ -505,6 +838,8 @@ class RemoteRepository:
                 raise NotLocked(args[0])
             elif error == "NotMyLock":
                 raise NotMyLock(args[0])
+            elif error == "NoManifestError":
+                raise NoManifestError
             else:
                 raise self.RPCError(unpacked)
 
@@ -745,6 +1080,22 @@ class RemoteRepository:
     def put_manifest(self, data):
         """actual remoting is done via self.call in the @api decorator"""
 
+    @api(since=parse_version("2.0.0b8"))
+    def store_list(self, name):
+        """actual remoting is done via self.call in the @api decorator"""
+
+    @api(since=parse_version("2.0.0b8"))
+    def store_load(self, name):
+        """actual remoting is done via self.call in the @api decorator"""
+
+    @api(since=parse_version("2.0.0b8"))
+    def store_store(self, name, value):
+        """actual remoting is done via self.call in the @api decorator"""
+
+    @api(since=parse_version("2.0.0b8"))
+    def store_delete(self, name):
+        """actual remoting is done via self.call in the @api decorator"""
+
 
 class RepositoryNoCache:
     """A not caching Repository wrapper, passes through to repository.

Різницю між файлами не показано, бо вона завелика
+ 188 - 1072
src/borg/repository.py


+ 0 - 418
src/borg/repository3.py

@@ -1,418 +0,0 @@
-import os
-
-from borgstore.store import Store
-from borgstore.store import ObjectNotFound as StoreObjectNotFound
-
-from .checksums import xxh64
-from .constants import *  # NOQA
-from .helpers import Error, ErrorWithTraceback, IntegrityError
-from .helpers import Location
-from .helpers import bin_to_hex, hex_to_bin
-from .locking3 import Lock
-from .logger import create_logger
-from .manifest import NoManifestError
-from .repoobj import RepoObj
-
-logger = create_logger(__name__)
-
-
-class Repository3:
-    """borgstore based key value store"""
-
-    class AlreadyExists(Error):
-        """A repository already exists at {}."""
-
-        exit_mcode = 10
-
-    class CheckNeeded(ErrorWithTraceback):
-        """Inconsistency detected. Please run "borg check {}"."""
-
-        exit_mcode = 12
-
-    class DoesNotExist(Error):
-        """Repository {} does not exist."""
-
-        exit_mcode = 13
-
-    class InsufficientFreeSpaceError(Error):
-        """Insufficient free space to complete transaction (required: {}, available: {})."""
-
-        exit_mcode = 14
-
-    class InvalidRepository(Error):
-        """{} is not a valid repository. Check repo config."""
-
-        exit_mcode = 15
-
-    class InvalidRepositoryConfig(Error):
-        """{} does not have a valid configuration. Check repo config [{}]."""
-
-        exit_mcode = 16
-
-    class ObjectNotFound(ErrorWithTraceback):
-        """Object with key {} not found in repository {}."""
-
-        exit_mcode = 17
-
-        def __init__(self, id, repo):
-            if isinstance(id, bytes):
-                id = bin_to_hex(id)
-            super().__init__(id, repo)
-
-    class ParentPathDoesNotExist(Error):
-        """The parent path of the repo directory [{}] does not exist."""
-
-        exit_mcode = 18
-
-    class PathAlreadyExists(Error):
-        """There is already something at {}."""
-
-        exit_mcode = 19
-
-    class StorageQuotaExceeded(Error):
-        """The storage quota ({}) has been exceeded ({}). Try deleting some archives."""
-
-        exit_mcode = 20
-
-    class PathPermissionDenied(Error):
-        """Permission denied to {}."""
-
-        exit_mcode = 21
-
-    def __init__(
-        self,
-        path,
-        create=False,
-        exclusive=False,
-        lock_wait=1.0,
-        lock=True,
-        append_only=False,
-        storage_quota=None,
-        make_parent_dirs=False,
-        send_log_cb=None,
-    ):
-        self.path = os.path.abspath(path)
-        url = "file://%s" % self.path
-        # use a Store with flat config storage and 2-levels-nested data storage
-        self.store = Store(url, levels={"config/": [0], "data/": [2]})
-        self._location = Location(url)
-        self.version = None
-        # long-running repository methods which emit log or progress output are responsible for calling
-        # the ._send_log method periodically to get log and progress output transferred to the borg client
-        # in a timely manner, in case we have a RemoteRepository.
-        # for local repositories ._send_log can be called also (it will just do nothing in that case).
-        self._send_log = send_log_cb or (lambda: None)
-        self.do_create = create
-        self.created = False
-        self.acceptable_repo_versions = (3,)
-        self.opened = False
-        self.append_only = append_only  # XXX not implemented / not implementable
-        self.storage_quota = storage_quota  # XXX not implemented
-        self.storage_quota_use = 0  # XXX not implemented
-        self.lock = None
-        self.do_lock = lock
-        self.lock_wait = lock_wait
-        self.exclusive = exclusive
-
-    def __repr__(self):
-        return f"<{self.__class__.__name__} {self.path}>"
-
-    def __enter__(self):
-        if self.do_create:
-            self.do_create = False
-            self.create()
-            self.created = True
-        self.open(exclusive=bool(self.exclusive), lock_wait=self.lock_wait, lock=self.do_lock)
-        return self
-
-    def __exit__(self, exc_type, exc_val, exc_tb):
-        self.close()
-
-    @property
-    def id_str(self):
-        return bin_to_hex(self.id)
-
-    def create(self):
-        """Create a new empty repository"""
-        self.store.create()
-        self.store.open()
-        self.store.store("config/readme", REPOSITORY_README.encode())
-        self.version = 3
-        self.store.store("config/version", str(self.version).encode())
-        self.store.store("config/id", bin_to_hex(os.urandom(32)).encode())
-        self.store.close()
-
-    def _set_id(self, id):
-        # for testing: change the id of an existing repository
-        assert self.opened
-        assert isinstance(id, bytes) and len(id) == 32
-        self.id = id
-        self.store.store("config/id", bin_to_hex(id).encode())
-
-    def _lock_refresh(self):
-        if self.lock is not None:
-            self.lock.refresh()
-
-    def save_key(self, keydata):
-        # note: saving an empty key means that there is no repokey anymore
-        self.store.store("keys/repokey", keydata)
-
-    def load_key(self):
-        keydata = self.store.load("keys/repokey")
-        # note: if we return an empty string, it means there is no repo key
-        return keydata
-
-    def destroy(self):
-        """Destroy the repository"""
-        self.close()
-        self.store.destroy()
-
-    def open(self, *, exclusive, lock_wait=None, lock=True):
-        assert lock_wait is not None
-        self.store.open()
-        if lock:
-            self.lock = Lock(self.store, exclusive, timeout=lock_wait).acquire()
-        else:
-            self.lock = None
-        readme = self.store.load("config/readme").decode()
-        if readme != REPOSITORY_README:
-            raise self.InvalidRepository(self.path)
-        self.version = int(self.store.load("config/version").decode())
-        if self.version not in self.acceptable_repo_versions:
-            self.close()
-            raise self.InvalidRepositoryConfig(
-                self.path, "repository version %d is not supported by this borg version" % self.version
-            )
-        self.id = hex_to_bin(self.store.load("config/id").decode(), length=32)
-        self.opened = True
-
-    def close(self):
-        if self.opened:
-            if self.lock:
-                self.lock.release()
-                self.lock = None
-            self.store.close()
-            self.opened = False
-
-    def info(self):
-        """return some infos about the repo (must be opened first)"""
-        info = dict(
-            id=self.id,
-            version=self.version,
-            storage_quota_use=self.storage_quota_use,
-            storage_quota=self.storage_quota,
-            append_only=self.append_only,
-        )
-        return info
-
-    def check(self, repair=False, max_duration=0):
-        """Check repository consistency"""
-
-        def log_error(msg):
-            nonlocal obj_corrupted
-            obj_corrupted = True
-            logger.error(f"Repo object {info.name} is corrupted: {msg}")
-
-        def check_object(obj):
-            """Check if obj looks valid."""
-            hdr_size = RepoObj.obj_header.size
-            obj_size = len(obj)
-            if obj_size >= hdr_size:
-                hdr = RepoObj.ObjHeader(*RepoObj.obj_header.unpack(obj[:hdr_size]))
-                meta = obj[hdr_size : hdr_size + hdr.meta_size]
-                if hdr.meta_size != len(meta):
-                    log_error("metadata size incorrect.")
-                elif hdr.meta_hash != xxh64(meta):
-                    log_error("metadata does not match checksum.")
-                data = obj[hdr_size + hdr.meta_size : hdr_size + hdr.meta_size + hdr.data_size]
-                if hdr.data_size != len(data):
-                    log_error("data size incorrect.")
-                elif hdr.data_hash != xxh64(data):
-                    log_error("data does not match checksum.")
-            else:
-                log_error("too small.")
-
-        # TODO: progress indicator, partial checks, ...
-        mode = "full"
-        logger.info("Starting repository check")
-        objs_checked = objs_errors = 0
-        infos = self.store.list("data")
-        try:
-            for info in infos:
-                self._lock_refresh()
-                key = "data/%s" % info.name
-                try:
-                    obj = self.store.load(key)
-                except StoreObjectNotFound:
-                    # looks like object vanished since store.list(), ignore that.
-                    continue
-                obj_corrupted = False
-                check_object(obj)
-                objs_checked += 1
-                if obj_corrupted:
-                    objs_errors += 1
-                    if repair:
-                        # if it is corrupted, we can't do much except getting rid of it.
-                        # but let's just retry loading it, in case the error goes away.
-                        try:
-                            obj = self.store.load(key)
-                        except StoreObjectNotFound:
-                            log_error("existing object vanished.")
-                        else:
-                            obj_corrupted = False
-                            check_object(obj)
-                            if obj_corrupted:
-                                log_error("reloading did not help, deleting it!")
-                                self.store.delete(key)
-                            else:
-                                log_error("reloading did help, inconsistent behaviour detected!")
-        except StoreObjectNotFound:
-            # it can be that there is no "data/" at all, then it crashes when iterating infos.
-            pass
-        logger.info(f"Checked {objs_checked} repository objects, {objs_errors} errors.")
-        if objs_errors == 0:
-            logger.info(f"Finished {mode} repository check, no problems found.")
-        else:
-            if repair:
-                logger.info(f"Finished {mode} repository check, errors found and repaired.")
-            else:
-                logger.error(f"Finished {mode} repository check, errors found.")
-        return objs_errors == 0 or repair
-
-    def __len__(self):
-        raise NotImplementedError
-
-    def __contains__(self, id):
-        raise NotImplementedError
-
-    def list(self, limit=None, marker=None):
-        """
-        list <limit> IDs starting from after id <marker>.
-        """
-        self._lock_refresh()
-        collect = True if marker is None else False
-        ids = []
-        infos = self.store.list("data")  # generator yielding ItemInfos
-        while True:
-            try:
-                info = next(infos)
-            except StoreObjectNotFound:
-                break  # can happen e.g. if "data" does not exist, pointless to continue in that case
-            except StopIteration:
-                break
-            else:
-                id = hex_to_bin(info.name)
-                if collect:
-                    ids.append(id)
-                    if len(ids) == limit:
-                        break
-                elif id == marker:
-                    collect = True
-                    # note: do not collect the marker id
-        return ids
-
-    def get(self, id, read_data=True):
-        self._lock_refresh()
-        id_hex = bin_to_hex(id)
-        key = "data/" + id_hex
-        try:
-            if read_data:
-                # read everything
-                return self.store.load(key)
-            else:
-                # RepoObj layout supports separately encrypted metadata and data.
-                # We return enough bytes so the client can decrypt the metadata.
-                hdr_size = RepoObj.obj_header.size
-                extra_size = 1024 - hdr_size  # load a bit more, 1024b, reduces round trips
-                obj = self.store.load(key, size=hdr_size + extra_size)
-                hdr = obj[0:hdr_size]
-                if len(hdr) != hdr_size:
-                    raise IntegrityError(f"Object too small [id {id_hex}]: expected {hdr_size}, got {len(hdr)} bytes")
-                meta_size = RepoObj.obj_header.unpack(hdr)[0]
-                if meta_size > extra_size:
-                    # we did not get enough, need to load more, but not all.
-                    # this should be rare, as chunk metadata is rather small usually.
-                    obj = self.store.load(key, size=hdr_size + meta_size)
-                meta = obj[hdr_size : hdr_size + meta_size]
-                if len(meta) != meta_size:
-                    raise IntegrityError(f"Object too small [id {id_hex}]: expected {meta_size}, got {len(meta)} bytes")
-                return hdr + meta
-        except StoreObjectNotFound:
-            raise self.ObjectNotFound(id, self.path) from None
-
-    def get_many(self, ids, read_data=True, is_preloaded=False):
-        for id_ in ids:
-            yield self.get(id_, read_data=read_data)
-
-    def put(self, id, data, wait=True):
-        """put a repo object
-
-        Note: when doing calls with wait=False this gets async and caller must
-              deal with async results / exceptions later.
-        """
-        self._lock_refresh()
-        data_size = len(data)
-        if data_size > MAX_DATA_SIZE:
-            raise IntegrityError(f"More than allowed put data [{data_size} > {MAX_DATA_SIZE}]")
-
-        key = "data/" + bin_to_hex(id)
-        self.store.store(key, data)
-
-    def delete(self, id, wait=True):
-        """delete a repo object
-
-        Note: when doing calls with wait=False this gets async and caller must
-              deal with async results / exceptions later.
-        """
-        self._lock_refresh()
-        key = "data/" + bin_to_hex(id)
-        try:
-            self.store.delete(key)
-        except StoreObjectNotFound:
-            raise self.ObjectNotFound(id, self.path) from None
-
-    def async_response(self, wait=True):
-        """Get one async result (only applies to remote repositories).
-
-        async commands (== calls with wait=False, e.g. delete and put) have no results,
-        but may raise exceptions. These async exceptions must get collected later via
-        async_response() calls. Repeat the call until it returns None.
-        The previous calls might either return one (non-None) result or raise an exception.
-        If wait=True is given and there are outstanding responses, it will wait for them
-        to arrive. With wait=False, it will only return already received responses.
-        """
-
-    def preload(self, ids):
-        """Preload objects (only applies to remote repositories)"""
-
-    def break_lock(self):
-        Lock(self.store).break_lock()
-
-    def migrate_lock(self, old_id, new_id):
-        # note: only needed for local repos
-        if self.lock is not None:
-            self.lock.migrate_lock(old_id, new_id)
-
-    def get_manifest(self):
-        try:
-            return self.store.load("config/manifest")
-        except StoreObjectNotFound:
-            raise NoManifestError
-
-    def put_manifest(self, data):
-        return self.store.store("config/manifest", data)
-
-    def store_list(self, name):
-        try:
-            return list(self.store.list(name))
-        except StoreObjectNotFound:
-            return []
-
-    def store_load(self, name):
-        return self.store.load(name)
-
-    def store_store(self, name, value):
-        return self.store.store(name, value)
-
-    def store_delete(self, name):
-        return self.store.delete(name)

+ 0 - 0
src/borg/locking3.py → src/borg/storelocking.py


+ 7 - 7
src/borg/testsuite/archiver/__init__.py

@@ -24,8 +24,8 @@ from ...helpers import init_ec_warnings
 from ...logger import flush_logging
 from ...manifest import Manifest
 from ...platform import get_flags
-from ...remote3 import RemoteRepository3
-from ...repository3 import Repository3
+from ...remote import RemoteRepository
+from ...repository import Repository
 from .. import has_lchflags, is_utime_fully_supported, have_fuse_mtime_ns, st_mtime_ns_round, no_selinux
 from .. import changedir
 from .. import are_symlinks_supported, are_hardlinks_supported, are_fifos_supported
@@ -166,7 +166,7 @@ def create_src_archive(archiver, name, ts=None):
 
 
 def open_archive(repo_path, name):
-    repository = Repository3(repo_path, exclusive=True)
+    repository = Repository(repo_path, exclusive=True)
     with repository:
         manifest = Manifest.load(repository, Manifest.NO_OPERATION_CHECK)
         archive = Archive(manifest, name)
@@ -175,9 +175,9 @@ def open_archive(repo_path, name):
 
 def open_repository(archiver):
     if archiver.get_kind() == "remote":
-        return RemoteRepository3(Location(archiver.repository_location))
+        return RemoteRepository(Location(archiver.repository_location))
     else:
-        return Repository3(archiver.repository_path, exclusive=True)
+        return Repository(archiver.repository_path, exclusive=True)
 
 
 def create_regular_file(input_path, name, size=0, contents=None):
@@ -253,12 +253,12 @@ def create_test_files(input_path, create_hardlinks=True):
 
 
 def _extract_repository_id(repo_path):
-    with Repository3(repo_path) as repository:
+    with Repository(repo_path) as repository:
         return repository.id
 
 
 def _set_repository_id(repo_path, id):
-    with Repository3(repo_path) as repository:
+    with Repository(repo_path) as repository:
         repository._set_id(id)
         return repository.id
 

+ 8 - 8
src/borg/testsuite/archiver/check_cmd.py

@@ -8,9 +8,9 @@ from ...archive import ChunkBuffer
 from ...constants import *  # NOQA
 from ...helpers import bin_to_hex, msgpack
 from ...manifest import Manifest
-from ...remote3 import RemoteRepository3
-from ...repository3 import Repository3
-from ..repository3 import fchunk
+from ...remote import RemoteRepository
+from ...repository import Repository
+from ..repository import fchunk
 from . import cmd, src_file, create_src_archive, open_archive, generate_archiver_tests, RK_ENCRYPTION
 
 pytest_generate_tests = lambda metafunc: generate_archiver_tests(metafunc, kinds="local,remote,binary")  # NOQA
@@ -191,7 +191,7 @@ def test_missing_manifest(archivers, request):
     check_cmd_setup(archiver)
     archive, repository = open_archive(archiver.repository_path, "archive1")
     with repository:
-        if isinstance(repository, (Repository3, RemoteRepository3)):
+        if isinstance(repository, (Repository, RemoteRepository)):
             repository.store_delete("config/manifest")
         else:
             repository.delete(Manifest.MANIFEST_ID)
@@ -342,7 +342,7 @@ def test_extra_chunks(archivers, request):
         pytest.skip("only works locally")
     check_cmd_setup(archiver)
     cmd(archiver, "check", exit_code=0)
-    with Repository3(archiver.repository_location, exclusive=True) as repository:
+    with Repository(archiver.repository_location, exclusive=True) as repository:
         chunk = fchunk(b"xxxx")
         repository.put(b"01234567890123456789012345678901", chunk)
     cmd(archiver, "check", "-v", exit_code=0)  # check does not deal with orphans anymore
@@ -362,9 +362,9 @@ def test_verify_data(archivers, request, init_args):
         return b"fakefake"
 
     import borg.repoobj
-    import borg.repository3
+    import borg.repository
 
-    with patch.object(borg.repoobj, "xxh64", fake_xxh64), patch.object(borg.repository3, "xxh64", fake_xxh64):
+    with patch.object(borg.repoobj, "xxh64", fake_xxh64), patch.object(borg.repository, "xxh64", fake_xxh64):
         check_cmd_setup(archiver)
         shutil.rmtree(archiver.repository_path)
         cmd(archiver, "rcreate", *init_args)
@@ -431,7 +431,7 @@ def test_empty_repository(archivers, request):
     if archiver.get_kind() == "remote":
         pytest.skip("only works locally")
     check_cmd_setup(archiver)
-    with Repository3(archiver.repository_location, exclusive=True) as repository:
+    with Repository(archiver.repository_location, exclusive=True) as repository:
         for id_ in repository.list():
             repository.delete(id_)
     cmd(archiver, "check", exit_code=1)

+ 12 - 12
src/borg/testsuite/archiver/checks.py

@@ -9,8 +9,8 @@ from ...constants import *  # NOQA
 from ...helpers import Location, get_security_dir, bin_to_hex
 from ...helpers import EXIT_ERROR
 from ...manifest import Manifest, MandatoryFeatureUnsupported
-from ...remote3 import RemoteRepository3, PathNotAllowed
-from ...repository3 import Repository3
+from ...remote import RemoteRepository, PathNotAllowed
+from ...repository import Repository
 from .. import llfuse
 from .. import changedir
 from . import cmd, _extract_repository_id, create_test_files
@@ -25,7 +25,7 @@ def get_security_directory(repo_path):
 
 
 def add_unknown_feature(repo_path, operation):
-    with Repository3(repo_path, exclusive=True) as repository:
+    with Repository(repo_path, exclusive=True) as repository:
         manifest = Manifest.load(repository, Manifest.NO_OPERATION_CHECK)
         manifest.config["feature_flags"] = {operation.value: {"mandatory": ["unknown-feature"]}}
         manifest.write()
@@ -259,7 +259,7 @@ def test_unknown_mandatory_feature_in_cache(archivers, request):
     remote_repo = archiver.get_kind() == "remote"
     print(cmd(archiver, "rcreate", RK_ENCRYPTION))
 
-    with Repository3(archiver.repository_path, exclusive=True) as repository:
+    with Repository(archiver.repository_path, exclusive=True) as repository:
         if remote_repo:
             repository._location = Location(archiver.repository_location)
         manifest = Manifest.load(repository, Manifest.NO_OPERATION_CHECK)
@@ -271,7 +271,7 @@ def test_unknown_mandatory_feature_in_cache(archivers, request):
     if archiver.FORK_DEFAULT:
         cmd(archiver, "create", "test", "input")
 
-    with Repository3(archiver.repository_path, exclusive=True) as repository:
+    with Repository(archiver.repository_path, exclusive=True) as repository:
         if remote_repo:
             repository._location = Location(archiver.repository_location)
         manifest = Manifest.load(repository, Manifest.NO_OPERATION_CHECK)
@@ -283,26 +283,26 @@ def test_unknown_mandatory_feature_in_cache(archivers, request):
 def test_remote_repo_restrict_to_path(remote_archiver):
     original_location, repo_path = remote_archiver.repository_location, remote_archiver.repository_path
     # restricted to repo directory itself:
-    with patch.object(RemoteRepository3, "extra_test_args", ["--restrict-to-path", repo_path]):
+    with patch.object(RemoteRepository, "extra_test_args", ["--restrict-to-path", repo_path]):
         cmd(remote_archiver, "rcreate", RK_ENCRYPTION)
     # restricted to repo directory itself, fail for other directories with same prefix:
-    with patch.object(RemoteRepository3, "extra_test_args", ["--restrict-to-path", repo_path]):
+    with patch.object(RemoteRepository, "extra_test_args", ["--restrict-to-path", repo_path]):
         with pytest.raises(PathNotAllowed):
             remote_archiver.repository_location = original_location + "_0"
             cmd(remote_archiver, "rcreate", RK_ENCRYPTION)
     # restricted to a completely different path:
-    with patch.object(RemoteRepository3, "extra_test_args", ["--restrict-to-path", "/foo"]):
+    with patch.object(RemoteRepository, "extra_test_args", ["--restrict-to-path", "/foo"]):
         with pytest.raises(PathNotAllowed):
             remote_archiver.repository_location = original_location + "_1"
             cmd(remote_archiver, "rcreate", RK_ENCRYPTION)
     path_prefix = os.path.dirname(repo_path)
     # restrict to repo directory's parent directory:
-    with patch.object(RemoteRepository3, "extra_test_args", ["--restrict-to-path", path_prefix]):
+    with patch.object(RemoteRepository, "extra_test_args", ["--restrict-to-path", path_prefix]):
         remote_archiver.repository_location = original_location + "_2"
         cmd(remote_archiver, "rcreate", RK_ENCRYPTION)
     # restrict to repo directory's parent directory and another directory:
     with patch.object(
-        RemoteRepository3, "extra_test_args", ["--restrict-to-path", "/foo", "--restrict-to-path", path_prefix]
+        RemoteRepository, "extra_test_args", ["--restrict-to-path", "/foo", "--restrict-to-path", path_prefix]
     ):
         remote_archiver.repository_location = original_location + "_3"
         cmd(remote_archiver, "rcreate", RK_ENCRYPTION)
@@ -311,10 +311,10 @@ def test_remote_repo_restrict_to_path(remote_archiver):
 def test_remote_repo_restrict_to_repository(remote_archiver):
     repo_path = remote_archiver.repository_path
     # restricted to repo directory itself:
-    with patch.object(RemoteRepository3, "extra_test_args", ["--restrict-to-repository", repo_path]):
+    with patch.object(RemoteRepository, "extra_test_args", ["--restrict-to-repository", repo_path]):
         cmd(remote_archiver, "rcreate", RK_ENCRYPTION)
     parent_path = os.path.join(repo_path, "..")
-    with patch.object(RemoteRepository3, "extra_test_args", ["--restrict-to-repository", parent_path]):
+    with patch.object(RemoteRepository, "extra_test_args", ["--restrict-to-repository", parent_path]):
         with pytest.raises(PathNotAllowed):
             cmd(remote_archiver, "rcreate", RK_ENCRYPTION)
 

+ 2 - 2
src/borg/testsuite/archiver/create_cmd.py

@@ -16,7 +16,7 @@ from ...cache import get_cache_impl
 from ...constants import *  # NOQA
 from ...manifest import Manifest
 from ...platform import is_cygwin, is_win32, is_darwin
-from ...repository3 import Repository3
+from ...repository import Repository
 from ...helpers import CommandError, BackupPermissionError
 from .. import has_lchflags
 from .. import changedir
@@ -644,7 +644,7 @@ def test_create_dry_run(archivers, request):
     cmd(archiver, "rcreate", RK_ENCRYPTION)
     cmd(archiver, "create", "--dry-run", "test", "input")
     # Make sure no archive has been created
-    with Repository3(archiver.repository_path) as repository:
+    with Repository(archiver.repository_path) as repository:
         manifest = Manifest.load(repository, Manifest.NO_OPERATION_CHECK)
     assert len(manifest.archives) == 0
 

+ 7 - 7
src/borg/testsuite/archiver/key_cmds.py

@@ -9,7 +9,7 @@ from ...crypto.keymanager import RepoIdMismatch, NotABorgKeyFile
 from ...helpers import CommandError
 from ...helpers import bin_to_hex, hex_to_bin
 from ...helpers import msgpack
-from ...repository3 import Repository3
+from ...repository import Repository
 from .. import key
 from . import RK_ENCRYPTION, KF_ENCRYPTION, cmd, _extract_repository_id, _set_repository_id, generate_archiver_tests
 
@@ -129,7 +129,7 @@ def test_key_export_repokey(archivers, request):
 
     assert export_contents.startswith("BORG_KEY " + bin_to_hex(repo_id) + "\n")
 
-    with Repository3(archiver.repository_path) as repository:
+    with Repository(archiver.repository_path) as repository:
         repo_key = AESOCBRepoKey(repository)
         repo_key.load(None, Passphrase.env_passphrase())
 
@@ -138,12 +138,12 @@ def test_key_export_repokey(archivers, request):
 
     assert repo_key.crypt_key == backup_key.crypt_key
 
-    with Repository3(archiver.repository_path) as repository:
+    with Repository(archiver.repository_path) as repository:
         repository.save_key(b"")
 
     cmd(archiver, "key", "import", export_file)
 
-    with Repository3(archiver.repository_path) as repository:
+    with Repository(archiver.repository_path) as repository:
         repo_key2 = AESOCBRepoKey(repository)
         repo_key2.load(None, Passphrase.env_passphrase())
 
@@ -302,7 +302,7 @@ def test_init_defaults_to_argon2(archivers, request):
     """https://github.com/borgbackup/borg/issues/747#issuecomment-1076160401"""
     archiver = request.getfixturevalue(archivers)
     cmd(archiver, "rcreate", RK_ENCRYPTION)
-    with Repository3(archiver.repository_path) as repository:
+    with Repository(archiver.repository_path) as repository:
         key = msgpack.unpackb(binascii.a2b_base64(repository.load_key()))
         assert key["algorithm"] == "argon2 chacha20-poly1305"
 
@@ -313,7 +313,7 @@ def test_change_passphrase_does_not_change_algorithm_argon2(archivers, request):
     os.environ["BORG_NEW_PASSPHRASE"] = "newpassphrase"
     cmd(archiver, "key", "change-passphrase")
 
-    with Repository3(archiver.repository_path) as repository:
+    with Repository(archiver.repository_path) as repository:
         key = msgpack.unpackb(binascii.a2b_base64(repository.load_key()))
         assert key["algorithm"] == "argon2 chacha20-poly1305"
 
@@ -323,6 +323,6 @@ def test_change_location_does_not_change_algorithm_argon2(archivers, request):
     cmd(archiver, "rcreate", KF_ENCRYPTION)
     cmd(archiver, "key", "change-location", "repokey")
 
-    with Repository3(archiver.repository_path) as repository:
+    with Repository(archiver.repository_path) as repository:
         key = msgpack.unpackb(binascii.a2b_base64(repository.load_key()))
         assert key["algorithm"] == "argon2 chacha20-poly1305"

+ 1 - 1
src/borg/testsuite/archiver/mount_cmds.py

@@ -7,7 +7,7 @@ import pytest
 
 from ... import xattr, platform
 from ...constants import *  # NOQA
-from ...locking3 import Lock
+from ...storelocking import Lock
 from ...helpers import flags_noatime, flags_normal
 from .. import has_lchflags, llfuse
 from .. import changedir, no_selinux, same_ts_ns

+ 2 - 2
src/borg/testsuite/archiver/rcompress_cmd.py

@@ -1,7 +1,7 @@
 import os
 
 from ...constants import *  # NOQA
-from ...repository3 import Repository3
+from ...repository import Repository
 from ...manifest import Manifest
 from ...compress import ZSTD, ZLIB, LZ4, CNONE
 from ...helpers import bin_to_hex
@@ -12,7 +12,7 @@ from . import create_regular_file, cmd, RK_ENCRYPTION
 def test_rcompress(archiver):
     def check_compression(ctype, clevel, olevel):
         """check if all the chunks in the repo are compressed/obfuscated like expected"""
-        repository = Repository3(archiver.repository_path, exclusive=True)
+        repository = Repository(archiver.repository_path, exclusive=True)
         with repository:
             manifest = Manifest.load(repository, Manifest.NO_OPERATION_CHECK)
             marker = None

+ 2 - 2
src/borg/testsuite/archiver/rename_cmd.py

@@ -1,6 +1,6 @@
 from ...constants import *  # NOQA
 from ...manifest import Manifest
-from ...repository3 import Repository3
+from ...repository import Repository
 from . import cmd, create_regular_file, generate_archiver_tests, RK_ENCRYPTION
 
 pytest_generate_tests = lambda metafunc: generate_archiver_tests(metafunc, kinds="local,remote,binary")  # NOQA
@@ -21,7 +21,7 @@ def test_rename(archivers, request):
     cmd(archiver, "extract", "test.3", "--dry-run")
     cmd(archiver, "extract", "test.4", "--dry-run")
     # Make sure both archives have been renamed
-    with Repository3(archiver.repository_path) as repository:
+    with Repository(archiver.repository_path) as repository:
         manifest = Manifest.load(repository, Manifest.NO_OPERATION_CHECK)
     assert len(manifest.archives) == 2
     assert "test.3" in manifest.archives

+ 3 - 3
src/borg/testsuite/cache.py

@@ -9,14 +9,14 @@ from ..cache import AdHocCache
 from ..crypto.key import AESOCBRepoKey
 from ..hashindex import ChunkIndex
 from ..manifest import Manifest
-from ..repository3 import Repository3
+from ..repository import Repository
 
 
 class TestAdHocCache:
     @pytest.fixture
     def repository(self, tmpdir):
         self.repository_location = os.path.join(str(tmpdir), "repository")
-        with Repository3(self.repository_location, exclusive=True, create=True) as repository:
+        with Repository(self.repository_location, exclusive=True, create=True) as repository:
             repository.put(H(1), b"1234")
             yield repository
 
@@ -51,7 +51,7 @@ class TestAdHocCache:
         assert cache.seen_chunk(H(5)) == 1
         cache.chunk_decref(H(5), 1, Statistics())
         assert not cache.seen_chunk(H(5))
-        with pytest.raises(Repository3.ObjectNotFound):
+        with pytest.raises(Repository.ObjectNotFound):
             repository.get(H(5))
 
     def test_files_cache(self, cache):

+ 1 - 1
src/borg/testsuite/locking.py → src/borg/testsuite/fslocking.py

@@ -6,7 +6,7 @@ from traceback import format_exc
 import pytest
 
 from ..platform import get_process_id, process_alive
-from ..locking import (
+from ..fslocking import (
     TimeoutTimer,
     ExclusiveLock,
     Lock,

+ 1115 - 0
src/borg/testsuite/legacyrepository.py

@@ -0,0 +1,1115 @@
+import logging
+import os
+import sys
+from typing import Optional
+from unittest.mock import patch
+
+import pytest
+
+from ..checksums import xxh64
+from ..hashindex import NSIndex
+from ..helpers import Location
+from ..helpers import IntegrityError
+from ..helpers import msgpack
+from ..fslocking import Lock, LockFailed
+from ..platformflags import is_win32
+from ..legacyremote import LegacyRemoteRepository, InvalidRPCMethod, PathNotAllowed
+from ..legacyrepository import LegacyRepository, LoggedIO
+from ..legacyrepository import MAGIC, MAX_DATA_SIZE, TAG_DELETE, TAG_PUT2, TAG_PUT, TAG_COMMIT
+from ..repoobj import RepoObj
+from .hashindex import H
+
+
+@pytest.fixture()
+def repository(tmp_path):
+    repository_location = os.fspath(tmp_path / "repository")
+    yield LegacyRepository(repository_location, exclusive=True, create=True)
+
+
+@pytest.fixture()
+def remote_repository(tmp_path):
+    if is_win32:
+        pytest.skip("Remote repository does not yet work on Windows.")
+    repository_location = Location("ssh://__testsuite__" + os.fspath(tmp_path / "repository"))
+    yield LegacyRemoteRepository(repository_location, exclusive=True, create=True)
+
+
+def pytest_generate_tests(metafunc):
+    # Generates tests that run on both local and remote repos
+    if "repo_fixtures" in metafunc.fixturenames:
+        metafunc.parametrize("repo_fixtures", ["repository", "remote_repository"])
+
+
+def get_repository_from_fixture(repo_fixtures, request):
+    # returns the repo object from the fixture for tests that run on both local and remote repos
+    return request.getfixturevalue(repo_fixtures)
+
+
+def reopen(repository, exclusive: Optional[bool] = True, create=False):
+    if isinstance(repository, LegacyRepository):
+        if repository.io is not None or repository.lock is not None:
+            raise RuntimeError("Repo must be closed before a reopen. Cannot support nested repository contexts.")
+        return LegacyRepository(repository.path, exclusive=exclusive, create=create)
+
+    if isinstance(repository, LegacyRemoteRepository):
+        if repository.p is not None or repository.sock is not None:
+            raise RuntimeError("Remote repo must be closed before a reopen. Cannot support nested repository contexts.")
+        return LegacyRemoteRepository(repository.location, exclusive=exclusive, create=create)
+
+    raise TypeError(
+        f"Invalid argument type. Expected 'Repository' or 'RemoteRepository', received '{type(repository).__name__}'."
+    )
+
+
+def get_path(repository):
+    if isinstance(repository, LegacyRepository):
+        return repository.path
+
+    if isinstance(repository, LegacyRemoteRepository):
+        return repository.location.path
+
+    raise TypeError(
+        f"Invalid argument type. Expected 'Repository' or 'RemoteRepository', received '{type(repository).__name__}'."
+    )
+
+
+def fchunk(data, meta=b""):
+    # create a raw chunk that has valid RepoObj layout, but does not use encryption or compression.
+    hdr = RepoObj.obj_header.pack(len(meta), len(data), xxh64(meta), xxh64(data))
+    assert isinstance(data, bytes)
+    chunk = hdr + meta + data
+    return chunk
+
+
+def pchunk(chunk):
+    # parse data and meta from a raw chunk made by fchunk
+    hdr_size = RepoObj.obj_header.size
+    hdr = chunk[:hdr_size]
+    meta_size, data_size = RepoObj.obj_header.unpack(hdr)[0:2]
+    meta = chunk[hdr_size : hdr_size + meta_size]
+    data = chunk[hdr_size + meta_size : hdr_size + meta_size + data_size]
+    return data, meta
+
+
+def pdchunk(chunk):
+    # parse only data from a raw chunk made by fchunk
+    return pchunk(chunk)[0]
+
+
+def add_keys(repository):
+    repository.put(H(0), fchunk(b"foo"))
+    repository.put(H(1), fchunk(b"bar"))
+    repository.put(H(3), fchunk(b"bar"))
+    repository.commit(compact=False)
+    repository.put(H(1), fchunk(b"bar2"))
+    repository.put(H(2), fchunk(b"boo"))
+    repository.delete(H(3))
+
+
+def repo_dump(repository, label=None):
+    label = label + ": " if label is not None else ""
+    H_trans = {H(i): i for i in range(10)}
+    H_trans[None] = -1  # key == None appears in commits
+    tag_trans = {TAG_PUT2: "put2", TAG_PUT: "put", TAG_DELETE: "del", TAG_COMMIT: "comm"}
+    for segment, fn in repository.io.segment_iterator():
+        for tag, key, offset, size, _ in repository.io.iter_objects(segment):
+            print("%s%s H(%d) -> %s[%d..+%d]" % (label, tag_trans[tag], H_trans[key], fn, offset, size))
+    print()
+
+
+def test_basic_operations(repo_fixtures, request):
+    with get_repository_from_fixture(repo_fixtures, request) as repository:
+        for x in range(100):
+            repository.put(H(x), fchunk(b"SOMEDATA"))
+        key50 = H(50)
+        assert pdchunk(repository.get(key50)) == b"SOMEDATA"
+        repository.delete(key50)
+        with pytest.raises(LegacyRepository.ObjectNotFound):
+            repository.get(key50)
+        repository.commit(compact=False)
+    with reopen(repository) as repository:
+        with pytest.raises(LegacyRepository.ObjectNotFound):
+            repository.get(key50)
+        for x in range(100):
+            if x == 50:
+                continue
+            assert pdchunk(repository.get(H(x))) == b"SOMEDATA"
+
+
+def test_multiple_transactions(repo_fixtures, request):
+    with get_repository_from_fixture(repo_fixtures, request) as repository:
+        repository.put(H(0), fchunk(b"foo"))
+        repository.put(H(1), fchunk(b"foo"))
+        repository.commit(compact=False)
+        repository.delete(H(0))
+        repository.put(H(1), fchunk(b"bar"))
+        repository.commit(compact=False)
+        assert pdchunk(repository.get(H(1))) == b"bar"
+
+
+def test_read_data(repo_fixtures, request):
+    with get_repository_from_fixture(repo_fixtures, request) as repository:
+        meta, data = b"meta", b"data"
+        hdr = RepoObj.obj_header.pack(len(meta), len(data), xxh64(meta), xxh64(data))
+        chunk_complete = hdr + meta + data
+        chunk_short = hdr + meta
+        repository.put(H(0), chunk_complete)
+        repository.commit(compact=False)
+        assert repository.get(H(0)) == chunk_complete
+        assert repository.get(H(0), read_data=True) == chunk_complete
+        assert repository.get(H(0), read_data=False) == chunk_short
+
+
+def test_consistency(repo_fixtures, request):
+    with get_repository_from_fixture(repo_fixtures, request) as repository:
+        repository.put(H(0), fchunk(b"foo"))
+        assert pdchunk(repository.get(H(0))) == b"foo"
+        repository.put(H(0), fchunk(b"foo2"))
+        assert pdchunk(repository.get(H(0))) == b"foo2"
+        repository.put(H(0), fchunk(b"bar"))
+        assert pdchunk(repository.get(H(0))) == b"bar"
+        repository.delete(H(0))
+        with pytest.raises(LegacyRepository.ObjectNotFound):
+            repository.get(H(0))
+
+
+def test_consistency2(repo_fixtures, request):
+    with get_repository_from_fixture(repo_fixtures, request) as repository:
+        repository.put(H(0), fchunk(b"foo"))
+        assert pdchunk(repository.get(H(0))) == b"foo"
+        repository.commit(compact=False)
+        repository.put(H(0), fchunk(b"foo2"))
+        assert pdchunk(repository.get(H(0))) == b"foo2"
+        repository.rollback()
+        assert pdchunk(repository.get(H(0))) == b"foo"
+
+
+def test_overwrite_in_same_transaction(repo_fixtures, request):
+    with get_repository_from_fixture(repo_fixtures, request) as repository:
+        repository.put(H(0), fchunk(b"foo"))
+        repository.put(H(0), fchunk(b"foo2"))
+        repository.commit(compact=False)
+        assert pdchunk(repository.get(H(0))) == b"foo2"
+
+
+def test_single_kind_transactions(repo_fixtures, request):
+    # put
+    with get_repository_from_fixture(repo_fixtures, request) as repository:
+        repository.put(H(0), fchunk(b"foo"))
+        repository.commit(compact=False)
+    # replace
+    with reopen(repository) as repository:
+        repository.put(H(0), fchunk(b"bar"))
+        repository.commit(compact=False)
+    # delete
+    with reopen(repository) as repository:
+        repository.delete(H(0))
+        repository.commit(compact=False)
+
+
+def test_list(repo_fixtures, request):
+    with get_repository_from_fixture(repo_fixtures, request) as repository:
+        for x in range(100):
+            repository.put(H(x), fchunk(b"SOMEDATA"))
+        repository.commit(compact=False)
+        repo_list = repository.list()
+        assert len(repo_list) == 100
+        first_half = repository.list(limit=50)
+        assert len(first_half) == 50
+        assert first_half == repo_list[:50]
+        second_half = repository.list(marker=first_half[-1])
+        assert len(second_half) == 50
+        assert second_half == repo_list[50:]
+        assert len(repository.list(limit=50)) == 50
+
+
+def test_max_data_size(repo_fixtures, request):
+    with get_repository_from_fixture(repo_fixtures, request) as repository:
+        max_data = b"x" * (MAX_DATA_SIZE - RepoObj.obj_header.size)
+        repository.put(H(0), fchunk(max_data))
+        assert pdchunk(repository.get(H(0))) == max_data
+        with pytest.raises(IntegrityError):
+            repository.put(H(1), fchunk(max_data + b"x"))
+
+
+def _assert_sparse(repository):
+    # the superseded 123456... PUT
+    assert repository.compact[0] == 41 + 8 + len(fchunk(b"123456789"))
+    # a COMMIT
+    assert repository.compact[1] == 9
+    # the DELETE issued by the superseding PUT (or issued directly)
+    assert repository.compact[2] == 41
+    repository._rebuild_sparse(0)
+    assert repository.compact[0] == 41 + 8 + len(fchunk(b"123456789"))  # 9 is chunk or commit?
+
+
+def test_sparse1(repository):
+    with repository:
+        repository.put(H(0), fchunk(b"foo"))
+        repository.put(H(1), fchunk(b"123456789"))
+        repository.commit(compact=False)
+        repository.put(H(1), fchunk(b"bar"))
+        _assert_sparse(repository)
+
+
+def test_sparse2(repository):
+    with repository:
+        repository.put(H(0), fchunk(b"foo"))
+        repository.put(H(1), fchunk(b"123456789"))
+        repository.commit(compact=False)
+        repository.delete(H(1))
+        _assert_sparse(repository)
+
+
+def test_sparse_delete(repository):
+    with repository:
+        chunk0 = fchunk(b"1245")
+        repository.put(H(0), chunk0)
+        repository.delete(H(0))
+        repository.io._write_fd.sync()
+        # the on-line tracking works on a per-object basis...
+        assert repository.compact[0] == 41 + 8 + 41 + len(chunk0)
+        repository._rebuild_sparse(0)
+        # ...while _rebuild_sparse can mark whole segments as completely sparse (which then includes the segment magic)
+        assert repository.compact[0] == 41 + 8 + 41 + len(chunk0) + len(MAGIC)
+        repository.commit(compact=True)
+        assert 0 not in [segment for segment, _ in repository.io.segment_iterator()]
+
+
+def test_uncommitted_garbage(repository):
+    with repository:
+        # uncommitted garbage should be no problem, it is cleaned up automatically.
+        # we just have to be careful with invalidation of cached FDs in LoggedIO.
+        repository.put(H(0), fchunk(b"foo"))
+        repository.commit(compact=False)
+        # write some crap to an uncommitted segment file
+        last_segment = repository.io.get_latest_segment()
+        with open(repository.io.segment_filename(last_segment + 1), "wb") as f:
+            f.write(MAGIC + b"crapcrapcrap")
+    with reopen(repository) as repository:
+        # usually, opening the repo and starting a transaction should trigger a cleanup.
+        repository.put(H(0), fchunk(b"bar"))  # this may trigger compact_segments()
+        repository.commit(compact=True)
+        # the point here is that nothing blows up with an exception.
+
+
+def test_replay_of_missing_index(repository):
+    with repository:
+        add_keys(repository)
+        for name in os.listdir(repository.path):
+            if name.startswith("index."):
+                os.unlink(os.path.join(repository.path, name))
+    with reopen(repository) as repository:
+        assert len(repository) == 3
+        assert repository.check() is True
+
+
+def test_crash_before_compact_segments(repository):
+    with repository:
+        add_keys(repository)
+        repository.compact_segments = None
+        try:
+            repository.commit(compact=True)
+        except TypeError:
+            pass
+    with reopen(repository) as repository:
+        assert len(repository) == 3
+        assert repository.check() is True
+
+
+def test_crash_before_write_index(repository):
+    with repository:
+        add_keys(repository)
+        repository.write_index = None
+        try:
+            repository.commit(compact=False)
+        except TypeError:
+            pass
+    with reopen(repository) as repository:
+        assert len(repository) == 3
+        assert repository.check() is True
+
+
+def test_replay_lock_upgrade_old(repository):
+    with repository:
+        add_keys(repository)
+        for name in os.listdir(repository.path):
+            if name.startswith("index."):
+                os.unlink(os.path.join(repository.path, name))
+    with patch.object(Lock, "upgrade", side_effect=LockFailed) as upgrade:
+        with reopen(repository, exclusive=None) as repository:
+            # simulate old client that always does lock upgrades
+            # the repo is only locked by a shared read lock, but to replay segments,
+            # we need an exclusive write lock - check if the lock gets upgraded.
+            with pytest.raises(LockFailed):
+                len(repository)
+            upgrade.assert_called_once_with()
+
+
+def test_replay_lock_upgrade(repository):
+    with repository:
+        add_keys(repository)
+        for name in os.listdir(repository.path):
+            if name.startswith("index."):
+                os.unlink(os.path.join(repository.path, name))
+    with patch.object(Lock, "upgrade", side_effect=LockFailed) as upgrade:
+        with reopen(repository, exclusive=False) as repository:
+            # current client usually does not do lock upgrade, except for replay
+            # the repo is only locked by a shared read lock, but to replay segments,
+            # we need an exclusive write lock - check if the lock gets upgraded.
+            with pytest.raises(LockFailed):
+                len(repository)
+            upgrade.assert_called_once_with()
+
+
+def test_crash_before_deleting_compacted_segments(repository):
+    with repository:
+        add_keys(repository)
+        repository.io.delete_segment = None
+        try:
+            repository.commit(compact=False)
+        except TypeError:
+            pass
+    with reopen(repository) as repository:
+        assert len(repository) == 3
+        assert repository.check() is True
+        assert len(repository) == 3
+
+
+def test_ignores_commit_tag_in_data(repository):
+    with repository:
+        repository.put(H(0), LoggedIO.COMMIT)
+    with reopen(repository) as repository:
+        io = repository.io
+        assert not io.is_committed_segment(io.get_latest_segment())
+
+
+def test_moved_deletes_are_tracked(repository):
+    with repository:
+        repository.put(H(1), fchunk(b"1"))
+        repository.put(H(2), fchunk(b"2"))
+        repository.commit(compact=False)
+        repo_dump(repository, "p1 p2 c")
+        repository.delete(H(1))
+        repository.commit(compact=True)
+        repo_dump(repository, "d1 cc")
+        last_segment = repository.io.get_latest_segment() - 1
+        num_deletes = 0
+        for tag, key, offset, size, _ in repository.io.iter_objects(last_segment):
+            if tag == TAG_DELETE:
+                assert key == H(1)
+                num_deletes += 1
+        assert num_deletes == 1
+        assert last_segment in repository.compact
+        repository.put(H(3), fchunk(b"3"))
+        repository.commit(compact=True)
+        repo_dump(repository, "p3 cc")
+        assert last_segment not in repository.compact
+        assert not repository.io.segment_exists(last_segment)
+        for segment, _ in repository.io.segment_iterator():
+            for tag, key, offset, size, _ in repository.io.iter_objects(segment):
+                assert tag != TAG_DELETE
+                assert key != H(1)
+        # after compaction, there should be no empty shadowed_segments lists left over.
+        # we have no put or del anymore for H(1), so we lost knowledge about H(1).
+        assert H(1) not in repository.shadow_index
+
+
+def test_shadowed_entries_are_preserved1(repository):
+    # this tests the shadowing-by-del behaviour
+    with repository:
+        get_latest_segment = repository.io.get_latest_segment
+        repository.put(H(1), fchunk(b"1"))
+        # This is the segment with our original PUT of interest
+        put_segment = get_latest_segment()
+        repository.commit(compact=False)
+        # we now delete H(1), and force this segment not to be compacted, which can happen
+        # if it's not sparse enough (symbolized by H(2) here).
+        repository.delete(H(1))
+        repository.put(H(2), fchunk(b"1"))
+        del_segment = get_latest_segment()
+        # we pretend these are mostly dense (not sparse) and won't be compacted
+        del repository.compact[put_segment]
+        del repository.compact[del_segment]
+        repository.commit(compact=True)
+        # we now perform an unrelated operation on the segment containing the DELETE,
+        # causing it to be compacted.
+        repository.delete(H(2))
+        repository.commit(compact=True)
+        assert repository.io.segment_exists(put_segment)
+        assert not repository.io.segment_exists(del_segment)
+        # basic case, since the index survived this must be ok
+        assert H(1) not in repository
+        # nuke index, force replay
+        os.unlink(os.path.join(repository.path, "index.%d" % get_latest_segment()))
+        # must not reappear
+        assert H(1) not in repository
+
+
+def test_shadowed_entries_are_preserved2(repository):
+    # this tests the shadowing-by-double-put behaviour, see issue #5661
+    # assume this repo state:
+    # seg1: PUT H1
+    # seg2: COMMIT
+    # seg3: DEL H1, PUT H1, DEL H1, PUT H2
+    # seg4: COMMIT
+    # Note how due to the final DEL H1 in seg3, H1 is effectively deleted.
+    #
+    # compaction of only seg3:
+    # PUT H1 gets dropped because it is not needed any more.
+    # DEL H1 must be kept, because there is still a PUT H1 in seg1 which must not
+    # "reappear" in the index if the index gets rebuilt.
+    with repository:
+        get_latest_segment = repository.io.get_latest_segment
+        repository.put(H(1), fchunk(b"1"))
+        # This is the segment with our original PUT of interest
+        put_segment = get_latest_segment()
+        repository.commit(compact=False)
+        # We now put H(1) again (which implicitly does DEL(H(1)) followed by PUT(H(1), ...)),
+        # delete H(1) afterwards, and force this segment to not be compacted, which can happen
+        # if it's not sparse enough (symbolized by H(2) here).
+        repository.put(H(1), fchunk(b"1"))
+        repository.delete(H(1))
+        repository.put(H(2), fchunk(b"1"))
+        delete_segment = get_latest_segment()
+        # We pretend these are mostly dense (not sparse) and won't be compacted
+        del repository.compact[put_segment]
+        del repository.compact[delete_segment]
+        repository.commit(compact=True)
+        # Now we perform an unrelated operation on the segment containing the DELETE,
+        # causing it to be compacted.
+        repository.delete(H(2))
+        repository.commit(compact=True)
+        assert repository.io.segment_exists(put_segment)
+        assert not repository.io.segment_exists(delete_segment)
+        # Basic case, since the index survived this must be ok
+        assert H(1) not in repository
+        # Nuke index, force replay
+        os.unlink(os.path.join(repository.path, "index.%d" % get_latest_segment()))
+        # Must not reappear
+        assert H(1) not in repository  # F
+
+
+def test_shadow_index_rollback(repository):
+    with repository:
+        repository.put(H(1), fchunk(b"1"))
+        repository.delete(H(1))
+        assert repository.shadow_index[H(1)] == [0]
+        repository.commit(compact=True)
+        repo_dump(repository, "p1 d1 cc")
+        # note how an empty list means that nothing is shadowed for sure
+        assert repository.shadow_index[H(1)] == []  # because the deletion is considered unstable
+        repository.put(H(1), b"1")
+        repository.delete(H(1))
+        repo_dump(repository, "p1 d1")
+        # 0 put/delete; 1 commit; 2 compacted; 3 commit; 4 put/delete
+        assert repository.shadow_index[H(1)] == [4]
+        repository.rollback()
+        repo_dump(repository, "r")
+        repository.put(H(2), fchunk(b"1"))
+        # after the rollback, segment 4 shouldn't be considered anymore
+        assert repository.shadow_index[H(1)] == []  # because the deletion is considered unstable
+
+
+def test_destroy_append_only(repository):
+    with repository:
+        # can't destroy append only repo (via the API)
+        repository.append_only = True
+        with pytest.raises(ValueError):
+            repository.destroy()
+        assert repository.append_only
+
+
+def test_append_only(repository):
+    def segments_in_repository(repo):
+        return len(list(repo.io.segment_iterator()))
+
+    with repository:
+        repository.append_only = True
+        repository.put(H(0), fchunk(b"foo"))
+        repository.commit(compact=False)
+
+        repository.append_only = False
+        assert segments_in_repository(repository) == 2
+        repository.put(H(0), fchunk(b"foo"))
+        repository.commit(compact=True)
+        # normal: compact squashes the data together, only one segment
+        assert segments_in_repository(repository) == 2
+
+        repository.append_only = True
+        assert segments_in_repository(repository) == 2
+        repository.put(H(0), fchunk(b"foo"))
+        repository.commit(compact=False)
+        # append only: does not compact, only new segments written
+        assert segments_in_repository(repository) == 4
+
+
+def test_additional_free_space(repository):
+    with repository:
+        add_keys(repository)
+        repository.config.set("repository", "additional_free_space", "1000T")
+        repository.save_key(b"shortcut to save_config")
+    with reopen(repository) as repository:
+        repository.put(H(0), fchunk(b"foobar"))
+        with pytest.raises(LegacyRepository.InsufficientFreeSpaceError):
+            repository.commit(compact=False)
+        assert os.path.exists(repository.path)
+
+
+def test_create_free_space(repository):
+    with repository:
+        repository.additional_free_space = 1e20
+        with pytest.raises(LegacyRepository.InsufficientFreeSpaceError):
+            add_keys(repository)
+        assert not os.path.exists(repository.path)
+
+
+def test_tracking(repository):
+    with repository:
+        assert repository.storage_quota_use == 0
+        ch1 = fchunk(bytes(1234))
+        repository.put(H(1), ch1)
+        assert repository.storage_quota_use == len(ch1) + 41 + 8
+        ch2 = fchunk(bytes(5678))
+        repository.put(H(2), ch2)
+        assert repository.storage_quota_use == len(ch1) + len(ch2) + 2 * (41 + 8)
+        repository.delete(H(1))
+        assert repository.storage_quota_use == len(ch1) + len(ch2) + 2 * (41 + 8)  # we have not compacted yet
+        repository.commit(compact=False)
+        assert repository.storage_quota_use == len(ch1) + len(ch2) + 2 * (41 + 8)  # we have not compacted yet
+    with reopen(repository) as repository:
+        # open new transaction; hints and thus quota data is not loaded unless needed.
+        ch3 = fchunk(b"")
+        repository.put(H(3), ch3)
+        repository.delete(H(3))
+        assert repository.storage_quota_use == len(ch1) + len(ch2) + len(ch3) + 3 * (
+            41 + 8
+        )  # we have not compacted yet
+        repository.commit(compact=True)
+        assert repository.storage_quota_use == len(ch2) + 41 + 8
+
+
+def test_exceed_quota(repository):
+    with repository:
+        assert repository.storage_quota_use == 0
+        repository.storage_quota = 80
+        ch1 = fchunk(b"x" * 7)
+        repository.put(H(1), ch1)
+        assert repository.storage_quota_use == len(ch1) + 41 + 8
+        repository.commit(compact=False)
+        with pytest.raises(LegacyRepository.StorageQuotaExceeded):
+            ch2 = fchunk(b"y" * 13)
+            repository.put(H(2), ch2)
+        assert repository.storage_quota_use == len(ch1) + len(ch2) + (41 + 8) * 2  # check ch2!?
+        with pytest.raises(LegacyRepository.StorageQuotaExceeded):
+            repository.commit(compact=False)
+        assert repository.storage_quota_use == len(ch1) + len(ch2) + (41 + 8) * 2  # check ch2!?
+    with reopen(repository) as repository:
+        repository.storage_quota = 161
+        # open new transaction; hints and thus quota data is not loaded unless needed.
+        repository.put(H(1), ch1)
+        # we have 2 puts for H(1) here and not yet compacted.
+        assert repository.storage_quota_use == len(ch1) * 2 + (41 + 8) * 2
+        repository.commit(compact=True)
+        assert repository.storage_quota_use == len(ch1) + 41 + 8  # now we have compacted.
+
+
+def make_auxiliary(repository):
+    with repository:
+        repository.put(H(0), fchunk(b"foo"))
+        repository.commit(compact=False)
+
+
+def do_commit(repository):
+    with repository:
+        repository.put(H(0), fchunk(b"fox"))
+        repository.commit(compact=False)
+
+
+def test_corrupted_hints(repository):
+    make_auxiliary(repository)
+    with open(os.path.join(repository.path, "hints.1"), "ab") as fd:
+        fd.write(b"123456789")
+    do_commit(repository)
+
+
+def test_deleted_hints(repository):
+    make_auxiliary(repository)
+    os.unlink(os.path.join(repository.path, "hints.1"))
+    do_commit(repository)
+
+
+def test_deleted_index(repository):
+    make_auxiliary(repository)
+    os.unlink(os.path.join(repository.path, "index.1"))
+    do_commit(repository)
+
+
+def test_unreadable_hints(repository):
+    make_auxiliary(repository)
+    hints = os.path.join(repository.path, "hints.1")
+    os.unlink(hints)
+    os.mkdir(hints)
+    with pytest.raises(OSError):
+        do_commit(repository)
+
+
+def test_index(repository):
+    make_auxiliary(repository)
+    with open(os.path.join(repository.path, "index.1"), "wb") as fd:
+        fd.write(b"123456789")
+    do_commit(repository)
+
+
+def test_index_outside_transaction(repository):
+    make_auxiliary(repository)
+    with open(os.path.join(repository.path, "index.1"), "wb") as fd:
+        fd.write(b"123456789")
+    with repository:
+        assert len(repository) == 1
+
+
+def _corrupt_index(repository):
+    # HashIndex is able to detect incorrect headers and file lengths,
+    # but on its own it can't tell if the data is correct.
+    index_path = os.path.join(repository.path, "index.1")
+    with open(index_path, "r+b") as fd:
+        index_data = fd.read()
+        # Flip one bit in a key stored in the index
+        corrupted_key = (int.from_bytes(H(0), "little") ^ 1).to_bytes(32, "little")
+        corrupted_index_data = index_data.replace(H(0), corrupted_key)
+        assert corrupted_index_data != index_data
+        assert len(corrupted_index_data) == len(index_data)
+        fd.seek(0)
+        fd.write(corrupted_index_data)
+
+
+def test_index_corrupted(repository):
+    make_auxiliary(repository)
+    _corrupt_index(repository)
+    with repository:
+        # data corruption is detected due to mismatching checksums, and fixed by rebuilding the index.
+        assert len(repository) == 1
+        assert pdchunk(repository.get(H(0))) == b"foo"
+
+
+def test_index_corrupted_without_integrity(repository):
+    make_auxiliary(repository)
+    _corrupt_index(repository)
+    integrity_path = os.path.join(repository.path, "integrity.1")
+    os.unlink(integrity_path)
+    with repository:
+        # since the corrupted key is not noticed, the repository still thinks it contains one key...
+        assert len(repository) == 1
+        with pytest.raises(LegacyRepository.ObjectNotFound):
+            # ... but the real, uncorrupted key is not found in the corrupted index.
+            repository.get(H(0))
+
+
+def test_unreadable_index(repository):
+    make_auxiliary(repository)
+    index = os.path.join(repository.path, "index.1")
+    os.unlink(index)
+    os.mkdir(index)
+    with pytest.raises(OSError):
+        do_commit(repository)
+
+
+def test_unknown_integrity_version(repository):
+    make_auxiliary(repository)
+    # for now an unknown integrity data version is ignored and not an error.
+    integrity_path = os.path.join(repository.path, "integrity.1")
+    with open(integrity_path, "r+b") as fd:
+        msgpack.pack({b"version": 4.7}, fd)  # borg only understands version 2
+        fd.truncate()
+    with repository:
+        # no issues accessing the repository
+        assert len(repository) == 1
+        assert pdchunk(repository.get(H(0))) == b"foo"
+
+
+def _subtly_corrupted_hints_setup(repository):
+    with repository:
+        repository.append_only = True
+        assert len(repository) == 1
+        assert pdchunk(repository.get(H(0))) == b"foo"
+        repository.put(H(1), fchunk(b"bar"))
+        repository.put(H(2), fchunk(b"baz"))
+        repository.commit(compact=False)
+        repository.put(H(2), fchunk(b"bazz"))
+        repository.commit(compact=False)
+    hints_path = os.path.join(repository.path, "hints.5")
+    with open(hints_path, "r+b") as fd:
+        hints = msgpack.unpack(fd)
+        fd.seek(0)
+        # corrupt segment refcount
+        assert hints["segments"][2] == 1
+        hints["segments"][2] = 0
+        msgpack.pack(hints, fd)
+        fd.truncate()
+
+
+def test_subtly_corrupted_hints(repository):
+    make_auxiliary(repository)
+    _subtly_corrupted_hints_setup(repository)
+    with repository:
+        repository.append_only = False
+        repository.put(H(3), fchunk(b"1234"))
+        # do a compaction run, which succeeds since the failed checksum prompted a rebuild of the index+hints.
+        repository.commit(compact=True)
+        assert len(repository) == 4
+        assert pdchunk(repository.get(H(0))) == b"foo"
+        assert pdchunk(repository.get(H(1))) == b"bar"
+        assert pdchunk(repository.get(H(2))) == b"bazz"
+
+
+def test_subtly_corrupted_hints_without_integrity(repository):
+    make_auxiliary(repository)
+    _subtly_corrupted_hints_setup(repository)
+    integrity_path = os.path.join(repository.path, "integrity.5")
+    os.unlink(integrity_path)
+    with repository:
+        repository.append_only = False
+        repository.put(H(3), fchunk(b"1234"))
+        # do a compaction run, which fails since the corrupted refcount wasn't detected and causes an assertion failure.
+        with pytest.raises(AssertionError) as exc_info:
+            repository.commit(compact=True)
+        assert "Corrupted segment reference count" in str(exc_info.value)
+
+
+def list_indices(repo_path):
+    return [name for name in os.listdir(repo_path) if name.startswith("index.")]
+
+
+def check(repository, repo_path, repair=False, status=True):
+    assert repository.check(repair=repair) == status
+    # Make sure no tmp files are left behind
+    tmp_files = [name for name in os.listdir(repo_path) if "tmp" in name]
+    assert tmp_files == [], "Found tmp files"
+
+
+def get_objects(repository, *ids):
+    for id_ in ids:
+        pdchunk(repository.get(H(id_)))
+
+
+def add_objects(repository, segments):
+    for ids in segments:
+        for id_ in ids:
+            repository.put(H(id_), fchunk(b"data"))
+        repository.commit(compact=False)
+
+
+def get_head(repo_path):
+    return sorted(int(n) for n in os.listdir(os.path.join(repo_path, "data", "0")) if n.isdigit())[-1]
+
+
+def open_index(repo_path):
+    return NSIndex.read(os.path.join(repo_path, f"index.{get_head(repo_path)}"))
+
+
+def corrupt_object(repo_path, id_):
+    idx = open_index(repo_path)
+    segment, offset, _ = idx[H(id_)]
+    with open(os.path.join(repo_path, "data", "0", str(segment)), "r+b") as fd:
+        fd.seek(offset)
+        fd.write(b"BOOM")
+
+
+def delete_segment(repository, segment):
+    repository.io.delete_segment(segment)
+
+
+def delete_index(repo_path):
+    os.unlink(os.path.join(repo_path, f"index.{get_head(repo_path)}"))
+
+
+def rename_index(repo_path, new_name):
+    os.replace(os.path.join(repo_path, f"index.{get_head(repo_path)}"), os.path.join(repo_path, new_name))
+
+
+def list_objects(repository):
+    return {int(key) for key in repository.list()}
+
+
+def test_repair_corrupted_segment(repo_fixtures, request):
+    with get_repository_from_fixture(repo_fixtures, request) as repository:
+        repo_path = get_path(repository)
+        add_objects(repository, [[1, 2, 3], [4, 5], [6]])
+        assert {1, 2, 3, 4, 5, 6} == list_objects(repository)
+        check(repository, repo_path, status=True)
+        corrupt_object(repo_path, 5)
+        with pytest.raises(IntegrityError):
+            get_objects(repository, 5)
+        repository.rollback()
+        # make sure a regular check does not repair anything
+        check(repository, repo_path, status=False)
+        check(repository, repo_path, status=False)
+        # make sure a repair actually repairs the repo
+        check(repository, repo_path, repair=True, status=True)
+        get_objects(repository, 4)
+        check(repository, repo_path, status=True)
+        assert {1, 2, 3, 4, 6} == list_objects(repository)
+
+
+def test_repair_missing_segment(repository):
+    # only test on local repo - files in RemoteRepository cannot be deleted
+    with repository:
+        add_objects(repository, [[1, 2, 3], [4, 5, 6]])
+        assert {1, 2, 3, 4, 5, 6} == list_objects(repository)
+        check(repository, repository.path, status=True)
+        delete_segment(repository, 2)
+        repository.rollback()
+        check(repository, repository.path, repair=True, status=True)
+        assert {1, 2, 3} == list_objects(repository)
+
+
+def test_repair_missing_commit_segment(repository):
+    # only test on local repo - files in RemoteRepository cannot be deleted
+    with repository:
+        add_objects(repository, [[1, 2, 3], [4, 5, 6]])
+        delete_segment(repository, 3)
+        with pytest.raises(LegacyRepository.ObjectNotFound):
+            get_objects(repository, 4)
+        assert {1, 2, 3} == list_objects(repository)
+
+
+def test_repair_corrupted_commit_segment(repo_fixtures, request):
+    with get_repository_from_fixture(repo_fixtures, request) as repository:
+        repo_path = get_path(repository)
+        add_objects(repository, [[1, 2, 3], [4, 5, 6]])
+        with open(os.path.join(repo_path, "data", "0", "3"), "r+b") as fd:
+            fd.seek(-1, os.SEEK_END)
+            fd.write(b"X")
+        with pytest.raises(LegacyRepository.ObjectNotFound):
+            get_objects(repository, 4)
+        check(repository, repo_path, status=True)
+        get_objects(repository, 3)
+        assert {1, 2, 3} == list_objects(repository)
+
+
+def test_repair_no_commits(repo_fixtures, request):
+    with get_repository_from_fixture(repo_fixtures, request) as repository:
+        repo_path = get_path(repository)
+        add_objects(repository, [[1, 2, 3]])
+        with open(os.path.join(repo_path, "data", "0", "1"), "r+b") as fd:
+            fd.seek(-1, os.SEEK_END)
+            fd.write(b"X")
+        with pytest.raises(LegacyRepository.CheckNeeded):
+            get_objects(repository, 4)
+        check(repository, repo_path, status=False)
+        check(repository, repo_path, status=False)
+        assert list_indices(repo_path) == ["index.1"]
+        check(repository, repo_path, repair=True, status=True)
+        assert list_indices(repo_path) == ["index.2"]
+        check(repository, repo_path, status=True)
+        get_objects(repository, 3)
+        assert {1, 2, 3} == list_objects(repository)
+
+
+def test_repair_missing_index(repo_fixtures, request):
+    with get_repository_from_fixture(repo_fixtures, request) as repository:
+        repo_path = get_path(repository)
+        add_objects(repository, [[1, 2, 3], [4, 5, 6]])
+        delete_index(repo_path)
+        check(repository, repo_path, status=True)
+        get_objects(repository, 4)
+        assert {1, 2, 3, 4, 5, 6} == list_objects(repository)
+
+
+def test_repair_index_too_new(repo_fixtures, request):
+    with get_repository_from_fixture(repo_fixtures, request) as repository:
+        repo_path = get_path(repository)
+        add_objects(repository, [[1, 2, 3], [4, 5, 6]])
+        assert list_indices(repo_path) == ["index.3"]
+        rename_index(repo_path, "index.100")
+        check(repository, repo_path, status=True)
+        assert list_indices(repo_path) == ["index.3"]
+        get_objects(repository, 4)
+        assert {1, 2, 3, 4, 5, 6} == list_objects(repository)
+
+
+def test_crash_before_compact(repository):
+    # only test on local repo - we can't mock-patch a RemoteRepository class in another process!
+    with repository:
+        repository.put(H(0), fchunk(b"data"))
+        repository.put(H(0), fchunk(b"data2"))
+        # simulate a crash before compact
+        with patch.object(LegacyRepository, "compact_segments") as compact:
+            repository.commit(compact=True)
+            compact.assert_called_once_with(0.1)
+    with reopen(repository) as repository:
+        check(repository, repository.path, repair=True)
+        assert pdchunk(repository.get(H(0))) == b"data2"
+
+
+def test_hints_persistence(repository):
+    with repository:
+        repository.put(H(0), fchunk(b"data"))
+        repository.delete(H(0))
+        repository.commit(compact=False)
+        shadow_index_expected = repository.shadow_index
+        compact_expected = repository.compact
+        segments_expected = repository.segments
+    # close and re-open the repository (create fresh Repository instance) to
+    # check whether hints were persisted to / reloaded from disk
+    with reopen(repository) as repository:
+        repository.put(H(42), fchunk(b"foobar"))  # this will call prepare_txn() and load the hints data
+        # check if hints persistence worked:
+        assert shadow_index_expected == repository.shadow_index
+        assert compact_expected == repository.compact
+        del repository.segments[2]  # ignore the segment created by put(H(42), ...)
+        assert segments_expected == repository.segments
+    with reopen(repository) as repository:
+        check(repository, repository.path, repair=True)
+    with reopen(repository) as repository:
+        repository.put(H(42), fchunk(b"foobar"))  # this will call prepare_txn() and load the hints data
+        assert shadow_index_expected == repository.shadow_index
+        # sizes do not match, with vs. without header?
+        # assert compact_expected == repository.compact
+        del repository.segments[2]  # ignore the segment created by put(H(42), ...)
+        assert segments_expected == repository.segments
+
+
+def test_hints_behaviour(repository):
+    with repository:
+        repository.put(H(0), fchunk(b"data"))
+        assert repository.shadow_index == {}
+        assert len(repository.compact) == 0
+        repository.delete(H(0))
+        repository.commit(compact=False)
+        # now there should be an entry for H(0) in shadow_index
+        assert H(0) in repository.shadow_index
+        assert len(repository.shadow_index[H(0)]) == 1
+        assert 0 in repository.compact  # segment 0 can be compacted
+        repository.put(H(42), fchunk(b"foobar"))  # see also do_compact()
+        repository.commit(compact=True, threshold=0.0)  # compact completely!
+        # nothing to compact anymore! no info left about stuff that does not exist anymore:
+        assert H(0) not in repository.shadow_index
+        # segment 0 was compacted away, no info about it left:
+        assert 0 not in repository.compact
+        assert 0 not in repository.segments
+
+
+def _get_mock_args():
+    class MockArgs:
+        remote_path = "borg"
+        umask = 0o077
+        debug_topics = []
+        rsh = None
+
+        def __contains__(self, item):
+            # to behave like argparse.Namespace
+            return hasattr(self, item)
+
+    return MockArgs()
+
+
+def test_remote_invalid_rpc(remote_repository):
+    with remote_repository:
+        with pytest.raises(InvalidRPCMethod):
+            remote_repository.call("__init__", {})
+
+
+def test_remote_rpc_exception_transport(remote_repository):
+    with remote_repository:
+        s1 = "test string"
+
+        try:
+            remote_repository.call("inject_exception", {"kind": "DoesNotExist"})
+        except LegacyRepository.DoesNotExist as e:
+            assert len(e.args) == 1
+            assert e.args[0] == remote_repository.location.processed
+
+        try:
+            remote_repository.call("inject_exception", {"kind": "AlreadyExists"})
+        except LegacyRepository.AlreadyExists as e:
+            assert len(e.args) == 1
+            assert e.args[0] == remote_repository.location.processed
+
+        try:
+            remote_repository.call("inject_exception", {"kind": "CheckNeeded"})
+        except LegacyRepository.CheckNeeded as e:
+            assert len(e.args) == 1
+            assert e.args[0] == remote_repository.location.processed
+
+        try:
+            remote_repository.call("inject_exception", {"kind": "IntegrityError"})
+        except IntegrityError as e:
+            assert len(e.args) == 1
+            assert e.args[0] == s1
+
+        try:
+            remote_repository.call("inject_exception", {"kind": "PathNotAllowed"})
+        except PathNotAllowed as e:
+            assert len(e.args) == 1
+            assert e.args[0] == "foo"
+
+        try:
+            remote_repository.call("inject_exception", {"kind": "ObjectNotFound"})
+        except LegacyRepository.ObjectNotFound as e:
+            assert len(e.args) == 2
+            assert e.args[0] == s1
+            assert e.args[1] == remote_repository.location.processed
+
+        try:
+            remote_repository.call("inject_exception", {"kind": "InvalidRPCMethod"})
+        except InvalidRPCMethod as e:
+            assert len(e.args) == 1
+            assert e.args[0] == s1
+
+        try:
+            remote_repository.call("inject_exception", {"kind": "divide"})
+        except LegacyRemoteRepository.RPCError as e:
+            assert e.unpacked
+            assert e.get_message() == "ZeroDivisionError: integer division or modulo by zero\n"
+            assert e.exception_class == "ZeroDivisionError"
+            assert len(e.exception_full) > 0
+
+
+def test_remote_ssh_cmd(remote_repository):
+    with remote_repository:
+        args = _get_mock_args()
+        remote_repository._args = args
+        assert remote_repository.ssh_cmd(Location("ssh://example.com/foo")) == ["ssh", "example.com"]
+        assert remote_repository.ssh_cmd(Location("ssh://user@example.com/foo")) == ["ssh", "user@example.com"]
+        assert remote_repository.ssh_cmd(Location("ssh://user@example.com:1234/foo")) == [
+            "ssh",
+            "-p",
+            "1234",
+            "user@example.com",
+        ]
+        os.environ["BORG_RSH"] = "ssh --foo"
+        assert remote_repository.ssh_cmd(Location("ssh://example.com/foo")) == ["ssh", "--foo", "example.com"]
+
+
+def test_remote_borg_cmd(remote_repository):
+    with remote_repository:
+        assert remote_repository.borg_cmd(None, testing=True) == [sys.executable, "-m", "borg", "serve"]
+        args = _get_mock_args()
+        # XXX without next line we get spurious test fails when using pytest-xdist, root cause unknown:
+        logging.getLogger().setLevel(logging.INFO)
+        # note: test logger is on info log level, so --info gets added automagically
+        assert remote_repository.borg_cmd(args, testing=False) == ["borg", "serve", "--info"]
+        args.remote_path = "borg-0.28.2"
+        assert remote_repository.borg_cmd(args, testing=False) == ["borg-0.28.2", "serve", "--info"]
+        args.debug_topics = ["something_client_side", "repository_compaction"]
+        assert remote_repository.borg_cmd(args, testing=False) == [
+            "borg-0.28.2",
+            "serve",
+            "--info",
+            "--debug-topic=borg.debug.repository_compaction",
+        ]
+        args = _get_mock_args()
+        args.storage_quota = 0
+        assert remote_repository.borg_cmd(args, testing=False) == ["borg", "serve", "--info"]
+        args.storage_quota = 314159265
+        assert remote_repository.borg_cmd(args, testing=False) == [
+            "borg",
+            "serve",
+            "--info",
+            "--storage-quota=314159265",
+        ]
+        args.rsh = "ssh -i foo"
+        remote_repository._args = args
+        assert remote_repository.ssh_cmd(Location("ssh://example.com/foo")) == ["ssh", "-i", "foo", "example.com"]

+ 1 - 1
src/borg/testsuite/platform.py

@@ -8,7 +8,7 @@ from ..platformflags import is_darwin, is_freebsd, is_linux, is_win32
 from ..platform import acl_get, acl_set
 from ..platform import get_process_id, process_alive
 from . import unopened_tempfile
-from .locking import free_pid  # NOQA
+from .fslocking import free_pid  # NOQA
 
 
 def fakeroot_detected():

+ 2 - 2
src/borg/testsuite/repoobj.py

@@ -3,14 +3,14 @@ import pytest
 from ..constants import ROBJ_FILE_STREAM, ROBJ_MANIFEST, ROBJ_ARCHIVE_META
 from ..crypto.key import PlaintextKey
 from ..helpers.errors import IntegrityError
-from ..repository3 import Repository3
+from ..repository import Repository
 from ..repoobj import RepoObj, RepoObj1
 from ..compress import LZ4
 
 
 @pytest.fixture
 def repository(tmpdir):
-    return Repository3(tmpdir, create=True)
+    return Repository(tmpdir, create=True)
 
 
 @pytest.fixture

+ 4 - 841
src/borg/testsuite/repository.py

@@ -2,19 +2,15 @@ import logging
 import os
 import sys
 from typing import Optional
-from unittest.mock import patch
 
 import pytest
 
 from ..checksums import xxh64
-from ..hashindex import NSIndex
 from ..helpers import Location
 from ..helpers import IntegrityError
-from ..helpers import msgpack
-from ..locking import Lock, LockFailed
 from ..platformflags import is_win32
 from ..remote import RemoteRepository, InvalidRPCMethod, PathNotAllowed
-from ..repository import Repository, LoggedIO, MAGIC, MAX_DATA_SIZE, TAG_DELETE, TAG_PUT2, TAG_PUT, TAG_COMMIT
+from ..repository import Repository, MAX_DATA_SIZE
 from ..repoobj import RepoObj
 from .hashindex import H
 
@@ -46,7 +42,7 @@ def get_repository_from_fixture(repo_fixtures, request):
 
 def reopen(repository, exclusive: Optional[bool] = True, create=False):
     if isinstance(repository, Repository):
-        if repository.io is not None or repository.lock is not None:
+        if repository.opened:
             raise RuntimeError("Repo must be closed before a reopen. Cannot support nested repository contexts.")
         return Repository(repository.path, exclusive=exclusive, create=create)
 
@@ -60,20 +56,8 @@ def reopen(repository, exclusive: Optional[bool] = True, create=False):
     )
 
 
-def get_path(repository):
-    if isinstance(repository, Repository):
-        return repository.path
-
-    if isinstance(repository, RemoteRepository):
-        return repository.location.path
-
-    raise TypeError(
-        f"Invalid argument type. Expected 'Repository' or 'RemoteRepository', received '{type(repository).__name__}'."
-    )
-
-
 def fchunk(data, meta=b""):
-    # create a raw chunk that has valid RepoObj layout, but does not use encryption or compression.
+    # format chunk: create a raw chunk that has valid RepoObj layout, but does not use encryption or compression.
     hdr = RepoObj.obj_header.pack(len(meta), len(data), xxh64(meta), xxh64(data))
     assert isinstance(data, bytes)
     chunk = hdr + meta + data
@@ -81,7 +65,7 @@ def fchunk(data, meta=b""):
 
 
 def pchunk(chunk):
-    # parse data and meta from a raw chunk made by fchunk
+    # parse chunk: parse data and meta from a raw chunk made by fchunk
     hdr_size = RepoObj.obj_header.size
     hdr = chunk[:hdr_size]
     meta_size, data_size = RepoObj.obj_header.unpack(hdr)[0:2]
@@ -95,27 +79,6 @@ def pdchunk(chunk):
     return pchunk(chunk)[0]
 
 
-def add_keys(repository):
-    repository.put(H(0), fchunk(b"foo"))
-    repository.put(H(1), fchunk(b"bar"))
-    repository.put(H(3), fchunk(b"bar"))
-    repository.commit(compact=False)
-    repository.put(H(1), fchunk(b"bar2"))
-    repository.put(H(2), fchunk(b"boo"))
-    repository.delete(H(3))
-
-
-def repo_dump(repository, label=None):
-    label = label + ": " if label is not None else ""
-    H_trans = {H(i): i for i in range(10)}
-    H_trans[None] = -1  # key == None appears in commits
-    tag_trans = {TAG_PUT2: "put2", TAG_PUT: "put", TAG_DELETE: "del", TAG_COMMIT: "comm"}
-    for segment, fn in repository.io.segment_iterator():
-        for tag, key, offset, size, _ in repository.io.iter_objects(segment):
-            print("%s%s H(%d) -> %s[%d..+%d]" % (label, tag_trans[tag], H_trans[key], fn, offset, size))
-    print()
-
-
 def test_basic_operations(repo_fixtures, request):
     with get_repository_from_fixture(repo_fixtures, request) as repository:
         for x in range(100):
@@ -125,7 +88,6 @@ def test_basic_operations(repo_fixtures, request):
         repository.delete(key50)
         with pytest.raises(Repository.ObjectNotFound):
             repository.get(key50)
-        repository.commit(compact=False)
     with reopen(repository) as repository:
         with pytest.raises(Repository.ObjectNotFound):
             repository.get(key50)
@@ -135,17 +97,6 @@ def test_basic_operations(repo_fixtures, request):
             assert pdchunk(repository.get(H(x))) == b"SOMEDATA"
 
 
-def test_multiple_transactions(repo_fixtures, request):
-    with get_repository_from_fixture(repo_fixtures, request) as repository:
-        repository.put(H(0), fchunk(b"foo"))
-        repository.put(H(1), fchunk(b"foo"))
-        repository.commit(compact=False)
-        repository.delete(H(0))
-        repository.put(H(1), fchunk(b"bar"))
-        repository.commit(compact=False)
-        assert pdchunk(repository.get(H(1))) == b"bar"
-
-
 def test_read_data(repo_fixtures, request):
     with get_repository_from_fixture(repo_fixtures, request) as repository:
         meta, data = b"meta", b"data"
@@ -153,7 +104,6 @@ def test_read_data(repo_fixtures, request):
         chunk_complete = hdr + meta + data
         chunk_short = hdr + meta
         repository.put(H(0), chunk_complete)
-        repository.commit(compact=False)
         assert repository.get(H(0)) == chunk_complete
         assert repository.get(H(0), read_data=True) == chunk_complete
         assert repository.get(H(0), read_data=False) == chunk_short
@@ -172,45 +122,10 @@ def test_consistency(repo_fixtures, request):
             repository.get(H(0))
 
 
-def test_consistency2(repo_fixtures, request):
-    with get_repository_from_fixture(repo_fixtures, request) as repository:
-        repository.put(H(0), fchunk(b"foo"))
-        assert pdchunk(repository.get(H(0))) == b"foo"
-        repository.commit(compact=False)
-        repository.put(H(0), fchunk(b"foo2"))
-        assert pdchunk(repository.get(H(0))) == b"foo2"
-        repository.rollback()
-        assert pdchunk(repository.get(H(0))) == b"foo"
-
-
-def test_overwrite_in_same_transaction(repo_fixtures, request):
-    with get_repository_from_fixture(repo_fixtures, request) as repository:
-        repository.put(H(0), fchunk(b"foo"))
-        repository.put(H(0), fchunk(b"foo2"))
-        repository.commit(compact=False)
-        assert pdchunk(repository.get(H(0))) == b"foo2"
-
-
-def test_single_kind_transactions(repo_fixtures, request):
-    # put
-    with get_repository_from_fixture(repo_fixtures, request) as repository:
-        repository.put(H(0), fchunk(b"foo"))
-        repository.commit(compact=False)
-    # replace
-    with reopen(repository) as repository:
-        repository.put(H(0), fchunk(b"bar"))
-        repository.commit(compact=False)
-    # delete
-    with reopen(repository) as repository:
-        repository.delete(H(0))
-        repository.commit(compact=False)
-
-
 def test_list(repo_fixtures, request):
     with get_repository_from_fixture(repo_fixtures, request) as repository:
         for x in range(100):
             repository.put(H(x), fchunk(b"SOMEDATA"))
-        repository.commit(compact=False)
         repo_list = repository.list()
         assert len(repo_list) == 100
         first_half = repository.list(limit=50)
@@ -231,555 +146,6 @@ def test_max_data_size(repo_fixtures, request):
             repository.put(H(1), fchunk(max_data + b"x"))
 
 
-def _assert_sparse(repository):
-    # the superseded 123456... PUT
-    assert repository.compact[0] == 41 + 8 + len(fchunk(b"123456789"))
-    # a COMMIT
-    assert repository.compact[1] == 9
-    # the DELETE issued by the superseding PUT (or issued directly)
-    assert repository.compact[2] == 41
-    repository._rebuild_sparse(0)
-    assert repository.compact[0] == 41 + 8 + len(fchunk(b"123456789"))  # 9 is chunk or commit?
-
-
-def test_sparse1(repository):
-    with repository:
-        repository.put(H(0), fchunk(b"foo"))
-        repository.put(H(1), fchunk(b"123456789"))
-        repository.commit(compact=False)
-        repository.put(H(1), fchunk(b"bar"))
-        _assert_sparse(repository)
-
-
-def test_sparse2(repository):
-    with repository:
-        repository.put(H(0), fchunk(b"foo"))
-        repository.put(H(1), fchunk(b"123456789"))
-        repository.commit(compact=False)
-        repository.delete(H(1))
-        _assert_sparse(repository)
-
-
-def test_sparse_delete(repository):
-    with repository:
-        chunk0 = fchunk(b"1245")
-        repository.put(H(0), chunk0)
-        repository.delete(H(0))
-        repository.io._write_fd.sync()
-        # the on-line tracking works on a per-object basis...
-        assert repository.compact[0] == 41 + 8 + 41 + len(chunk0)
-        repository._rebuild_sparse(0)
-        # ...while _rebuild_sparse can mark whole segments as completely sparse (which then includes the segment magic)
-        assert repository.compact[0] == 41 + 8 + 41 + len(chunk0) + len(MAGIC)
-        repository.commit(compact=True)
-        assert 0 not in [segment for segment, _ in repository.io.segment_iterator()]
-
-
-def test_uncommitted_garbage(repository):
-    with repository:
-        # uncommitted garbage should be no problem, it is cleaned up automatically.
-        # we just have to be careful with invalidation of cached FDs in LoggedIO.
-        repository.put(H(0), fchunk(b"foo"))
-        repository.commit(compact=False)
-        # write some crap to an uncommitted segment file
-        last_segment = repository.io.get_latest_segment()
-        with open(repository.io.segment_filename(last_segment + 1), "wb") as f:
-            f.write(MAGIC + b"crapcrapcrap")
-    with reopen(repository) as repository:
-        # usually, opening the repo and starting a transaction should trigger a cleanup.
-        repository.put(H(0), fchunk(b"bar"))  # this may trigger compact_segments()
-        repository.commit(compact=True)
-        # the point here is that nothing blows up with an exception.
-
-
-def test_replay_of_missing_index(repository):
-    with repository:
-        add_keys(repository)
-        for name in os.listdir(repository.path):
-            if name.startswith("index."):
-                os.unlink(os.path.join(repository.path, name))
-    with reopen(repository) as repository:
-        assert len(repository) == 3
-        assert repository.check() is True
-
-
-def test_crash_before_compact_segments(repository):
-    with repository:
-        add_keys(repository)
-        repository.compact_segments = None
-        try:
-            repository.commit(compact=True)
-        except TypeError:
-            pass
-    with reopen(repository) as repository:
-        assert len(repository) == 3
-        assert repository.check() is True
-
-
-def test_crash_before_write_index(repository):
-    with repository:
-        add_keys(repository)
-        repository.write_index = None
-        try:
-            repository.commit(compact=False)
-        except TypeError:
-            pass
-    with reopen(repository) as repository:
-        assert len(repository) == 3
-        assert repository.check() is True
-
-
-def test_replay_lock_upgrade_old(repository):
-    with repository:
-        add_keys(repository)
-        for name in os.listdir(repository.path):
-            if name.startswith("index."):
-                os.unlink(os.path.join(repository.path, name))
-    with patch.object(Lock, "upgrade", side_effect=LockFailed) as upgrade:
-        with reopen(repository, exclusive=None) as repository:
-            # simulate old client that always does lock upgrades
-            # the repo is only locked by a shared read lock, but to replay segments,
-            # we need an exclusive write lock - check if the lock gets upgraded.
-            with pytest.raises(LockFailed):
-                len(repository)
-            upgrade.assert_called_once_with()
-
-
-def test_replay_lock_upgrade(repository):
-    with repository:
-        add_keys(repository)
-        for name in os.listdir(repository.path):
-            if name.startswith("index."):
-                os.unlink(os.path.join(repository.path, name))
-    with patch.object(Lock, "upgrade", side_effect=LockFailed) as upgrade:
-        with reopen(repository, exclusive=False) as repository:
-            # current client usually does not do lock upgrade, except for replay
-            # the repo is only locked by a shared read lock, but to replay segments,
-            # we need an exclusive write lock - check if the lock gets upgraded.
-            with pytest.raises(LockFailed):
-                len(repository)
-            upgrade.assert_called_once_with()
-
-
-def test_crash_before_deleting_compacted_segments(repository):
-    with repository:
-        add_keys(repository)
-        repository.io.delete_segment = None
-        try:
-            repository.commit(compact=False)
-        except TypeError:
-            pass
-    with reopen(repository) as repository:
-        assert len(repository) == 3
-        assert repository.check() is True
-        assert len(repository) == 3
-
-
-def test_ignores_commit_tag_in_data(repository):
-    with repository:
-        repository.put(H(0), LoggedIO.COMMIT)
-    with reopen(repository) as repository:
-        io = repository.io
-        assert not io.is_committed_segment(io.get_latest_segment())
-
-
-def test_moved_deletes_are_tracked(repository):
-    with repository:
-        repository.put(H(1), fchunk(b"1"))
-        repository.put(H(2), fchunk(b"2"))
-        repository.commit(compact=False)
-        repo_dump(repository, "p1 p2 c")
-        repository.delete(H(1))
-        repository.commit(compact=True)
-        repo_dump(repository, "d1 cc")
-        last_segment = repository.io.get_latest_segment() - 1
-        num_deletes = 0
-        for tag, key, offset, size, _ in repository.io.iter_objects(last_segment):
-            if tag == TAG_DELETE:
-                assert key == H(1)
-                num_deletes += 1
-        assert num_deletes == 1
-        assert last_segment in repository.compact
-        repository.put(H(3), fchunk(b"3"))
-        repository.commit(compact=True)
-        repo_dump(repository, "p3 cc")
-        assert last_segment not in repository.compact
-        assert not repository.io.segment_exists(last_segment)
-        for segment, _ in repository.io.segment_iterator():
-            for tag, key, offset, size, _ in repository.io.iter_objects(segment):
-                assert tag != TAG_DELETE
-                assert key != H(1)
-        # after compaction, there should be no empty shadowed_segments lists left over.
-        # we have no put or del anymore for H(1), so we lost knowledge about H(1).
-        assert H(1) not in repository.shadow_index
-
-
-def test_shadowed_entries_are_preserved1(repository):
-    # this tests the shadowing-by-del behaviour
-    with repository:
-        get_latest_segment = repository.io.get_latest_segment
-        repository.put(H(1), fchunk(b"1"))
-        # This is the segment with our original PUT of interest
-        put_segment = get_latest_segment()
-        repository.commit(compact=False)
-        # we now delete H(1), and force this segment not to be compacted, which can happen
-        # if it's not sparse enough (symbolized by H(2) here).
-        repository.delete(H(1))
-        repository.put(H(2), fchunk(b"1"))
-        del_segment = get_latest_segment()
-        # we pretend these are mostly dense (not sparse) and won't be compacted
-        del repository.compact[put_segment]
-        del repository.compact[del_segment]
-        repository.commit(compact=True)
-        # we now perform an unrelated operation on the segment containing the DELETE,
-        # causing it to be compacted.
-        repository.delete(H(2))
-        repository.commit(compact=True)
-        assert repository.io.segment_exists(put_segment)
-        assert not repository.io.segment_exists(del_segment)
-        # basic case, since the index survived this must be ok
-        assert H(1) not in repository
-        # nuke index, force replay
-        os.unlink(os.path.join(repository.path, "index.%d" % get_latest_segment()))
-        # must not reappear
-        assert H(1) not in repository
-
-
-def test_shadowed_entries_are_preserved2(repository):
-    # this tests the shadowing-by-double-put behaviour, see issue #5661
-    # assume this repo state:
-    # seg1: PUT H1
-    # seg2: COMMIT
-    # seg3: DEL H1, PUT H1, DEL H1, PUT H2
-    # seg4: COMMIT
-    # Note how due to the final DEL H1 in seg3, H1 is effectively deleted.
-    #
-    # compaction of only seg3:
-    # PUT H1 gets dropped because it is not needed any more.
-    # DEL H1 must be kept, because there is still a PUT H1 in seg1 which must not
-    # "reappear" in the index if the index gets rebuilt.
-    with repository:
-        get_latest_segment = repository.io.get_latest_segment
-        repository.put(H(1), fchunk(b"1"))
-        # This is the segment with our original PUT of interest
-        put_segment = get_latest_segment()
-        repository.commit(compact=False)
-        # We now put H(1) again (which implicitly does DEL(H(1)) followed by PUT(H(1), ...)),
-        # delete H(1) afterwards, and force this segment to not be compacted, which can happen
-        # if it's not sparse enough (symbolized by H(2) here).
-        repository.put(H(1), fchunk(b"1"))
-        repository.delete(H(1))
-        repository.put(H(2), fchunk(b"1"))
-        delete_segment = get_latest_segment()
-        # We pretend these are mostly dense (not sparse) and won't be compacted
-        del repository.compact[put_segment]
-        del repository.compact[delete_segment]
-        repository.commit(compact=True)
-        # Now we perform an unrelated operation on the segment containing the DELETE,
-        # causing it to be compacted.
-        repository.delete(H(2))
-        repository.commit(compact=True)
-        assert repository.io.segment_exists(put_segment)
-        assert not repository.io.segment_exists(delete_segment)
-        # Basic case, since the index survived this must be ok
-        assert H(1) not in repository
-        # Nuke index, force replay
-        os.unlink(os.path.join(repository.path, "index.%d" % get_latest_segment()))
-        # Must not reappear
-        assert H(1) not in repository  # F
-
-
-def test_shadow_index_rollback(repository):
-    with repository:
-        repository.put(H(1), fchunk(b"1"))
-        repository.delete(H(1))
-        assert repository.shadow_index[H(1)] == [0]
-        repository.commit(compact=True)
-        repo_dump(repository, "p1 d1 cc")
-        # note how an empty list means that nothing is shadowed for sure
-        assert repository.shadow_index[H(1)] == []  # because the deletion is considered unstable
-        repository.put(H(1), b"1")
-        repository.delete(H(1))
-        repo_dump(repository, "p1 d1")
-        # 0 put/delete; 1 commit; 2 compacted; 3 commit; 4 put/delete
-        assert repository.shadow_index[H(1)] == [4]
-        repository.rollback()
-        repo_dump(repository, "r")
-        repository.put(H(2), fchunk(b"1"))
-        # after the rollback, segment 4 shouldn't be considered anymore
-        assert repository.shadow_index[H(1)] == []  # because the deletion is considered unstable
-
-
-def test_destroy_append_only(repository):
-    with repository:
-        # can't destroy append only repo (via the API)
-        repository.append_only = True
-        with pytest.raises(ValueError):
-            repository.destroy()
-        assert repository.append_only
-
-
-def test_append_only(repository):
-    def segments_in_repository(repo):
-        return len(list(repo.io.segment_iterator()))
-
-    with repository:
-        repository.append_only = True
-        repository.put(H(0), fchunk(b"foo"))
-        repository.commit(compact=False)
-
-        repository.append_only = False
-        assert segments_in_repository(repository) == 2
-        repository.put(H(0), fchunk(b"foo"))
-        repository.commit(compact=True)
-        # normal: compact squashes the data together, only one segment
-        assert segments_in_repository(repository) == 2
-
-        repository.append_only = True
-        assert segments_in_repository(repository) == 2
-        repository.put(H(0), fchunk(b"foo"))
-        repository.commit(compact=False)
-        # append only: does not compact, only new segments written
-        assert segments_in_repository(repository) == 4
-
-
-def test_additional_free_space(repository):
-    with repository:
-        add_keys(repository)
-        repository.config.set("repository", "additional_free_space", "1000T")
-        repository.save_key(b"shortcut to save_config")
-    with reopen(repository) as repository:
-        repository.put(H(0), fchunk(b"foobar"))
-        with pytest.raises(Repository.InsufficientFreeSpaceError):
-            repository.commit(compact=False)
-        assert os.path.exists(repository.path)
-
-
-def test_create_free_space(repository):
-    with repository:
-        repository.additional_free_space = 1e20
-        with pytest.raises(Repository.InsufficientFreeSpaceError):
-            add_keys(repository)
-        assert not os.path.exists(repository.path)
-
-
-def test_tracking(repository):
-    with repository:
-        assert repository.storage_quota_use == 0
-        ch1 = fchunk(bytes(1234))
-        repository.put(H(1), ch1)
-        assert repository.storage_quota_use == len(ch1) + 41 + 8
-        ch2 = fchunk(bytes(5678))
-        repository.put(H(2), ch2)
-        assert repository.storage_quota_use == len(ch1) + len(ch2) + 2 * (41 + 8)
-        repository.delete(H(1))
-        assert repository.storage_quota_use == len(ch1) + len(ch2) + 2 * (41 + 8)  # we have not compacted yet
-        repository.commit(compact=False)
-        assert repository.storage_quota_use == len(ch1) + len(ch2) + 2 * (41 + 8)  # we have not compacted yet
-    with reopen(repository) as repository:
-        # open new transaction; hints and thus quota data is not loaded unless needed.
-        ch3 = fchunk(b"")
-        repository.put(H(3), ch3)
-        repository.delete(H(3))
-        assert repository.storage_quota_use == len(ch1) + len(ch2) + len(ch3) + 3 * (
-            41 + 8
-        )  # we have not compacted yet
-        repository.commit(compact=True)
-        assert repository.storage_quota_use == len(ch2) + 41 + 8
-
-
-def test_exceed_quota(repository):
-    with repository:
-        assert repository.storage_quota_use == 0
-        repository.storage_quota = 80
-        ch1 = fchunk(b"x" * 7)
-        repository.put(H(1), ch1)
-        assert repository.storage_quota_use == len(ch1) + 41 + 8
-        repository.commit(compact=False)
-        with pytest.raises(Repository.StorageQuotaExceeded):
-            ch2 = fchunk(b"y" * 13)
-            repository.put(H(2), ch2)
-        assert repository.storage_quota_use == len(ch1) + len(ch2) + (41 + 8) * 2  # check ch2!?
-        with pytest.raises(Repository.StorageQuotaExceeded):
-            repository.commit(compact=False)
-        assert repository.storage_quota_use == len(ch1) + len(ch2) + (41 + 8) * 2  # check ch2!?
-    with reopen(repository) as repository:
-        repository.storage_quota = 161
-        # open new transaction; hints and thus quota data is not loaded unless needed.
-        repository.put(H(1), ch1)
-        # we have 2 puts for H(1) here and not yet compacted.
-        assert repository.storage_quota_use == len(ch1) * 2 + (41 + 8) * 2
-        repository.commit(compact=True)
-        assert repository.storage_quota_use == len(ch1) + 41 + 8  # now we have compacted.
-
-
-def make_auxiliary(repository):
-    with repository:
-        repository.put(H(0), fchunk(b"foo"))
-        repository.commit(compact=False)
-
-
-def do_commit(repository):
-    with repository:
-        repository.put(H(0), fchunk(b"fox"))
-        repository.commit(compact=False)
-
-
-def test_corrupted_hints(repository):
-    make_auxiliary(repository)
-    with open(os.path.join(repository.path, "hints.1"), "ab") as fd:
-        fd.write(b"123456789")
-    do_commit(repository)
-
-
-def test_deleted_hints(repository):
-    make_auxiliary(repository)
-    os.unlink(os.path.join(repository.path, "hints.1"))
-    do_commit(repository)
-
-
-def test_deleted_index(repository):
-    make_auxiliary(repository)
-    os.unlink(os.path.join(repository.path, "index.1"))
-    do_commit(repository)
-
-
-def test_unreadable_hints(repository):
-    make_auxiliary(repository)
-    hints = os.path.join(repository.path, "hints.1")
-    os.unlink(hints)
-    os.mkdir(hints)
-    with pytest.raises(OSError):
-        do_commit(repository)
-
-
-def test_index(repository):
-    make_auxiliary(repository)
-    with open(os.path.join(repository.path, "index.1"), "wb") as fd:
-        fd.write(b"123456789")
-    do_commit(repository)
-
-
-def test_index_outside_transaction(repository):
-    make_auxiliary(repository)
-    with open(os.path.join(repository.path, "index.1"), "wb") as fd:
-        fd.write(b"123456789")
-    with repository:
-        assert len(repository) == 1
-
-
-def _corrupt_index(repository):
-    # HashIndex is able to detect incorrect headers and file lengths,
-    # but on its own it can't tell if the data is correct.
-    index_path = os.path.join(repository.path, "index.1")
-    with open(index_path, "r+b") as fd:
-        index_data = fd.read()
-        # Flip one bit in a key stored in the index
-        corrupted_key = (int.from_bytes(H(0), "little") ^ 1).to_bytes(32, "little")
-        corrupted_index_data = index_data.replace(H(0), corrupted_key)
-        assert corrupted_index_data != index_data
-        assert len(corrupted_index_data) == len(index_data)
-        fd.seek(0)
-        fd.write(corrupted_index_data)
-
-
-def test_index_corrupted(repository):
-    make_auxiliary(repository)
-    _corrupt_index(repository)
-    with repository:
-        # data corruption is detected due to mismatching checksums, and fixed by rebuilding the index.
-        assert len(repository) == 1
-        assert pdchunk(repository.get(H(0))) == b"foo"
-
-
-def test_index_corrupted_without_integrity(repository):
-    make_auxiliary(repository)
-    _corrupt_index(repository)
-    integrity_path = os.path.join(repository.path, "integrity.1")
-    os.unlink(integrity_path)
-    with repository:
-        # since the corrupted key is not noticed, the repository still thinks it contains one key...
-        assert len(repository) == 1
-        with pytest.raises(Repository.ObjectNotFound):
-            # ... but the real, uncorrupted key is not found in the corrupted index.
-            repository.get(H(0))
-
-
-def test_unreadable_index(repository):
-    make_auxiliary(repository)
-    index = os.path.join(repository.path, "index.1")
-    os.unlink(index)
-    os.mkdir(index)
-    with pytest.raises(OSError):
-        do_commit(repository)
-
-
-def test_unknown_integrity_version(repository):
-    make_auxiliary(repository)
-    # for now an unknown integrity data version is ignored and not an error.
-    integrity_path = os.path.join(repository.path, "integrity.1")
-    with open(integrity_path, "r+b") as fd:
-        msgpack.pack({b"version": 4.7}, fd)  # borg only understands version 2
-        fd.truncate()
-    with repository:
-        # no issues accessing the repository
-        assert len(repository) == 1
-        assert pdchunk(repository.get(H(0))) == b"foo"
-
-
-def _subtly_corrupted_hints_setup(repository):
-    with repository:
-        repository.append_only = True
-        assert len(repository) == 1
-        assert pdchunk(repository.get(H(0))) == b"foo"
-        repository.put(H(1), fchunk(b"bar"))
-        repository.put(H(2), fchunk(b"baz"))
-        repository.commit(compact=False)
-        repository.put(H(2), fchunk(b"bazz"))
-        repository.commit(compact=False)
-    hints_path = os.path.join(repository.path, "hints.5")
-    with open(hints_path, "r+b") as fd:
-        hints = msgpack.unpack(fd)
-        fd.seek(0)
-        # corrupt segment refcount
-        assert hints["segments"][2] == 1
-        hints["segments"][2] = 0
-        msgpack.pack(hints, fd)
-        fd.truncate()
-
-
-def test_subtly_corrupted_hints(repository):
-    make_auxiliary(repository)
-    _subtly_corrupted_hints_setup(repository)
-    with repository:
-        repository.append_only = False
-        repository.put(H(3), fchunk(b"1234"))
-        # do a compaction run, which succeeds since the failed checksum prompted a rebuild of the index+hints.
-        repository.commit(compact=True)
-        assert len(repository) == 4
-        assert pdchunk(repository.get(H(0))) == b"foo"
-        assert pdchunk(repository.get(H(1))) == b"bar"
-        assert pdchunk(repository.get(H(2))) == b"bazz"
-
-
-def test_subtly_corrupted_hints_without_integrity(repository):
-    make_auxiliary(repository)
-    _subtly_corrupted_hints_setup(repository)
-    integrity_path = os.path.join(repository.path, "integrity.5")
-    os.unlink(integrity_path)
-    with repository:
-        repository.append_only = False
-        repository.put(H(3), fchunk(b"1234"))
-        # do a compaction run, which fails since the corrupted refcount wasn't detected and causes an assertion failure.
-        with pytest.raises(AssertionError) as exc_info:
-            repository.commit(compact=True)
-        assert "Corrupted segment reference count" in str(exc_info.value)
-
-
-def list_indices(repo_path):
-    return [name for name in os.listdir(repo_path) if name.startswith("index.")]
-
-
 def check(repository, repo_path, repair=False, status=True):
     assert repository.check(repair=repair) == status
     # Make sure no tmp files are left behind
@@ -787,209 +153,6 @@ def check(repository, repo_path, repair=False, status=True):
     assert tmp_files == [], "Found tmp files"
 
 
-def get_objects(repository, *ids):
-    for id_ in ids:
-        pdchunk(repository.get(H(id_)))
-
-
-def add_objects(repository, segments):
-    for ids in segments:
-        for id_ in ids:
-            repository.put(H(id_), fchunk(b"data"))
-        repository.commit(compact=False)
-
-
-def get_head(repo_path):
-    return sorted(int(n) for n in os.listdir(os.path.join(repo_path, "data", "0")) if n.isdigit())[-1]
-
-
-def open_index(repo_path):
-    return NSIndex.read(os.path.join(repo_path, f"index.{get_head(repo_path)}"))
-
-
-def corrupt_object(repo_path, id_):
-    idx = open_index(repo_path)
-    segment, offset, _ = idx[H(id_)]
-    with open(os.path.join(repo_path, "data", "0", str(segment)), "r+b") as fd:
-        fd.seek(offset)
-        fd.write(b"BOOM")
-
-
-def delete_segment(repository, segment):
-    repository.io.delete_segment(segment)
-
-
-def delete_index(repo_path):
-    os.unlink(os.path.join(repo_path, f"index.{get_head(repo_path)}"))
-
-
-def rename_index(repo_path, new_name):
-    os.replace(os.path.join(repo_path, f"index.{get_head(repo_path)}"), os.path.join(repo_path, new_name))
-
-
-def list_objects(repository):
-    return {int(key) for key in repository.list()}
-
-
-def test_repair_corrupted_segment(repo_fixtures, request):
-    with get_repository_from_fixture(repo_fixtures, request) as repository:
-        repo_path = get_path(repository)
-        add_objects(repository, [[1, 2, 3], [4, 5], [6]])
-        assert {1, 2, 3, 4, 5, 6} == list_objects(repository)
-        check(repository, repo_path, status=True)
-        corrupt_object(repo_path, 5)
-        with pytest.raises(IntegrityError):
-            get_objects(repository, 5)
-        repository.rollback()
-        # make sure a regular check does not repair anything
-        check(repository, repo_path, status=False)
-        check(repository, repo_path, status=False)
-        # make sure a repair actually repairs the repo
-        check(repository, repo_path, repair=True, status=True)
-        get_objects(repository, 4)
-        check(repository, repo_path, status=True)
-        assert {1, 2, 3, 4, 6} == list_objects(repository)
-
-
-def test_repair_missing_segment(repository):
-    # only test on local repo - files in RemoteRepository cannot be deleted
-    with repository:
-        add_objects(repository, [[1, 2, 3], [4, 5, 6]])
-        assert {1, 2, 3, 4, 5, 6} == list_objects(repository)
-        check(repository, repository.path, status=True)
-        delete_segment(repository, 2)
-        repository.rollback()
-        check(repository, repository.path, repair=True, status=True)
-        assert {1, 2, 3} == list_objects(repository)
-
-
-def test_repair_missing_commit_segment(repository):
-    # only test on local repo - files in RemoteRepository cannot be deleted
-    with repository:
-        add_objects(repository, [[1, 2, 3], [4, 5, 6]])
-        delete_segment(repository, 3)
-        with pytest.raises(Repository.ObjectNotFound):
-            get_objects(repository, 4)
-        assert {1, 2, 3} == list_objects(repository)
-
-
-def test_repair_corrupted_commit_segment(repo_fixtures, request):
-    with get_repository_from_fixture(repo_fixtures, request) as repository:
-        repo_path = get_path(repository)
-        add_objects(repository, [[1, 2, 3], [4, 5, 6]])
-        with open(os.path.join(repo_path, "data", "0", "3"), "r+b") as fd:
-            fd.seek(-1, os.SEEK_END)
-            fd.write(b"X")
-        with pytest.raises(Repository.ObjectNotFound):
-            get_objects(repository, 4)
-        check(repository, repo_path, status=True)
-        get_objects(repository, 3)
-        assert {1, 2, 3} == list_objects(repository)
-
-
-def test_repair_no_commits(repo_fixtures, request):
-    with get_repository_from_fixture(repo_fixtures, request) as repository:
-        repo_path = get_path(repository)
-        add_objects(repository, [[1, 2, 3]])
-        with open(os.path.join(repo_path, "data", "0", "1"), "r+b") as fd:
-            fd.seek(-1, os.SEEK_END)
-            fd.write(b"X")
-        with pytest.raises(Repository.CheckNeeded):
-            get_objects(repository, 4)
-        check(repository, repo_path, status=False)
-        check(repository, repo_path, status=False)
-        assert list_indices(repo_path) == ["index.1"]
-        check(repository, repo_path, repair=True, status=True)
-        assert list_indices(repo_path) == ["index.2"]
-        check(repository, repo_path, status=True)
-        get_objects(repository, 3)
-        assert {1, 2, 3} == list_objects(repository)
-
-
-def test_repair_missing_index(repo_fixtures, request):
-    with get_repository_from_fixture(repo_fixtures, request) as repository:
-        repo_path = get_path(repository)
-        add_objects(repository, [[1, 2, 3], [4, 5, 6]])
-        delete_index(repo_path)
-        check(repository, repo_path, status=True)
-        get_objects(repository, 4)
-        assert {1, 2, 3, 4, 5, 6} == list_objects(repository)
-
-
-def test_repair_index_too_new(repo_fixtures, request):
-    with get_repository_from_fixture(repo_fixtures, request) as repository:
-        repo_path = get_path(repository)
-        add_objects(repository, [[1, 2, 3], [4, 5, 6]])
-        assert list_indices(repo_path) == ["index.3"]
-        rename_index(repo_path, "index.100")
-        check(repository, repo_path, status=True)
-        assert list_indices(repo_path) == ["index.3"]
-        get_objects(repository, 4)
-        assert {1, 2, 3, 4, 5, 6} == list_objects(repository)
-
-
-def test_crash_before_compact(repository):
-    # only test on local repo - we can't mock-patch a RemoteRepository class in another process!
-    with repository:
-        repository.put(H(0), fchunk(b"data"))
-        repository.put(H(0), fchunk(b"data2"))
-        # simulate a crash before compact
-        with patch.object(Repository, "compact_segments") as compact:
-            repository.commit(compact=True)
-            compact.assert_called_once_with(0.1)
-    with reopen(repository) as repository:
-        check(repository, repository.path, repair=True)
-        assert pdchunk(repository.get(H(0))) == b"data2"
-
-
-def test_hints_persistence(repository):
-    with repository:
-        repository.put(H(0), fchunk(b"data"))
-        repository.delete(H(0))
-        repository.commit(compact=False)
-        shadow_index_expected = repository.shadow_index
-        compact_expected = repository.compact
-        segments_expected = repository.segments
-    # close and re-open the repository (create fresh Repository instance) to
-    # check whether hints were persisted to / reloaded from disk
-    with reopen(repository) as repository:
-        repository.put(H(42), fchunk(b"foobar"))  # this will call prepare_txn() and load the hints data
-        # check if hints persistence worked:
-        assert shadow_index_expected == repository.shadow_index
-        assert compact_expected == repository.compact
-        del repository.segments[2]  # ignore the segment created by put(H(42), ...)
-        assert segments_expected == repository.segments
-    with reopen(repository) as repository:
-        check(repository, repository.path, repair=True)
-    with reopen(repository) as repository:
-        repository.put(H(42), fchunk(b"foobar"))  # this will call prepare_txn() and load the hints data
-        assert shadow_index_expected == repository.shadow_index
-        # sizes do not match, with vs. without header?
-        # assert compact_expected == repository.compact
-        del repository.segments[2]  # ignore the segment created by put(H(42), ...)
-        assert segments_expected == repository.segments
-
-
-def test_hints_behaviour(repository):
-    with repository:
-        repository.put(H(0), fchunk(b"data"))
-        assert repository.shadow_index == {}
-        assert len(repository.compact) == 0
-        repository.delete(H(0))
-        repository.commit(compact=False)
-        # now there should be an entry for H(0) in shadow_index
-        assert H(0) in repository.shadow_index
-        assert len(repository.shadow_index[H(0)]) == 1
-        assert 0 in repository.compact  # segment 0 can be compacted
-        repository.put(H(42), fchunk(b"foobar"))  # see also do_compact()
-        repository.commit(compact=True, threshold=0.0)  # compact completely!
-        # nothing to compact anymore! no info left about stuff that does not exist anymore:
-        assert H(0) not in repository.shadow_index
-        # segment 0 was compacted away, no info about it left:
-        assert 0 not in repository.compact
-        assert 0 not in repository.segments
-
-
 def _get_mock_args():
     class MockArgs:
         remote_path = "borg"

+ 0 - 277
src/borg/testsuite/repository3.py

@@ -1,277 +0,0 @@
-import logging
-import os
-import sys
-from typing import Optional
-
-import pytest
-
-from ..checksums import xxh64
-from ..helpers import Location
-from ..helpers import IntegrityError
-from ..platformflags import is_win32
-from ..remote3 import RemoteRepository3, InvalidRPCMethod, PathNotAllowed
-from ..repository3 import Repository3, MAX_DATA_SIZE
-from ..repoobj import RepoObj
-from .hashindex import H
-
-
-@pytest.fixture()
-def repository(tmp_path):
-    repository_location = os.fspath(tmp_path / "repository")
-    yield Repository3(repository_location, exclusive=True, create=True)
-
-
-@pytest.fixture()
-def remote_repository(tmp_path):
-    if is_win32:
-        pytest.skip("Remote repository does not yet work on Windows.")
-    repository_location = Location("ssh://__testsuite__" + os.fspath(tmp_path / "repository"))
-    yield RemoteRepository3(repository_location, exclusive=True, create=True)
-
-
-def pytest_generate_tests(metafunc):
-    # Generates tests that run on both local and remote repos
-    if "repo_fixtures" in metafunc.fixturenames:
-        metafunc.parametrize("repo_fixtures", ["repository", "remote_repository"])
-
-
-def get_repository_from_fixture(repo_fixtures, request):
-    # returns the repo object from the fixture for tests that run on both local and remote repos
-    return request.getfixturevalue(repo_fixtures)
-
-
-def reopen(repository, exclusive: Optional[bool] = True, create=False):
-    if isinstance(repository, Repository3):
-        if repository.opened:
-            raise RuntimeError("Repo must be closed before a reopen. Cannot support nested repository contexts.")
-        return Repository3(repository.path, exclusive=exclusive, create=create)
-
-    if isinstance(repository, RemoteRepository3):
-        if repository.p is not None or repository.sock is not None:
-            raise RuntimeError("Remote repo must be closed before a reopen. Cannot support nested repository contexts.")
-        return RemoteRepository3(repository.location, exclusive=exclusive, create=create)
-
-    raise TypeError(
-        f"Invalid argument type. Expected 'Repository3' or 'RemoteRepository3', received '{type(repository).__name__}'."
-    )
-
-
-def fchunk(data, meta=b""):
-    # format chunk: create a raw chunk that has valid RepoObj layout, but does not use encryption or compression.
-    hdr = RepoObj.obj_header.pack(len(meta), len(data), xxh64(meta), xxh64(data))
-    assert isinstance(data, bytes)
-    chunk = hdr + meta + data
-    return chunk
-
-
-def pchunk(chunk):
-    # parse chunk: parse data and meta from a raw chunk made by fchunk
-    hdr_size = RepoObj.obj_header.size
-    hdr = chunk[:hdr_size]
-    meta_size, data_size = RepoObj.obj_header.unpack(hdr)[0:2]
-    meta = chunk[hdr_size : hdr_size + meta_size]
-    data = chunk[hdr_size + meta_size : hdr_size + meta_size + data_size]
-    return data, meta
-
-
-def pdchunk(chunk):
-    # parse only data from a raw chunk made by fchunk
-    return pchunk(chunk)[0]
-
-
-def test_basic_operations(repo_fixtures, request):
-    with get_repository_from_fixture(repo_fixtures, request) as repository:
-        for x in range(100):
-            repository.put(H(x), fchunk(b"SOMEDATA"))
-        key50 = H(50)
-        assert pdchunk(repository.get(key50)) == b"SOMEDATA"
-        repository.delete(key50)
-        with pytest.raises(Repository3.ObjectNotFound):
-            repository.get(key50)
-    with reopen(repository) as repository:
-        with pytest.raises(Repository3.ObjectNotFound):
-            repository.get(key50)
-        for x in range(100):
-            if x == 50:
-                continue
-            assert pdchunk(repository.get(H(x))) == b"SOMEDATA"
-
-
-def test_read_data(repo_fixtures, request):
-    with get_repository_from_fixture(repo_fixtures, request) as repository:
-        meta, data = b"meta", b"data"
-        hdr = RepoObj.obj_header.pack(len(meta), len(data), xxh64(meta), xxh64(data))
-        chunk_complete = hdr + meta + data
-        chunk_short = hdr + meta
-        repository.put(H(0), chunk_complete)
-        assert repository.get(H(0)) == chunk_complete
-        assert repository.get(H(0), read_data=True) == chunk_complete
-        assert repository.get(H(0), read_data=False) == chunk_short
-
-
-def test_consistency(repo_fixtures, request):
-    with get_repository_from_fixture(repo_fixtures, request) as repository:
-        repository.put(H(0), fchunk(b"foo"))
-        assert pdchunk(repository.get(H(0))) == b"foo"
-        repository.put(H(0), fchunk(b"foo2"))
-        assert pdchunk(repository.get(H(0))) == b"foo2"
-        repository.put(H(0), fchunk(b"bar"))
-        assert pdchunk(repository.get(H(0))) == b"bar"
-        repository.delete(H(0))
-        with pytest.raises(Repository3.ObjectNotFound):
-            repository.get(H(0))
-
-
-def test_list(repo_fixtures, request):
-    with get_repository_from_fixture(repo_fixtures, request) as repository:
-        for x in range(100):
-            repository.put(H(x), fchunk(b"SOMEDATA"))
-        repo_list = repository.list()
-        assert len(repo_list) == 100
-        first_half = repository.list(limit=50)
-        assert len(first_half) == 50
-        assert first_half == repo_list[:50]
-        second_half = repository.list(marker=first_half[-1])
-        assert len(second_half) == 50
-        assert second_half == repo_list[50:]
-        assert len(repository.list(limit=50)) == 50
-
-
-def test_max_data_size(repo_fixtures, request):
-    with get_repository_from_fixture(repo_fixtures, request) as repository:
-        max_data = b"x" * (MAX_DATA_SIZE - RepoObj.obj_header.size)
-        repository.put(H(0), fchunk(max_data))
-        assert pdchunk(repository.get(H(0))) == max_data
-        with pytest.raises(IntegrityError):
-            repository.put(H(1), fchunk(max_data + b"x"))
-
-
-def check(repository, repo_path, repair=False, status=True):
-    assert repository.check(repair=repair) == status
-    # Make sure no tmp files are left behind
-    tmp_files = [name for name in os.listdir(repo_path) if "tmp" in name]
-    assert tmp_files == [], "Found tmp files"
-
-
-def _get_mock_args():
-    class MockArgs:
-        remote_path = "borg"
-        umask = 0o077
-        debug_topics = []
-        rsh = None
-
-        def __contains__(self, item):
-            # to behave like argparse.Namespace
-            return hasattr(self, item)
-
-    return MockArgs()
-
-
-def test_remote_invalid_rpc(remote_repository):
-    with remote_repository:
-        with pytest.raises(InvalidRPCMethod):
-            remote_repository.call("__init__", {})
-
-
-def test_remote_rpc_exception_transport(remote_repository):
-    with remote_repository:
-        s1 = "test string"
-
-        try:
-            remote_repository.call("inject_exception", {"kind": "DoesNotExist"})
-        except Repository3.DoesNotExist as e:
-            assert len(e.args) == 1
-            assert e.args[0] == remote_repository.location.processed
-
-        try:
-            remote_repository.call("inject_exception", {"kind": "AlreadyExists"})
-        except Repository3.AlreadyExists as e:
-            assert len(e.args) == 1
-            assert e.args[0] == remote_repository.location.processed
-
-        try:
-            remote_repository.call("inject_exception", {"kind": "CheckNeeded"})
-        except Repository3.CheckNeeded as e:
-            assert len(e.args) == 1
-            assert e.args[0] == remote_repository.location.processed
-
-        try:
-            remote_repository.call("inject_exception", {"kind": "IntegrityError"})
-        except IntegrityError as e:
-            assert len(e.args) == 1
-            assert e.args[0] == s1
-
-        try:
-            remote_repository.call("inject_exception", {"kind": "PathNotAllowed"})
-        except PathNotAllowed as e:
-            assert len(e.args) == 1
-            assert e.args[0] == "foo"
-
-        try:
-            remote_repository.call("inject_exception", {"kind": "ObjectNotFound"})
-        except Repository3.ObjectNotFound as e:
-            assert len(e.args) == 2
-            assert e.args[0] == s1
-            assert e.args[1] == remote_repository.location.processed
-
-        try:
-            remote_repository.call("inject_exception", {"kind": "InvalidRPCMethod"})
-        except InvalidRPCMethod as e:
-            assert len(e.args) == 1
-            assert e.args[0] == s1
-
-        try:
-            remote_repository.call("inject_exception", {"kind": "divide"})
-        except RemoteRepository3.RPCError as e:
-            assert e.unpacked
-            assert e.get_message() == "ZeroDivisionError: integer division or modulo by zero\n"
-            assert e.exception_class == "ZeroDivisionError"
-            assert len(e.exception_full) > 0
-
-
-def test_remote_ssh_cmd(remote_repository):
-    with remote_repository:
-        args = _get_mock_args()
-        remote_repository._args = args
-        assert remote_repository.ssh_cmd(Location("ssh://example.com/foo")) == ["ssh", "example.com"]
-        assert remote_repository.ssh_cmd(Location("ssh://user@example.com/foo")) == ["ssh", "user@example.com"]
-        assert remote_repository.ssh_cmd(Location("ssh://user@example.com:1234/foo")) == [
-            "ssh",
-            "-p",
-            "1234",
-            "user@example.com",
-        ]
-        os.environ["BORG_RSH"] = "ssh --foo"
-        assert remote_repository.ssh_cmd(Location("ssh://example.com/foo")) == ["ssh", "--foo", "example.com"]
-
-
-def test_remote_borg_cmd(remote_repository):
-    with remote_repository:
-        assert remote_repository.borg_cmd(None, testing=True) == [sys.executable, "-m", "borg", "serve"]
-        args = _get_mock_args()
-        # XXX without next line we get spurious test fails when using pytest-xdist, root cause unknown:
-        logging.getLogger().setLevel(logging.INFO)
-        # note: test logger is on info log level, so --info gets added automagically
-        assert remote_repository.borg_cmd(args, testing=False) == ["borg", "serve", "--info"]
-        args.remote_path = "borg-0.28.2"
-        assert remote_repository.borg_cmd(args, testing=False) == ["borg-0.28.2", "serve", "--info"]
-        args.debug_topics = ["something_client_side", "repository_compaction"]
-        assert remote_repository.borg_cmd(args, testing=False) == [
-            "borg-0.28.2",
-            "serve",
-            "--info",
-            "--debug-topic=borg.debug.repository_compaction",
-        ]
-        args = _get_mock_args()
-        args.storage_quota = 0
-        assert remote_repository.borg_cmd(args, testing=False) == ["borg", "serve", "--info"]
-        args.storage_quota = 314159265
-        assert remote_repository.borg_cmd(args, testing=False) == [
-            "borg",
-            "serve",
-            "--info",
-            "--storage-quota=314159265",
-        ]
-        args.rsh = "ssh -i foo"
-        remote_repository._args = args
-        assert remote_repository.ssh_cmd(Location("ssh://example.com/foo")) == ["ssh", "-i", "foo", "example.com"]

+ 1 - 1
src/borg/testsuite/locking3.py → src/borg/testsuite/storelocking.py

@@ -4,7 +4,7 @@ import pytest
 
 from borgstore.store import Store
 
-from ..locking3 import Lock, LockFailed, NotLocked
+from ..storelocking import Lock, LockFailed, NotLocked
 
 ID1 = "foo", 1, 1
 ID2 = "bar", 2, 2

Деякі файли не було показано, через те що забагато файлів було змінено