Răsfoiți Sursa

Merge pull request #6763 from ThomasWaldmann/remove-csize

borg2: there is no csize
TW 3 ani în urmă
părinte
comite
fc8a289647

+ 5 - 13
docs/internals/frontends.rst

@@ -269,12 +269,8 @@ stats
         Number of unique chunks
     total_size
         Total uncompressed size of all chunks multiplied with their reference counts
-    total_csize
-        Total compressed and encrypted size of all chunks multiplied with their reference counts
     unique_size
         Uncompressed size of all chunks
-    unique_csize
-        Compressed and encrypted size of all chunks
 
 .. highlight: json
 
@@ -285,10 +281,8 @@ Example *borg info* output::
             "path": "/home/user/.cache/borg/0cbe6166b46627fd26b97f8831e2ca97584280a46714ef84d2b668daf8271a23",
             "stats": {
                 "total_chunks": 511533,
-                "total_csize": 17948017540,
                 "total_size": 22635749792,
                 "total_unique_chunks": 54892,
-                "unique_csize": 1920405405,
                 "unique_size": 2449675468
             }
         },
@@ -424,10 +418,8 @@ The same archive with more information (``borg info --last 1 --json``)::
             "path": "/home/user/.cache/borg/0cbe6166b46627fd26b97f8831e2ca97584280a46714ef84d2b668daf8271a23",
             "stats": {
                 "total_chunks": 511533,
-                "total_csize": 17948017540,
                 "total_size": 22635749792,
                 "total_unique_chunks": 54892,
-                "unique_csize": 1920405405,
                 "unique_size": 2449675468
             }
         },
@@ -495,26 +487,26 @@ added:
 
 removed:
     See **added** property.
-    
+
 old_mode:
     If **type** == '*mode*', then **old_mode** and **new_mode** provide the mode and permissions changes.
 
 new_mode:
     See **old_mode** property.
- 
+
 old_user:
     If **type** == '*owner*', then **old_user**, **new_user**, **old_group** and **new_group** provide the user
     and group ownership changes.
 
 old_group:
     See **old_user** property.
- 
+
 new_user:
     See **old_user** property.
- 
+
 new_group:
     See **old_user** property.
-    
+
 
 Example (excerpt) of ``borg diff --json-lines``::
 

+ 47 - 85
src/borg/archive.py

@@ -58,49 +58,44 @@ class Statistics:
     def __init__(self, output_json=False, iec=False):
         self.output_json = output_json
         self.iec = iec
-        self.osize = self.csize = self.usize = self.nfiles = 0
-        self.osize_parts = self.csize_parts = self.usize_parts = self.nfiles_parts = 0
+        self.osize = self.usize = self.nfiles = 0
+        self.osize_parts = self.usize_parts = self.nfiles_parts = 0
         self.last_progress = 0  # timestamp when last progress was shown
 
-    def update(self, size, csize, unique, part=False):
+    def update(self, size, unique, part=False):
         if not part:
             self.osize += size
-            self.csize += csize
             if unique:
-                self.usize += csize
+                self.usize += size
         else:
             self.osize_parts += size
-            self.csize_parts += csize
             if unique:
-                self.usize_parts += csize
+                self.usize_parts += size
 
     def __add__(self, other):
         if not isinstance(other, Statistics):
             raise TypeError('can only add Statistics objects')
         stats = Statistics(self.output_json, self.iec)
         stats.osize = self.osize + other.osize
-        stats.csize = self.csize + other.csize
         stats.usize = self.usize + other.usize
         stats.nfiles = self.nfiles + other.nfiles
         stats.osize_parts = self.osize_parts + other.osize_parts
-        stats.csize_parts = self.csize_parts + other.csize_parts
         stats.usize_parts = self.usize_parts + other.usize_parts
         stats.nfiles_parts = self.nfiles_parts + other.nfiles_parts
         return stats
 
-    summary = "{label:15} {stats.osize_fmt:>20s} {stats.csize_fmt:>20s} {stats.usize_fmt:>20s}"
+    summary = "{label:15} {stats.osize_fmt:>20s} {stats.usize_fmt:>20s}"
 
     def __str__(self):
         return self.summary.format(stats=self, label='This archive:')
 
     def __repr__(self):
-        return "<{cls} object at {hash:#x} ({self.osize}, {self.csize}, {self.usize})>".format(
+        return "<{cls} object at {hash:#x} ({self.osize}, {self.usize})>".format(
             cls=type(self).__name__, hash=id(self), self=self)
 
     def as_dict(self):
         return {
             'original_size': FileSize(self.osize, iec=self.iec),
-            'compressed_size': FileSize(self.csize, iec=self.iec),
             'deduplicated_size': FileSize(self.usize, iec=self.iec),
             'nfiles': self.nfiles,
         }
@@ -108,10 +103,8 @@ class Statistics:
     def as_raw_dict(self):
         return {
             'size': self.osize,
-            'csize': self.csize,
             'nfiles': self.nfiles,
             'size_parts': self.osize_parts,
-            'csize_parts': self.csize_parts,
             'nfiles_parts': self.nfiles_parts,
         }
 
@@ -119,10 +112,8 @@ class Statistics:
     def from_raw_dict(cls, **kw):
         self = cls()
         self.osize = kw['size']
-        self.csize = kw['csize']
         self.nfiles = kw['nfiles']
         self.osize_parts = kw['size_parts']
-        self.csize_parts = kw['csize_parts']
         self.nfiles_parts = kw['nfiles_parts']
         return self
 
@@ -134,10 +125,6 @@ class Statistics:
     def usize_fmt(self):
         return format_file_size(self.usize, iec=self.iec)
 
-    @property
-    def csize_fmt(self):
-        return format_file_size(self.csize, iec=self.iec)
-
     def show_progress(self, item=None, final=False, stream=None, dt=None):
         now = time.monotonic()
         if dt is None or now - self.last_progress > dt:
@@ -158,7 +145,7 @@ class Statistics:
             else:
                 columns, lines = get_terminal_size()
                 if not final:
-                    msg = '{0.osize_fmt} O {0.csize_fmt} C {0.usize_fmt} D {0.nfiles} N '.format(self)
+                    msg = '{0.osize_fmt} O {0.usize_fmt} U {0.nfiles} N '.format(self)
                     path = remove_surrogates(item.path) if item else ''
                     space = columns - swidth(msg)
                     if space < 12:
@@ -383,7 +370,7 @@ class CacheChunkBuffer(ChunkBuffer):
         self.stats = stats
 
     def write_chunk(self, chunk):
-        id_, _, _ = self.cache.add_chunk(self.key.id_hash(chunk), chunk, self.stats, wait=False)
+        id_, _ = self.cache.add_chunk(self.key.id_hash(chunk), chunk, self.stats, wait=False)
         self.cache.repository.async_response(wait=False)
         return id_
 
@@ -507,6 +494,10 @@ class Archive:
     def duration_from_meta(self):
         return format_timedelta(self.ts_end - self.ts)
 
+    def _archive_csize(self):
+        cdata = self.repository.get(self.id)
+        return len(cdata)
+
     def info(self):
         if self.create:
             stats = self.stats
@@ -524,7 +515,7 @@ class Archive:
             'duration': (end - start).total_seconds(),
             'stats': stats.as_dict(),
             'limits': {
-                'max_archive_size': self.cache.chunks[self.id].csize / MAX_DATA_SIZE,
+                'max_archive_size': self._archive_csize() / MAX_DATA_SIZE,
             },
         }
         if self.create:
@@ -553,7 +544,7 @@ Utilization of max. archive size: {csize_max:.0%}
             self,
             start=OutputTimestamp(self.start.replace(tzinfo=timezone.utc)),
             end=OutputTimestamp(self.end.replace(tzinfo=timezone.utc)),
-            csize_max=self.cache.chunks[self.id].csize / MAX_DATA_SIZE,
+            csize_max=self._archive_csize() / MAX_DATA_SIZE,
             location=self.repository._location.canonical_path()
 )
 
@@ -614,10 +605,8 @@ Utilization of max. archive size: {csize_max:.0%}
         if stats is not None:
             metadata.update({
                 'size': stats.osize,
-                'csize': stats.csize,
                 'nfiles': stats.nfiles,
                 'size_parts': stats.osize_parts,
-                'csize_parts': stats.csize_parts,
                 'nfiles_parts': stats.nfiles_parts})
         metadata.update(additional_metadata or {})
         metadata = ArchiveItem(metadata)
@@ -640,25 +629,12 @@ Utilization of max. archive size: {csize_max:.0%}
         self.cache.commit()
 
     def calc_stats(self, cache, want_unique=True):
-        # caching wrapper around _calc_stats which is rather slow for archives made with borg < 1.2
-        have_borg12_meta = self.metadata.get('nfiles') is not None
-        try:
-            stats = Statistics.from_raw_dict(**cache.pre12_meta[self.fpr])
-        except KeyError:  # not in pre12_meta cache
-            stats = self._calc_stats(cache, want_unique=want_unique)
-            if not have_borg12_meta:
-                cache.pre12_meta[self.fpr] = stats.as_raw_dict()
-        return stats
-
-    def _calc_stats(self, cache, want_unique=True):
-        have_borg12_meta = self.metadata.get('nfiles') is not None
-
-        if have_borg12_meta and not want_unique:
-            unique_csize = 0
+        if not want_unique:
+            unique_size = 0
         else:
             def add(id):
                 entry = cache.chunks[id]
-                archive_index.add(id, 1, entry.size, entry.csize)
+                archive_index.add(id, 1, entry.size)
 
             archive_index = ChunkIndex()
             sync = CacheSynchronizer(archive_index)
@@ -673,29 +649,16 @@ Utilization of max. archive size: {csize_max:.0%}
                 add(id)
                 data = self.key.decrypt(id, chunk)
                 sync.feed(data)
-            unique_csize = archive_index.stats_against(cache.chunks)[3]
+            unique_size = archive_index.stats_against(cache.chunks)[1]
             pi.finish()
 
         stats = Statistics(iec=self.iec)
-        stats.usize = unique_csize  # the part files use same chunks as the full file
-        if not have_borg12_meta:
-            if self.consider_part_files:
-                stats.nfiles = sync.num_files_totals
-                stats.osize = sync.size_totals
-                stats.csize = sync.csize_totals
-            else:
-                stats.nfiles = sync.num_files_totals - sync.num_files_parts
-                stats.osize = sync.size_totals - sync.size_parts
-                stats.csize = sync.csize_totals - sync.csize_parts
-        else:
-            if self.consider_part_files:
-                stats.nfiles = self.metadata.nfiles_parts + self.metadata.nfiles
-                stats.osize = self.metadata.size_parts + self.metadata.size
-                stats.csize = self.metadata.csize_parts + self.metadata.csize
-            else:
-                stats.nfiles = self.metadata.nfiles
-                stats.osize = self.metadata.size
-                stats.csize = self.metadata.csize
+        stats.usize = unique_size  # the part files use same chunks as the full file
+        stats.nfiles = self.metadata.nfiles
+        stats.osize = self.metadata.size
+        if self.consider_part_files:
+            stats.nfiles += self.metadata.nfiles_parts
+            stats.osize += self.metadata.size_parts
         return stats
 
     @contextmanager
