فهرست منبع

refactor hardlink_master processing globally

borg now has the chunks list in every item with content.
due to the symmetric way how borg now deals with hardlinks using
item.hlid, processing gets much simpler.

but some places where borg deals with other "sources" of hardlinks
still need to do some hardlink management:
borg uses the HardLinkManager there now (which is not much more
than a dict, but keeps documentation at one place and avoids some
code duplication we had before).

item.hlid is computed via hardlink_id function.

support hardlinked symlinks, fixes #2379
as we use item.hlid now to group hardlinks together,
there is no conflict with the item.source usage for
symlink targets any more.

2nd+ hardlinks now add to the files count as did the 1st one.
for borg, now all hardlinks are created equal.
so any hardlink item with chunks now adds to the "file" count.

ItemFormatter: support {hlid} instead of {source} for hardlinks
Thomas Waldmann 3 سال پیش
والد
کامیت
6bfdb3f630

+ 0 - 3
docs/faq.rst

@@ -132,9 +132,6 @@ Which file types, attributes, etc. are *not* preserved?
       Archive extraction has optional support to extract all-zero chunks as
       holes in a sparse file.
     * Some filesystem specific attributes, like btrfs NOCOW, see :ref:`platforms`.
-    * For hardlinked symlinks, the hardlinking can not be archived (and thus,
-      the hardlinking will not be done at extraction time). The symlinks will
-      be archived and extracted as non-hardlinked symlinks, see :issue:`2379`.
 
 Are there other known limitations?
 ----------------------------------

+ 1 - 1
docs/internals/data-structures.rst

@@ -567,7 +567,7 @@ dictionary created by the ``Item`` class that contains:
 * uid
 * gid
 * mode (item type + permissions)
-* source (for symlinks, and for hardlinks within one archive)
+* source (for symlinks)
 * rdev (for device files)
 * mtime, atime, ctime in nanoseconds
 * xattrs

+ 1 - 1
docs/usage/general/file-metadata.rst.inc

@@ -10,7 +10,7 @@ Besides regular file and directory structures, Borg can preserve
   * FIFOs ("named pipes")
   * special file *contents* can be backed up in ``--read-special`` mode.
     By default the metadata to create them with mknod(2), mkfifo(2) etc. is stored.
-* hardlinked regular files, devices, FIFOs (considering all items in the same archive)
+* hardlinked regular files, devices, symlinks, FIFOs (considering all items in the same archive)
 * timestamps in nanosecond precision: mtime, atime, ctime
 * other timestamps: birthtime (on platforms supporting it)
 * permissions:

+ 112 - 174
src/borg/archive.py

@@ -28,7 +28,7 @@ from .constants import *  # NOQA
 from .crypto.low_level import IntegrityError as IntegrityErrorBase
 from .hashindex import ChunkIndex, ChunkIndexEntry, CacheSynchronizer
 from .helpers import Manifest
-from .helpers import hardlinkable
+from .helpers import HardLinkManager
 from .helpers import ChunkIteratorFileWrapper, open_item
 from .helpers import Error, IntegrityError, set_ec
 from .platform import uid2user, user2uid, gid2group, group2gid
@@ -280,7 +280,7 @@ class DownloadPipeline:
         self.repository = repository
         self.key = key
 
-    def unpack_many(self, ids, filter=None, partial_extract=False, preload=False, hardlink_masters=None):
+    def unpack_many(self, ids, *, filter=None, preload=False):
         """
         Return iterator of items.
 
@@ -290,10 +290,7 @@ class DownloadPipeline:
         Warning: if *preload* is True then all data chunks of every yielded item have to be retrieved,
         otherwise preloaded chunks will accumulate in RemoteRepository and create a memory leak.
         """
-        def _preload(chunks):
-            self.repository.preload([c.id for c in chunks])
-
-        masters_preloaded = set()
+        hlids_preloaded = set()
         unpacker = msgpack.Unpacker(use_list=False)
         for data in self.fetch_many(ids):
             unpacker.feed(data)
@@ -306,33 +303,20 @@ class DownloadPipeline:
                 items = [item for item in items if filter(item)]
 
             if preload:
-                if filter and partial_extract:
-                    # if we do only a partial extraction, it gets a bit
-                    # complicated with computing the preload items: if a hardlink master item is not
-                    # selected (== not extracted), we will still need to preload its chunks if a
-                    # corresponding hardlink slave is selected (== is extracted).
-                    # due to a side effect of the filter() call, we now have hardlink_masters dict populated.
-                    for item in items:
-                        if hardlinkable(item.mode):
-                            source = item.get('source')
-                            if source is None:  # maybe a hardlink master
-                                if 'chunks' in item:
-                                    _preload(item.chunks)
-                                # if this is a hl master, remember that we already preloaded all chunks of it (if any):
-                                if item.get('hardlink_master', True):
-                                    masters_preloaded.add(item.path)
-                            else:  # hardlink slave
-                                if source not in masters_preloaded:
-                                    # we only need to preload *once* (for the 1st selected slave)
-                                    chunks, _ = hardlink_masters[source]
-                                    if chunks is not None:
-                                        _preload(chunks)
-                                    masters_preloaded.add(source)
-                else:
-                    # easy: we do not have a filter, thus all items are selected, thus we need to preload all chunks.
-                    for item in items:
-                        if 'chunks' in item:
-                            _preload(item.chunks)
+                for item in items:
+                    if 'chunks' in item:
+                        hlid = item.get('hlid', None)
+                        if hlid is None:
+                            preload_chunks = True
+                        else:
+                            if hlid in hlids_preloaded:
+                                preload_chunks = False
+                            else:
+                                # not having the hardlink's chunks already preloaded for other hardlink to same inode
+                                preload_chunks = True
+                                hlids_preloaded.add(hlid)
+                        if preload_chunks:
+                            self.repository.preload([c.id for c in item.chunks])
 
             for item in items:
                 yield item
@@ -443,7 +427,6 @@ class Archive:
         self.repository = repository
         self.cache = cache
         self.manifest = manifest
-        self.hard_links = {}
         self.stats = Statistics(output_json=log_json, iec=iec)
         self.iec = iec
         self.show_progress = progress
@@ -584,12 +567,10 @@ Utilization of max. archive size: {csize_max:.0%}
             return False
         return filter(item) if filter else True
 
-    def iter_items(self, filter=None, partial_extract=False, preload=False, hardlink_masters=None):
+    def iter_items(self, filter=None, preload=False):
         # note: when calling this with preload=True, later fetch_many() must be called with
         # is_preloaded=True or the RemoteRepository code will leak memory!
