浏览代码

prepare to remove csize (set it to 0 for now)

Thomas Waldmann 3 年之前
父节点
当前提交
b9f9623a6d

+ 24 - 90
src/borg/archive.py

@@ -58,60 +58,46 @@ 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.nfiles = 0
+        self.osize_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, part=False):
         if not part:
             self.osize += size
-            self.csize += csize
-            if unique:
-                self.usize += csize
         else:
             self.osize_parts += size
-            self.csize_parts += csize
-            if unique:
-                self.usize_parts += csize
 
     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}"
 
     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})>".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,
         }
 
     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 +105,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
 
@@ -130,14 +114,6 @@ class Statistics:
     def osize_fmt(self):
         return format_file_size(self.osize, iec=self.iec)
 
-    @property
-    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 +134,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.nfiles} N '.format(self)
                     path = remove_surrogates(item.path) if item else ''
                     space = columns - swidth(msg)
                     if space < 12:
@@ -614,10 +590,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)
@@ -651,51 +625,12 @@ Utilization of max. archive size: {csize_max:.0%}
         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
-        else:
-            def add(id):
-                entry = cache.chunks[id]
-                archive_index.add(id, 1, entry.size, entry.csize)
-
-            archive_index = ChunkIndex()
-            sync = CacheSynchronizer(archive_index)
-            add(self.id)
-            # we must escape any % char in the archive name, because we use it in a format string, see #6500
-            arch_name_escd = self.name.replace('%', '%%')
-            pi = ProgressIndicatorPercent(total=len(self.metadata.items),
-                                          msg='Calculating statistics for archive %s ... %%3.0f%%%%' % arch_name_escd,
-                                          msgid='archive.calc_stats')
-            for id, chunk in zip(self.metadata.items, self.repository.get_many(self.metadata.items)):
-                pi.show(increase=1)
-                add(id)
-                data = self.key.decrypt(id, chunk)
-                sync.feed(data)
-            unique_csize = archive_index.stats_against(cache.chunks)[3]
-            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.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 +921,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,
@@ -1789,15 +1724,15 @@ class ArchiveChecker:
         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, csize=0)  # was: csize=csize
                 if self.repair:
                     self.repository.put(id_, cdata)
 
@@ -1811,8 +1746,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 = []
@@ -1835,30 +1769,30 @@ 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
                         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
                 offset += size
@@ -2005,7 +1939,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()
 

+ 3 - 3
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"
 
 PURE_PYTHON_MSGPACK_WARNING = "Using a pure-python msgpack! This will result in lower performance."
 
@@ -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
+                This archive:   {stats[original_size]:>20s}
                 {cache}
                 """).strip().format(cache=cache, **info))
             if self.exit_code:

+ 18 - 66
src/borg/cache.py

@@ -406,7 +406,7 @@ 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}
 
                        Unique chunks         Total chunks
 Chunk index:    {0.total_unique_chunks:20d} {0.total_chunks:20d}"""
@@ -418,39 +418,32 @@ Chunk index:    {0.total_unique_chunks:20d} {0.total_chunks:20d}"""
     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', ]:
             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):
     """
@@ -679,8 +672,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,39 +709,6 @@ 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
@@ -766,7 +724,6 @@ class LocalCache(CacheStatsMixin):
                 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 +819,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,10 +903,10 @@ 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)
+        csize = 0  # len(data)
         self.repository.put(id, data, wait=wait)
         self.chunks.add(id, 1, size, csize)
-        stats.update(size, csize, not refcount)
+        stats.update(size)
         return ChunkListEntry(id, size, csize)
 
     def seen_chunk(self, id, size=None):
@@ -970,7 +922,7 @@ class LocalCache(CacheStatsMixin):
         if not self.txn_active:
             self.begin_txn()
         count, _size, csize = self.chunks.incref(id)
-        stats.update(_size, csize, False, part=part)
+        stats.update(_size, part=part)
         return ChunkListEntry(id, _size, csize)
 
     def chunk_decref(self, id, stats, wait=True, part=False):
@@ -980,9 +932,9 @@ class LocalCache(CacheStatsMixin):
         if count == 0:
             del self.chunks[id]
             self.repository.delete(id, wait=wait)
