Browse Source

Item: remove .chunks_healthy, fixes #8559

Well, it's not totally removed, some code in Item, Archive and
borg transfer --from-borg1 needs to stay in place, so that we
can pick the CORRECT chunks list that is in .chunks_healthy
for all-zero-replacement-chunk-patched items when transferring
archives from borg1 to borg2 repos.

transfer: do not transfer replacement chunks, deal with missing chunks in other_repo

FUSE fs read: IOError or all-zero result
Thomas Waldmann 6 months ago
parent
commit
6357f2ebc1

+ 2 - 2
docs/internals/frontends.rst

@@ -480,8 +480,8 @@ Refer to the *borg list* documentation for the available keys and their meaning.
 
 Example (excerpt) of ``borg list --json-lines``::
 
-    {"type": "d", "mode": "drwxr-xr-x", "user": "user", "group": "user", "uid": 1000, "gid": 1000, "path": "linux", "healthy": true, "target": "", "flags": null, "mtime": "2017-02-27T12:27:20.023407", "size": 0}
-    {"type": "d", "mode": "drwxr-xr-x", "user": "user", "group": "user", "uid": 1000, "gid": 1000, "path": "linux/baz", "healthy": true, "target": "", "flags": null, "mtime": "2017-02-27T12:27:20.585407", "size": 0}
+    {"type": "d", "mode": "drwxr-xr-x", "user": "user", "group": "user", "uid": 1000, "gid": 1000, "path": "linux", "target": "", "flags": null, "mtime": "2017-02-27T12:27:20.023407", "size": 0}
+    {"type": "d", "mode": "drwxr-xr-x", "user": "user", "group": "user", "uid": 1000, "gid": 1000, "path": "linux/baz", "target": "", "flags": null, "mtime": "2017-02-27T12:27:20.585407", "size": 0}
 
 
 Archive Differencing

+ 13 - 90
src/borg/archive.py

@@ -280,7 +280,7 @@ class DownloadPipeline:
                 if filter is None or filter(item):
                     if "chunks" in item:
                         item.chunks = [ChunkListEntry(*e) for e in item.chunks]
-                    if "chunks_healthy" in item:
+                    if "chunks_healthy" in item:  # legacy
                         item.chunks_healthy = [ChunkListEntry(*e) for e in item.chunks_healthy]
                     yield item
 
@@ -762,7 +762,6 @@ Duration: {0.duration}
             # if a previous extraction was interrupted between setting the mtime and setting non-default flags.
             return True
 
-        has_damaged_chunks = "chunks_healthy" in item
         if dry_run or stdout:
             with self.extract_helper(item, "", hlm, dry_run=dry_run or stdout) as hardlink_set:
                 if not hardlink_set:
@@ -789,8 +788,6 @@ Duration: {0.duration}
                                         item_size, item_chunks_size
                                     )
                                 )
-            if has_damaged_chunks:
-                raise BackupError("File has damaged (all-zero) chunks. Try running borg check --repair.")
             return
 
         dest = self.cwd
@@ -845,8 +842,6 @@ Duration: {0.duration}
                         raise BackupError(
                             f"Size inconsistency detected: size {item_size}, chunks size {item_chunks_size}"
                         )
-                if has_damaged_chunks:
-                    raise BackupError("File has damaged (all-zero) chunks. Try running borg check --repair.")
             return
         with backup_io:
             # No repository access beyond this point.
@@ -1159,10 +1154,6 @@ class ChunksProcessor:
                 return chunk_entry
 
         item.chunks = []
-        # if we rechunkify, we'll get a fundamentally different chunks list, thus we need
-        # to get rid of .chunks_healthy, as it might not correspond to .chunks any more.
-        if self.rechunkify and "chunks_healthy" in item:
-            del item.chunks_healthy
         for chunk in chunk_iter:
             chunk_entry = chunk_processor(chunk)
             item.chunks.append(chunk_entry)
@@ -1779,13 +1770,10 @@ class ArchiveChecker:
         if defect_chunks:
             if self.repair:
                 # if we kill the defect chunk here, subsequent actions within this "borg check"
-                # run will find missing chunks and replace them with all-zero replacement
-                # chunks and flag the files as "repaired".
-                # if another backup is done later and the missing chunks get backed up again,
-                # a "borg check" afterwards can heal all files where this chunk was missing.
+                # run will find missing chunks.
                 logger.warning(
-                    "Found defect chunks. They will be deleted now, so affected files can "
-                    "get repaired now and maybe healed later."
+                    "Found defect chunks and will delete them now. "
+                    "Reading files referencing these chunks will result in an I/O error."
                 )
                 for defect_chunk in defect_chunks:
                     # remote repo (ssh): retry might help for strange network / NIC / RAM errors