-        assert not (filter and partial_extract and preload) or hardlink_masters is not None
-        for item in self.pipeline.unpack_many(self.metadata.items, partial_extract=partial_extract,
-                                              preload=preload, hardlink_masters=hardlink_masters,
+        for item in self.pipeline.unpack_many(self.metadata.items, preload=preload,
                                               filter=lambda item: self.item_filter(item, filter)):
             yield item
 
@@ -719,33 +700,30 @@ Utilization of max. archive size: {csize_max:.0%}
         return stats
 
     @contextmanager
-    def extract_helper(self, dest, item, path, stripped_components, original_path, hardlink_masters):
+    def extract_helper(self, item, path, hlm, *, dry_run=False):
         hardlink_set = False
         # Hard link?
-        if 'source' in item:
-            source = os.path.join(dest, *item.source.split(os.sep)[stripped_components:])
-            chunks, link_target = hardlink_masters.get(item.source, (None, source))
-            if link_target and has_link:
-                # Hard link was extracted previously, just link
-                with backup_io('link'):
-                    os.link(link_target, path)
-                    hardlink_set = True
-            elif chunks is not None:
-                # assign chunks to this item, since the item which had the chunks was not extracted
-                item.chunks = chunks
+        if 'hlid' in item:
+            link_target = hlm.retrieve(id=item.hlid)
+            if link_target is not None and has_link:
+                if not dry_run:
+                    # another hardlink to same inode (same hlid) was extracted previously, just link to it
+                    with backup_io('link'):
+                        os.link(link_target, path, follow_symlinks=False)
+                hardlink_set = True
         yield hardlink_set
-        if not hardlink_set and hardlink_masters:
-            if has_link:
-                # Update master entry with extracted item path, so that following hardlinks don't extract twice.
+        if not hardlink_set:
+            if 'hlid' in item and has_link:
+                # Update entry with extracted item path, so that following hardlinks don't extract twice.
                 # We have hardlinking support, so we will hardlink not extract.
-                hardlink_masters[item.get('source') or original_path] = (None, path)
+                hlm.remember(id=item.hlid, info=path)
             else:
                 # Broken platform with no hardlinking support.
                 # In this case, we *want* to extract twice, because there is no other way.
                 pass
 
     def extract_item(self, item, restore_attrs=True, dry_run=False, stdout=False, sparse=False,
-                     hardlink_masters=None, stripped_components=0, original_path=None, pi=None):
+                     hlm=None, stripped_components=0, original_path=None, pi=None):
         """
         Extract archive item.
 
@@ -754,29 +732,33 @@ Utilization of max. archive size: {csize_max:.0%}
         :param dry_run: do not write any data
         :param stdout: write extracted data to stdout
         :param sparse: write sparse files (chunk-granularity, independent of the original being sparse)
-        :param hardlink_masters: maps paths to (chunks, link_target) for extracting subtrees with hardlinks correctly
+        :param hlm: maps hlid to link_target for extracting subtrees with hardlinks correctly
         :param stripped_components: stripped leading path components to correct hard link extraction
         :param original_path: 'path' key as stored in archive
         :param pi: ProgressIndicatorPercent (or similar) for file extraction progress (in bytes)
         """
-        hardlink_masters = hardlink_masters or {}
         has_damaged_chunks = 'chunks_healthy' in item
         if dry_run or stdout:
-            if 'chunks' in item:
-                item_chunks_size = 0
-                for data in self.pipeline.fetch_many([c.id for c in item.chunks], is_preloaded=True):
-                    if pi:
-                        pi.show(increase=len(data), info=[remove_surrogates(item.path)])
-                    if stdout:
-                        sys.stdout.buffer.write(data)
-                    item_chunks_size += len(data)
-                if stdout:
-                    sys.stdout.buffer.flush()
-                if 'size' in item:
-                    item_size = item.size
-                    if item_size != item_chunks_size:
-                        raise BackupError('Size inconsistency detected: size {}, chunks size {}'.format(
-                                          item_size, item_chunks_size))
+            with self.extract_helper(item, '', hlm, dry_run=dry_run or stdout) as hardlink_set:
+                if not hardlink_set:
+                    # it does not really set hardlinks due to dry_run, but we need to behave same
+                    # as non-dry_run concerning fetching preloaded chunks from the pipeline or
+                    # it would get stuck.
+                    if 'chunks' in item:
+                        item_chunks_size = 0
+                        for data in self.pipeline.fetch_many([c.id for c in item.chunks], is_preloaded=True):
+                            if pi:
+                                pi.show(increase=len(data), info=[remove_surrogates(item.path)])
+                            if stdout:
+                                sys.stdout.buffer.write(data)
+                            item_chunks_size += len(data)
+                        if stdout:
+                            sys.stdout.buffer.flush()
+                        if 'size' in item:
+                            item_size = item.size
+                            if item_size != item_chunks_size:
+                                raise BackupError('Size inconsistency detected: size {}, chunks size {}'.format(
+                                                  item_size, item_chunks_size))
             if has_damaged_chunks:
                 raise BackupError('File has damaged (all-zero) chunks. Try running borg check --repair.')
             return
@@ -807,8 +789,7 @@ Utilization of max. archive size: {csize_max:.0%}
         if stat.S_ISREG(mode):
             with backup_io('makedirs'):
                 make_parent(path)
-            with self.extract_helper(dest, item, path, stripped_components, original_path,
-                                     hardlink_masters) as hardlink_set:
+            with self.extract_helper(item, path, hlm) as hardlink_set:
                 if hardlink_set:
                     return
                 with backup_io('open'):
@@ -847,24 +828,26 @@ Utilization of max. archive size: {csize_max:.0%}
                     self.restore_attrs(path, item)
             elif stat.S_ISLNK(mode):
                 make_parent(path)
-                source = item.source
-                try:
-                    os.symlink(source, path)
-                except UnicodeEncodeError:
-                    raise self.IncompatibleFilesystemEncodingError(source, sys.getfilesystemencoding()) from None
-                self.restore_attrs(path, item, symlink=True)
+                with self.extract_helper(item, path, hlm) as hardlink_set:
+                    if hardlink_set:
+                        # unusual, but possible: this is a hardlinked symlink.
+                        return
+                    source = item.source
+                    try:
+                        os.symlink(source, path)
+                    except UnicodeEncodeError:
+                        raise self.IncompatibleFilesystemEncodingError(source, sys.getfilesystemencoding()) from None
+                    self.restore_attrs(path, item, symlink=True)
             elif stat.S_ISFIFO(mode):
                 make_parent(path)
-                with self.extract_helper(dest, item, path, stripped_components, original_path,
-                                         hardlink_masters) as hardlink_set:
+                with self.extract_helper(item, path, hlm) as hardlink_set:
                     if hardlink_set:
                         return
                     os.mkfifo(path)
                     self.restore_attrs(path, item)
             elif stat.S_ISCHR(mode) or stat.S_ISBLK(mode):
                 make_parent(path)
-                with self.extract_helper(dest, item, path, stripped_components, original_path,
-                                         hardlink_masters) as hardlink_set:
+                with self.extract_helper(item, path, hlm) as hardlink_set:
                     if hardlink_set:
                         return
                     os.mknod(path, item.mode, item.rdev)