-            stats.update(-size, -csize, True, part=part)
+            stats.update(-size, part=part)
         else:
-            stats.update(-size, -csize, False, part=part)
+            stats.update(-size, part=part)
 
     def file_known_and_unchanged(self, hashed_path, path_hash, st):
         """
@@ -1122,7 +1074,7 @@ Chunk index:    {0.total_unique_chunks:20d}             unknown"""
         csize = len(data)
         self.repository.put(id, data, wait=wait)
         self.chunks.add(id, 1, size, csize)
-        stats.update(size, csize, not refcount)
+        stats.update(size)
         return ChunkListEntry(id, size, csize)
 
     def seen_chunk(self, id, size=None):
@@ -1144,7 +1096,7 @@ Chunk index:    {0.total_unique_chunks:20d}             unknown"""
         # 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)
+        stats.update(size, part=part)
         return ChunkListEntry(id, size, csize)
 
     def chunk_decref(self, id, stats, wait=True, part=False):
@@ -1154,9 +1106,9 @@ Chunk index:    {0.total_unique_chunks:20d}             unknown"""
         if count == 0:
             del self.chunks[id]
             self.repository.delete(id, wait=wait)
-            stats.update(-size, -csize, True, part=part)
+            stats.update(-size, part=part)
         else:
-            stats.update(-size, -csize, False, part=part)
+            stats.update(-size, part=part)
 
     def commit(self):
         if not self._txn_active:

+ 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', ])

+ 0 - 2
src/borg/hashindex.pyx

@@ -375,9 +375,7 @@ 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
 

+ 2 - 12
src/borg/helpers/parseformat.py

@@ -698,9 +698,6 @@ 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 +705,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', 'num_chunks', 'unique_chunks'),
         ('mtime', 'ctime', 'atime', 'isomtime', 'isoctime', 'isoatime'),
         tuple(sorted(hash_algorithms)),
         ('archiveid', 'archivename', 'extra'),
@@ -716,7 +713,7 @@ class ItemFormatter(BaseFormatter):
     )
 
     KEYS_REQUIRING_CACHE = (
-        'dsize', 'dcsize', 'unique_chunks',
+        'unique_chunks',
     )
 
     @classmethod
@@ -774,9 +771,6 @@ 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'),
@@ -848,10 +842,6 @@ class ItemFormatter(BaseFormatter):
         # 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)
-
     def hash_item(self, hash_function, item):
         if 'chunks' not in item:
             return ""

+ 3 - 2
src/borg/item.pyx

@@ -293,8 +293,9 @@ class Item(PropDict):
         :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.'
+        if compressed:
+            return 0  # try to live without csize
+        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:

+ 8 - 15
src/borg/testsuite/archive.py

@@ -19,47 +19,44 @@ from ..platform import uid2user, gid2group
 @pytest.fixture()
 def stats():
     stats = Statistics()
-    stats.update(20, 10, unique=True)
+    stats.update(20)
     return stats
 
 
 def test_stats_basic(stats):
     assert stats.osize == 20
-    assert stats.csize == stats.usize == 10
-    stats.update(20, 10, unique=False)
+    stats.update(20)
     assert stats.osize == 40
-    assert stats.csize == 20
-    assert stats.usize == 10
 
 
 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 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)
     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 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 0 N foofoofoofoofoofoofoofoofoofoo...foofoofoofoofoofoofoofoofoofoofoo'
     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"""
     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)>'
 
 
 def test_stats_progress_json(stats):
@@ -73,8 +70,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 +80,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
 
 

+ 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):

+ 1 - 3
src/borg/testsuite/hashindex.py

@@ -147,11 +147,9 @@ class HashIndexTestCase(BaseTestCase):
         idx[H(2)] = 2, 2000, 200
         idx[H(3)] = 3, 3000, 300
 
-        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
 

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

@@ -143,13 +143,11 @@ 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(csize=0, size=1000, id=None),
+        ChunkListEntry(csize=0, 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
+    assert item.get_size(compressed=True) == 0  # no csize any more
     item.get_size(memorize=True)
     assert item.size == 3000