@@ -1805,10 +1793,7 @@ class ArchiveChecker:
                     else:
                         logger.warning("chunk %s not deleted, did not consistently fail.", bin_to_hex(defect_chunk))
             else:
-                logger.warning(
-                    "Found defect chunks. With --repair, they would get deleted, so affected "
-                    "files could get repaired then and maybe healed later."
-                )
+                logger.warning("Found defect chunks. With --repair, they would get deleted.")
                 for defect_chunk in defect_chunks:
                     logger.debug("chunk %s is defect.", bin_to_hex(defect_chunk))
         log = logger.error if errors else logger.info
@@ -1919,80 +1904,18 @@ class ArchiveChecker:
                     self.repository.put(id_, cdata)
 
         def verify_file_chunks(archive_name, item):
-            """Verifies that all file chunks are present.
-
-            Missing file chunks will be replaced with new chunks of the same length containing all zeros.
-            If a previously missing file chunk re-appears, the replacement chunk is replaced by the correct one.
-            """
-
-            def replacement_chunk(size):
-                chunk = Chunk(None, allocation=CH_ALLOC, size=size)
-                chunk_id, data = cached_hash(chunk, self.key.id_hash)
-                cdata = self.repo_objs.format(chunk_id, {}, data, ro_type=ROBJ_FILE_STREAM)
-                return chunk_id, size, cdata
-
+            """Verifies that all file chunks are present. Missing file chunks will be logged."""
             offset = 0
-            chunk_list = []
-            chunks_replaced = False
-            has_chunks_healthy = "chunks_healthy" in item
-            chunks_current = item.chunks
-            chunks_healthy = item.chunks_healthy if has_chunks_healthy else chunks_current
-            if has_chunks_healthy and len(chunks_current) != len(chunks_healthy):
-                # should never happen, but there was issue #3218.
-                logger.warning(f"{archive_name}: {item.path}: Invalid chunks_healthy metadata removed!")
-                del item.chunks_healthy
-                has_chunks_healthy = False
-                chunks_healthy = chunks_current
-            for chunk_current, chunk_healthy in zip(chunks_current, chunks_healthy):
-                chunk_id, size = chunk_healthy
+            for chunk in item.chunks:
+                chunk_id, size = chunk
                 if chunk_id not in self.chunks:
-                    # a chunk of the healthy list is missing
-                    if chunk_current == chunk_healthy:
-                        logger.error(
-                            "{}: {}: New missing file chunk detected (Byte {}-{}, Chunk {}). "
-                            "Replacing with all-zero chunk.".format(
-                                archive_name, item.path, offset, offset + size, bin_to_hex(chunk_id)
-                            )
+                    logger.error(
+                        "{}: {}: Missing file chunk detected (Byte {}-{}, Chunk {}).".format(
+                            archive_name, item.path, offset, offset + size, bin_to_hex(chunk_id)
                         )
-                        self.error_found = chunks_replaced = True
-                        chunk_id, size, cdata = replacement_chunk(size)
-                        add_reference(chunk_id, size, cdata)
-                    else:
-                        logger.info(
-                            "{}: {}: Previously missing file chunk is still missing (Byte {}-{}, Chunk {}). "
-                            "It has an all-zero replacement chunk already.".format(
-                                archive_name, item.path, offset, offset + size, bin_to_hex(chunk_id)
-                            )
-                        )
-                        chunk_id, size = chunk_current
-                        if chunk_id not in self.chunks:
-                            logger.warning(
-                                "{}: {}: Missing all-zero replacement chunk detected (Byte {}-{}, Chunk {}). "
-                                "Generating new replacement chunk.".format(
-                                    archive_name, item.path, offset, offset + size, bin_to_hex(chunk_id)
-                                )
-                            )
-                            self.error_found = chunks_replaced = True
-                            chunk_id, size, cdata = replacement_chunk(size)
-                            add_reference(chunk_id, size, cdata)
-                else:
-                    if chunk_current == chunk_healthy:
-                        pass  # normal case, all fine.
-                    else:
-                        logger.info(
-                            "{}: {}: Healed previously missing file chunk! (Byte {}-{}, Chunk {}).".format(
-                                archive_name, item.path, offset, offset + size, bin_to_hex(chunk_id)
-                            )
-                        )
-                chunk_list.append([chunk_id, size])  # list-typed element as chunks_healthy is list-of-lists
+                    )
+                    self.error_found = True
                 offset += size
-            if chunks_replaced and not has_chunks_healthy:
-                # if this is first repair, remember the correct chunk IDs, so we can maybe heal the file later
-                item.chunks_healthy = item.chunks
-            if has_chunks_healthy and chunk_list == chunks_healthy:
-                logger.info(f"{archive_name}: {item.path}: Completely healed previously damaged file!")
-                del item.chunks_healthy
-            item.chunks = chunk_list
             if "size" in item:
                 item_size = item.size
                 item_chunks_size = item.get_size(from_chunks=True)

+ 1 - 22
src/borg/archiver/check_cmd.py

@@ -168,28 +168,7 @@ class CheckMixIn:
 
         2. When checking the consistency and correctness of archives, repair mode might
            remove whole archives from the manifest if their archive metadata chunk is
-           corrupt or lost. On a chunk level (i.e. the contents of files), repair mode
-           will replace corrupt or lost chunks with a same-size replacement chunk of
-           zeroes. If a previously zeroed chunk reappears, repair mode will restore
-           this lost chunk using the new chunk.
-
-        Most steps taken by repair mode have a one-time effect on the repository, like
-        removing a lost archive from the repository. However, replacing a corrupt or
-        lost chunk with an all-zero replacement will have an ongoing effect on the
-        repository: When attempting to extract a file referencing an all-zero chunk,
-        the ``extract`` command will distinctly warn about it. The FUSE filesystem
-        created by the ``mount`` command will reject reading such a "zero-patched"
-        file unless a special mount option is given.
-
-        As mentioned earlier, Borg might be able to "heal" a "zero-patched" file in
-        repair mode, if all its previously lost chunks reappear (e.g. via a later
-        backup). This is achieved by Borg not only keeping track of the all-zero
-        replacement chunks, but also by keeping metadata about the lost chunks. In
-        repair mode Borg will check whether a previously lost chunk reappeared and will
-        replace the all-zero replacement chunk by the reappeared chunk. If all lost
-        chunks of a "zero-patched" file reappear, this effectively "heals" the file.
-        Consequently, if lost chunks were repaired earlier, it is advised to run
-        ``--repair`` a second time after creating some new backups.
+           corrupt or lost. Borg will also report files that reference missing chunks.
 
         If ``--repair --find-lost-archives`` is given, previously lost entries will
         be recreated in the archive directory. This is only possible before

+ 8 - 25
src/borg/archiver/compact_cmd.py

@@ -6,7 +6,7 @@ from ..archive import Archive
 from ..cache import write_chunkindex_to_repo_cache, build_chunkindex_from_repo
 from ..constants import *  # NOQA
 from ..hashindex import ChunkIndex, ChunkIndexEntry
-from ..helpers import set_ec, EXIT_WARNING, EXIT_ERROR, format_file_size, bin_to_hex
+from ..helpers import set_ec, EXIT_ERROR, format_file_size, bin_to_hex
 from ..helpers import ProgressIndicatorPercent
 from ..manifest import Manifest
 from ..remote import RemoteRepository
@@ -39,9 +39,7 @@ class ArchiveGarbageCollector:
         logger.info("Starting compaction / garbage collection...")
         self.chunks = self.get_repository_chunks()
         logger.info("Computing object IDs used by archives...")
-        (self.missing_chunks, self.reappeared_chunks, self.total_files, self.total_size, self.archives_count) = (
-            self.analyze_archives()
-        )
+        (self.missing_chunks, self.total_files, self.total_size, self.archives_count) = self.analyze_archives()
         self.report_and_delete()
         self.save_chunk_index()
         logger.info("Finished compaction / garbage collection...")
@@ -73,28 +71,24 @@ class ArchiveGarbageCollector:
             self.chunks.clear()  # we already have updated the repo cache in get_repository_chunks
         self.chunks = None  # nothing there (cleared!)
 
-    def analyze_archives(self) -> Tuple[Set, Set, int, int, int]:
-        """Iterate over all items in all archives, create the dicts id -> size of all used/wanted chunks."""
+    def analyze_archives(self) -> Tuple[Set, int, int, int]:
+        """Iterate over all items in all archives, create the dicts id -> size of all used chunks."""
 
-        def use_it(id, *, wanted=False):
+        def use_it(id):
             entry = self.chunks.get(id)
             if entry is not None:
                 # the chunk is in the repo, mark it used.
                 self.chunks[id] = entry._replace(flags=entry.flags | ChunkIndex.F_USED)
-                if wanted:
-                    # chunk id is from chunks_healthy list: a lost chunk has re-appeared!
-                    reappeared_chunks.add(id)
             else:
                 # with --stats: we do NOT have this chunk in the repository!
                 # without --stats: we do not have this chunk or the chunks index is incomplete.
                 missing_chunks.add(id)
 
         missing_chunks: set[bytes] = set()
-        reappeared_chunks: set[bytes] = set()
         archive_infos = self.manifest.archives.list(sort_by=["ts"])
         num_archives = len(archive_infos)
         pi = ProgressIndicatorPercent(
-            total=num_archives, msg="Computing used/wanted chunks %3.1f%%", step=0.1, msgid="compact.analyze_archives"
+            total=num_archives, msg="Computing used chunks %3.1f%%", step=0.1, msgid="compact.analyze_archives"
         )
         total_size, total_files = 0, 0
         for i, info in enumerate(archive_infos):
@@ -114,25 +108,14 @@ class ArchiveGarbageCollector:
                     for id, size in item.chunks:
                         total_size += size  # original, uncompressed file content size
                         use_it(id)
-                    if "chunks_healthy" in item:
-                        # we also consider the chunks_healthy chunks as referenced - do not throw away
-                        # anything that borg check --repair might still need.
-                        for id, size in item.chunks_healthy:
-                            use_it(id, wanted=True)
         pi.finish()
-        return missing_chunks, reappeared_chunks, total_files, total_size, num_archives
+        return missing_chunks, total_files, total_size, num_archives
 
     def report_and_delete(self):
-        run_repair = " Run borg check --repair!"
-
         if self.missing_chunks:
-            logger.error(f"Repository has {len(self.missing_chunks)} missing objects." + run_repair)
+            logger.error(f"Repository has {len(self.missing_chunks)} missing objects!")
             set_ec(EXIT_ERROR)
 
-        if self.reappeared_chunks:
-            logger.warning(f"{len(self.reappeared_chunks)} previously missing objects re-appeared!" + run_repair)
-            set_ec(EXIT_WARNING)
-
         logger.info("Cleaning archives directory from soft-deleted archives...")
         archive_infos = self.manifest.archives.list(sort_by=["ts"], deleted=True)
         for archive_info in archive_infos:

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

@@ -104,9 +104,9 @@ class MountMixIn:
 
         - ``versions``: when used with a repository mount, this gives a merged, versioned
           view of the files in the archives. EXPERIMENTAL, layout may change in future.
-        - ``allow_damaged_files``: by default damaged files (where missing chunks were
-          replaced with runs of zeros by ``borg check --repair``) are not readable and
-          return EIO (I/O error). Set this option to read such files.
+        - ``allow_damaged_files``: by default damaged files (where chunks are missing)
+          will return EIO (I/O error) when trying to read the related parts of the file.
+          Set this option to replace the missing parts with all-zero bytes.
         - ``ignore_permissions``: for security reasons the ``default_permissions`` mount
           option is internally enforced by borg. ``ignore_permissions`` can be given to
           not enforce ``default_permissions``.

+ 4 - 10
src/borg/archiver/recreate_cmd.py

@@ -95,16 +95,10 @@ class RecreateMixIn:
         at least the entire deduplicated size of the archives using the previous
         chunker params.
 
-        If you recently ran borg check --repair and it had to fix lost chunks with all-zero
-        replacement chunks, please first run another backup for the same data and re-run
-        borg check --repair afterwards to heal any archives that had lost chunks which are
-        still generated from the input data.
-
-        Important: running borg recreate to re-chunk will remove the chunks_healthy
-        metadata of all items with replacement chunks, so healing will not be possible
-        any more after re-chunking (it is also unlikely it would ever work: due to the
-        change of chunking parameters, the missing chunk likely will never be seen again
-        even if you still have the data that produced it).
+        If your most recent borg check found missing chunks, please first run another
+        backup for the same data, before doing any rechunking. If you are lucky, that
+        will re-create the missing chunks. Optionally, do another borg check, to see
+        if the chunks are still missing).
         """
         )
         subparser = subparsers.add_parser(

+ 56 - 41
src/borg/archiver/transfer_cmd.py

@@ -9,6 +9,8 @@ from ..helpers import Error
 from ..helpers import location_validator, Location, archivename_validator, comment_validator
 from ..helpers import format_file_size, bin_to_hex
 from ..manifest import Manifest
+from ..legacyrepository import LegacyRepository
+from ..repository import Repository
 
 from ..logger import create_logger
 
@@ -111,51 +113,64 @@ class TransferMixIn:
                         # so let's remove them from old archives also, considering there is no
                         # code any more that deals with them in special ways (e.g. to get stats right).
                         continue
-                    if "chunks" in item:
+                    if "chunks_healthy" in item:  # legacy
+                        other_chunks = item.chunks_healthy  # chunks_healthy has the CORRECT chunks list, if present.
+                    elif "chunks" in item:
+                        other_chunks = item.chunks
+                    else:
+                        other_chunks = None
+                    if other_chunks is not None:
                         chunks = []
-                        for chunk_id, size in item.chunks:
+                        for chunk_id, size in other_chunks:
                             chunk_present = cache.seen_chunk(chunk_id, size)
                             if not chunk_present:  # target repo does not yet have this chunk
                                 if not dry_run:
-                                    cdata = other_repository.get(chunk_id)
-                                    if args.recompress == "never":
-                                        # keep compressed payload same, verify via assert_id (that will
-                                        # decompress, but avoid needing to compress it again):
-                                        meta, data = other_manifest.repo_objs.parse(
-                                            chunk_id,
-                                            cdata,
-                                            decompress=True,
-                                            want_compressed=True,
-                                            ro_type=ROBJ_FILE_STREAM,
-                                        )
-                                        meta, data = upgrader.upgrade_compressed_chunk(meta, data)
-                                        chunk_entry = cache.add_chunk(
-                                            chunk_id,
-                                            meta,
-                                            data,
-                                            stats=archive.stats,
-                                            wait=False,
-                                            compress=False,
-                                            size=size,
-                                            ctype=meta["ctype"],
-                                            clevel=meta["clevel"],
-                                            ro_type=ROBJ_FILE_STREAM,
-                                        )
-                                    elif args.recompress == "always":
-                                        # always decompress and re-compress file data chunks
-                                        meta, data = other_manifest.repo_objs.parse(
-                                            chunk_id, cdata, ro_type=ROBJ_FILE_STREAM
-                                        )
-                                        chunk_entry = cache.add_chunk(
-                                            chunk_id,
-                                            meta,
-                                            data,
-                                            stats=archive.stats,
-                                            wait=False,
-                                            ro_type=ROBJ_FILE_STREAM,
-                                        )
+                                    try:
+                                        cdata = other_repository.get(chunk_id)
+                                    except (Repository.ObjectNotFound, LegacyRepository.ObjectNotFound):
+                                        # missing correct chunk in other_repository (source) will result in
+                                        # a missing chunk in repository (destination).
+                                        # we do NOT want to transfer all-zero replacement chunks from borg1 repos.
+                                        pass
                                     else:
-                                        raise ValueError(f"unsupported recompress mode: {args.recompress}")
+                                        if args.recompress == "never":
+                                            # keep compressed payload same, verify via assert_id (that will
+                                            # decompress, but avoid needing to compress it again):
+                                            meta, data = other_manifest.repo_objs.parse(
+                                                chunk_id,
+                                                cdata,
+                                                decompress=True,
+                                                want_compressed=True,
+                                                ro_type=ROBJ_FILE_STREAM,
+                                            )
+                                            meta, data = upgrader.upgrade_compressed_chunk(meta, data)
+                                            chunk_entry = cache.add_chunk(
+                                                chunk_id,
+                                                meta,
+                                                data,
+                                                stats=archive.stats,
+                                                wait=False,
+                                                compress=False,
+                                                size=size,
+                                                ctype=meta["ctype"],
+                                                clevel=meta["clevel"],
+                                                ro_type=ROBJ_FILE_STREAM,
+                                            )
+                                        elif args.recompress == "always":
+                                            # always decompress and re-compress file data chunks
+                                            meta, data = other_manifest.repo_objs.parse(
+                                                chunk_id, cdata, ro_type=ROBJ_FILE_STREAM
+                                            )
+                                            chunk_entry = cache.add_chunk(
+                                                chunk_id,
+                                                meta,
+                                                data,
+                                                stats=archive.stats,
+                                                wait=False,
+                                                ro_type=ROBJ_FILE_STREAM,
+                                            )
+                                        else:
+                                            raise ValueError(f"unsupported recompress mode: {args.recompress}")
                                     cache.repository.async_response(wait=False)
                                     chunks.append(chunk_entry)
                                 transfer_size += size
@@ -165,7 +180,7 @@ class TransferMixIn:
                                     chunks.append(chunk_entry)
                                 present_size += size
                         if not dry_run:
-                            item.chunks = chunks  # TODO: overwrite? IDs and sizes are same.
+                            item.chunks = chunks
                             archive.stats.nfiles += 1
                     if not dry_run:
                         item = upgrader.upgrade_item(item=item)

+ 12 - 13
src/borg/fuse.py

@@ -10,7 +10,7 @@ import time
 from collections import defaultdict, Counter
 from signal import SIGINT
 
-from .constants import ROBJ_FILE_STREAM
+from .constants import ROBJ_FILE_STREAM, zeros
 from .fuse_impl import llfuse, has_pyfuse3
 
 
@@ -46,6 +46,7 @@ from .helpers.lrucache import LRUCache
 from .item import Item
 from .platform import uid2user, gid2group
 from .platformflags import is_darwin
+from .repository import Repository
 from .remote import RemoteRepository
 
 
@@ -652,17 +653,6 @@ class FuseOperations(llfuse.Operations, FuseBackend):
 
     @async_wrapper
     def open(self, inode, flags, ctx=None):
-        if not self.allow_damaged_files:
-            item = self.get_item(inode)
-            if "chunks_healthy" in item:
-                # Processed archive items don't carry the path anymore; for converting the inode
-                # to the path we'd either have to store the inverse of the current structure,
-                # or search the entire archive. So we just don't print it. It's easy to correlate anyway.
-                logger.warning(
-                    "File has damaged (all-zero) chunks. Try running borg check --repair. "
-                    "Mount with allow_damaged_files to read damaged files."
-                )
-                raise llfuse.FUSEError(errno.EIO)
         return llfuse.FileInfo(fh=inode) if has_pyfuse3 else inode
 
     @async_wrapper
@@ -699,7 +689,16 @@ class FuseOperations(llfuse.Operations, FuseBackend):
                     # evict fully read chunk from cache
                     del self.data_cache[id]
             else:
-                _, data = self.repo_objs.parse(id, self.repository_uncached.get(id), ro_type=ROBJ_FILE_STREAM)
+                try:
+                    cdata = self.repository_uncached.get(id)
+                except Repository.ObjectNotFound:
+                    if self.allow_damaged_files:
+                        data = zeros[:s]
+                        assert len(data) == s
+                    else:
+                        raise llfuse.FUSEError(errno.EIO) from None
+                else:
+                    _, data = self.repo_objs.parse(id, cdata, ro_type=ROBJ_FILE_STREAM)
                 if offset + n < len(data):
                     # chunk was only partially read, cache it
                     self.data_cache[id] = data

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

@@ -308,8 +308,8 @@ class HardLinkManager:
        If we encounter the same hlid again later, we hardlink to the path of the already extracted content of same hlid.
 
     C) When transferring from a borg1 archive, we need:
-       path -> chunks, chunks_healthy  # for borg1_hl_targets
-       If we encounter a regular file item with source == path later, we reuse chunks and chunks_healthy
+       path -> chunks_correct  # for borg1_hl_targets, chunks_correct must be either from .chunks_healthy or .chunks.
+       If we encounter a regular file item with source == path later, we reuse chunks_correct
        and create the same hlid = hardlink_id_from_path(source).
 
     D) When importing a tar file (simplified 1-pass way for now, not creating borg hardlink items):
@@ -353,7 +353,7 @@ class HardLinkManager:
                    a hlid (new borg style) [bytes]
                    a (dev, inode) tuple (filesystem)
         :param info: information to remember, could be:
-                     chunks / chunks_healthy list
+                     chunks list
                      hlid
         """
         assert isinstance(id, self.id_type), f"id is {id!r}, not of type {self.id_type}"