@@ -986,7 +949,7 @@ Utilization of max. archive size: {csize_max:.0%}
                         item = Item(internal_dict=item)
                         if 'chunks' in item:
                             part = not self.consider_part_files and 'part' in item
-                            for chunk_id, size, csize in item.chunks:
+                            for chunk_id, size in item.chunks:
                                 chunk_decref(chunk_id, stats, part=part)
                 except (TypeError, ValueError):
                     # if items metadata spans multiple chunks and one chunk got dropped somehow,
@@ -1344,7 +1307,7 @@ class FilesystemObjectProcessors:
                     # 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 '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]
+                    [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))
@@ -1626,7 +1589,7 @@ class ArchiveChecker:
             if not result:
                 break
             marker = result[-1]
-            init_entry = ChunkIndexEntry(refcount=0, size=0, csize=0)
+            init_entry = ChunkIndexEntry(refcount=0, size=0)
             for id_ in result:
                 self.chunks[id_] = init_entry
 
@@ -1783,21 +1746,21 @@ class ArchiveChecker:
         self.chunks.pop(Manifest.MANIFEST_ID, None)
 
         def mark_as_possibly_superseded(id_):
-            if self.chunks.get(id_, ChunkIndexEntry(0, 0, 0)).refcount == 0:
+            if self.chunks.get(id_, ChunkIndexEntry(0, 0)).refcount == 0:
                 self.possibly_superseded.add(id_)
 
         def add_callback(chunk):
             id_ = self.key.id_hash(chunk)
             cdata = self.key.encrypt(id_, chunk)
-            add_reference(id_, len(chunk), len(cdata), cdata)
+            add_reference(id_, len(chunk), cdata)
             return id_
 
-        def add_reference(id_, size, csize, cdata=None):
+        def add_reference(id_, size, cdata=None):
             try:
                 self.chunks.incref(id_)
             except KeyError:
                 assert cdata is not None
-                self.chunks[id_] = ChunkIndexEntry(refcount=1, size=size, csize=csize)
+                self.chunks[id_] = ChunkIndexEntry(refcount=1, size=size)
                 if self.repair:
                     self.repository.put(id_, cdata)
 
@@ -1811,8 +1774,7 @@ class ArchiveChecker:
                 chunk = Chunk(None, allocation=CH_ALLOC, size=size)
                 chunk_id, data = cached_hash(chunk, self.key.id_hash)
                 cdata = self.key.encrypt(chunk_id, data)
-                csize = len(cdata)
-                return chunk_id, size, csize, cdata
+                return chunk_id, size, cdata
 
             offset = 0
             chunk_list = []
@@ -1827,7 +1789,7 @@ class ArchiveChecker:
                 has_chunks_healthy = False
                 chunks_healthy = chunks_current
             for chunk_current, chunk_healthy in zip(chunks_current, chunks_healthy):
-                chunk_id, size, csize = chunk_healthy
+                chunk_id, size = chunk_healthy
                 if chunk_id not in self.chunks:
                     # a chunk of the healthy list is missing
                     if chunk_current == chunk_healthy:
@@ -1835,32 +1797,32 @@ class ArchiveChecker:
                                      'Replacing with all-zero chunk.'.format(
                                      archive_name, item.path, offset, offset + size, bin_to_hex(chunk_id)))
                         self.error_found = chunks_replaced = True
-                        chunk_id, size, csize, cdata = replacement_chunk(size)
-                        add_reference(chunk_id, size, csize, cdata)
+                        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, csize = chunk_current
+                        chunk_id, size = chunk_current
                         if chunk_id in self.chunks:
-                            add_reference(chunk_id, size, csize)
+                            add_reference(chunk_id, size)
                         else:
                             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, csize, cdata = replacement_chunk(size)
-                            add_reference(chunk_id, size, csize, cdata)
+                            chunk_id, size, cdata = replacement_chunk(size)
+                            add_reference(chunk_id, size, cdata)
                 else:
                     if chunk_current == chunk_healthy:
                         # normal case, all fine.
-                        add_reference(chunk_id, size, csize)
+                        add_reference(chunk_id, size)
                     else:
                         logger.info('{}: {}: Healed previously missing file chunk! (Byte {}-{}, Chunk {}).'.format(
                             archive_name, item.path, offset, offset + size, bin_to_hex(chunk_id)))
-                        add_reference(chunk_id, size, csize)
+                        add_reference(chunk_id, size)
                         mark_as_possibly_superseded(chunk_current[0])  # maybe orphaned the all-zero replacement chunk
-                chunk_list.append([chunk_id, size, csize])  # list-typed element as chunks_healthy is list-of-lists
+                chunk_list.append([chunk_id, size])  # list-typed element as chunks_healthy is list-of-lists
                 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
@@ -1871,7 +1833,7 @@ class ArchiveChecker:
             item.chunks = chunk_list
             if 'size' in item:
                 item_size = item.size