@@ -1041,79 +1024,43 @@ Utilization of max. archive size: {csize_max:.0%}
         :param can_compare_chunk_ids: Whether --chunker-params are the same for both archives.
         """
 
-        def hardlink_master_seen(item):
-            return 'source' not in item or not hardlinkable(item.mode) or item.source in hardlink_masters
-
-        def is_hardlink_master(item):
-            return item.get('hardlink_master', True) and 'source' not in item and hardlinkable(item.mode)
-
-        def update_hardlink_masters(item1, item2):
-            if is_hardlink_master(item1) or is_hardlink_master(item2):
-                hardlink_masters[item1.path] = (item1, item2)
-
-        def has_hardlink_master(item, hardlink_masters):
-            return hardlinkable(item.mode) and item.get('source') in hardlink_masters
-
         def compare_items(item1, item2):
-            if has_hardlink_master(item1, hardlink_masters):
-                item1 = hardlink_masters[item1.source][0]
-            if has_hardlink_master(item2, hardlink_masters):
-                item2 = hardlink_masters[item2.source][1]
             return ItemDiff(item1, item2,
                             archive1.pipeline.fetch_many([c.id for c in item1.get('chunks', [])]),
                             archive2.pipeline.fetch_many([c.id for c in item2.get('chunks', [])]),
                             can_compare_chunk_ids=can_compare_chunk_ids)
 
-        def defer_if_necessary(item1, item2):
-            """Adds item tuple to deferred if necessary and returns True, if items were deferred"""
-            update_hardlink_masters(item1, item2)
-            defer = not hardlink_master_seen(item1) or not hardlink_master_seen(item2)
-            if defer:
-                deferred.append((item1, item2))
-            return defer
-
         orphans_archive1 = OrderedDict()
         orphans_archive2 = OrderedDict()
-        deferred = []
-        hardlink_masters = {}
 
         for item1, item2 in zip_longest(
                 archive1.iter_items(lambda item: matcher.match(item.path)),
                 archive2.iter_items(lambda item: matcher.match(item.path)),
         ):
             if item1 and item2 and item1.path == item2.path:
-                if not defer_if_necessary(item1, item2):
-                    yield (item1.path, compare_items(item1, item2))
+                yield (item1.path, compare_items(item1, item2))
                 continue
             if item1:
                 matching_orphan = orphans_archive2.pop(item1.path, None)
                 if matching_orphan:
-                    if not defer_if_necessary(item1, matching_orphan):
-                        yield (item1.path, compare_items(item1, matching_orphan))
+                    yield (item1.path, compare_items(item1, matching_orphan))
                 else:
                     orphans_archive1[item1.path] = item1
             if item2:
                 matching_orphan = orphans_archive1.pop(item2.path, None)
                 if matching_orphan:
-                    if not defer_if_necessary(matching_orphan, item2):
-                        yield (matching_orphan.path, compare_items(matching_orphan, item2))
+                    yield (matching_orphan.path, compare_items(matching_orphan, item2))
                 else:
                     orphans_archive2[item2.path] = item2
         # At this point orphans_* contain items that had no matching partner in the other archive
         for added in orphans_archive2.values():
             path = added.path
             deleted_item = Item.create_deleted(path)
-            update_hardlink_masters(deleted_item, added)
             yield (path, compare_items(deleted_item, added))
         for deleted in orphans_archive1.values():
             path = deleted.path
             deleted_item = Item.create_deleted(path)
-            update_hardlink_masters(deleted, deleted_item)
             yield (path, compare_items(deleted, deleted_item))
-        for item1, item2 in deferred:
-            assert hardlink_master_seen(item1)
-            assert hardlink_master_seen(item2)
-            yield (path, compare_items(item1, item2))
 
 
 class MetadataCollector:
@@ -1289,7 +1236,7 @@ class FilesystemObjectProcessors:
         self.show_progress = show_progress
         self.print_file_status = file_status_printer or (lambda *args: None)
 
-        self.hard_links = {}
+        self.hlm = HardLinkManager(id_type=tuple, info_type=tuple)  # (dev, ino) -> (hlid, chunks)
         self.stats = Statistics(output_json=log_json, iec=iec)  # threading: done by cache (including progress)
         self.cwd = os.getcwd()
         self.chunker = get_chunker(*chunker_params, seed=key.chunk_seed, sparse=sparse)
@@ -1298,29 +1245,32 @@ class FilesystemObjectProcessors:
     def create_helper(self, path, st, status=None, hardlinkable=True):
         safe_path = make_path_safe(path)
         item = Item(path=safe_path)
-        hardlink_master = False
         hardlinked = hardlinkable and st.st_nlink > 1
+        update_map = False
         if hardlinked:
-            source = self.hard_links.get((st.st_ino, st.st_dev))
-            if source is not None:
-                item.source = source
-                status = 'h'  # hardlink (to already seen inodes)
-            else:
-                hardlink_master = True
-        yield item, status, hardlinked, hardlink_master
-        # if we get here, "with"-block worked ok without error/exception, the item was processed ok...
+            status = 'h'  # hardlink
+            hlid, chunks = self.hlm.retrieve(id=(st.st_ino, st.st_dev), default=(None, None))
+            if hlid is None:
+                update_map = True
+                hlid = self.hlm.hardlink_id(item._dict['path'])
+            item.hlid = hlid
+            if chunks is not None:
+                item.chunks = chunks
+        yield item, status, hardlinked
         self.add_item(item, stats=self.stats)
-        # ... and added to the archive, so we can remember it to refer to it later in the archive:
-        if hardlink_master:
-            self.hard_links[(st.st_ino, st.st_dev)] = safe_path
+        if update_map:
+            # remember the hlid of this fs object and if the item has chunks,
+            # also remember them, so we do not have to re-chunk a hardlink.
+            chunks = item.chunks if 'chunks' in item else None
+            self.hlm.remember(id=(st.st_ino, st.st_dev), info=(hlid, chunks))
 
     def process_dir_with_fd(self, *, path, fd, st):
-        with self.create_helper(path, st, 'd', hardlinkable=False) as (item, status, hardlinked, hardlink_master):
+        with self.create_helper(path, st, 'd', hardlinkable=False) as (item, status, hardlinked):
             item.update(self.metadata_collector.stat_attrs(st, path, fd=fd))
             return status
 
     def process_dir(self, *, path, parent_fd, name, st):
-        with self.create_helper(path, st, 'd', hardlinkable=False) as (item, status, hardlinked, hardlink_master):
+        with self.create_helper(path, st, 'd', hardlinkable=False) as (item, status, hardlinked):
             with OsOpen(path=path, parent_fd=parent_fd, name=name, flags=flags_dir,
                         noatime=True, op='dir_open') as fd:
                 # fd is None for directories on windows, in that case a race condition check is not possible.
@@ -1331,7 +1281,7 @@ class FilesystemObjectProcessors:
                 return status
 
     def process_fifo(self, *, path, parent_fd, name, st):
-        with self.create_helper(path, st, 'f') as (item, status, hardlinked, hardlink_master):  # fifo
+        with self.create_helper(path, st, 'f') as (item, status, hardlinked):  # fifo
             with OsOpen(path=path, parent_fd=parent_fd, name=name, flags=flags_normal, noatime=True) as fd:
                 with backup_io('fstat'):
                     st = stat_update_check(st, os.fstat(fd))
@@ -1339,7 +1289,7 @@ class FilesystemObjectProcessors:
                 return status
 
     def process_dev(self, *, path, parent_fd, name, st, dev_type):
-        with self.create_helper(path, st, dev_type) as (item, status, hardlinked, hardlink_master):  # char/block device
+        with self.create_helper(path, st, dev_type) as (item, status, hardlinked):  # char/block device
             # looks like we can not work fd-based here without causing issues when trying to open/close the device
             with backup_io('stat'):
                 st = stat_update_check(st, os_stat(path=path, parent_fd=parent_fd, name=name, follow_symlinks=False))
@@ -1348,10 +1298,7 @@ class FilesystemObjectProcessors:
             return status
 
     def process_symlink(self, *, path, parent_fd, name, st):
-        # note: using hardlinkable=False because we can not support hardlinked symlinks,
-        #       due to the dual-use of item.source, see issue #2343:
-        # hardlinked symlinks will be archived [and extracted] as non-hardlinked symlinks.
-        with self.create_helper(path, st, 's', hardlinkable=False) as (item, status, hardlinked, hardlink_master):
+        with self.create_helper(path, st, 's', hardlinkable=True) as (item, status, hardlinked):
             fname = name if name is not None and parent_fd is not None else path
             with backup_io('readlink'):
                 source = os.readlink(fname, dir_fd=parent_fd)
@@ -1384,7 +1331,7 @@ class FilesystemObjectProcessors:
         return status
 
     def process_file(self, *, path, parent_fd, name, st, cache, flags=flags_normal):
-        with self.create_helper(path, st, None) as (item, status, hardlinked, hardlink_master):  # no status yet
+        with self.create_helper(path, st, None) as (item, status, hardlinked):  # no status yet
             with OsOpen(path=path, parent_fd=parent_fd, name=name, flags=flags, noatime=True) as fd:
                 with backup_io('fstat'):
                     st = stat_update_check(st, os.fstat(fd))
@@ -1395,7 +1342,9 @@ class FilesystemObjectProcessors:
                     # so it can be extracted / accessed in FUSE mount like a regular file.
                     # this needs to be done early, so that part files also get the patched mode.
                     item.mode = stat.S_IFREG | stat.S_IMODE(item.mode)
-                if not hardlinked or hardlink_master:
+                if 'chunks' in item:  # create_helper might have put chunks from a previous hardlink there
+                    [cache.chunk_incref(id_, self.stats) for id_, _, _ in item.chunks]
+                else:  # normal case, no "2nd+" hardlink
                     if not is_special_file:
                         hashed_path = safe_encode(os.path.join(self.cwd, path))
                         path_hash = self.key.id_hash(hashed_path)
@@ -1420,7 +1369,6 @@ class FilesystemObjectProcessors:
                         status = 'M' if known else 'A'  # regular file, modified or added
                     self.print_file_status(status, path)
                     status = None  # we already printed the status
-                    item.hardlink_master = hardlinked
                     # Only chunkify the file if needed
                     if chunks is not None:
                         item.chunks = chunks
@@ -1444,7 +1392,7 @@ class FilesystemObjectProcessors:
                             # also, we must not memorize a potentially inconsistent/corrupt file that
                             # changed while we backed it up.
                             cache.memorize_file(hashed_path, path_hash, st, [c.id for c in item.chunks])
-                    self.stats.nfiles += 1
+                self.stats.nfiles += 1
                 item.update(self.metadata_collector.stat_ext_attrs(st, path, fd=fd))
                 item.get_size(memorize=True)
                 return status
@@ -1464,6 +1412,7 @@ class TarfileObjectProcessors:
 
         self.stats = Statistics(output_json=log_json, iec=iec)  # threading: done by cache (including progress)
         self.chunker = get_chunker(*chunker_params, seed=key.chunk_seed, sparse=False)
+        self.hlm = HardLinkManager(id_type=str, info_type=list)  # path -> chunks
 
     @contextmanager
     def create_helper(self, tarinfo, status=None, type=None):
@@ -1504,11 +1453,21 @@ class TarfileObjectProcessors:
             item.rdev = os.makedev(tarinfo.devmajor, tarinfo.devminor)
             return status
 
-    def process_link(self, *, tarinfo, status, type):
+    def process_symlink(self, *, tarinfo, status, type):
         with self.create_helper(tarinfo, status, type) as (item, status):
             item.source = tarinfo.linkname
             return status
 
+    def process_hardlink(self, *, tarinfo, status, type):
+        with self.create_helper(tarinfo, status, type) as (item, status):
+            # create a not hardlinked borg item, reusing the chunks, see HardLinkManager.__doc__
+            chunks = self.hlm.retrieve(tarinfo.linkname)
+            if chunks is not None:
+                item.chunks = chunks
+            item.get_size(memorize=True, from_chunks=True)
+            self.stats.nfiles += 1
+            return status
+
     def process_file(self, *, tarinfo, status, type, tar):
         with self.create_helper(tarinfo, status, type) as (item, status):
             self.print_file_status(status, tarinfo.name)
@@ -1516,8 +1475,10 @@ class TarfileObjectProcessors:
             fd = tar.extractfile(tarinfo)
             self.process_file_chunks(item, self.cache, self.stats, self.show_progress,
                                      backup_io_iter(self.chunker.chunkify(fd)))
-            item.get_size(memorize=True)
+            item.get_size(memorize=True, from_chunks=True)
             self.stats.nfiles += 1
+            # we need to remember ALL files, see HardLinkManager.__doc__
+            self.hlm.remember(id=tarinfo.name, info=item.chunks)
             return status
 
 
@@ -2127,34 +2088,11 @@ class ArchiveRecreater:
 
     def process_items(self, archive, target):
         matcher = self.matcher
-        target_is_subset = not matcher.empty()
-        hardlink_masters = {} if target_is_subset else None
-
-        def item_is_hardlink_master(item):
-            return (target_is_subset and
-                    hardlinkable(item.mode) and
-                    item.get('hardlink_master', True) and
-                    'source' not in item)
 
         for item in archive.iter_items():
             if not matcher.match(item.path):
                 self.print_file_status('x', item.path)
-                if item_is_hardlink_master(item):
-                    hardlink_masters[item.path] = (item.get('chunks'), item.get('chunks_healthy'), None)
                 continue
-            if target_is_subset and hardlinkable(item.mode) and item.get('source') in hardlink_masters:
-                # master of this hard link is outside the target subset
-                chunks, chunks_healthy, new_source = hardlink_masters[item.source]
-                if new_source is None:
-                    # First item to use this master, move the chunks
-                    item.chunks = chunks
-                    if chunks_healthy is not None:
-                        item.chunks_healthy = chunks_healthy
-                    hardlink_masters[item.source] = (None, None, item.path)
-                    del item.source
-                else:
-                    # Master was already moved, only update this item's source
-                    item.source = new_source
             if self.dry_run:
                 self.print_file_status('-', item.path)
             else:
@@ -2261,7 +2199,7 @@ class ArchiveRecreater:
         tag_files = []
         tagged_dirs = []
 
-        # to support reading hard-linked CACHEDIR.TAGs (aka CACHE_TAG_NAME), similar to hardlink_masters:
+        # to support reading hard-linked CACHEDIR.TAGs (aka CACHE_TAG_NAME):
         cachedir_masters = {}
 
         if self.exclude_caches:

+ 31 - 64
src/borg/archiver.py

@@ -59,7 +59,7 @@ try:
     from .helpers import timestamp
     from .helpers import get_cache_dir, os_stat
     from .helpers import Manifest, AI_HUMAN_SORT_KEYS
-    from .helpers import hardlinkable
+    from .helpers import HardLinkManager
     from .helpers import StableDict
     from .helpers import check_python, check_extension_modules
     from .helpers import dir_is_tagged, is_slow_msgpack, is_supported_msgpack, yes, sysinfo
@@ -347,12 +347,12 @@ class Archiver:
 
         def upgrade_item(item):
             """upgrade item as needed, get rid of legacy crap"""
-            if item.get('hardlink_master', True) and 'source' not in item and hardlinkable(item.mode):
-                item._dict['hlid'] = hlid = hashlib.sha256(item._dict['path'])
-                hardlink_masters[hlid] = (item._dict.get('chunks'), item._dict.get('chunks_healthy'))
-            elif 'source' in item and hardlinkable(item.mode):
-                item._dict['hlid'] = hlid = hashlib.sha256(item._dict['source'])
-                chunks, chunks_healthy = hardlink_masters.get(hlid, (None, None))
+            if hlm.borg1_hardlink_master(item):
+                item._dict['hlid'] = hlid = hlm.hardlink_id(item._dict['path'])
+                hlm.remember(id=hlid, info=(item._dict.get('chunks'), item._dict.get('chunks_healthy')))
+            elif hlm.borg1_hardlink_slave(item):
+                item._dict['hlid'] = hlid = hlm.hardlink_id(item._dict['source'])
+                chunks, chunks_healthy = hlm.retrieve(id=hlid, default=(None, None))
                 if chunks is not None:
                     item._dict['chunks'] = chunks
                     for chunk_id, _, _ in chunks:
@@ -389,7 +389,7 @@ class Archiver:
             else:
                 if not dry_run:
                     print(f"{name}: copying archive to destination repo...")
-                hardlink_masters = {}
+                hlm = HardLinkManager(id_type=bytes, info_type=tuple)  # hlid -> (chunks, chunks_healthy)
                 other_archive = Archive(other_repository, other_key, other_manifest, name)
                 archive = Archive(repository, key, manifest, name, cache=cache, create=True) if not dry_run else None
                 for item in other_archive.iter_items():
@@ -1154,16 +1154,14 @@ class Archiver:
             self.print_file_status(status, path)
 
     @staticmethod
-    def build_filter(matcher, peek_and_store_hardlink_masters, strip_components):
+    def build_filter(matcher, strip_components):
         if strip_components:
             def item_filter(item):
                 matched = matcher.match(item.path) and os.sep.join(item.path.split(os.sep)[strip_components:])
-                peek_and_store_hardlink_masters(item, matched)
                 return matched
         else:
             def item_filter(item):
                 matched = matcher.match(item.path)
-                peek_and_store_hardlink_masters(item, matched)
                 return matched
         return item_filter
 
@@ -1186,33 +1184,18 @@ class Archiver:
         sparse = args.sparse
         strip_components = args.strip_components
         dirs = []
-        partial_extract = not matcher.empty() or strip_components
-        hardlink_masters = {} if partial_extract or not has_link else None
+        hlm = HardLinkManager(id_type=bytes, info_type=str)  # hlid -> path
 
-        def peek_and_store_hardlink_masters(item, matched):
-            # not has_link:
-            # OS does not have hardlink capability thus we need to remember the chunks so that
-            # we can extract all hardlinks as separate normal (not-hardlinked) files instead.
-            #
-            # partial_extract and not matched and hardlinkable:
-            # we do not extract the very first hardlink, so we need to remember the chunks
-            # in hardlinks_master, so we can use them when we extract some 2nd+ hardlink item
-            # that has no chunks list.
-            if ((not has_link or (partial_extract and not matched and hardlinkable(item.mode))) and
-                    (item.get('hardlink_master', True) and 'source' not in item)):
-                hardlink_masters[item.get('path')] = (item.get('chunks'), None)
-
-        filter = self.build_filter(matcher, peek_and_store_hardlink_masters, strip_components)
+        filter = self.build_filter(matcher, strip_components)
         if progress:
             pi = ProgressIndicatorPercent(msg='%5.1f%% Extracting: %s', step=0.1, msgid='extract')
             pi.output('Calculating total archive size for the progress indicator (might take long for large archives)')
-            extracted_size = sum(item.get_size(hardlink_masters) for item in archive.iter_items(filter))
+            extracted_size = sum(item.get_size() for item in archive.iter_items(filter))
             pi.total = extracted_size
         else:
             pi = None
 
-        for item in archive.iter_items(filter, partial_extract=partial_extract,
-                                       preload=True, hardlink_masters=hardlink_masters):
+        for item in archive.iter_items(filter, preload=True):
             orig_path = item.path
             if strip_components:
                 item.path = os.sep.join(orig_path.split(os.sep)[strip_components:])
@@ -1227,13 +1210,13 @@ class Archiver:
                 logging.getLogger('borg.output.list').info(remove_surrogates(item.path))
             try:
                 if dry_run:
-                    archive.extract_item(item, dry_run=True, pi=pi)
+                    archive.extract_item(item, dry_run=True, hlm=hlm, pi=pi)
                 else:
                     if stat.S_ISDIR(item.mode):
                         dirs.append(item)
                         archive.extract_item(item, stdout=stdout, restore_attrs=False)
                     else:
-                        archive.extract_item(item, stdout=stdout, sparse=sparse, hardlink_masters=hardlink_masters,
+                        archive.extract_item(item, stdout=stdout, sparse=sparse, hlm=hlm,
                                              stripped_components=strip_components, original_path=orig_path, pi=pi)
             except (BackupOSError, BackupError) as e:
                 self.print_warning('%s: %s', remove_surrogates(orig_path), e)
@@ -1298,15 +1281,9 @@ class Archiver:
         progress = args.progress
         output_list = args.output_list
         strip_components = args.strip_components
-        partial_extract = not matcher.empty() or strip_components
-        hardlink_masters = {} if partial_extract else None
-
-        def peek_and_store_hardlink_masters(item, matched):
-            if ((partial_extract and not matched and hardlinkable(item.mode)) and
-                    (item.get('hardlink_master', True) and 'source' not in item)):
-                hardlink_masters[item.get('path')] = (item.get('chunks'), None)
+        hlm = HardLinkManager(id_type=bytes, info_type=str)  # hlid -> path
 
-        filter = self.build_filter(matcher, peek_and_store_hardlink_masters, strip_components)
+        filter = self.build_filter(matcher, strip_components)
 
         # The | (pipe) symbol instructs tarfile to use a streaming mode of operation
         # where it never seeks on the passed fileobj.
@@ -1316,7 +1293,7 @@ class Archiver:
         if progress:
             pi = ProgressIndicatorPercent(msg='%5.1f%% Processing: %s', step=0.1, msgid='extract')
             pi.output('Calculating size')
-            extracted_size = sum(item.get_size(hardlink_masters) for item in archive.iter_items(filter))
+            extracted_size = sum(item.get_size() for item in archive.iter_items(filter))
             pi.total = extracted_size
         else:
             pi = None
@@ -1351,9 +1328,8 @@ class Archiver:
             tarinfo.gid = item.gid
             tarinfo.uname = item.user or ''
             tarinfo.gname = item.group or ''
-            # The linkname in tar has the same dual use the 'source' attribute of Borg items,
-            # i.e. for symlinks it means the destination, while for hardlinks it refers to the
-            # file.
+            # The linkname in tar has 2 uses:
+            # for symlinks it means the destination, while for hardlinks it refers to the file.
             # Since hardlinks in tar have a different type code (LNKTYPE) the format might
             # support hardlinking arbitrary objects (including symlinks and directories), but
             # whether implementations actually support that is a whole different question...
@@ -1362,23 +1338,16 @@ class Archiver:
             modebits = stat.S_IFMT(item.mode)
             if modebits == stat.S_IFREG:
                 tarinfo.type = tarfile.REGTYPE
-                if 'source' in item:
-                    source = os.sep.join(item.source.split(os.sep)[strip_components:])
-                    if hardlink_masters is None:
-                        linkname = source
-                    else:
-                        chunks, linkname = hardlink_masters.get(item.source, (None, source))
-                    if linkname:
-                        # Master was already added to the archive, add a hardlink reference to it.
+                if 'hlid' in item:
+                    linkname = hlm.retrieve(id=item.hlid)
+                    if linkname is not None:
+                        # the first hardlink was already added to the archive, add a tar-hardlink reference to it.
                         tarinfo.type = tarfile.LNKTYPE
                         tarinfo.linkname = linkname
-                    elif chunks is not None:
-                        # The item which has the chunks was not put into the tar, therefore
-                        # we do that now and update hardlink_masters to reflect that.
-                        item.chunks = chunks
+                    else:
                         tarinfo.size = item.get_size()
                         stream = item_content_stream(item)
-                        hardlink_masters[item.get('source') or original_path] = (None, item.path)
+                        hlm.remember(id=item.hlid, info=item.path)
                 else:
                     tarinfo.size = item.get_size()
                     stream = item_content_stream(item)
@@ -1436,8 +1405,7 @@ class Archiver:
                 ph['BORG.item.meta'] = meta_text
             return ph
 
-        for item in archive.iter_items(filter, partial_extract=partial_extract,
-                                       preload=True, hardlink_masters=hardlink_masters):
+        for item in archive.iter_items(filter, preload=True):
             orig_path = item.path
             if strip_components:
                 item.path = os.sep.join(orig_path.split(os.sep)[strip_components:])
@@ -2072,12 +2040,11 @@ class Archiver:
             elif tarinfo.isdir():
                 status = tfo.process_dir(tarinfo=tarinfo, status='d', type=stat.S_IFDIR)
             elif tarinfo.issym():
-                status = tfo.process_link(tarinfo=tarinfo, status='s', type=stat.S_IFLNK)
+                status = tfo.process_symlink(tarinfo=tarinfo, status='s', type=stat.S_IFLNK)
             elif tarinfo.islnk():
-                # tar uses the same hardlink model as borg (rather vice versa); the first instance of a hardlink
-                # is stored as a regular file, later instances are special entries referencing back to the
-                # first instance.
-                status = tfo.process_link(tarinfo=tarinfo, status='h', type=stat.S_IFREG)
+                # tar uses a hardlink model like: the first instance of a hardlink is stored as a regular file,
+                # later instances are special entries referencing back to the first instance.
+                status = tfo.process_hardlink(tarinfo=tarinfo, status='h', type=stat.S_IFREG)
             elif tarinfo.isblk():
                 status = tfo.process_dev(tarinfo=tarinfo, status='b', type=stat.S_IFBLK)
             elif tarinfo.ischr():

+ 14 - 25
src/borg/fuse.py

@@ -35,7 +35,8 @@ from .crypto.low_level import blake2b_128
 from .archiver import Archiver
 from .archive import Archive, get_item_uid_gid
 from .hashindex import FuseVersionsIndex
-from .helpers import daemonize, daemonizing, hardlinkable, signal_handler, format_file_size, Error
+from .helpers import daemonize, daemonizing, signal_handler, format_file_size, Error
+from .helpers import HardLinkManager
 from .helpers import msgpack
 from .item import Item
 from .lrucache import LRUCache
@@ -339,15 +340,9 @@ class FuseBackend:
                           consider_part_files=self._args.consider_part_files)
         strip_components = self._args.strip_components
         matcher = Archiver.build_matcher(self._args.patterns, self._args.paths)
-        partial_extract = not matcher.empty() or strip_components
-        hardlink_masters = {} if partial_extract else None
+        hlm = HardLinkManager(id_type=bytes, info_type=str)  # hlid -> path
 
-        def peek_and_store_hardlink_masters(item, matched):
-            if (partial_extract and not matched and hardlinkable(item.mode) and
-                    item.get('hardlink_master', True) and 'source' not in item):
-                hardlink_masters[item.get('path')] = (item.get('chunks'), None)
-
-        filter = Archiver.build_filter(matcher, peek_and_store_hardlink_masters, strip_components)
+        filter = Archiver.build_filter(matcher, strip_components)
         for item_inode, item in self.cache.iter_archive_items(archive.metadata.items, filter=filter,
                                                               consider_part_files=self._args.consider_part_files):
             if strip_components:
@@ -369,15 +364,13 @@ class FuseBackend:
             parent = 1
             for segment in segments[:-1]:
                 parent = self._process_inner(segment, parent)
-            self._process_leaf(segments[-1], item, parent, prefix, is_dir, item_inode,
-                               hardlink_masters, strip_components)
+            self._process_leaf(segments[-1], item, parent, prefix, is_dir, item_inode, hlm)
         duration = time.perf_counter() - t0
         logger.debug('fuse: _process_archive completed in %.1f s for archive %s', duration, archive.name)
 
-    def _process_leaf(self, name, item, parent, prefix, is_dir, item_inode, hardlink_masters, stripped_components):
+    def _process_leaf(self, name, item, parent, prefix, is_dir, item_inode, hlm):
         path = item.path
         del item.path  # save some space
-        hardlink_masters = hardlink_masters or {}
 
         def file_version(item, path):
             if 'chunks' in item:
@@ -402,10 +395,9 @@ class FuseBackend:
             version_enc = os.fsencode('.%05d' % version)
             return name + version_enc + ext
 
-        if 'source' in item and hardlinkable(item.mode):
-            source = os.sep.join(item.source.split(os.sep)[stripped_components:])
-            chunks, link_target = hardlink_masters.get(item.source, (None, source))
-            if link_target:
+        if 'hlid' in item:
+            link_target = hlm.retrieve(id=item.hlid, default=None)
+            if link_target is not None:
                 # Hard link was extracted previously, just link
                 link_target = os.fsencode(link_target)
                 if self.versions:
@@ -415,19 +407,16 @@ class FuseBackend:
                 try:
                     inode = self.find_inode(link_target, prefix)
                 except KeyError:
-                    logger.warning('Skipping broken hard link: %s -> %s', path, source)
+                    logger.warning('Skipping broken hard link: %s -> %s', path, link_target)
                     return
                 item = self.get_item(inode)
                 item.nlink = item.get('nlink', 1) + 1
                 self._items[inode] = item
-            elif chunks is not None:
-                # assign chunks to this item, since the item which had the chunks was not extracted
-                item.chunks = chunks
+            else:
                 inode = item_inode
                 self._items[inode] = item
-                if hardlink_masters:
-                    # Update master entry with extracted item path, so that following hardlinks don't extract twice.
-                    hardlink_masters[item.source] = (None, path)
+                # remember extracted item path, so that following hardlinks don't extract twice.
+                hlm.remember(id=item.hlid, info=path)
         else:
             inode = item_inode
 
@@ -436,7 +425,7 @@ class FuseBackend:
             enc_path = os.fsencode(path)
             version = file_version(item, enc_path)
             if version is not None:
-                # regular file, with contents - maybe a hardlink master
+                # regular file, with contents
                 name = make_versioned_name(name, version)
                 self.file_versions[enc_path] = version
 

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

@@ -1,4 +1,5 @@
 import errno
+import hashlib
 import os
 import os.path
 import re
@@ -165,9 +166,70 @@ def make_path_safe(path):
     return _safe_re.sub('', path) or '.'
 
 
-def hardlinkable(mode):
-    """return True if we support hardlinked items of this type"""
-    return stat.S_ISREG(mode) or stat.S_ISBLK(mode) or stat.S_ISCHR(mode) or stat.S_ISFIFO(mode)
+class HardLinkManager:
+    """
+    Manage hardlinks (and avoid code duplication doing so).
+
+    A) When creating a borg2 archive from the filesystem, we have to maintain a mapping like:
+       (dev, ino) -> (hlid, chunks)  # for fs_hl_targets
+       If we encounter the same (dev, ino) again later, we'll just re-use the hlid and chunks list.
+
+    B) When extracting a borg2 archive to the filesystem, we have to maintain a mapping like:
+       hlid -> path
+       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
+       and create the same hlid = hardlink_id(source).
+
+    D) When importing a tar file (simplified 1-pass way for now, not creating borg hardlink items):
+       path -> chunks
+       If we encounter a LNK tar entry later with linkname==path, we re-use the chunks and create a regular file item.
+       For better hardlink support (including the very first hardlink item for each group of same-target hardlinks),
+       we would need a 2-pass processing, which is not yet implemented.
+    """
+    def __init__(self, *, id_type, info_type):
+        self._map = {}
+        self.id_type = id_type
+        self.info_type = info_type
+
+    def borg1_hardlinkable(self, mode):  # legacy
+        return stat.S_ISREG(mode) or stat.S_ISBLK(mode) or stat.S_ISCHR(mode) or stat.S_ISFIFO(mode)
+
+    def borg1_hardlink_master(self, item):  # legacy
+        return item.get('hardlink_master', True) and 'source' not in item and self.borg1_hardlinkable(item.mode)
+
+    def borg1_hardlink_slave(self, item):  # legacy
+        return 'source' in item and self.borg1_hardlinkable(item.mode)
+
+    def hardlink_id(self, path):
+        """compute a hardlink id from a path"""
+        assert isinstance(path, bytes)
+        return hashlib.sha256(path).digest()
+
+    def remember(self, *, id, info):
+        """
+        remember stuff from a (usually contentful) item.
+
+        :param id: some id used to reference to the contentful item, could be:
+                   a path (tar style, old borg style) [bytes]
+                   a hlid (new borg style) [bytes]
+                   a (dev, inode) tuple (filesystem)
+        :param info: information to remember, could be:
+                     chunks / chunks_healthy list
+                     hlid
+        """
+        assert isinstance(id, self.id_type)
+        assert isinstance(info, self.info_type)
+        self._map[id] = info
+
+    def retrieve(self, id, *, default=None):
+        """
+        retrieve stuff to use it in a (usually contentless) item.
+        """
+        assert isinstance(id, self.id_type)
+        return self._map.get(id, default)
 
 
 def scandir_keyfunc(dirent):

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

@@ -695,7 +695,8 @@ class ItemFormatter(BaseFormatter):
     KEY_DESCRIPTIONS = {
         'bpath': 'verbatim POSIX path, can contain any character except NUL',
         'path': 'path interpreted as text (might be missing non-text characters, see bpath)',
-        'source': 'link target for links (identical to linktarget)',
+        'source': 'link target for symlinks (identical to linktarget)',
+        'hlid': 'hard link identity (same if hardlinking same fs object)',
         'extra': 'prepends {source} with " -> " for soft links and " link to " for hard links',
         'csize': 'compressed size',
         'dsize': 'deduplicated size',
@@ -706,7 +707,7 @@ class ItemFormatter(BaseFormatter):
         'health': 'either "healthy" (file ok) or "broken" (if file has all-zero replacement chunks)',
     }
     KEY_GROUPS = (
-        ('type', 'mode', 'uid', 'gid', 'user', 'group', 'path', 'bpath', 'source', 'linktarget', 'flags'),
+        ('type', 'mode', 'uid', 'gid', 'user', 'group', 'path', 'bpath', 'source', 'linktarget', 'hlid', 'flags'),
         ('size', 'csize', 'dsize', 'dcsize', 'num_chunks', 'unique_chunks'),
         ('mtime', 'ctime', 'atime', 'isomtime', 'isoctime', 'isoatime'),
         tuple(sorted(hash_algorithms)),
@@ -802,11 +803,9 @@ class ItemFormatter(BaseFormatter):
         extra = ''
         if source:
             source = remove_surrogates(source)
-            if item_type == 'l':
-                extra = ' -> %s' % source
-            else:
-                mode = 'h' + mode[1:]
-                extra = ' link to %s' % source
+            extra = ' -> %s' % source
+        hlid = item.get('hlid')
+        hlid = bin_to_hex(hlid) if hlid else ''
         item_data['type'] = item_type
         item_data['mode'] = mode
         item_data['user'] = item.user or item.uid
@@ -822,6 +821,7 @@ class ItemFormatter(BaseFormatter):
             item_data['health'] = 'broken' if 'chunks_healthy' in item else 'healthy'
         item_data['source'] = source
         item_data['linktarget'] = source
+        item_data['hlid'] = hlid
         item_data['flags'] = item.get('bsdflags')
         for key in self.used_call_keys:
             item_data[key] = self.call_keys[key](item)

+ 0 - 1
src/borg/item.pyx

@@ -214,7 +214,6 @@ class Item(PropDict):
         except AttributeError:
             if stat.S_ISLNK(self.mode):
                 # get out of here quickly. symlinks have no own chunks, their fs size is the length of the target name.
-                # also, there is the dual-use issue of .source (#2343), so don't confuse it with a hardlink slave.
                 return len(self.source)
             # no precomputed (c)size value available, compute it:
             try:

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

@@ -321,7 +321,7 @@ class ArchiverTestCaseBase(BaseTestCase):
                 contents = b'X' * size
             fd.write(contents)
 
-    def create_test_files(self):
+    def create_test_files(self, create_hardlinks=True):
         """Create a minimal test case including all supported file types
         """
         # File
@@ -332,7 +332,7 @@ class ArchiverTestCaseBase(BaseTestCase):
         # File mode
         os.chmod('input/file1', 0o4755)
         # Hard link
-        if are_hardlinks_supported():
+        if are_hardlinks_supported() and create_hardlinks:
             os.link(os.path.join(self.input_path, 'file1'),
                     os.path.join(self.input_path, 'hardlink'))
         # Symlink
@@ -432,7 +432,7 @@ class ArchiverTestCase(ArchiverTestCaseBase):
             self.assert_in(name, list_output)
         self.assert_dirs_equal('input', 'output/input')
         info_output = self.cmd('info', self.repository_location + '::test')
-        item_count = 4 if has_lchflags else 5  # one file is UF_NODUMP
+        item_count = 5 if has_lchflags else 6  # one file is UF_NODUMP
         self.assert_in('Number of files: %d' % item_count, info_output)
         shutil.rmtree(self.cache_path)
         info_output2 = self.cmd('info', self.repository_location + '::test')
@@ -506,6 +506,29 @@ class ArchiverTestCase(ArchiverTestCaseBase):
             self.cmd('extract', self.repository_location + '::test')
             assert os.readlink('input/link1') == 'somewhere'
 
+    @pytest.mark.skipif(not are_symlinks_supported() or not are_hardlinks_supported(),
+                        reason='symlinks or hardlinks not supported')
+    def test_hardlinked_symlinks_extract(self):
+        self.create_regular_file('target', size=1024)
+        with changedir('input'):
+            os.symlink('target', 'symlink1')
+            os.link('symlink1', 'symlink2', follow_symlinks=False)
+        self.cmd('init', '--encryption=repokey', self.repository_location)
+        self.cmd('create', self.repository_location + '::test', 'input')
+        with changedir('output'):
+            output = self.cmd('extract', self.repository_location + '::test')
+            print(output)
+            with changedir('input'):
+                assert os.path.exists('target')
+                assert os.readlink('symlink1') == 'target'
+                assert os.readlink('symlink2') == 'target'
+                st1 = os.stat('symlink1', follow_symlinks=False)
+                st2 = os.stat('symlink2', follow_symlinks=False)
+                assert st1.st_nlink == 2
+                assert st2.st_nlink == 2
+                assert st1.st_ino == st2.st_ino
+                assert st1.st_size == st2.st_size
+
     @pytest.mark.skipif(not is_utime_fully_supported(), reason='cannot properly setup and execute test without utime')
     def test_atime(self):
         def has_noatime(some_file):
@@ -2661,7 +2684,7 @@ class ArchiverTestCase(ArchiverTestCaseBase):
                 hl3 = os.path.join(mountpoint, 'input', 'hardlink3', 'hardlink3.00001')
                 assert os.stat(hl1).st_ino == os.stat(hl2).st_ino == os.stat(hl3).st_ino
                 assert open(hl3, 'rb').read() == b'123456'
-        # similar again, but exclude the hardlink master:
+        # similar again, but exclude the 1st hardlink:
         with self.fuse_mount(self.repository_location, mountpoint, '-o', 'versions', '-e', 'input/hardlink1'):
             if are_hardlinks_supported():
                 hl2 = os.path.join(mountpoint, 'input', 'hardlink2', 'hardlink2.00001')
@@ -3475,7 +3498,7 @@ id: 2 / e29442 3506da 4e1ea7 / 25f62a 5a3d41 - 02
             assert os.stat('input/dir1/source2').st_nlink == 2
 
     def test_import_tar(self, tar_format='PAX'):
-        self.create_test_files()
+        self.create_test_files(create_hardlinks=False)  # hardlinks become separate files
         os.unlink('input/flagfile')
         self.cmd('init', '--encryption=none', self.repository_location)
         self.cmd('create', self.repository_location + '::src', 'input')
@@ -3489,7 +3512,7 @@ id: 2 / e29442 3506da 4e1ea7 / 25f62a 5a3d41 - 02
     def test_import_tar_gz(self, tar_format='GNU'):
         if not shutil.which('gzip'):
             pytest.skip('gzip is not installed')
-        self.create_test_files()
+        self.create_test_files(create_hardlinks=False)  # hardlinks become separate files
         os.unlink('input/flagfile')
         self.cmd('init', '--encryption=none', self.repository_location)
         self.cmd('create', self.repository_location + '::src', 'input')
@@ -4444,26 +4467,23 @@ def test_chunk_content_equal():
 
 
 class TestBuildFilter:
-    @staticmethod
-    def peek_and_store_hardlink_masters(item, matched):
-        pass
 
     def test_basic(self):
         matcher = PatternMatcher()
         matcher.add([parse_pattern('included')], IECommand.Include)
-        filter = Archiver.build_filter(matcher, self.peek_and_store_hardlink_masters, 0)
+        filter = Archiver.build_filter(matcher, 0)
         assert filter(Item(path='included'))
         assert filter(Item(path='included/file'))
         assert not filter(Item(path='something else'))
 
     def test_empty(self):
         matcher = PatternMatcher(fallback=True)
-        filter = Archiver.build_filter(matcher, self.peek_and_store_hardlink_masters, 0)
+        filter = Archiver.build_filter(matcher, 0)
         assert filter(Item(path='anything'))
 
     def test_strip_components(self):
         matcher = PatternMatcher(fallback=True)
-        filter = Archiver.build_filter(matcher, self.peek_and_store_hardlink_masters, strip_components=1)
+        filter = Archiver.build_filter(matcher, strip_components=1)
         assert not filter(Item(path='shallow'))
         assert not filter(Item(path='shallow/'))  # can this even happen? paths are normalized...
         assert filter(Item(path='deep enough/file'))