+ 0 - 6
src/borg/helpers/parseformat.py

@@ -827,7 +827,6 @@ class ItemFormatter(BaseFormatter):
         "isoctime": "file change time (ISO 8601 format)",
         "isoatime": "file access time (ISO 8601 format)",
         "xxh64": "XXH64 checksum of this file (note: this is NOT a cryptographic hash!)",
-        "health": 'either "healthy" (file ok) or "broken" (if file has all-zero replacement chunks)',
         "archiveid": "internal ID of the archive",
         "archivename": "name of the archive",
     }
@@ -837,7 +836,6 @@ class ItemFormatter(BaseFormatter):
         ("mtime", "ctime", "atime", "isomtime", "isoctime", "isoatime"),
         tuple(sorted(hash_algorithms)),
         ("archiveid", "archivename", "extra"),
-        ("health",),
     )
 
     KEYS_REQUIRING_CACHE = ()
@@ -894,10 +892,6 @@ class ItemFormatter(BaseFormatter):
         item_data.update(text_to_json("user", item.get("user", str(item_data["uid"]))))
         item_data.update(text_to_json("group", item.get("group", str(item_data["gid"]))))
 
-        if jsonline:
-            item_data["healthy"] = "chunks_healthy" not in item
-        else:
-            item_data["health"] = "broken" if "chunks_healthy" in item else "healthy"
         item_data["flags"] = item.get("bsdflags")  # int if flags known, else (if flags unknown) None
         for key in self.used_call_keys:
             item_data[key] = self.call_keys[key](item)