-                item_chunks_size = item.get_size(compressed=False, from_chunks=True)
+                item_chunks_size = item.get_size(from_chunks=True)
                 if item_size != item_chunks_size:
                     # just warn, but keep the inconsistency, so that borg extract can warn about it.
                     logger.warning('{}: {}: size inconsistency detected: size {}, chunks size {}'.format(
@@ -2005,7 +1967,7 @@ class ArchiveChecker:
                 data = msgpack.packb(archive.as_dict())
                 new_archive_id = self.key.id_hash(data)
                 cdata = self.key.encrypt(new_archive_id, data)
-                add_reference(new_archive_id, len(data), len(cdata), cdata)
+                add_reference(new_archive_id, len(data), cdata)
                 self.manifest.archives[info.name] = (new_archive_id, info.ts)
             pi.finish()
 
@@ -2112,7 +2074,7 @@ class ArchiveRecreater:
 
     def process_chunks(self, archive, target, item):
         if not self.recompress and not target.recreate_rechunkify:
-            for chunk_id, size, csize in item.chunks:
+            for chunk_id, size in item.chunks:
                 self.cache.chunk_incref(chunk_id, target.stats)
             return item.chunks
         chunk_iterator = self.iter_chunks(archive, target, list(item.chunks))
@@ -2136,7 +2098,7 @@ class ArchiveRecreater:
         return chunk_entry
 
     def iter_chunks(self, archive, target, chunks):
-        chunk_iterator = archive.pipeline.fetch_many([chunk_id for chunk_id, _, _ in chunks])
+        chunk_iterator = archive.pipeline.fetch_many([chunk_id for chunk_id, _ in chunks])
         if target.recreate_rechunkify:
             # The target.chunker will read the file contents through ChunkIteratorFileWrapper chunk-by-chunk
             # (does not load the entire file into memory)

+ 7 - 7
src/borg/archiver.py

@@ -99,7 +99,7 @@ except BaseException:
 assert EXIT_ERROR == 2, "EXIT_ERROR is not 2, as expected - fix assert AND exception handler right above this line."
 
 
-STATS_HEADER = "                       Original size      Compressed size    Deduplicated size"
+STATS_HEADER = "                       Original size    Deduplicated size"
 
 PURE_PYTHON_MSGPACK_WARNING = "Using a pure-python msgpack! This will result in lower performance."
 
@@ -361,7 +361,7 @@ class Archiver:
                 chunks, chunks_healthy = hlm.retrieve(id=hlid, default=(None, None))
                 if chunks is not None:
                     item._dict['chunks'] = chunks
-                    for chunk_id, _, _ in chunks:
+                    for chunk_id, _ in chunks:
                         cache.chunk_incref(chunk_id, archive.stats)
                 if chunks_healthy is not None:
                     item._dict['chunks_healthy'] = chunks
@@ -424,7 +424,7 @@ class Archiver:
                 for item in other_archive.iter_items():
                     if 'chunks' in item:
                         chunks = []
-                        for chunk_id, size, _ in item.chunks:
+                        for chunk_id, size in item.chunks:
                             refcount = cache.seen_chunk(chunk_id, size)
                             if refcount == 0:  # target repo does not yet have this chunk
                                 if not dry_run:
@@ -443,7 +443,7 @@ class Archiver:
                                     chunks.append(chunk_entry)
                                 present_size += size
                         if not dry_run:
-                            item.chunks = chunks  # overwrite! IDs and sizes are same, csizes are likely different
+                            item.chunks = chunks  # TODO: overwrite? IDs and sizes are same.
                             archive.stats.nfiles += 1
                     if not dry_run:
                         archive.add_item(upgrade_item(item))
@@ -1331,7 +1331,7 @@ class Archiver:
             """
             Return a file-like object that reads from the chunks of *item*.
             """
-            chunk_iterator = archive.pipeline.fetch_many([chunk_id for chunk_id, _, _ in item.chunks],
+            chunk_iterator = archive.pipeline.fetch_many([chunk_id for chunk_id, _ in item.chunks],
                                                          is_preloaded=True)
             if pi:
                 info = [remove_surrogates(item.path)]
@@ -1797,8 +1797,8 @@ class Archiver:
                 Command line: {command_line}
                 Utilization of maximum supported archive size: {limits[max_archive_size]:.0%}
                 ------------------------------------------------------------------------------
-                                       Original size      Compressed size    Deduplicated size
-                This archive:   {stats[original_size]:>20s} {stats[compressed_size]:>20s} {stats[deduplicated_size]:>20s}
+                                       Original size    Deduplicated size
+                This archive:   {stats[original_size]:>20s} {stats[deduplicated_size]:>20s}
                 {cache}
                 """).strip().format(cache=cache, **info))
             if self.exit_code:

+ 33 - 95
src/borg/cache.py

@@ -406,51 +406,43 @@ class Cache:
 
 class CacheStatsMixin:
     str_format = """\
-All archives:   {0.total_size:>20s} {0.total_csize:>20s} {0.unique_csize:>20s}
+All archives:   {0.total_size:>20s} {0.unique_size:>20s}
 
                        Unique chunks         Total chunks
 Chunk index:    {0.total_unique_chunks:20d} {0.total_chunks:20d}"""
 
     def __init__(self, iec=False):
         self.iec = iec
-        self.pre12_meta = {}  # here we cache archive metadata for borg < 1.2
 
     def __str__(self):
         return self.str_format.format(self.format_tuple())
 
-    Summary = namedtuple('Summary', ['total_size', 'total_csize', 'unique_size', 'unique_csize', 'total_unique_chunks',
-                                     'total_chunks'])
+    Summary = namedtuple('Summary', ['total_size', 'unique_size', 'total_unique_chunks', 'total_chunks'])
 
     def stats(self):
         from .archive import Archive
         # XXX: this should really be moved down to `hashindex.pyx`
-        total_size, total_csize, unique_size, unique_csize, total_unique_chunks, total_chunks = self.chunks.summarize()
+        total_size, unique_size, total_unique_chunks, total_chunks = self.chunks.summarize()
         # the above values have the problem that they do not consider part files,
-        # thus the total_size and total_csize might be too high (chunks referenced
+        # thus the total_size might be too high (chunks referenced
         # by the part files AND by the complete file).
-        # since borg 1.2 we have new archive metadata telling the total size and
-        # csize per archive, so we can just sum up all archives to get the "all
-        # archives" stats:
-        total_size, total_csize = 0, 0
+        # since borg 1.2 we have new archive metadata telling the total size per archive,
+        # so we can just sum up all archives to get the "all archives" stats:
+        total_size = 0
         for archive_name in self.manifest.archives:
             archive = Archive(self.repository, self.key, self.manifest, archive_name,
                               consider_part_files=self.consider_part_files)
             stats = archive.calc_stats(self, want_unique=False)
             total_size += stats.osize
-            total_csize += stats.csize
-        stats = self.Summary(total_size, total_csize, unique_size, unique_csize,
-                             total_unique_chunks, total_chunks)._asdict()
+        stats = self.Summary(total_size, unique_size, total_unique_chunks, total_chunks)._asdict()
         return stats
 
     def format_tuple(self):
         stats = self.stats()
-        for field in ['total_size', 'total_csize', 'unique_csize']:
+        for field in ['total_size', 'unique_size']:
             stats[field] = format_file_size(stats[field], iec=self.iec)
         return self.Summary(**stats)
 
-    def chunks_stored_size(self):
-        return self.stats()['unique_csize']
-
 
 class LocalCache(CacheStatsMixin):
     """
@@ -518,8 +510,6 @@ class LocalCache(CacheStatsMixin):
         os.makedirs(os.path.join(self.path, 'chunks.archive.d'))
         with SaveFile(os.path.join(self.path, files_cache_name()), binary=True):
             pass  # empty file
-        with SaveFile(os.path.join(self.path, 'pre12-meta'), binary=False) as fd:
-            json.dump(self.pre12_meta, fd, indent=4)
 
     def _do_open(self):
         self.cache_config.load()
@@ -530,11 +520,6 @@ class LocalCache(CacheStatsMixin):
             self.files = None
         else:
             self._read_files()
-        try:
-            with open(os.path.join(self.path, 'pre12-meta')) as fd:
-                self.pre12_meta = json.load(fd)
-        except (FileNotFoundError, json.JSONDecodeError):
-            pass
 
     def open(self):
         if not os.path.isdir(self.path):
@@ -543,9 +528,6 @@ class LocalCache(CacheStatsMixin):
         self.rollback()
 
     def close(self):
-        # save the pre12_meta cache in any case
-        with open(os.path.join(self.path, 'pre12-meta'), 'w') as fd:
-            json.dump(self.pre12_meta, fd, indent=4)
         if self.cache_config is not None:
             self.cache_config.close()
             self.cache_config = None
@@ -679,8 +661,6 @@ class LocalCache(CacheStatsMixin):
         processed_item_metadata_bytes = 0
         processed_item_metadata_chunks = 0
         compact_chunks_archive_saved_space = 0
-        fetched_chunks_for_csize = 0
-        fetched_bytes_for_csize = 0
 
         def mkpath(id, suffix=''):
             id_hex = bin_to_hex(id)
@@ -718,55 +698,21 @@ class LocalCache(CacheStatsMixin):
             except FileNotFoundError:
                 pass
 
-        def fetch_missing_csize(chunk_idx):
-            """
-            Archives created with AdHocCache will have csize=0 in all chunk list entries whose
-            chunks were already in the repository.
-
-            Scan *chunk_idx* for entries where csize=0 and fill in the correct information.
-            """
-            nonlocal fetched_chunks_for_csize
-            nonlocal fetched_bytes_for_csize
-
-            all_missing_ids = chunk_idx.zero_csize_ids()
-            fetch_ids = []
-            if len(chunks_fetched_size_index):
-                for id_ in all_missing_ids:
-                    already_fetched_entry = chunks_fetched_size_index.get(id_)
-                    if already_fetched_entry:
-                        entry = chunk_idx[id_]._replace(csize=already_fetched_entry.csize)
-                        assert entry.size == already_fetched_entry.size, 'Chunk size mismatch'
-                        chunk_idx[id_] = entry
-                    else:
-                        fetch_ids.append(id_)
-            else:
-                fetch_ids = all_missing_ids
-
-            # This is potentially a rather expensive operation, but it's hard to tell at this point
-            # if it's a problem in practice (hence the experimental status of --no-cache-sync).
-            for id_, data in zip(fetch_ids, decrypted_repository.repository.get_many(fetch_ids)):
-                entry = chunk_idx[id_]._replace(csize=len(data))
-                chunk_idx[id_] = entry
-                chunks_fetched_size_index[id_] = entry
-                fetched_chunks_for_csize += 1
-                fetched_bytes_for_csize += len(data)
-
         def fetch_and_build_idx(archive_id, decrypted_repository, chunk_idx):
             nonlocal processed_item_metadata_bytes
             nonlocal processed_item_metadata_chunks
             csize, data = decrypted_repository.get(archive_id)
-            chunk_idx.add(archive_id, 1, len(data), csize)
+            chunk_idx.add(archive_id, 1, len(data))
             archive = ArchiveItem(internal_dict=msgpack.unpackb(data))
             if archive.version not in (1, 2):  # legacy
                 raise Exception('Unknown archive metadata version')
             sync = CacheSynchronizer(chunk_idx)
             for item_id, (csize, data) in zip(archive.items, decrypted_repository.get_many(archive.items)):
-                chunk_idx.add(item_id, 1, len(data), csize)
+                chunk_idx.add(item_id, 1, len(data))
                 processed_item_metadata_bytes += len(data)
                 processed_item_metadata_chunks += 1
                 sync.feed(data)
             if self.do_cache:
-                fetch_missing_csize(chunk_idx)
                 write_archive_index(archive_id, chunk_idx)
 
         def write_archive_index(archive_id, chunk_idx):
@@ -862,12 +808,7 @@ class LocalCache(CacheStatsMixin):
                         chunk_idx = chunk_idx or ChunkIndex(usable=master_index_capacity)
                         logger.info('Fetching archive index for %s ...', archive_name)
                         fetch_and_build_idx(archive_id, decrypted_repository, chunk_idx)
-                if not self.do_cache:
-                    fetch_missing_csize(chunk_idx)
                 pi.finish()
-                logger.debug('Cache sync: had to fetch %s (%d chunks) because no archive had a csize set for them '
-                             '(due to --no-cache-sync)',
-                             format_file_size(fetched_bytes_for_csize), fetched_chunks_for_csize)
                 logger.debug('Cache sync: processed %s (%d chunks) of metadata',
                              format_file_size(processed_item_metadata_bytes), processed_item_metadata_chunks)
                 logger.debug('Cache sync: compact chunks.archive.d storage saved %s bytes',
@@ -951,14 +892,13 @@ class LocalCache(CacheStatsMixin):
         if size is None:
             raise ValueError("when giving compressed data for a new chunk, the uncompressed size must be given also")
         data = self.key.encrypt(id, chunk, compress=compress)
-        csize = len(data)
         self.repository.put(id, data, wait=wait)
-        self.chunks.add(id, 1, size, csize)
-        stats.update(size, csize, not refcount)
-        return ChunkListEntry(id, size, csize)
+        self.chunks.add(id, 1, size)
+        stats.update(size, not refcount)
+        return ChunkListEntry(id, size)
 
     def seen_chunk(self, id, size=None):
-        refcount, stored_size, _ = self.chunks.get(id, ChunkIndexEntry(0, None, None))
+        refcount, stored_size = self.chunks.get(id, ChunkIndexEntry(0, None))
         if size is not None and stored_size is not None and size != stored_size:
             # we already have a chunk with that id, but different size.
             # this is either a hash collision (unlikely) or corruption or a bug.
@@ -969,20 +909,20 @@ class LocalCache(CacheStatsMixin):
     def chunk_incref(self, id, stats, size=None, part=False):
         if not self.txn_active:
             self.begin_txn()
-        count, _size, csize = self.chunks.incref(id)
-        stats.update(_size, csize, False, part=part)
-        return ChunkListEntry(id, _size, csize)
+        count, _size = self.chunks.incref(id)
+        stats.update(_size, False, part=part)
+        return ChunkListEntry(id, _size)
 
     def chunk_decref(self, id, stats, wait=True, part=False):
         if not self.txn_active:
             self.begin_txn()
-        count, size, csize = self.chunks.decref(id)
+        count, size = self.chunks.decref(id)
         if count == 0:
             del self.chunks[id]
             self.repository.delete(id, wait=wait)
-            stats.update(-size, -csize, True, part=part)
+            stats.update(-size, True, part=part)
         else:
-            stats.update(-size, -csize, False, part=part)
+            stats.update(-size, False, part=part)
 
     def file_known_and_unchanged(self, hashed_path, path_hash, st):
         """
@@ -1064,7 +1004,7 @@ class AdHocCache(CacheStatsMixin):
 
     Compared to the standard LocalCache the AdHocCache does not maintain accurate reference count,
     nor does it provide a files cache (which would require persistence). Chunks that were not added
-    during the current AdHocCache lifetime won't have correct size/csize set (0 bytes) and will
+    during the current AdHocCache lifetime won't have correct size set (0 bytes) and will
     have an infinite reference count (MAX_VALUE).
     """
 
@@ -1086,7 +1026,6 @@ Chunk index:    {0.total_unique_chunks:20d}             unknown"""
         self.security_manager = SecurityManager(repository)
         self.security_manager.assert_secure(manifest, key, lock_wait=lock_wait)
 
-        self.pre12_meta = {}
         logger.warning('Note: --no-cache-sync is an experimental feature.')
 
     # Public API
@@ -1119,16 +1058,15 @@ Chunk index:    {0.total_unique_chunks:20d}             unknown"""
         if refcount:
             return self.chunk_incref(id, stats, size=size)
         data = self.key.encrypt(id, chunk, compress=compress)
-        csize = len(data)
         self.repository.put(id, data, wait=wait)
-        self.chunks.add(id, 1, size, csize)
-        stats.update(size, csize, not refcount)
-        return ChunkListEntry(id, size, csize)
+        self.chunks.add(id, 1, size)
+        stats.update(size, not refcount)
+        return ChunkListEntry(id, size)
 
     def seen_chunk(self, id, size=None):
         if not self._txn_active:
             self.begin_txn()
-        entry = self.chunks.get(id, ChunkIndexEntry(0, None, None))
+        entry = self.chunks.get(id, ChunkIndexEntry(0, None))
         if entry.refcount and size and not entry.size:
             # The LocalCache has existing size information and uses *size* to make an effort at detecting collisions.
             # This is of course not possible for the AdHocCache.
@@ -1139,24 +1077,24 @@ Chunk index:    {0.total_unique_chunks:20d}             unknown"""
     def chunk_incref(self, id, stats, size=None, part=False):
         if not self._txn_active:
             self.begin_txn()
-        count, _size, csize = self.chunks.incref(id)
+        count, _size = self.chunks.incref(id)
         # When _size is 0 and size is not given, then this chunk has not been locally visited yet (seen_chunk with
         # size or add_chunk); we can't add references to those (size=0 is invalid) and generally don't try to.
         size = _size or size
         assert size
-        stats.update(size, csize, False, part=part)
-        return ChunkListEntry(id, size, csize)
+        stats.update(size, False, part=part)
+        return ChunkListEntry(id, size)
 
     def chunk_decref(self, id, stats, wait=True, part=False):
         if not self._txn_active:
             self.begin_txn()
-        count, size, csize = self.chunks.decref(id)
+        count, size = self.chunks.decref(id)
         if count == 0:
             del self.chunks[id]
             self.repository.delete(id, wait=wait)
-            stats.update(-size, -csize, True, part=part)
+            stats.update(-size, True, part=part)
         else:
-            stats.update(-size, -csize, False, part=part)
+            stats.update(-size, False, part=part)
 
     def commit(self):
         if not self._txn_active:
@@ -1190,7 +1128,7 @@ Chunk index:    {0.total_unique_chunks:20d}             unknown"""
             # All chunks from the repository have a refcount of MAX_VALUE, which is sticky,
             # therefore we can't/won't delete them. Chunks we added ourselves in this transaction
             # (e.g. checkpoint archives) are tracked correctly.
-            init_entry = ChunkIndexEntry(refcount=ChunkIndex.MAX_VALUE, size=0, csize=0)
+            init_entry = ChunkIndexEntry(refcount=ChunkIndex.MAX_VALUE, size=0)
             for id_ in result:
                 self.chunks[id_] = init_entry
         assert len(self.chunks) == num_chunks

+ 0 - 14
src/borg/cache_sync/cache_sync.c

@@ -39,10 +39,8 @@ cache_sync_init(HashIndex *chunks)
     /* needs to be set only once */
     ctx->ctx.user.chunks = chunks;
     ctx->ctx.user.parts.size = 0;
-    ctx->ctx.user.parts.csize = 0;
     ctx->ctx.user.parts.num_files = 0;
     ctx->ctx.user.totals.size = 0;
-    ctx->ctx.user.totals.csize = 0;
     ctx->ctx.user.totals.num_files = 0;
     ctx->buf = NULL;
     ctx->head = 0;
@@ -91,18 +89,6 @@ cache_sync_size_parts(const CacheSyncCtx *ctx)
     return ctx->ctx.user.parts.size;
 }
 
-static uint64_t
-cache_sync_csize_totals(const CacheSyncCtx *ctx)
-{
-    return ctx->ctx.user.totals.csize;
-}
-
-static uint64_t
-cache_sync_csize_parts(const CacheSyncCtx *ctx)
-{
-    return ctx->ctx.user.parts.csize;
-}
-
 /**
  * feed data to the cache synchronizer
  * 0 = abort, 1 = continue

+ 6 - 19
src/borg/cache_sync/unpack.h

@@ -86,14 +86,12 @@ typedef struct unpack_user {
 
         /*
          * processing ChunkListEntry tuple:
-         * expect_key, expect_size, expect_csize, expect_entry_end
+         * expect_key, expect_size, expect_entry_end
          */
         /* next thing must be the key (raw, l=32) */
         expect_key,
         /* next thing must be the size (int) */
         expect_size,
-        /* next thing must be the csize (int) */
-        expect_csize,
         /* next thing must be the end of the CLE (array_end) */
         expect_entry_end,
 
@@ -103,23 +101,22 @@ typedef struct unpack_user {
     /* collect values here for current chunklist entry */
     struct {
         unsigned char key[32];
-        uint32_t csize;
         uint32_t size;
     } current;
 
     /* summing up chunks sizes here within a single item */
     struct {
-        uint64_t size, csize;
+        uint64_t size;
     } item;
 
     /* total sizes and files count coming from all files */
     struct {
-        uint64_t size, csize, num_files;
+        uint64_t size, num_files;
     } totals;
 
     /* total sizes and files count coming from part files */
     struct {
-        uint64_t size, csize, num_files;
+        uint64_t size, num_files;
     } parts;
 
 } unpack_user;
@@ -147,10 +144,6 @@ static inline int unpack_callback_uint64(unpack_user* u, int64_t d)
     switch(u->expect) {
         case expect_size:
             u->current.size = d;
-            u->expect = expect_csize;
-            break;
-        case expect_csize:
-            u->current.csize = d;
             u->expect = expect_entry_end;
             break;
         default:
@@ -239,7 +232,7 @@ static inline int unpack_callback_array(unpack_user* u, unsigned int n)
     case expect_entry_begin_or_chunks_end:
         /* b'chunks': [ (
          *              ^ */
-        if(n != 3) {
+        if(n != 2) {
             SET_LAST_ERROR("Invalid chunk list entry length");
             return -1;
         }
@@ -283,18 +276,15 @@ static inline int unpack_callback_array_end(unpack_user* u)
             refcount += 1;
             cache_entry[0] = _htole32(MIN(refcount, _MAX_VALUE));
         } else {
-            /* refcount, size, csize */
+            /* refcount, size */
             cache_values[0] = _htole32(1);
             cache_values[1] = _htole32(u->current.size);
-            cache_values[2] = _htole32(u->current.csize);
             if(!hashindex_set(u->chunks, u->current.key, cache_values)) {
                 SET_LAST_ERROR("hashindex_set failed");
                 return -1;
             }
         }
         u->item.size += u->current.size;
-        u->item.csize += u->current.csize;
-
         u->expect = expect_entry_begin_or_chunks_end;
         break;
     case expect_entry_begin_or_chunks_end:
@@ -330,7 +320,6 @@ static inline int unpack_callback_map(unpack_user* u, unsigned int n)
         u->part = 0;
         u->has_chunks = 0;
         u->item.size = 0;
-        u->item.csize = 0;
     }
 
     if(u->inside_chunks) {
@@ -372,11 +361,9 @@ static inline int unpack_callback_map_end(unpack_user* u)
             if(u->part) {
                 u->parts.num_files += 1;
                 u->parts.size += u->item.size;
-                u->parts.csize += u->item.csize;
             }
             u->totals.num_files += 1;
             u->totals.size += u->item.size;
-            u->totals.csize += u->item.csize;
         }
     }
     return 0;

+ 1 - 1
src/borg/constants.py

@@ -12,7 +12,7 @@ ARCHIVE_KEYS = frozenset(['version', 'name', 'items', 'cmdline', 'hostname', 'us
                           'comment', 'chunker_params',
                           'recreate_cmdline',
                           'recreate_source_id', 'recreate_args', 'recreate_partial_chunks',  # used in 1.1.0b1 .. b2
-                          'size', 'csize', 'nfiles', 'size_parts', 'csize_parts', 'nfiles_parts', ])
+                          'size', 'nfiles', 'size_parts', 'nfiles_parts', ])
 
 # this is the set of keys that are always present in archives:
 REQUIRED_ARCHIVE_KEYS = frozenset(['version', 'name', 'items', 'cmdline', 'time', ])

+ 2 - 2
src/borg/fuse.py

@@ -377,7 +377,7 @@ class FuseBackend:
                 file_id = blake2b_128(path)
                 current_version, previous_id = self.versions_index.get(file_id, (0, None))
 
-                contents_id = blake2b_128(b''.join(chunk_id for chunk_id, _, _ in item.chunks))
+                contents_id = blake2b_128(b''.join(chunk_id for chunk_id, _ in item.chunks))
 
                 if contents_id != previous_id:
                     current_version += 1
@@ -658,7 +658,7 @@ class FuseOperations(llfuse.Operations, FuseBackend):
         chunks = item.chunks
         # note: using index iteration to avoid frequently copying big (sub)lists by slicing
         for idx in range(chunk_no, len(chunks)):
-            id, s, csize = chunks[idx]
+            id, s = chunks[idx]
             if s < offset:
                 offset -= s
                 chunk_offset += s

+ 18 - 52
src/borg/hashindex.pyx

@@ -49,8 +49,6 @@ cdef extern from "cache_sync/cache_sync.c":
     uint64_t cache_sync_num_files_parts(const CacheSyncCtx *ctx)
     uint64_t cache_sync_size_totals(const CacheSyncCtx *ctx)
     uint64_t cache_sync_size_parts(const CacheSyncCtx *ctx)
-    uint64_t cache_sync_csize_totals(const CacheSyncCtx *ctx)
-    uint64_t cache_sync_csize_parts(const CacheSyncCtx *ctx)
     int cache_sync_feed(CacheSyncCtx *ctx, void *data, uint32_t length)
     void cache_sync_free(CacheSyncCtx *ctx)
 
@@ -271,12 +269,12 @@ cdef class NSKeyIterator:
         return (<char *>self.key)[:self.key_size], (segment, _le32toh(value[1]))
 
 
-ChunkIndexEntry = namedtuple('ChunkIndexEntry', 'refcount size csize')
+ChunkIndexEntry = namedtuple('ChunkIndexEntry', 'refcount size')
 
 
 cdef class ChunkIndex(IndexBase):
     """
-    Mapping of 32 byte keys to (refcount, size, csize), which are all 32-bit unsigned.
+    Mapping of 32 byte keys to (refcount, size), which are all 32-bit unsigned.
 
     The reference count cannot overflow. If an overflow would occur, the refcount
     is fixed to MAX_VALUE and will neither increase nor decrease by incref(), decref()
@@ -291,7 +289,7 @@ cdef class ChunkIndex(IndexBase):
     Assigning refcounts in this reserved range is an invalid operation and raises AssertionError.
     """
 
-    value_size = 12
+    value_size = 8
 
     def __getitem__(self, key):
         assert len(key) == self.key_size