+ 17 - 46
src/borg/testsuite/archiver/check_cmd_test.py

@@ -155,28 +155,19 @@ def test_missing_file_chunk(archivers, request):
         else:
             pytest.fail("should not happen")  # convert 'fail'
 
-    cmd(archiver, "check", exit_code=1)
+    output = cmd(archiver, "check", exit_code=1)
+    assert "Missing file chunk detected" in output
     output = cmd(archiver, "check", "--repair", exit_code=0)
-    assert "New missing file chunk detected" in output
-
-    cmd(archiver, "check", exit_code=0)
-    output = cmd(archiver, "list", "archive1", "--format={health}#{path}{NL}", exit_code=0)
-    assert "broken#" in output
+    assert "Missing file chunk detected" in output  # repair is not changing anything, just reporting.
 
-    # check that the file in the old archives has now a different chunk list without the killed chunk.
-    # also check that the correct original chunks list is preserved in item.chunks_healthy.
+    # check does not modify the chunks list.
     for archive_name in ("archive1", "archive2"):
         archive, repository = open_archive(archiver.repository_path, archive_name)
         with repository:
             for item in archive.iter_items():
                 if item.path.endswith(src_file):
                     assert len(valid_chunks) == len(item.chunks)
-                    assert killed_chunk not in item.chunks
-                    assert valid_chunks != item.chunks
-                    assert "chunks_healthy" in item
-                    assert len(valid_chunks) == len(item.chunks_healthy)
-                    assert killed_chunk in item.chunks_healthy
-                    assert valid_chunks == item.chunks_healthy
+                    assert valid_chunks == item.chunks
                     break
             else:
                 pytest.fail("should not happen")  # convert 'fail'