@@ -300,16 +298,15 @@ cdef class ChunkIndex(IndexBase):
             raise KeyError(key)
         cdef uint32_t refcount = _le32toh(data[0])
         assert refcount <= _MAX_VALUE, "invalid reference count"
-        return ChunkIndexEntry(refcount, _le32toh(data[1]), _le32toh(data[2]))
+        return ChunkIndexEntry(refcount, _le32toh(data[1]))
 
     def __setitem__(self, key, value):
         assert len(key) == self.key_size
-        cdef uint32_t[3] data
+        cdef uint32_t[2] data
         cdef uint32_t refcount = value[0]
         assert refcount <= _MAX_VALUE, "invalid reference count"
         data[0] = _htole32(refcount)
         data[1] = _htole32(value[1])
-        data[2] = _htole32(value[2])
         if not hashindex_set(self.index, <unsigned char *>key, data):
             raise Exception('hashindex_set failed')
 
@@ -321,7 +318,7 @@ cdef class ChunkIndex(IndexBase):
         return data != NULL
 
     def incref(self, key):
-        """Increase refcount for 'key', return (refcount, size, csize)"""
+        """Increase refcount for 'key', return (refcount, size)"""
         assert len(key) == self.key_size
         data = <uint32_t *>hashindex_get(self.index, <unsigned char *>key)
         if not data:
@@ -331,10 +328,10 @@ cdef class ChunkIndex(IndexBase):
         if refcount != _MAX_VALUE:
             refcount += 1
         data[0] = _htole32(refcount)
-        return refcount, _le32toh(data[1]), _le32toh(data[2])
+        return refcount, _le32toh(data[1])
 
     def decref(self, key):
-        """Decrease refcount for 'key', return (refcount, size, csize)"""
+        """Decrease refcount for 'key', return (refcount, size)"""
         assert len(key) == self.key_size
         data = <uint32_t *>hashindex_get(self.index, <unsigned char *>key)
         if not data:
@@ -345,7 +342,7 @@ cdef class ChunkIndex(IndexBase):
         if refcount != _MAX_VALUE:
             refcount -= 1
         data[0] = _htole32(refcount)
-        return refcount, _le32toh(data[1]), _le32toh(data[2])
+        return refcount, _le32toh(data[1])
 
     def iteritems(self, marker=None):
         cdef const unsigned char *key
@@ -360,7 +357,7 @@ cdef class ChunkIndex(IndexBase):
         return iter
 
     def summarize(self):
-        cdef uint64_t size = 0, csize = 0, unique_size = 0, unique_csize = 0, chunks = 0, unique_chunks = 0
+        cdef uint64_t size = 0, unique_size = 0, chunks = 0, unique_chunks = 0
         cdef uint32_t *values
         cdef uint32_t refcount
         cdef unsigned char *key = NULL
@@ -375,11 +372,9 @@ cdef class ChunkIndex(IndexBase):
             assert refcount <= _MAX_VALUE, "invalid reference count"
             chunks += refcount
             unique_size += _le32toh(values[1])
-            unique_csize += _le32toh(values[2])
             size += <uint64_t> _le32toh(values[1]) * _le32toh(values[0])
-            csize += <uint64_t> _le32toh(values[2]) * _le32toh(values[0])
 
-        return size, csize, unique_size, unique_csize, unique_chunks, chunks
+        return size, unique_size, unique_chunks, chunks
 
     def stats_against(self, ChunkIndex master_index):
         """
@@ -391,10 +386,10 @@ cdef class ChunkIndex(IndexBase):
         This index must be a subset of *master_index*.
 
         Return the same statistics tuple as summarize:
-        size, csize, unique_size, unique_csize, unique_chunks, chunks.
+        size, unique_size, unique_chunks, chunks.
         """
-        cdef uint64_t size = 0, csize = 0, unique_size = 0, unique_csize = 0, chunks = 0, unique_chunks = 0
-        cdef uint32_t our_refcount, chunk_size, chunk_csize
+        cdef uint64_t size = 0, unique_size = 0, chunks = 0, unique_chunks = 0
+        cdef uint32_t our_refcount, chunk_size
         cdef const uint32_t *our_values
         cdef const uint32_t *master_values
         cdef const unsigned char *key = NULL
@@ -410,25 +405,21 @@ cdef class ChunkIndex(IndexBase):
                 raise ValueError('stats_against: key contained in self but not in master_index.')
             our_refcount = _le32toh(our_values[0])
             chunk_size = _le32toh(master_values[1])
-            chunk_csize = _le32toh(master_values[2])
 
             chunks += our_refcount
             size += <uint64_t> chunk_size * our_refcount
-            csize += <uint64_t> chunk_csize * our_refcount
             if our_values[0] == master_values[0]:
                 # our refcount equals the master's refcount, so this chunk is unique to us
                 unique_chunks += 1
                 unique_size += chunk_size
-                unique_csize += chunk_csize
 
-        return size, csize, unique_size, unique_csize, unique_chunks, chunks
+        return size, unique_size, unique_chunks, chunks
 
-    def add(self, key, refs, size, csize):
+    def add(self, key, refs, size):
         assert len(key) == self.key_size
-        cdef uint32_t[3] data
+        cdef uint32_t[2] data
         data[0] = _htole32(refs)
         data[1] = _htole32(size)
-        data[2] = _htole32(csize)
         self._add(<unsigned char*> key, data)
 
     cdef _add(self, unsigned char *key, uint32_t *data):
@@ -442,7 +433,6 @@ cdef class ChunkIndex(IndexBase):
             result64 = refcount1 + refcount2
             values[0] = _htole32(min(result64, _MAX_VALUE))
             values[1] = data[1]
-            values[2] = data[2]
         else:
             if not hashindex_set(self.index, key, data):
                 raise Exception('hashindex_set failed')
@@ -456,22 +446,6 @@ cdef class ChunkIndex(IndexBase):
                 break
             self._add(key, <uint32_t*> (key + self.key_size))
 
-    def zero_csize_ids(self):
-        cdef unsigned char *key = NULL
-        cdef uint32_t *values
-        entries = []
-        while True:
-            key = hashindex_next_key(self.index, key)
-            if not key:
-                break
-            values = <uint32_t*> (key + self.key_size)
-            refcount = _le32toh(values[0])
-            assert refcount <= _MAX_VALUE, "invalid reference count"
-            if _le32toh(values[2]) == 0:
-                # csize == 0
-                entries.append(PyBytes_FromStringAndSize(<char*> key, self.key_size))
-        return entries
-
 
 cdef class ChunkKeyIterator:
     cdef ChunkIndex idx
@@ -498,7 +472,7 @@ cdef class ChunkKeyIterator:
         cdef uint32_t *value = <uint32_t *>(self.key + self.key_size)
         cdef uint32_t refcount = _le32toh(value[0])
         assert refcount <= _MAX_VALUE, "invalid reference count"
-        return (<char *>self.key)[:self.key_size], ChunkIndexEntry(refcount, _le32toh(value[1]), _le32toh(value[2]))
+        return (<char *>self.key)[:self.key_size], ChunkIndexEntry(refcount, _le32toh(value[1]))
 
 
 cdef Py_buffer ro_buffer(object data) except *:
@@ -546,11 +520,3 @@ cdef class CacheSynchronizer:
     @property
     def size_parts(self):
         return cache_sync_size_parts(self.sync)
-
-    @property
-    def csize_totals(self):
-        return cache_sync_csize_totals(self.sync)
-
-    @property
-    def csize_parts(self):
-        return cache_sync_csize_parts(self.sync)

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

@@ -698,9 +698,7 @@ class ItemFormatter(BaseFormatter):
         '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',
-        'dcsize': 'deduplicated compressed size',
         'num_chunks': 'number of chunks in this file',
         'unique_chunks': 'number of unique chunks in this file',
         'xxh64': 'XXH64 checksum of this file (note: this is NOT a cryptographic hash!)',
@@ -708,7 +706,7 @@ class ItemFormatter(BaseFormatter):
     }
     KEY_GROUPS = (
         ('type', 'mode', 'uid', 'gid', 'user', 'group', 'path', 'bpath', 'source', 'linktarget', 'hlid', 'flags'),
-        ('size', 'csize', 'dsize', 'dcsize', 'num_chunks', 'unique_chunks'),
+        ('size', 'dsize', 'num_chunks', 'unique_chunks'),
         ('mtime', 'ctime', 'atime', 'isomtime', 'isoctime', 'isoatime'),
         tuple(sorted(hash_algorithms)),
         ('archiveid', 'archivename', 'extra'),
@@ -716,7 +714,7 @@ class ItemFormatter(BaseFormatter):
     )
 
     KEYS_REQUIRING_CACHE = (
-        'dsize', 'dcsize', 'unique_chunks',
+        'dsize', 'unique_chunks',
     )
 
     @classmethod
@@ -774,9 +772,7 @@ class ItemFormatter(BaseFormatter):
         self.format_keys = {f[1] for f in Formatter().parse(format)}
         self.call_keys = {
             'size': self.calculate_size,
-            'csize': self.calculate_csize,
             'dsize': partial(self.sum_unique_chunks_metadata, lambda chunk: chunk.size),
-            'dcsize': partial(self.sum_unique_chunks_metadata, lambda chunk: chunk.csize),
             'num_chunks': self.calculate_num_chunks,
             'unique_chunks': partial(self.sum_unique_chunks_metadata, lambda chunk: 1),
             'isomtime': partial(self.format_iso_time, 'mtime'),
@@ -846,11 +842,7 @@ class ItemFormatter(BaseFormatter):
 
     def calculate_size(self, item):
         # note: does not support hardlink slaves, they will be size 0
-        return item.get_size(compressed=False)
-
-    def calculate_csize(self, item):
-        # note: does not support hardlink slaves, they will be csize 0
-        return item.get_size(compressed=True)
+        return item.get_size()
 
     def hash_item(self, hash_function, item):
         if 'chunks' not in item:

+ 7 - 11
src/borg/item.pyx

@@ -61,10 +61,10 @@ def fix_list_of_chunkentries(v):
     chunks = []
     for ce in v:
         assert isinstance(ce, (tuple, list))
-        assert len(ce) == 3  # id, size, csize
+        assert len(ce) in (2, 3)  # id, size[, csize]
         assert isinstance(ce[1], int)
-        assert isinstance(ce[2], int)
-        ce_fixed = [want_bytes(ce[0]), ce[1], ce[2]]  # list!
+        assert len(ce) == 2 or isinstance(ce[2], int)
+        ce_fixed = [want_bytes(ce[0]), ce[1]]  # list! id, size only, drop csize
         chunks.append(ce_fixed)  # create a list of lists
     return chunks
 
@@ -227,7 +227,7 @@ class PropDict:
         return property(_get, _set, _del, doc=doc)
 
 
-ChunkListEntry = namedtuple('ChunkListEntry', 'id size csize')
+ChunkListEntry = namedtuple('ChunkListEntry', 'id size')
 
 class Item(PropDict):
     """
@@ -284,17 +284,15 @@ class Item(PropDict):
 
     part = PropDict._make_property('part', int)
 
-    def get_size(self, memorize=False, compressed=False, from_chunks=False, consider_ids=None):
+    def get_size(self, *, memorize=False, from_chunks=False, consider_ids=None):
         """
-        Determine the (uncompressed or compressed) size of this item.
+        Determine the uncompressed size of this item.
 
         :param memorize: Whether the computed size value will be stored into the item.
-        :param compressed: Whether the compressed or uncompressed size will be returned.
         :param from_chunks: If true, size is computed from chunks even if a precomputed value is available.
         :param consider_ids: Returns the size of the given ids only.
         """
-        attr = 'csize' if compressed else 'size'
-        assert not (compressed and memorize), 'Item does not have a csize field.'
+        attr = 'size'
         assert not (consider_ids is not None and memorize), "Can't store size when considering only certain ids"
         try:
             if from_chunks or consider_ids is not None:
@@ -496,10 +494,8 @@ class ArchiveItem(PropDict):
     recreate_args = PropDict._make_property('recreate_args', list)  # list of s-e-str
     recreate_partial_chunks = PropDict._make_property('recreate_partial_chunks', list)  # list of tuples
     size = PropDict._make_property('size', int)
-    csize = PropDict._make_property('csize', int)
     nfiles = PropDict._make_property('nfiles', int)
     size_parts = PropDict._make_property('size_parts', int)
-    csize_parts = PropDict._make_property('csize_parts', int)
     nfiles_parts = PropDict._make_property('nfiles_parts', int)
 
     def update_internal(self, d):

+ 11 - 16
src/borg/testsuite/archive.py

@@ -19,47 +19,46 @@ from ..platform import uid2user, gid2group
 @pytest.fixture()
 def stats():
     stats = Statistics()
-    stats.update(20, 10, unique=True)
+    stats.update(20, unique=True)
     return stats
 
 
 def test_stats_basic(stats):
     assert stats.osize == 20
-    assert stats.csize == stats.usize == 10
-    stats.update(20, 10, unique=False)
+    assert stats.usize == 20
+    stats.update(20, unique=False)
     assert stats.osize == 40
-    assert stats.csize == 20
-    assert stats.usize == 10
+    assert stats.usize == 20
 
 
 def tests_stats_progress(stats, monkeypatch, columns=80):
     monkeypatch.setenv('COLUMNS', str(columns))
     out = StringIO()
     stats.show_progress(stream=out)
-    s = '20 B O 10 B C 10 B D 0 N '
+    s = '20 B O 20 B U 0 N '
     buf = ' ' * (columns - len(s))
     assert out.getvalue() == s + buf + "\r"
 
     out = StringIO()
-    stats.update(10**3, 0, unique=False)
+    stats.update(10 ** 3, unique=False)
     stats.show_progress(item=Item(path='foo'), final=False, stream=out)
-    s = '1.02 kB O 10 B C 10 B D 0 N foo'
+    s = '1.02 kB O 20 B U 0 N foo'
     buf = ' ' * (columns - len(s))
     assert out.getvalue() == s + buf + "\r"
     out = StringIO()
     stats.show_progress(item=Item(path='foo'*40), final=False, stream=out)
-    s = '1.02 kB O 10 B C 10 B D 0 N foofoofoofoofoofoofoofo...oofoofoofoofoofoofoofoofoo'
+    s = '1.02 kB O 20 B U 0 N foofoofoofoofoofoofoofoofo...foofoofoofoofoofoofoofoofoofoo'
     buf = ' ' * (columns - len(s))
     assert out.getvalue() == s + buf + "\r"
 
 
 def test_stats_format(stats):
     assert str(stats) == """\
-This archive:                   20 B                 10 B                 10 B"""
+This archive:                   20 B                 20 B"""
     s = f"{stats.osize_fmt}"
     assert s == "20 B"
     # kind of redundant, but id is variable so we can't match reliably
-    assert repr(stats) == f'<Statistics object at {id(stats):#x} (20, 10, 10)>'
+    assert repr(stats) == f'<Statistics object at {id(stats):#x} (20, 20)>'
 
 
 def test_stats_progress_json(stats):
@@ -73,8 +72,6 @@ def test_stats_progress_json(stats):
     assert result['finished'] is False
     assert result['path'] == 'foo'
     assert result['original_size'] == 20
-    assert result['compressed_size'] == 10
-    assert result['deduplicated_size'] == 10
     assert result['nfiles'] == 0  # this counter gets updated elsewhere
 
     out = StringIO()
@@ -85,8 +82,6 @@ def test_stats_progress_json(stats):
     assert result['finished'] is True  # see #6570
     assert 'path' not in result
     assert 'original_size' not in result
-    assert 'compressed_size' not in result
-    assert 'deduplicated_size' not in result
     assert 'nfiles' not in result
 
 
@@ -102,7 +97,7 @@ class MockCache:
 
     def add_chunk(self, id, chunk, stats=None, wait=True):
         self.objects[id] = chunk
-        return id, len(chunk), len(chunk)
+        return id, len(chunk)
 
 
 class ArchiveTimestampTestCase(BaseTestCase):

+ 11 - 80
src/borg/testsuite/archiver.py

@@ -1543,7 +1543,7 @@ class ArchiverTestCase(ArchiverTestCaseBase):
         cache = info_repo['cache']
         stats = cache['stats']
         assert all(isinstance(o, int) for o in stats.values())
-        assert all(key in stats for key in ('total_chunks', 'total_csize', 'total_size', 'total_unique_chunks', 'unique_csize', 'unique_size'))
+        assert all(key in stats for key in ('total_chunks', 'total_size', 'total_unique_chunks', 'unique_size'))
 
         info_archive = json.loads(self.cmd('info', '--json', self.repository_location + '::test'))
         assert info_repo['repository'] == info_archive['repository']
@@ -2363,12 +2363,9 @@ class ArchiverTestCase(ArchiverTestCaseBase):
         self.cmd('init', '--encryption=repokey', self.repository_location)
         test_archive = self.repository_location + '::test'
         self.cmd('create', '-C', 'lz4', test_archive, 'input')
-        output = self.cmd('list', '--format', '{size} {csize} {dsize} {dcsize} {path}{NL}', test_archive)
-        size, csize, dsize, dcsize, path = output.split("\n")[1].split(" ")
-        assert int(csize) < int(size)
-        assert int(dcsize) < int(dsize)
-        assert int(dsize) <= int(size)
-        assert int(dcsize) <= int(csize)
+        output = self.cmd('list', '--format', '{size} {path}{NL}', test_archive)
+        size, path = output.split("\n")[1].split(" ")
+        assert int(size) == 10000
 
     def test_list_json(self):
         self.create_regular_file('file1', size=1024 * 80)
@@ -2441,69 +2438,6 @@ class ArchiverTestCase(ArchiverTestCaseBase):
         log = self.cmd('--debug', 'create', self.repository_location + '::test', 'input')
         assert 'security: read previous location' in log
 
-    def _get_sizes(self, compression, compressible, size=10000):
-        if compressible:
-            contents = b'X' * size
-        else:
-            contents = os.urandom(size)
-        self.create_regular_file('file', contents=contents)
-        self.cmd('init', '--encryption=none', self.repository_location)
-        archive = self.repository_location + '::test'
-        self.cmd('create', '-C', compression, archive, 'input')
-        output = self.cmd('list', '--format', '{size} {csize} {path}{NL}', archive)
-        size, csize, path = output.split("\n")[1].split(" ")
-        return int(size), int(csize)
-
-    def test_compression_none_compressible(self):
-        size, csize = self._get_sizes('none', compressible=True)
-        assert csize == size + 3
-
-    def test_compression_none_uncompressible(self):
-        size, csize = self._get_sizes('none', compressible=False)
-        assert csize == size + 3
-
-    def test_compression_zlib_compressible(self):
-        size, csize = self._get_sizes('zlib', compressible=True)
-        assert csize < size * 0.1
-        assert csize == 37
-
-    def test_compression_zlib_uncompressible(self):
-        size, csize = self._get_sizes('zlib', compressible=False)
-        assert csize >= size
-
-    def test_compression_auto_compressible(self):
-        size, csize = self._get_sizes('auto,zlib', compressible=True)
-        assert csize < size * 0.1
-        assert csize == 37  # same as compression 'zlib'
-
-    def test_compression_auto_uncompressible(self):
-        size, csize = self._get_sizes('auto,zlib', compressible=False)
-        assert csize == size + 3  # same as compression 'none'
-
-    def test_compression_lz4_compressible(self):
-        size, csize = self._get_sizes('lz4', compressible=True)
-        assert csize < size * 0.1
-
-    def test_compression_lz4_uncompressible(self):
-        size, csize = self._get_sizes('lz4', compressible=False)
-        assert csize == size + 3  # same as compression 'none'
-
-    def test_compression_lzma_compressible(self):
-        size, csize = self._get_sizes('lzma', compressible=True)
-        assert csize < size * 0.1
-
-    def test_compression_lzma_uncompressible(self):
-        size, csize = self._get_sizes('lzma', compressible=False)
-        assert csize == size + 3  # same as compression 'none'
-
-    def test_compression_zstd_compressible(self):
-        size, csize = self._get_sizes('zstd', compressible=True)
-        assert csize < size * 0.1
-
-    def test_compression_zstd_uncompressible(self):
-        size, csize = self._get_sizes('zstd', compressible=False)
-        assert csize == size + 3  # same as compression 'none'
-
     def test_change_passphrase(self):
         self.cmd('init', '--encryption=repokey', self.repository_location)
         os.environ['BORG_NEW_PASSPHRASE'] = 'newpassphrase'
@@ -2951,13 +2885,12 @@ class ArchiverTestCase(ArchiverTestCaseBase):
                 correct_chunks = cache.chunks
         assert original_chunks is not correct_chunks
         seen = set()
-        for id, (refcount, size, csize) in correct_chunks.iteritems():
-            o_refcount, o_size, o_csize = original_chunks[id]
+        for id, (refcount, size) in correct_chunks.iteritems():
+            o_refcount, o_size = original_chunks[id]
             assert refcount == o_refcount
             assert size == o_size
-            assert csize == o_csize
             seen.add(id)
-        for id, (refcount, size, csize) in original_chunks.iteritems():
+        for id, (refcount, size) in original_chunks.iteritems():
             assert id in seen
 
     def test_check_cache(self):
@@ -3051,15 +2984,13 @@ class ArchiverTestCase(ArchiverTestCaseBase):
         self.cmd('init', '--encryption=repokey', self.repository_location)
         self.cmd('create', self.repository_location + '::test', 'input', '-C', 'none')
         file_list = self.cmd('list', self.repository_location + '::test', 'input/compressible',
-                             '--format', '{size} {csize} {sha256}')
-        size, csize, sha256_before = file_list.split(' ')
-        assert int(csize) >= int(size)  # >= due to metadata overhead
+                             '--format', '{size} {sha256}')
+        size, sha256_before = file_list.split(' ')
         self.cmd('recreate', self.repository_location, '-C', 'lz4', '--recompress')
         self.check_cache()
         file_list = self.cmd('list', self.repository_location + '::test', 'input/compressible',
-                             '--format', '{size} {csize} {sha256}')
-        size, csize, sha256_after = file_list.split(' ')
-        assert int(csize) < int(size)
+                             '--format', '{size} {sha256}')
+        size, sha256_after = file_list.split(' ')
         assert sha256_before == sha256_after
 
     def test_recreate_timestamp(self):

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