@@ -185,32 +176,9 @@ def test_missing_file_chunk(archivers, request):
     with patch.object(ChunkBuffer, "BUFFER_SIZE", 10):
         create_src_archive(archiver, "archive3")
 
-    # check should be able to heal the file now:
+    # check should not complain anymore about missing chunks:
     output = cmd(archiver, "check", "-v", "--repair", exit_code=0)
-    assert "Healed previously missing file chunk" in output
-    assert f"{src_file}: Completely healed previously damaged file!" in output
-
-    # check that the file in the old archives has the correct chunks again.
-    # also check that chunks_healthy list is removed as it is not needed any more.
-    for archive_name in ("archive1", "archive2"):
-        archive, repository = open_archive(archiver.repository_path, archive_name)
-        with repository:
-            for item in archive.iter_items():
-                if item.path.endswith(src_file):
-                    assert valid_chunks == item.chunks
-                    assert "chunks_healthy" not in item
-                    break
-            else:
-                pytest.fail("should not happen")
-
-    # list is also all-healthy again
-    output = cmd(archiver, "list", "archive1", "--format={health}#{path}{NL}", exit_code=0)
-    assert "broken#" not in output
-
-    # check should be fine now (and not show it has healed anything).
-    output = cmd(archiver, "check", "-v", "--repair", exit_code=0)
-    assert "Healed previously missing file chunk" not in output
-    assert "testsuite/archiver.py: Completely healed previously damaged file!" not in output
+    assert "Missing file chunk detected" not in output
 
 
 def test_missing_archive_item_chunk(archivers, request):