@@ -43,14 +43,14 @@ class TestCacheSynchronizer:
             'bar': 5678,
             'user': 'chunks',
             'chunks': [
-                (H(1), 1, 2),
-                (H(2), 2, 3),
+                (H(1), 1),
+                (H(2), 2),
             ]
         })
         sync.feed(data)
         assert len(index) == 2
-        assert index[H(1)] == (1, 1, 2)
-        assert index[H(2)] == (1, 2, 3)
+        assert index[H(1)] == (1, 1)
+        assert index[H(2)] == (1, 2)
 
     def test_multiple(self, index, sync):
         data = packb({
@@ -59,8 +59,8 @@ class TestCacheSynchronizer:
             'bar': 5678,
             'user': 'chunks',
             'chunks': [
-                (H(1), 1, 2),
-                (H(2), 2, 3),
+                (H(1), 1),
+                (H(2), 2),
             ]
         })
         data += packb({
@@ -78,8 +78,8 @@ class TestCacheSynchronizer:
                 'chunks': '123456',
             },
             'chunks': [
-                (H(1), 1, 2),
-                (H(2), 2, 3),
+                (H(1), 1),
+                (H(2), 2),
             ],
             'stuff': [
                 (1, 2, 3),
@@ -87,12 +87,12 @@ class TestCacheSynchronizer:
         })
         data += packb({
             'chunks': [
-                (H(3), 1, 2),
+                (H(3), 1),
             ],
         })
         data += packb({
             'chunks': [
-                (H(1), 1, 2),
+                (H(1), 1),
             ],
         })
 
@@ -103,9 +103,9 @@ class TestCacheSynchronizer:
         sync.feed(part2)
         sync.feed(part3)
         assert len(index) == 3