@@ -425,13 +393,14 @@ def test_verify_data(archivers, request, init_args):
         output = cmd(archiver, "check", "--archives-only", "--verify-data", exit_code=1)
         assert f"{bin_to_hex(chunk.id)}, integrity error" in output
 
-        # repair (heal is tested in another test)
+        # repair will find the defect chunk and remove it
         output = cmd(archiver, "check", "--repair", "--verify-data", exit_code=0)
         assert f"{bin_to_hex(chunk.id)}, integrity error" in output
-        assert f"{src_file}: New missing file chunk detected" in output
+        assert f"{src_file}: Missing file chunk detected" in output
 
-        # run with --verify-data again, all fine now (file was patched with a replacement chunk).
-        cmd(archiver, "check", "--archives-only", "--verify-data", exit_code=0)
+        # run with --verify-data again, it will notice the missing chunk.
+        output = cmd(archiver, "check", "--archives-only", "--verify-data", exit_code=1)
+        assert f"{src_file}: Missing file chunk detected" in output
 
 
 @pytest.mark.parametrize("init_args", [["--encryption=repokey-aes-ocb"], ["--encryption", "none"]])
@@ -457,13 +426,15 @@ def test_corrupted_file_chunk(archivers, request, init_args):
     output = cmd(archiver, "check", "--repository-only", exit_code=1)
     assert f"{bin_to_hex(chunk.id)} is corrupted: data does not match checksum." in output
 
-    # repair (heal is tested in another test)
+    # repair: the defect chunk will be removed by repair.
     output = cmd(archiver, "check", "--repair", exit_code=0)
     assert f"{bin_to_hex(chunk.id)} is corrupted: data does not match checksum." in output