-        assert index[H(1)] == (3, 1, 2)
-        assert index[H(2)] == (2, 2, 3)
-        assert index[H(3)] == (1, 1, 2)
+        assert index[H(1)] == (3, 1)
+        assert index[H(2)] == (2, 2)
+        assert index[H(3)] == (1, 1)
 
     @pytest.mark.parametrize('elem,error', (
         ({1: 2}, 'Unexpected object: map'),
@@ -121,7 +121,7 @@ class TestCacheSynchronizer:
     @pytest.mark.parametrize('structure', (
         lambda elem: {'chunks': elem},
         lambda elem: {'chunks': [elem]},
-        lambda elem: {'chunks': [(elem, 1, 2)]},
+        lambda elem: {'chunks': [(elem, 1)]},
     ))
     def test_corrupted(self, sync, structure, elem, error):
         packed = packb(structure(elem))
@@ -135,11 +135,11 @@ class TestCacheSynchronizer:
     @pytest.mark.parametrize('data,error', (
         # Incorrect tuple length
         ({'chunks': [(bytes(32), 2, 3, 4)]}, 'Invalid chunk list entry length'),
-        ({'chunks': [(bytes(32), 2)]}, 'Invalid chunk list entry length'),
+        ({'chunks': [(bytes(32), )]}, 'Invalid chunk list entry length'),
         # Incorrect types
-        ({'chunks': [(1, 2, 3)]}, 'Unexpected object: integer'),
-        ({'chunks': [(1, bytes(32), 2)]}, 'Unexpected object: integer'),
-        ({'chunks': [(bytes(32), 1.0, 2)]}, 'Unexpected object: double'),
+        ({'chunks': [(1, 2)]}, 'Unexpected object: integer'),
+        ({'chunks': [(1, bytes(32))]}, 'Unexpected object: integer'),
+        ({'chunks': [(bytes(32), 1.0)]}, 'Unexpected object: double'),
     ))
     def test_corrupted_ancillary(self, index, sync, data, error):
         packed = packb(data)
@@ -173,7 +173,7 @@ class TestCacheSynchronizer:
         sync = CacheSynchronizer(index)
         data = packb({
             'chunks': [
-                (H(0), 1, 2),
+                (H(0), 1),
             ]
         })
         with pytest.raises(ValueError) as excinfo:
@@ -185,25 +185,25 @@ class TestCacheSynchronizer:
         sync = CacheSynchronizer(index)
         data = packb({
             'chunks': [
-                (H(0), 1, 2),
+                (H(0), 1),
             ]
         })
         sync.feed(data)
-        assert index[H(0)] == (ChunkIndex.MAX_VALUE, 1234, 5678)
+        assert index[H(0)] == (ChunkIndex.MAX_VALUE, 1234)
 
     def test_refcount_one_below_max_value(self):
         index = self.make_index_with_refcount(ChunkIndex.MAX_VALUE - 1)
         sync = CacheSynchronizer(index)
         data = packb({
             'chunks': [
-                (H(0), 1, 2),
+                (H(0), 1),
             ]
         })
         sync.feed(data)
         # Incremented to maximum
-        assert index[H(0)] == (ChunkIndex.MAX_VALUE, 1234, 5678)
+        assert index[H(0)] == (ChunkIndex.MAX_VALUE, 1234)
         sync.feed(data)
-        assert index[H(0)] == (ChunkIndex.MAX_VALUE, 1234, 5678)
+        assert index[H(0)] == (ChunkIndex.MAX_VALUE, 1234)
 
 
 class TestAdHocCache:
@@ -244,7 +244,7 @@ class TestAdHocCache:
             cache.add_chunk(H(1), b'5678', Statistics(), overwrite=True)
 
     def test_seen_chunk_add_chunk_size(self, cache):
-        assert cache.add_chunk(H(1), b'5678', Statistics()) == (H(1), 4, 0)
+        assert cache.add_chunk(H(1), b'5678', Statistics()) == (H(1), 4)
 
     def test_deletes_chunks_during_lifetime(self, cache, repository):
         """E.g. checkpoint archives"""
@@ -270,10 +270,10 @@ class TestAdHocCache:
         assert not hasattr(cache, 'chunks')
 
     def test_incref_after_add_chunk(self, cache):
-        assert cache.add_chunk(H(3), b'5678', Statistics()) == (H(3), 4, 47)
-        assert cache.chunk_incref(H(3), Statistics()) == (H(3), 4, 47)
+        assert cache.add_chunk(H(3), b'5678', Statistics()) == (H(3), 4)
+        assert cache.chunk_incref(H(3), Statistics()) == (H(3), 4)
 
     def test_existing_incref_after_add_chunk(self, cache):
         """This case occurs with part files, see Archive.chunk_file."""
-        assert cache.add_chunk(H(1), b'5678', Statistics()) == (H(1), 4, 0)
-        assert cache.chunk_incref(H(1), Statistics()) == (H(1), 4, 0)
+        assert cache.add_chunk(H(1), b'5678', Statistics()) == (H(1), 4)
+        assert cache.chunk_incref(H(1), Statistics()) == (H(1), 4)

+ 57 - 58
src/borg/testsuite/hashindex.py

@@ -91,8 +91,8 @@ class HashIndexTestCase(BaseTestCase):
                            '85f72b036c692c8266e4f51ccf0cff2147204282b5e316ae508d30a448d88fef')
 
     def test_chunkindex(self):
-        self._generic_test(ChunkIndex, lambda x: (x, x, x),
-                           'c83fdf33755fc37879285f2ecfc5d1f63b97577494902126b6fb6f3e4d852488')
+        self._generic_test(ChunkIndex, lambda x: (x, x),
+                           '85f72b036c692c8266e4f51ccf0cff2147204282b5e316ae508d30a448d88fef')
 
     def test_resize(self):
         n = 2000  # Must be >= MIN_BUCKETS
@@ -126,32 +126,30 @@ class HashIndexTestCase(BaseTestCase):
 
     def test_chunkindex_merge(self):
         idx1 = ChunkIndex()
-        idx1[H(1)] = 1, 100, 100
-        idx1[H(2)] = 2, 200, 200
-        idx1[H(3)] = 3, 300, 300
+        idx1[H(1)] = 1, 100
+        idx1[H(2)] = 2, 200
+        idx1[H(3)] = 3, 300
         # no H(4) entry
         idx2 = ChunkIndex()
-        idx2[H(1)] = 4, 100, 100
-        idx2[H(2)] = 5, 200, 200
+        idx2[H(1)] = 4, 100
+        idx2[H(2)] = 5, 200
         # no H(3) entry
-        idx2[H(4)] = 6, 400, 400
+        idx2[H(4)] = 6, 400
         idx1.merge(idx2)
-        assert idx1[H(1)] == (5, 100, 100)
-        assert idx1[H(2)] == (7, 200, 200)
-        assert idx1[H(3)] == (3, 300, 300)
-        assert idx1[H(4)] == (6, 400, 400)
+        assert idx1[H(1)] == (5, 100)
+        assert idx1[H(2)] == (7, 200)
+        assert idx1[H(3)] == (3, 300)
+        assert idx1[H(4)] == (6, 400)
 
     def test_chunkindex_summarize(self):
         idx = ChunkIndex()
-        idx[H(1)] = 1, 1000, 100
-        idx[H(2)] = 2, 2000, 200
-        idx[H(3)] = 3, 3000, 300
+        idx[H(1)] = 1, 1000
+        idx[H(2)] = 2, 2000
+        idx[H(3)] = 3, 3000
 
-        size, csize, unique_size, unique_csize, unique_chunks, chunks = idx.summarize()
+        size, unique_size, unique_chunks, chunks = idx.summarize()
         assert size == 1000 + 2 * 2000 + 3 * 3000
-        assert csize == 100 + 2 * 200 + 3 * 300
         assert unique_size == 1000 + 2000 + 3000
-        assert unique_csize == 100 + 200 + 300
         assert chunks == 1 + 2 + 3
         assert unique_chunks == 3
 
@@ -169,14 +167,14 @@ class HashIndexExtraTestCase(BaseTestCase):
         keys, to_delete_keys = all_keys[0:(2*key_count//3)], all_keys[(2*key_count//3):]
 
         for i, key in enumerate(keys):
-            index[key] = (i, i, i)
+            index[key] = (i, i)
         for i, key in enumerate(to_delete_keys):
-            index[key] = (i, i, i)
+            index[key] = (i, i)
 
         for key in to_delete_keys:
             del index[key]
         for i, key in enumerate(keys):
-            assert index[key] == (i, i, i)
+            assert index[key] == (i, i)
         for key in to_delete_keys:
             assert index.get(key) is None
 
@@ -190,12 +188,12 @@ class HashIndexExtraTestCase(BaseTestCase):
 class HashIndexSizeTestCase(BaseTestCase):
     def test_size_on_disk(self):
         idx = ChunkIndex()
-        assert idx.size() == 18 + 1031 * (32 + 3 * 4)
+        assert idx.size() == 18 + 1031 * (32 + 2 * 4)
 
     def test_size_on_disk_accurate(self):
         idx = ChunkIndex()
         for i in range(1234):
-            idx[H(i)] = i, i**2, i**3
+            idx[H(i)] = i, i**2
         with tempfile.NamedTemporaryFile() as file:
             idx.write(file.name)
             size = os.path.getsize(file.name)
@@ -205,7 +203,7 @@ class HashIndexSizeTestCase(BaseTestCase):
 class HashIndexRefcountingTestCase(BaseTestCase):
     def test_chunkindex_limit(self):
         idx = ChunkIndex()
-        idx[H(1)] = ChunkIndex.MAX_VALUE - 1, 1, 2
+        idx[H(1)] = ChunkIndex.MAX_VALUE - 1, 1
 
         # 5 is arbitrary, any number of incref/decrefs shouldn't move it once it's limited
         for i in range(5):
@@ -219,9 +217,9 @@ class HashIndexRefcountingTestCase(BaseTestCase):
     def _merge(self, refcounta, refcountb):
         def merge(refcount1, refcount2):
             idx1 = ChunkIndex()
-            idx1[H(1)] = refcount1, 1, 2
+            idx1[H(1)] = refcount1, 1
             idx2 = ChunkIndex()
-            idx2[H(1)] = refcount2, 1, 2
+            idx2[H(1)] = refcount2, 1
             idx1.merge(idx2)
             refcount, *_ = idx1[H(1)]
             return refcount
@@ -253,44 +251,44 @@ class HashIndexRefcountingTestCase(BaseTestCase):
 
     def test_chunkindex_add(self):
         idx1 = ChunkIndex()
-        idx1.add(H(1), 5, 6, 7)
-        assert idx1[H(1)] == (5, 6, 7)
-        idx1.add(H(1), 1, 2, 3)
-        assert idx1[H(1)] == (6, 2, 3)
+        idx1.add(H(1), 5, 6)
+        assert idx1[H(1)] == (5, 6)
+        idx1.add(H(1), 1, 2)
+        assert idx1[H(1)] == (6, 2)
 
     def test_incref_limit(self):
         idx1 = ChunkIndex()
-        idx1[H(1)] = (ChunkIndex.MAX_VALUE, 6, 7)
+        idx1[H(1)] = ChunkIndex.MAX_VALUE, 6
         idx1.incref(H(1))
         refcount, *_ = idx1[H(1)]
         assert refcount == ChunkIndex.MAX_VALUE
 
     def test_decref_limit(self):
         idx1 = ChunkIndex()
-        idx1[H(1)] = ChunkIndex.MAX_VALUE, 6, 7
+        idx1[H(1)] = ChunkIndex.MAX_VALUE, 6
         idx1.decref(H(1))
         refcount, *_ = idx1[H(1)]
         assert refcount == ChunkIndex.MAX_VALUE
 
     def test_decref_zero(self):
         idx1 = ChunkIndex()
-        idx1[H(1)] = 0, 0, 0
+        idx1[H(1)] = 0, 0
         with self.assert_raises(AssertionError):
             idx1.decref(H(1))
 
     def test_incref_decref(self):
         idx1 = ChunkIndex()
-        idx1.add(H(1), 5, 6, 7)
-        assert idx1[H(1)] == (5, 6, 7)
+        idx1.add(H(1), 5, 6)
+        assert idx1[H(1)] == (5, 6)
         idx1.incref(H(1))
-        assert idx1[H(1)] == (6, 6, 7)
+        assert idx1[H(1)] == (6, 6)
         idx1.decref(H(1))
-        assert idx1[H(1)] == (5, 6, 7)
+        assert idx1[H(1)] == (5, 6)
 
     def test_setitem_raises(self):
         idx1 = ChunkIndex()
         with self.assert_raises(AssertionError):
-            idx1[H(1)] = ChunkIndex.MAX_VALUE + 1, 0, 0
+            idx1[H(1)] = ChunkIndex.MAX_VALUE + 1, 0
 
     def test_keyerror(self):
         idx = ChunkIndex()
@@ -301,14 +299,15 @@ class HashIndexRefcountingTestCase(BaseTestCase):
         with self.assert_raises(KeyError):
             idx[H(1)]
         with self.assert_raises(OverflowError):
-            idx.add(H(1), -1, 0, 0)
+            idx.add(H(1), -1, 0)
 
 
 class HashIndexDataTestCase(BaseTestCase):
-    # This bytestring was created with 1.0-maint at c2f9533
-    HASHINDEX = b'eJzt0L0NgmAUhtHLT0LDEI6AuAEhMVYmVnSuYefC7AB3Aj9KNedJbnfyFne6P67P27w0EdG1Eac+Cm1ZybAsy7Isy7Isy7Isy7I' \
-                b'sy7Isy7Isy7Isy7Isy7Isy7Isy7Isy7Isy7Isy7Isy7Isy7Isy7Isy7Isy7Isy7Isy7Isy7Isy7Isy7Isy7Isy7LsL9nhc+cqTZ' \
-                b'3XlO2Ys++Du5fX+l1/YFmWZVmWZVmWZVmWZVmWZVmWZVmWZVmWZVmWZVmWZVmWZVmWZVmWZVmWZVmWZVmWZVn2/+0O2rYccw=='
+    # This bytestring was created with borg2-pre 2022-06-10
+    HASHINDEX = b'eJzt0LEJg1AYhdE/JqBjOEJMNhBBrAQrO9ewc+HsoG+CPMsEz1cfbnHbceqXoZvvEVE+IuoqMu2pnOE4' \
+                b'juM4juM4juM4juM4juM4juM4juM4juM4juM4juM4juM4juM4juM4juM4juM4juM4juM4juM4juM4juM4' \
+                b'juM4juM4juM4jruie36vuSVT5N0rzW0n9t7r5z9+4TiO4ziO4ziO4ziO4ziO4ziO4ziO4ziO4ziO4ziO' \
+                b'4ziO4ziO4ziO4ziO4ziO437LHbSVHGw='
 
     def _serialize_hashindex(self, idx):
         with tempfile.TemporaryDirectory() as tempdir:
@@ -332,23 +331,23 @@ class HashIndexDataTestCase(BaseTestCase):
 
     def test_identical_creation(self):
         idx1 = ChunkIndex()
-        idx1[H(1)] = 1, 2, 3
-        idx1[H(2)] = 2**31 - 1, 0, 0
-        idx1[H(3)] = 4294962296, 0, 0  # 4294962296 is -5000 interpreted as an uint32_t
+        idx1[H(1)] = 1, 2
+        idx1[H(2)] = 2**31 - 1, 0
+        idx1[H(3)] = 4294962296, 0  # 4294962296 is -5000 interpreted as an uint32_t
 
         serialized = self._serialize_hashindex(idx1)
         assert self._unpack(serialized) == self._unpack(self.HASHINDEX)
 
     def test_read_known_good(self):
         idx1 = self._deserialize_hashindex(self.HASHINDEX)
-        assert idx1[H(1)] == (1, 2, 3)
-        assert idx1[H(2)] == (2**31 - 1, 0, 0)
-        assert idx1[H(3)] == (4294962296, 0, 0)
+        assert idx1[H(1)] == (1, 2)
+        assert idx1[H(2)] == (2**31 - 1, 0)
+        assert idx1[H(3)] == (4294962296, 0)
 
         idx2 = ChunkIndex()
-        idx2[H(3)] = 2**32 - 123456, 6, 7
+        idx2[H(3)] = 2**32 - 123456, 6
         idx1.merge(idx2)
-        assert idx1[H(3)] == (ChunkIndex.MAX_VALUE, 6, 7)
+        assert idx1[H(3)] == (ChunkIndex.MAX_VALUE, 6)
 
 
 class HashIndexIntegrityTestCase(HashIndexDataTestCase):
@@ -499,16 +498,16 @@ class HashIndexCompactTestCase(HashIndexDataTestCase):
     def test_merge(self):
         master = ChunkIndex()
         idx1 = ChunkIndex()
-        idx1[H(1)] = 1, 100, 100
-        idx1[H(2)] = 2, 200, 200
-        idx1[H(3)] = 3, 300, 300
+        idx1[H(1)] = 1, 100
+        idx1[H(2)] = 2, 200
+        idx1[H(3)] = 3, 300
         idx1.compact()
-        assert idx1.size() == 18 + 3 * (32 + 3 * 4)
+        assert idx1.size() == 18 + 3 * (32 + 2 * 4)
 
         master.merge(idx1)
-        assert master[H(1)] == (1, 100, 100)
-        assert master[H(2)] == (2, 200, 200)
-        assert master[H(3)] == (3, 300, 300)
+        assert master[H(1)] == (1, 100)
+        assert master[H(2)] == (2, 200)
+        assert master[H(3)] == (3, 300)
 
 
 class NSIndexTestCase(BaseTestCase):

+ 2 - 5
src/borg/testsuite/item.py

@@ -143,13 +143,10 @@ def test_unknown_property():
 
 def test_item_file_size():
     item = Item(mode=0o100666, chunks=[
-        ChunkListEntry(csize=1, size=1000, id=None),
-        ChunkListEntry(csize=1, size=2000, id=None),
+        ChunkListEntry(size=1000, id=None),
+        ChunkListEntry(size=2000, id=None),
     ])
     assert item.get_size() == 3000
-    with pytest.raises(AssertionError):
-        item.get_size(compressed=True, memorize=True)
-    assert item.get_size(compressed=True) == 2
     item.get_size(memorize=True)
     assert item.size == 3000