-    assert f"{src_file}: New missing file chunk detected" in output
+    assert f"{src_file}: Missing file chunk detected" in output
 
-    # run normal check again, all fine now (file was patched with a replacement chunk).
+    # run normal check again
     cmd(archiver, "check", "--repository-only", exit_code=0)
+    output = cmd(archiver, "check", "--archives-only", exit_code=1)
+    assert f"{src_file}: Missing file chunk detected" in output
 
 
 def test_empty_repository(archivers, request):

+ 9 - 5
src/borg/testsuite/archiver/mount_cmds_test.py

@@ -233,15 +233,19 @@ def test_fuse_allow_damaged_files(archivers, request):
                 break
         else:
             assert False  # missed the file
-    cmd(archiver, "check", "--repair", exit_code=0)
 
     mountpoint = os.path.join(archiver.tmpdir, "mountpoint")
     with fuse_mount(archiver, mountpoint, "-a", "archive"):
-        with pytest.raises(OSError) as excinfo:
-            open(os.path.join(mountpoint, "archive", path))
-        assert excinfo.value.errno == errno.EIO
+        with open(os.path.join(mountpoint, "archive", path), "rb") as f:
+            with pytest.raises(OSError) as excinfo:
+                f.read()
+            assert excinfo.value.errno == errno.EIO
+
     with fuse_mount(archiver, mountpoint, "-a", "archive", "-o", "allow_damaged_files"):
-        open(os.path.join(mountpoint, "archive", path)).close()
+        with open(os.path.join(mountpoint, "archive", path), "rb") as f:
+            # no exception raised, missing data will be all-zero
+            data = f.read()
+        assert data.endswith(b"\0\0")
 
 
 @pytest.mark.skipif(not llfuse, reason="llfuse not installed")

+ 3 - 1
src/borg/testsuite/archiver/transfer_cmd_test.py

@@ -164,7 +164,6 @@ def test_transfer_upgrade(archivers, request):
                     # fix expectation for size
                     e["size"] = g["size"]
                 # Note: size == 0 for all items without a size or chunks list (like e.g. directories)
-                # Note: healthy == True indicates the *absence* of the additional chunks_healthy list
             del g["hlid"]
 
             # borg 1 used "linktarget" and "source" for links, borg 2 uses "target" for symlinks.
@@ -177,6 +176,9 @@ def test_transfer_upgrade(archivers, request):
                 # The S_IFBLK macro is broken on MINGW
                 del e["type"], g["type"]
                 del e["mode"], g["mode"]
+
+            del e["healthy"]  # not supported anymore
+
             assert g == e
 
         if name == "archive1":

+ 3 - 6
src/borg/upgrade.py

@@ -48,7 +48,7 @@ class UpgraderFrom12To20:
 
     def new_archive(self, *, archive):
         self.archive = archive
-        self.hlm = HardLinkManager(id_type=bytes, info_type=tuple)  # hlid -> (chunks, chunks_healthy)
+        self.hlm = HardLinkManager(id_type=bytes, info_type=list)  # hlid -> chunks_correct
 
     def upgrade_item(self, *, item):
         """upgrade item as needed, get rid of legacy crap"""
@@ -56,7 +56,6 @@ class UpgraderFrom12To20:
             "path",
             "rdev",
             "chunks",
-            "chunks_healthy",
             "hlid",
             "mode",
             "user",
@@ -78,16 +77,14 @@ class UpgraderFrom12To20:
 
         if self.hlm.borg1_hardlink_master(item):
             item.hlid = hlid = self.hlm.hardlink_id_from_path(item.path)
-            self.hlm.remember(id=hlid, info=(item.get("chunks"), item.get("chunks_healthy")))
+            self.hlm.remember(id=hlid, info=item.get("chunks"))
         elif self.hlm.borg1_hardlink_slave(item):
             item.hlid = hlid = self.hlm.hardlink_id_from_path(item.source)
-            chunks, chunks_healthy = self.hlm.retrieve(id=hlid, default=(None, None))
+            chunks = self.hlm.retrieve(id=hlid)
             if chunks is not None:
                 item.chunks = chunks
                 for chunk_id, chunk_size in chunks:
                     self.cache.reuse_chunk(chunk_id, chunk_size, self.archive.stats)
-            if chunks_healthy is not None:
-                item.chunks_healthy = chunks
             del item.source  # not used for hardlinks any more, replaced by hlid
         # make sure we only have desired stuff in the new item. specifically, make sure to get rid of:
         # - 'acl' remnants of bug in attic <= 0.13