Browse Source

Merge pull request #2364 from enkore/f/autocomp

Refactor compression decision stuff
enkore 8 years ago
parent
commit
f878678b0c

+ 35 - 46
src/borg/archive.py

@@ -21,11 +21,11 @@ logger = create_logger()
 from . import xattr
 from . import xattr
 from .cache import ChunkListEntry
 from .cache import ChunkListEntry
 from .chunker import Chunker
 from .chunker import Chunker
-from .compress import Compressor
+from .compress import Compressor, CompressionSpec
 from .constants import *  # NOQA
 from .constants import *  # NOQA
 from .hashindex import ChunkIndex, ChunkIndexEntry
 from .hashindex import ChunkIndex, ChunkIndexEntry
 from .helpers import Manifest
 from .helpers import Manifest
-from .helpers import Chunk, ChunkIteratorFileWrapper, open_item
+from .helpers import ChunkIteratorFileWrapper, open_item
 from .helpers import Error, IntegrityError, set_ec
 from .helpers import Error, IntegrityError, set_ec
 from .helpers import uid2user, user2uid, gid2group, group2gid
 from .helpers import uid2user, user2uid, gid2group, group2gid
 from .helpers import parse_timestamp, to_localtime
 from .helpers import parse_timestamp, to_localtime
@@ -36,7 +36,6 @@ from .helpers import bin_to_hex
 from .helpers import safe_ns
 from .helpers import safe_ns
 from .helpers import ellipsis_truncate, ProgressIndicatorPercent, log_multi
 from .helpers import ellipsis_truncate, ProgressIndicatorPercent, log_multi
 from .helpers import PathPrefixPattern, FnmatchPattern
 from .helpers import PathPrefixPattern, FnmatchPattern
-from .helpers import CompressionDecider1, CompressionDecider2, CompressionSpec
 from .item import Item, ArchiveItem
 from .item import Item, ArchiveItem
 from .key import key_factory
 from .key import key_factory
 from .platform import acl_get, acl_set, set_flags, get_flags, swidth
 from .platform import acl_get, acl_set, set_flags, get_flags, swidth
@@ -196,7 +195,7 @@ class DownloadPipeline:
         otherwise preloaded chunks will accumulate in RemoteRepository and create a memory leak.
         otherwise preloaded chunks will accumulate in RemoteRepository and create a memory leak.
         """
         """
         unpacker = msgpack.Unpacker(use_list=False)
         unpacker = msgpack.Unpacker(use_list=False)
-        for _, data in self.fetch_many(ids):
+        for data in self.fetch_many(ids):
             unpacker.feed(data)
             unpacker.feed(data)
             items = [Item(internal_dict=item) for item in unpacker]
             items = [Item(internal_dict=item) for item in unpacker]
             for item in items:
             for item in items:
@@ -238,7 +237,9 @@ class ChunkBuffer:
         if self.buffer.tell() == 0:
         if self.buffer.tell() == 0:
             return
             return
         self.buffer.seek(0)
         self.buffer.seek(0)
-        chunks = list(Chunk(bytes(s)) for s in self.chunker.chunkify(self.buffer))
+        # The chunker returns a memoryview to its internal buffer,
+        # thus a copy is needed before resuming the chunker iterator.
+        chunks = list(bytes(s) for s in self.chunker.chunkify(self.buffer))
         self.buffer.seek(0)
         self.buffer.seek(0)
         self.buffer.truncate(0)
         self.buffer.truncate(0)
         # Leave the last partial chunk in the buffer unless flush is True
         # Leave the last partial chunk in the buffer unless flush is True
@@ -246,7 +247,7 @@ class ChunkBuffer:
         for chunk in chunks[:end]:
         for chunk in chunks[:end]:
             self.chunks.append(self.write_chunk(chunk))
             self.chunks.append(self.write_chunk(chunk))
         if end == -1:
         if end == -1:
-            self.buffer.write(chunks[-1].data)
+            self.buffer.write(chunks[-1])
 
 
     def is_full(self):
     def is_full(self):
         return self.buffer.tell() > self.BUFFER_SIZE
         return self.buffer.tell() > self.BUFFER_SIZE
@@ -260,7 +261,7 @@ class CacheChunkBuffer(ChunkBuffer):
         self.stats = stats
         self.stats = stats
 
 
     def write_chunk(self, chunk):
     def write_chunk(self, chunk):
-        id_, _, _ = self.cache.add_chunk(self.key.id_hash(chunk.data), 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)
         self.cache.repository.async_response(wait=False)
         return id_
         return id_
 
 
@@ -278,7 +279,7 @@ class Archive:
 
 
     def __init__(self, repository, key, manifest, name, cache=None, create=False,
     def __init__(self, repository, key, manifest, name, cache=None, create=False,
                  checkpoint_interval=300, numeric_owner=False, noatime=False, noctime=False, progress=False,
                  checkpoint_interval=300, numeric_owner=False, noatime=False, noctime=False, progress=False,
-                 chunker_params=CHUNKER_PARAMS, start=None, start_monotonic=None, end=None, compression=None, compression_files=None,
+                 chunker_params=CHUNKER_PARAMS, start=None, start_monotonic=None, end=None,
                  consider_part_files=False, log_json=False):
                  consider_part_files=False, log_json=False):
         self.cwd = os.getcwd()
         self.cwd = os.getcwd()
         self.key = key
         self.key = key
@@ -307,12 +308,8 @@ class Archive:
         self.pipeline = DownloadPipeline(self.repository, self.key)
         self.pipeline = DownloadPipeline(self.repository, self.key)
         self.create = create
         self.create = create
         if self.create:
         if self.create:
-            self.file_compression_logger = create_logger('borg.debug.file-compression')
             self.items_buffer = CacheChunkBuffer(self.cache, self.key, self.stats)
             self.items_buffer = CacheChunkBuffer(self.cache, self.key, self.stats)
             self.chunker = Chunker(self.key.chunk_seed, *chunker_params)
             self.chunker = Chunker(self.key.chunk_seed, *chunker_params)
-            self.compression_decider1 = CompressionDecider1(compression or CompressionSpec('none'),
-                                                            compression_files or [])
-            key.compression_decider2 = CompressionDecider2(compression or CompressionSpec('none'))
             if name in manifest.archives:
             if name in manifest.archives:
                 raise self.AlreadyExists(name)
                 raise self.AlreadyExists(name)
             self.last_checkpoint = time.monotonic()
             self.last_checkpoint = time.monotonic()
@@ -330,7 +327,7 @@ class Archive:
             self.zeros = None
             self.zeros = None
 
 
     def _load_meta(self, id):
     def _load_meta(self, id):
-        _, data = self.key.decrypt(id, self.repository.get(id))
+        data = self.key.decrypt(id, self.repository.get(id))
         metadata = ArchiveItem(internal_dict=msgpack.unpackb(data, unicode_errors='surrogateescape'))
         metadata = ArchiveItem(internal_dict=msgpack.unpackb(data, unicode_errors='surrogateescape'))
         if metadata.version != 1:
         if metadata.version != 1:
             raise Exception('Unknown archive metadata version')
             raise Exception('Unknown archive metadata version')
@@ -469,7 +466,7 @@ Utilization of max. archive size: {csize_max:.0%}
         metadata = ArchiveItem(metadata)
         metadata = ArchiveItem(metadata)
         data = self.key.pack_and_authenticate_metadata(metadata.as_dict(), context=b'archive')
         data = self.key.pack_and_authenticate_metadata(metadata.as_dict(), context=b'archive')
         self.id = self.key.id_hash(data)
         self.id = self.key.id_hash(data)
-        self.cache.add_chunk(self.id, Chunk(data), self.stats)
+        self.cache.add_chunk(self.id, data, self.stats)
         while self.repository.async_response(wait=True) is not None:
         while self.repository.async_response(wait=True) is not None:
             pass
             pass
         self.manifest.archives[name] = (self.id, metadata.time)
         self.manifest.archives[name] = (self.id, metadata.time)
@@ -495,7 +492,7 @@ Utilization of max. archive size: {csize_max:.0%}
         add(self.id)
         add(self.id)
         for id, chunk in zip(self.metadata.items, self.repository.get_many(self.metadata.items)):
         for id, chunk in zip(self.metadata.items, self.repository.get_many(self.metadata.items)):
             add(id)
             add(id)
-            _, data = self.key.decrypt(id, chunk)
+            data = self.key.decrypt(id, chunk)
             unpacker.feed(data)
             unpacker.feed(data)
             for item in unpacker:
             for item in unpacker:
                 chunks = item.get(b'chunks')
                 chunks = item.get(b'chunks')
@@ -525,7 +522,7 @@ Utilization of max. archive size: {csize_max:.0%}
         if dry_run or stdout:
         if dry_run or stdout:
             if 'chunks' in item:
             if 'chunks' in item:
                 item_chunks_size = 0
                 item_chunks_size = 0
-                for _, data in self.pipeline.fetch_many([c.id for c in item.chunks], is_preloaded=True):
+                for data in self.pipeline.fetch_many([c.id for c in item.chunks], is_preloaded=True):
                     if pi:
                     if pi:
                         pi.show(increase=len(data), info=[remove_surrogates(item.path)])
                         pi.show(increase=len(data), info=[remove_surrogates(item.path)])
                     if stdout:
                     if stdout:
@@ -589,7 +586,7 @@ Utilization of max. archive size: {csize_max:.0%}
                 self.zeros = b'\0' * (1 << self.chunker_params[1])
                 self.zeros = b'\0' * (1 << self.chunker_params[1])
             with fd:
             with fd:
                 ids = [c.id for c in item.chunks]
                 ids = [c.id for c in item.chunks]
-                for _, data in self.pipeline.fetch_many(ids, is_preloaded=True):
+                for data in self.pipeline.fetch_many(ids, is_preloaded=True):
                     if pi:
                     if pi:
                         pi.show(increase=len(data), info=[remove_surrogates(item.path)])
                         pi.show(increase=len(data), info=[remove_surrogates(item.path)])
                     with backup_io('write'):
                     with backup_io('write'):
@@ -717,7 +714,7 @@ Utilization of max. archive size: {csize_max:.0%}
         setattr(metadata, key, value)
         setattr(metadata, key, value)
         data = msgpack.packb(metadata.as_dict(), unicode_errors='surrogateescape')
         data = msgpack.packb(metadata.as_dict(), unicode_errors='surrogateescape')
         new_id = self.key.id_hash(data)
         new_id = self.key.id_hash(data)
-        self.cache.add_chunk(new_id, Chunk(data), self.stats)
+        self.cache.add_chunk(new_id, data, self.stats)
         self.manifest.archives[self.name] = (new_id, metadata.time)
         self.manifest.archives[self.name] = (new_id, metadata.time)
         self.cache.chunk_decref(self.id, self.stats)
         self.cache.chunk_decref(self.id, self.stats)
         self.id = new_id
         self.id = new_id
@@ -764,7 +761,7 @@ Utilization of max. archive size: {csize_max:.0%}
             for (i, (items_id, data)) in enumerate(zip(items_ids, self.repository.get_many(items_ids))):
             for (i, (items_id, data)) in enumerate(zip(items_ids, self.repository.get_many(items_ids))):
                 if progress:
                 if progress:
                     pi.show(i)
                     pi.show(i)
-                _, data = self.key.decrypt(items_id, data)
+                data = self.key.decrypt(items_id, data)
                 unpacker.feed(data)
                 unpacker.feed(data)
                 chunk_decref(items_id, stats)
                 chunk_decref(items_id, stats)
                 try:
                 try:
@@ -879,10 +876,10 @@ Utilization of max. archive size: {csize_max:.0%}
         self.write_checkpoint()
         self.write_checkpoint()
         return length, number
         return length, number
 
 
-    def chunk_file(self, item, cache, stats, chunk_iter, chunk_processor=None, **chunk_kw):
+    def chunk_file(self, item, cache, stats, chunk_iter, chunk_processor=None):
         if not chunk_processor:
         if not chunk_processor:
             def chunk_processor(data):
             def chunk_processor(data):
-                chunk_entry = cache.add_chunk(self.key.id_hash(data), Chunk(data, **chunk_kw), stats, wait=False)
+                chunk_entry = cache.add_chunk(self.key.id_hash(data), data, stats, wait=False)
                 self.cache.repository.async_response(wait=False)
                 self.cache.repository.async_response(wait=False)
                 return chunk_entry
                 return chunk_entry
 
 
@@ -971,12 +968,10 @@ Utilization of max. archive size: {csize_max:.0%}
         if chunks is not None:
         if chunks is not None:
             item.chunks = chunks
             item.chunks = chunks
         else:
         else:
-            compress = self.compression_decider1.decide(path)
-            self.file_compression_logger.debug('%s -> compression %s', path, compress.name)
             with backup_io('open'):
             with backup_io('open'):
                 fh = Archive._open_rb(path)
                 fh = Archive._open_rb(path)
             with os.fdopen(fh, 'rb') as fd:
             with os.fdopen(fh, 'rb') as fd:
-                self.chunk_file(item, cache, self.stats, backup_io_iter(self.chunker.chunkify(fd, fh)), compress=compress)
+                self.chunk_file(item, cache, self.stats, backup_io_iter(self.chunker.chunkify(fd, fh)))
             if not is_special_file:
             if not is_special_file:
                 # we must not memorize special files, because the contents of e.g. a
                 # we must not memorize special files, because the contents of e.g. a
                 # block or char device will change without its mtime/size/inode changing.
                 # block or char device will change without its mtime/size/inode changing.
@@ -1212,9 +1207,9 @@ class ArchiveChecker:
                         chunk_ids = list(reversed(chunk_ids_revd))
                         chunk_ids = list(reversed(chunk_ids_revd))
                         chunk_data_iter = self.repository.get_many(chunk_ids)
                         chunk_data_iter = self.repository.get_many(chunk_ids)
                 else:
                 else:
+                    _chunk_id = None if chunk_id == Manifest.MANIFEST_ID else chunk_id
                     try:
                     try:
-                        _chunk_id = None if chunk_id == Manifest.MANIFEST_ID else chunk_id
-                        _, data = self.key.decrypt(_chunk_id, encrypted_data)
+                        self.key.decrypt(_chunk_id, encrypted_data)
                     except IntegrityError as integrity_error:
                     except IntegrityError as integrity_error:
                         self.error_found = True
                         self.error_found = True
                         errors += 1
                         errors += 1
@@ -1284,7 +1279,7 @@ class ArchiveChecker:
         for chunk_id, _ in self.chunks.iteritems():
         for chunk_id, _ in self.chunks.iteritems():
             cdata = self.repository.get(chunk_id)
             cdata = self.repository.get(chunk_id)
             try:
             try:
-                _, data = self.key.decrypt(chunk_id, cdata)
+                data = self.key.decrypt(chunk_id, cdata)
             except IntegrityError as exc:
             except IntegrityError as exc:
                 logger.error('Skipping corrupted chunk: %s', exc)
                 logger.error('Skipping corrupted chunk: %s', exc)
                 self.error_found = True
                 self.error_found = True
@@ -1329,9 +1324,9 @@ class ArchiveChecker:
                 self.possibly_superseded.add(id_)
                 self.possibly_superseded.add(id_)
 
 
         def add_callback(chunk):
         def add_callback(chunk):
-            id_ = self.key.id_hash(chunk.data)
+            id_ = self.key.id_hash(chunk)
             cdata = self.key.encrypt(chunk)
             cdata = self.key.encrypt(chunk)
-            add_reference(id_, len(chunk.data), len(cdata), cdata)
+            add_reference(id_, len(chunk), len(cdata), cdata)
             return id_
             return id_
 
 
         def add_reference(id_, size, csize, cdata=None):
         def add_reference(id_, size, csize, cdata=None):
@@ -1352,7 +1347,7 @@ class ArchiveChecker:
             def replacement_chunk(size):
             def replacement_chunk(size):
                 data = bytes(size)
                 data = bytes(size)
                 chunk_id = self.key.id_hash(data)
                 chunk_id = self.key.id_hash(data)
-                cdata = self.key.encrypt(Chunk(data))
+                cdata = self.key.encrypt(data)
                 csize = len(cdata)
                 csize = len(cdata)
                 return chunk_id, size, csize, cdata
                 return chunk_id, size, csize, cdata
 
 
@@ -1461,7 +1456,7 @@ class ArchiveChecker:
                 if state > 0:
                 if state > 0:
                     unpacker.resync()
                     unpacker.resync()
                 for chunk_id, cdata in zip(items, repository.get_many(items)):
                 for chunk_id, cdata in zip(items, repository.get_many(items)):
-                    _, data = self.key.decrypt(chunk_id, cdata)
+                    data = self.key.decrypt(chunk_id, cdata)
                     unpacker.feed(data)
                     unpacker.feed(data)
                     try:
                     try:
                         for item in unpacker:
                         for item in unpacker:
@@ -1511,7 +1506,7 @@ class ArchiveChecker:
                     continue
                     continue
                 mark_as_possibly_superseded(archive_id)
                 mark_as_possibly_superseded(archive_id)
                 cdata = self.repository.get(archive_id)
                 cdata = self.repository.get(archive_id)
-                _, data = self.key.decrypt(archive_id, cdata)
+                data = self.key.decrypt(archive_id, cdata)
                 archive = ArchiveItem(internal_dict=msgpack.unpackb(data))
                 archive = ArchiveItem(internal_dict=msgpack.unpackb(data))
                 if archive.version != 1:
                 if archive.version != 1:
                     raise Exception('Unknown archive metadata version')
                     raise Exception('Unknown archive metadata version')
@@ -1528,7 +1523,7 @@ class ArchiveChecker:
                 archive.items = items_buffer.chunks
                 archive.items = items_buffer.chunks
                 data = msgpack.packb(archive.as_dict(), unicode_errors='surrogateescape')
                 data = msgpack.packb(archive.as_dict(), unicode_errors='surrogateescape')
                 new_archive_id = self.key.id_hash(data)
                 new_archive_id = self.key.id_hash(data)
-                cdata = self.key.encrypt(Chunk(data))
+                cdata = self.key.encrypt(data)
                 add_reference(new_archive_id, len(data), len(cdata), cdata)
                 add_reference(new_archive_id, len(data), len(cdata), cdata)
                 self.manifest.archives[info.name] = (new_archive_id, info.ts)
                 self.manifest.archives[info.name] = (new_archive_id, info.ts)
 
 
@@ -1562,7 +1557,7 @@ class ArchiveRecreater:
 
 
     def __init__(self, repository, manifest, key, cache, matcher,
     def __init__(self, repository, manifest, key, cache, matcher,
                  exclude_caches=False, exclude_if_present=None, keep_exclude_tags=False,
                  exclude_caches=False, exclude_if_present=None, keep_exclude_tags=False,
-                 chunker_params=None, compression=None, compression_files=None, always_recompress=False,
+                 chunker_params=None, compression=None, always_recompress=False,
                  dry_run=False, stats=False, progress=False, file_status_printer=None,
                  dry_run=False, stats=False, progress=False, file_status_printer=None,
                  checkpoint_interval=1800):
                  checkpoint_interval=1800):
         self.repository = repository
         self.repository = repository
@@ -1583,9 +1578,6 @@ class ArchiveRecreater:
         self.always_recompress = always_recompress
         self.always_recompress = always_recompress
         self.compression = compression or CompressionSpec('none')
         self.compression = compression or CompressionSpec('none')
         self.seen_chunks = set()
         self.seen_chunks = set()
-        self.compression_decider1 = CompressionDecider1(compression or CompressionSpec('none'),
-                                                        compression_files or [])
-        key.compression_decider2 = CompressionDecider2(compression or CompressionSpec('none'))
 
 
         self.dry_run = dry_run
         self.dry_run = dry_run
         self.stats = stats
         self.stats = stats
@@ -1654,24 +1646,21 @@ class ArchiveRecreater:
                 self.cache.chunk_incref(chunk_id, target.stats)
                 self.cache.chunk_incref(chunk_id, target.stats)
             return item.chunks
             return item.chunks
         chunk_iterator = self.iter_chunks(archive, target, list(item.chunks))
         chunk_iterator = self.iter_chunks(archive, target, list(item.chunks))
-        compress = self.compression_decider1.decide(item.path)
-        chunk_processor = partial(self.chunk_processor, target, compress)
+        chunk_processor = partial(self.chunk_processor, target)
         target.chunk_file(item, self.cache, target.stats, chunk_iterator, chunk_processor)
         target.chunk_file(item, self.cache, target.stats, chunk_iterator, chunk_processor)
 
 
-    def chunk_processor(self, target, compress, data):
+    def chunk_processor(self, target, data):
         chunk_id = self.key.id_hash(data)
         chunk_id = self.key.id_hash(data)
         if chunk_id in self.seen_chunks:
         if chunk_id in self.seen_chunks:
             return self.cache.chunk_incref(chunk_id, target.stats)
             return self.cache.chunk_incref(chunk_id, target.stats)
-        chunk = Chunk(data, compress=compress)
-        compression_spec, chunk = self.key.compression_decider2.decide(chunk)
         overwrite = self.recompress
         overwrite = self.recompress
         if self.recompress and not self.always_recompress and chunk_id in self.cache.chunks:
         if self.recompress and not self.always_recompress and chunk_id in self.cache.chunks:
             # Check if this chunk is already compressed the way we want it
             # Check if this chunk is already compressed the way we want it
             old_chunk = self.key.decrypt(None, self.repository.get(chunk_id), decompress=False)
             old_chunk = self.key.decrypt(None, self.repository.get(chunk_id), decompress=False)
-            if Compressor.detect(old_chunk.data).name == compression_spec.name:
+            if Compressor.detect(old_chunk).name == self.key.compressor.decide(data).name:
                 # Stored chunk has the same compression we wanted
                 # Stored chunk has the same compression we wanted
                 overwrite = False
                 overwrite = False
-        chunk_entry = self.cache.add_chunk(chunk_id, chunk, target.stats, overwrite=overwrite, wait=False)
+        chunk_entry = self.cache.add_chunk(chunk_id, data, target.stats, overwrite=overwrite, wait=False)
         self.cache.repository.async_response(wait=False)
         self.cache.repository.async_response(wait=False)
         self.seen_chunks.add(chunk_entry.id)
         self.seen_chunks.add(chunk_entry.id)
         return chunk_entry
         return chunk_entry
@@ -1685,7 +1674,7 @@ class ArchiveRecreater:
             yield from target.chunker.chunkify(file)
             yield from target.chunker.chunkify(file)
         else:
         else:
             for chunk in chunk_iterator:
             for chunk in chunk_iterator:
-                yield chunk.data
+                yield chunk
 
 
     def save(self, archive, target, comment=None, replace_original=True):
     def save(self, archive, target, comment=None, replace_original=True):
         if self.dry_run:
         if self.dry_run:
@@ -1756,7 +1745,7 @@ class ArchiveRecreater:
     def create_target_archive(self, name):
     def create_target_archive(self, name):
         target = Archive(self.repository, self.key, self.manifest, name, create=True,
         target = Archive(self.repository, self.key, self.manifest, name, create=True,
                           progress=self.progress, chunker_params=self.chunker_params, cache=self.cache,
                           progress=self.progress, chunker_params=self.chunker_params, cache=self.cache,
-                          checkpoint_interval=self.checkpoint_interval, compression=self.compression)
+                          checkpoint_interval=self.checkpoint_interval)
         return target
         return target
 
 
     def open_archive(self, name, **kwargs):
     def open_archive(self, name, **kwargs):

+ 22 - 54
src/borg/archiver.py

@@ -34,10 +34,11 @@ from .archive import Archive, ArchiveChecker, ArchiveRecreater, Statistics, is_s
 from .archive import BackupOSError, backup_io
 from .archive import BackupOSError, backup_io
 from .cache import Cache
 from .cache import Cache
 from .constants import *  # NOQA
 from .constants import *  # NOQA
+from .compress import CompressionSpec
 from .crc32 import crc32
 from .crc32 import crc32
 from .helpers import EXIT_SUCCESS, EXIT_WARNING, EXIT_ERROR
 from .helpers import EXIT_SUCCESS, EXIT_WARNING, EXIT_ERROR
 from .helpers import Error, NoManifestError, set_ec
 from .helpers import Error, NoManifestError, set_ec
-from .helpers import location_validator, archivename_validator, ChunkerParams, CompressionSpec, ComprSpec
+from .helpers import location_validator, archivename_validator, ChunkerParams
 from .helpers import PrefixSpec, SortBySpec, HUMAN_SORT_KEYS
 from .helpers import PrefixSpec, SortBySpec, HUMAN_SORT_KEYS
 from .helpers import BaseFormatter, ItemFormatter, ArchiveFormatter
 from .helpers import BaseFormatter, ItemFormatter, ArchiveFormatter
 from .helpers import format_time, format_timedelta, format_file_size, format_archive
 from .helpers import format_time, format_timedelta, format_file_size, format_archive
@@ -107,6 +108,15 @@ def with_repository(fake=False, invert_fake=False, create=False, lock=True, excl
             with repository:
             with repository:
                 if manifest or cache:
                 if manifest or cache:
                     kwargs['manifest'], kwargs['key'] = Manifest.load(repository)
                     kwargs['manifest'], kwargs['key'] = Manifest.load(repository)
+                    # do_recreate uses args.compression is None as in band signalling for "don't recompress",
+                    # note that it does not look at key.compressor. In this case the default compressor applies
+                    # to new chunks.
+                    #
+                    # We can't use a check like `'compression' in args` (an argparse.Namespace speciality),
+                    # since the compression attribute is set. So we need to see whether it's set to something
+                    # true-ish, like a CompressionSpec instance.
+                    if getattr(args, 'compression', False):
+                        kwargs['key'].compressor = args.compression.compressor
                 if cache:
                 if cache:
                     with Cache(repository, kwargs['key'], kwargs['manifest'],
                     with Cache(repository, kwargs['key'], kwargs['manifest'],
                                do_files=getattr(args, 'cache_files', False),
                                do_files=getattr(args, 'cache_files', False),
@@ -167,14 +177,14 @@ class Archiver:
                 a = next(chunks1, end)
                 a = next(chunks1, end)
                 if a is end:
                 if a is end:
                     return not blen - bi and next(chunks2, end) is end
                     return not blen - bi and next(chunks2, end) is end
-                a = memoryview(a.data)
+                a = memoryview(a)
                 alen = len(a)
                 alen = len(a)
                 ai = 0
                 ai = 0
             if not blen - bi:
             if not blen - bi:
                 b = next(chunks2, end)
                 b = next(chunks2, end)
                 if b is end:
                 if b is end:
                     return not alen - ai and next(chunks1, end) is end
                     return not alen - ai and next(chunks1, end) is end
-                b = memoryview(b.data)
+                b = memoryview(b)
                 blen = len(b)
                 blen = len(b)
                 bi = 0
                 bi = 0
             slicelen = min(alen - ai, blen - bi)
             slicelen = min(alen - ai, blen - bi)
@@ -471,7 +481,6 @@ class Archiver:
                                   numeric_owner=args.numeric_owner, noatime=args.noatime, noctime=args.noctime,
                                   numeric_owner=args.numeric_owner, noatime=args.noatime, noctime=args.noctime,
                                   progress=args.progress,
                                   progress=args.progress,
                                   chunker_params=args.chunker_params, start=t0, start_monotonic=t0_monotonic,
                                   chunker_params=args.chunker_params, start=t0, start_monotonic=t0_monotonic,
-                                  compression=args.compression, compression_files=args.compression_files,
                                   log_json=args.log_json)
                                   log_json=args.log_json)
                 create_inner(archive, cache)
                 create_inner(archive, cache)
         else:
         else:
@@ -1325,8 +1334,7 @@ class Archiver:
         recreater = ArchiveRecreater(repository, manifest, key, cache, matcher,
         recreater = ArchiveRecreater(repository, manifest, key, cache, matcher,
                                      exclude_caches=args.exclude_caches, exclude_if_present=args.exclude_if_present,
                                      exclude_caches=args.exclude_caches, exclude_if_present=args.exclude_if_present,
                                      keep_exclude_tags=args.keep_exclude_tags, chunker_params=args.chunker_params,
                                      keep_exclude_tags=args.keep_exclude_tags, chunker_params=args.chunker_params,
-                                     compression=args.compression, compression_files=args.compression_files,
-                                     always_recompress=args.always_recompress,
+                                     compression=args.compression, always_recompress=args.always_recompress,
                                      progress=args.progress, stats=args.stats,
                                      progress=args.progress, stats=args.stats,
                                      file_status_printer=self.print_file_status,
                                      file_status_printer=self.print_file_status,
                                      checkpoint_interval=args.checkpoint_interval,
                                      checkpoint_interval=args.checkpoint_interval,
@@ -1387,7 +1395,7 @@ class Archiver:
         archive = Archive(repository, key, manifest, args.location.archive,
         archive = Archive(repository, key, manifest, args.location.archive,
                           consider_part_files=args.consider_part_files)
                           consider_part_files=args.consider_part_files)
         for i, item_id in enumerate(archive.metadata.items):
         for i, item_id in enumerate(archive.metadata.items):
-            _, data = key.decrypt(item_id, repository.get(item_id))
+            data = key.decrypt(item_id, repository.get(item_id))
             filename = '%06d_%s.items' % (i, bin_to_hex(item_id))
             filename = '%06d_%s.items' % (i, bin_to_hex(item_id))
             print('Dumping', filename)
             print('Dumping', filename)
             with open(filename, 'wb') as fd:
             with open(filename, 'wb') as fd:
@@ -1417,7 +1425,7 @@ class Archiver:
             fd.write(do_indent(prepare_dump_dict(archive_meta_orig)))
             fd.write(do_indent(prepare_dump_dict(archive_meta_orig)))
             fd.write(',\n')
             fd.write(',\n')
 
 
-            _, data = key.decrypt(archive_meta_orig[b'id'], repository.get(archive_meta_orig[b'id']))
+            data = key.decrypt(archive_meta_orig[b'id'], repository.get(archive_meta_orig[b'id']))
             archive_org_dict = msgpack.unpackb(data, object_hook=StableDict, unicode_errors='surrogateescape')
             archive_org_dict = msgpack.unpackb(data, object_hook=StableDict, unicode_errors='surrogateescape')
 
 
             fd.write('    "_meta":\n')
             fd.write('    "_meta":\n')
@@ -1428,7 +1436,7 @@ class Archiver:
             unpacker = msgpack.Unpacker(use_list=False, object_hook=StableDict)
             unpacker = msgpack.Unpacker(use_list=False, object_hook=StableDict)
             first = True
             first = True
             for item_id in archive_org_dict[b'items']:
             for item_id in archive_org_dict[b'items']:
-                _, data = key.decrypt(item_id, repository.get(item_id))
+                data = key.decrypt(item_id, repository.get(item_id))
                 unpacker.feed(data)
                 unpacker.feed(data)
                 for item in unpacker:
                 for item in unpacker:
                     item = prepare_dump_dict(item)
                     item = prepare_dump_dict(item)
@@ -1452,7 +1460,7 @@ class Archiver:
     def do_debug_dump_manifest(self, args, repository, manifest, key):
     def do_debug_dump_manifest(self, args, repository, manifest, key):
         """dump decoded repository manifest"""
         """dump decoded repository manifest"""
 
 
-        _, data = key.decrypt(None, repository.get(manifest.MANIFEST_ID))
+        data = key.decrypt(None, repository.get(manifest.MANIFEST_ID))
 
 
         meta = prepare_dump_dict(msgpack.fallback.unpackb(data, object_hook=StableDict, unicode_errors='surrogateescape'))
         meta = prepare_dump_dict(msgpack.fallback.unpackb(data, object_hook=StableDict, unicode_errors='surrogateescape'))
 
 
@@ -1476,7 +1484,7 @@ class Archiver:
             for id in result:
             for id in result:
                 cdata = repository.get(id)
                 cdata = repository.get(id)
                 give_id = id if id != Manifest.MANIFEST_ID else None
                 give_id = id if id != Manifest.MANIFEST_ID else None
-                _, data = key.decrypt(give_id, cdata)
+                data = key.decrypt(give_id, cdata)
                 filename = '%06d_%s.obj' % (i, bin_to_hex(id))
                 filename = '%06d_%s.obj' % (i, bin_to_hex(id))
                 print('Dumping', filename)
                 print('Dumping', filename)
                 with open(filename, 'wb') as fd:
                 with open(filename, 'wb') as fd:
@@ -1789,43 +1797,13 @@ class Archiver:
             For compressible data, it uses the given C[,L] compression - with C[,L]
             For compressible data, it uses the given C[,L] compression - with C[,L]
             being any valid compression specifier.
             being any valid compression specifier.
 
 
-        The decision about which compression to use is done by borg like this:
-
-        1. find a compression specifier (per file):
-           match the path/filename against all patterns in all --compression-from
-           files (if any). If a pattern matches, use the compression spec given for
-           that pattern. If no pattern matches (and also if you do not give any
-           --compression-from option), default to the compression spec given by
-           --compression. See docs/misc/compression.conf for an example config.
-
-        2. if the found compression spec is not "auto", the decision is taken:
-           use the found compression spec.
-
-        3. if the found compression spec is "auto", test compressibility of each
-           chunk using lz4.
-           If it is compressible, use the C,[L] compression spec given within the
-           "auto" specifier. If it is not compressible, use no compression.
-
         Examples::
         Examples::
 
 
             borg create --compression lz4 REPO::ARCHIVE data
             borg create --compression lz4 REPO::ARCHIVE data
             borg create --compression zlib REPO::ARCHIVE data
             borg create --compression zlib REPO::ARCHIVE data
             borg create --compression zlib,1 REPO::ARCHIVE data
             borg create --compression zlib,1 REPO::ARCHIVE data
             borg create --compression auto,lzma,6 REPO::ARCHIVE data
             borg create --compression auto,lzma,6 REPO::ARCHIVE data
-            borg create --compression-from compression.conf --compression auto,lzma ...
-
-        compression.conf has entries like::
-
-            # example config file for --compression-from option
-            #
-            # Format of non-comment / non-empty lines:
-            # <compression-spec>:<path/filename pattern>
-            # compression-spec is same format as for --compression option
-            # path/filename pattern is same format as for --exclude option
-            none:*.gz
-            none:*.zip
-            none:*.mp3
-            none:*.ogg
+            borg create --compression auto,lzma ...
 
 
         General remarks:
         General remarks:
 
 
@@ -2411,14 +2389,9 @@ class Archiver:
                                    help='specify the chunker parameters (CHUNK_MIN_EXP, CHUNK_MAX_EXP, '
                                    help='specify the chunker parameters (CHUNK_MIN_EXP, CHUNK_MAX_EXP, '
                                         'HASH_MASK_BITS, HASH_WINDOW_SIZE). default: %d,%d,%d,%d' % CHUNKER_PARAMS)
                                         'HASH_MASK_BITS, HASH_WINDOW_SIZE). default: %d,%d,%d,%d' % CHUNKER_PARAMS)
         archive_group.add_argument('-C', '--compression', dest='compression',
         archive_group.add_argument('-C', '--compression', dest='compression',
-                                   type=CompressionSpec, default=ComprSpec(name='lz4', spec=None), metavar='COMPRESSION',
+                                   type=CompressionSpec, default=CompressionSpec('lz4'), metavar='COMPRESSION',
                                    help='select compression algorithm, see the output of the '
                                    help='select compression algorithm, see the output of the '
                                         '"borg help compression" command for details.')
                                         '"borg help compression" command for details.')
-        archive_group.add_argument('--compression-from', dest='compression_files',
-                                   type=argparse.FileType('r'), action='append',
-                                   metavar='COMPRESSIONCONFIG',
-                                   help='read compression patterns from COMPRESSIONCONFIG, see the output of the '
-                                        '"borg help compression" command for details.')
 
 
         subparser.add_argument('location', metavar='ARCHIVE',
         subparser.add_argument('location', metavar='ARCHIVE',
                                type=location_validator(archive=True),
                                type=location_validator(archive=True),
@@ -2954,7 +2927,7 @@ class Archiver:
         resulting archive will only contain files from these PATHs.
         resulting archive will only contain files from these PATHs.
 
 
         Note that all paths in an archive are relative, therefore absolute patterns/paths
         Note that all paths in an archive are relative, therefore absolute patterns/paths
-        will *not* match (--exclude, --exclude-from, --compression-from, PATHs).
+        will *not* match (--exclude, --exclude-from, PATHs).
 
 
         --compression: all chunks seen will be stored using the given method.
         --compression: all chunks seen will be stored using the given method.
         Due to how Borg stores compressed size information this might display
         Due to how Borg stores compressed size information this might display
@@ -3049,11 +3022,6 @@ class Archiver:
         archive_group.add_argument('--always-recompress', dest='always_recompress', action='store_true',
         archive_group.add_argument('--always-recompress', dest='always_recompress', action='store_true',
                                    help='always recompress chunks, don\'t skip chunks already compressed with the same '
                                    help='always recompress chunks, don\'t skip chunks already compressed with the same '
                                         'algorithm.')
                                         'algorithm.')
-        archive_group.add_argument('--compression-from', dest='compression_files',
-                                   type=argparse.FileType('r'), action='append',
-                                   metavar='COMPRESSIONCONFIG',
-                                   help='read compression patterns from COMPRESSIONCONFIG, see the output of the '
-                                        '"borg help compression" command for details.')
         archive_group.add_argument('--chunker-params', dest='chunker_params',
         archive_group.add_argument('--chunker-params', dest='chunker_params',
                                    type=ChunkerParams, default=CHUNKER_PARAMS,
                                    type=ChunkerParams, default=CHUNKER_PARAMS,
                                    metavar='PARAMS',
                                    metavar='PARAMS',

+ 3 - 3
src/borg/cache.py

@@ -424,14 +424,14 @@ Chunk index:    {0.total_unique_chunks:20d} {0.total_chunks:20d}"""
 
 
         def fetch_and_build_idx(archive_id, repository, key, chunk_idx):
         def fetch_and_build_idx(archive_id, repository, key, chunk_idx):
             cdata = repository.get(archive_id)
             cdata = repository.get(archive_id)
-            _, data = key.decrypt(archive_id, cdata)
+            data = key.decrypt(archive_id, cdata)
             chunk_idx.add(archive_id, 1, len(data), len(cdata))
             chunk_idx.add(archive_id, 1, len(data), len(cdata))
             archive = ArchiveItem(internal_dict=msgpack.unpackb(data))
             archive = ArchiveItem(internal_dict=msgpack.unpackb(data))
             if archive.version != 1:
             if archive.version != 1:
                 raise Exception('Unknown archive metadata version')
                 raise Exception('Unknown archive metadata version')
             unpacker = msgpack.Unpacker()
             unpacker = msgpack.Unpacker()
             for item_id, chunk in zip(archive.items, repository.get_many(archive.items)):
             for item_id, chunk in zip(archive.items, repository.get_many(archive.items)):
-                _, data = key.decrypt(item_id, chunk)
+                data = key.decrypt(item_id, chunk)
                 chunk_idx.add(item_id, 1, len(data), len(chunk))
                 chunk_idx.add(item_id, 1, len(data), len(chunk))
                 unpacker.feed(data)
                 unpacker.feed(data)
                 for item in unpacker:
                 for item in unpacker:
@@ -527,7 +527,7 @@ Chunk index:    {0.total_unique_chunks:20d} {0.total_chunks:20d}"""
     def add_chunk(self, id, chunk, stats, overwrite=False, wait=True):
     def add_chunk(self, id, chunk, stats, overwrite=False, wait=True):
         if not self.txn_active:
         if not self.txn_active:
             self.begin_txn()
             self.begin_txn()
-        size = len(chunk.data)
+        size = len(chunk)
         refcount = self.seen_chunk(id, size)
         refcount = self.seen_chunk(id, size)
         if refcount and not overwrite:
         if refcount and not overwrite:
             return self.chunk_incref(id, stats)
             return self.chunk_incref(id, stats)

+ 133 - 1
src/borg/compress.pyx

@@ -1,4 +1,22 @@
+"""
+borg.compress
+=============
+
+Compression is applied to chunks after ID hashing (so the ID is a direct function of the
+plain chunk, compression is irrelevant to it), and of course before encryption.
+
+The "auto" mode (e.g. --compression auto,lzma,4) is implemented as a meta Compressor,
+meaning that Auto acts like a Compressor, but defers actual work to others (namely
+LZ4 as a heuristic whether compression is worth it, and the specified Compressor
+for the actual compression).
+
+Decompression is normally handled through Compressor.decompress which will detect
+which compressor has been used to compress the data and dispatch to the correct
+decompressor.
+"""
+
 import zlib
 import zlib
+
 try:
 try:
     import lzma
     import lzma
 except ImportError:
 except ImportError:
@@ -6,7 +24,7 @@ except ImportError:
 
 
 from .helpers import Buffer, DecompressionError
 from .helpers import Buffer, DecompressionError
 
 
-API_VERSION = '1.1_02'
+API_VERSION = '1.1_03'
 
 
 cdef extern from "lz4.h":
 cdef extern from "lz4.h":
     int LZ4_compress_limitedOutput(const char* source, char* dest, int inputSize, int maxOutputSize) nogil
     int LZ4_compress_limitedOutput(const char* source, char* dest, int inputSize, int maxOutputSize) nogil
@@ -34,11 +52,34 @@ cdef class CompressorBase:
     def __init__(self, **kwargs):
     def __init__(self, **kwargs):
         pass
         pass
 
 
+    def decide(self, data):
+        """
+        Return which compressor will perform the actual compression for *data*.
+
+        This exists for a very specific case: If borg recreate is instructed to recompress
+        using Auto compression it needs to determine the _actual_ target compression of a chunk
+        in order to detect whether it should be recompressed.
+        
+        For all Compressors that are not Auto this always returns *self*.
+        """
+        return self
+
     def compress(self, data):
     def compress(self, data):
+        """
+        Compress *data* (bytes) and return bytes result. Prepend the ID bytes of this compressor,
+        which is needed so that the correct decompressor can be used for decompression.
+        """
         # add ID bytes
         # add ID bytes
         return self.ID + data
         return self.ID + data
 
 
     def decompress(self, data):
     def decompress(self, data):
+        """
+        Decompress *data* (bytes) and return bytes result. The leading Compressor ID
+        bytes need to be present.
+
+        Only handles input generated by _this_ Compressor - for a general purpose
+        decompression method see *Compressor.decompress*.
+        """
         # strip ID bytes
         # strip ID bytes
         return data[2:]
         return data[2:]
 
 
@@ -179,12 +220,64 @@ class ZLIB(CompressorBase):
             raise DecompressionError(str(e)) from None
             raise DecompressionError(str(e)) from None
 
 
 
 
+class Auto(CompressorBase):
+    """
+    Meta-Compressor that decides which compression to use based on LZ4's ratio.
+
+    As a meta-Compressor the actual compression is deferred to other Compressors,
+    therefore this Compressor has no ID, no detect() and no decompress().
+    """
+
+    ID = None
+    name = 'auto'
+
+    def __init__(self, compressor):
+        super().__init__()
+        self.compressor = compressor
+        self.lz4 = get_compressor('lz4')
+        self.none = get_compressor('none')
+
+    def _decide(self, data):
+        """
+        Decides what to do with *data*. Returns (compressor, lz4_data).
+
+        *lz4_data* is the LZ4 result if *compressor* is LZ4 as well, otherwise it is None.
+        """
+        lz4_data = self.lz4.compress(data)
+        ratio = len(lz4_data) / len(data)
+        if ratio < 0.97:
+            return self.compressor, None
+        elif ratio < 1:
+            return self.lz4, lz4_data
+        else:
+            return self.none, None
+
+    def decide(self, data):
+        return self._decide(data)[0]
+
+    def compress(self, data):
+        compressor, lz4_data = self._decide(data)
+        if lz4_data is None:
+            return compressor.compress(data)
+        else:
+            return lz4_data
+
+    def decompress(self, data):
+        raise NotImplementedError
+
+    def detect(cls, data):
+        raise NotImplementedError
+
+
+# Maps valid compressor names to their class
 COMPRESSOR_TABLE = {
 COMPRESSOR_TABLE = {
     CNONE.name: CNONE,
     CNONE.name: CNONE,
     LZ4.name: LZ4,
     LZ4.name: LZ4,
     ZLIB.name: ZLIB,
     ZLIB.name: ZLIB,
     LZMA.name: LZMA,
     LZMA.name: LZMA,
+    Auto.name: Auto,
 }
 }
+# List of possible compression types. Does not include Auto, since it is a meta-Compressor.
 COMPRESSOR_LIST = [LZ4, CNONE, ZLIB, LZMA, ]  # check fast stuff first
 COMPRESSOR_LIST = [LZ4, CNONE, ZLIB, LZMA, ]  # check fast stuff first
 
 
 def get_compressor(name, **kwargs):
 def get_compressor(name, **kwargs):
@@ -216,3 +309,42 @@ class Compressor:
                 return cls
                 return cls
         else:
         else:
             raise ValueError('No decompressor for this data found: %r.', data[:2])
             raise ValueError('No decompressor for this data found: %r.', data[:2])
+
+
+class CompressionSpec:
+    def __init__(self, s):
+        values = s.split(',')
+        count = len(values)
+        if count < 1:
+            raise ValueError
+        # --compression algo[,level]
+        self.name = values[0]
+        if self.name in ('none', 'lz4', ):
+            return
+        elif self.name in ('zlib', 'lzma', ):
+            if count < 2:
+                level = 6  # default compression level in py stdlib
+            elif count == 2:
+                level = int(values[1])
+                if not 0 <= level <= 9:
+                    raise ValueError
+            else:
+                raise ValueError
+            self.level = level
+        elif self.name == 'auto':
+            if 2 <= count <= 3:
+                compression = ','.join(values[1:])
+            else:
+                raise ValueError
+            self.inner = CompressionSpec(compression)
+        else:
+            raise ValueError
+
+    @property
+    def compressor(self):
+        if self.name in ('none', 'lz4', ):
+            return get_compressor(self.name)
+        elif self.name in ('zlib', 'lzma', ):
+            return get_compressor(self.name, level=self.level)
+        elif self.name == 'auto':
+            return get_compressor(self.name, compressor=self.inner.compressor)

+ 2 - 2
src/borg/fuse.py

@@ -144,7 +144,7 @@ class FuseOperations(llfuse.Operations):
         self.file_versions = {}  # for versions mode: original path -> version
         self.file_versions = {}  # for versions mode: original path -> version
         unpacker = msgpack.Unpacker()
         unpacker = msgpack.Unpacker()
         for key, chunk in zip(archive.metadata.items, self.repository.get_many(archive.metadata.items)):
         for key, chunk in zip(archive.metadata.items, self.repository.get_many(archive.metadata.items)):
-            _, data = self.key.decrypt(key, chunk)
+            data = self.key.decrypt(key, chunk)
             unpacker.feed(data)
             unpacker.feed(data)
             for item in unpacker:
             for item in unpacker:
                 item = Item(internal_dict=item)
                 item = Item(internal_dict=item)
@@ -340,7 +340,7 @@ class FuseOperations(llfuse.Operations):
                     # evict fully read chunk from cache
                     # evict fully read chunk from cache
                     del self.data_cache[id]
                     del self.data_cache[id]
             else:
             else:
-                _, data = self.key.decrypt(id, self.repository.get(id))
+                data = self.key.decrypt(id, self.repository.get(id))
                 if offset + n < len(data):
                 if offset + n < len(data):
                     # chunk was only partially read, cache it
                     # chunk was only partially read, cache it
                     self.data_cache[id] = data
                     self.data_cache[id] = data

+ 5 - 110
src/borg/helpers.py

@@ -44,13 +44,6 @@ from . import hashindex
 from . import shellpattern
 from . import shellpattern
 from .constants import *  # NOQA
 from .constants import *  # NOQA
 
 
-# meta dict, data bytes
-_Chunk = namedtuple('_Chunk', 'meta data')
-
-
-def Chunk(data, **meta):
-    return _Chunk(meta, data)
-
 
 
 '''
 '''
 The global exit_code variable is used so that modules other than archiver can increase the program exit code if a
 The global exit_code variable is used so that modules other than archiver can increase the program exit code if a
@@ -123,7 +116,7 @@ def check_extension_modules():
         raise ExtensionModuleError
         raise ExtensionModuleError
     if chunker.API_VERSION != '1.1_01':
     if chunker.API_VERSION != '1.1_01':
         raise ExtensionModuleError
         raise ExtensionModuleError
-    if compress.API_VERSION != '1.1_02':
+    if compress.API_VERSION != '1.1_03':
         raise ExtensionModuleError
         raise ExtensionModuleError
     if crypto.API_VERSION != '1.1_01':
     if crypto.API_VERSION != '1.1_01':
         raise ExtensionModuleError
         raise ExtensionModuleError
@@ -247,7 +240,7 @@ class Manifest:
         if not key:
         if not key:
             key = key_factory(repository, cdata)
             key = key_factory(repository, cdata)
         manifest = cls(key, repository)
         manifest = cls(key, repository)
-        data = key.decrypt(None, cdata).data
+        data = key.decrypt(None, cdata)
         manifest_dict, manifest.tam_verified = key.unpack_and_verify_manifest(data, force_tam_not_required=force_tam_not_required)
         manifest_dict, manifest.tam_verified = key.unpack_and_verify_manifest(data, force_tam_not_required=force_tam_not_required)
         m = ManifestItem(internal_dict=manifest_dict)
         m = ManifestItem(internal_dict=manifest_dict)
         manifest.id = key.id_hash(data)
         manifest.id = key.id_hash(data)
@@ -292,7 +285,7 @@ class Manifest:
         self.tam_verified = True
         self.tam_verified = True
         data = self.key.pack_and_authenticate_metadata(manifest.as_dict())
         data = self.key.pack_and_authenticate_metadata(manifest.as_dict())
         self.id = self.key.id_hash(data)
         self.id = self.key.id_hash(data)
-        self.repository.put(self.MANIFEST_ID, self.key.encrypt(Chunk(data, compression={'name': 'none'})))
+        self.repository.put(self.MANIFEST_ID, self.key.encrypt(data))
 
 
 
 
 def prune_within(archives, within):
 def prune_within(archives, within):
@@ -726,37 +719,6 @@ def ChunkerParams(s):
     return int(chunk_min), int(chunk_max), int(chunk_mask), int(window_size)
     return int(chunk_min), int(chunk_max), int(chunk_mask), int(window_size)
 
 
 
 
-ComprSpec = namedtuple('ComprSpec', ('name', 'spec'))
-
-
-def CompressionSpec(s):
-    values = s.split(',')
-    count = len(values)
-    if count < 1:
-        raise ValueError
-    # --compression algo[,level]
-    name = values[0]
-    if name in ('none', 'lz4', ):
-        return ComprSpec(name=name, spec=None)
-    if name in ('zlib', 'lzma', ):
-        if count < 2:
-            level = 6  # default compression level in py stdlib
-        elif count == 2:
-            level = int(values[1])
-            if not 0 <= level <= 9:
-                raise ValueError
-        else:
-            raise ValueError
-        return ComprSpec(name=name, spec=level)
-    if name == 'auto':
-        if 2 <= count <= 3:
-            compression = ','.join(values[1:])
-        else:
-            raise ValueError
-        return ComprSpec(name=name, spec=CompressionSpec(compression))
-    raise ValueError
-
-
 def dir_is_cachedir(path):
 def dir_is_cachedir(path):
     """Determines whether the specified path is a cache directory (and
     """Determines whether the specified path is a cache directory (and
     therefore should potentially be excluded from the backup) according to
     therefore should potentially be excluded from the backup) according to
@@ -1940,7 +1902,7 @@ class ItemFormatter(BaseFormatter):
         if 'chunks' not in item:
         if 'chunks' not in item:
             return ""
             return ""
         hash = hashlib.new(hash_function)
         hash = hashlib.new(hash_function)
-        for _, data in self.archive.pipeline.fetch_many([c.id for c in item.chunks]):
+        for data in self.archive.pipeline.fetch_many([c.id for c in item.chunks]):
             hash.update(data)
             hash.update(data)
         return hash.hexdigest()
         return hash.hexdigest()
 
 
@@ -1965,7 +1927,7 @@ class ChunkIteratorFileWrapper:
         if not remaining:
         if not remaining:
             try:
             try:
                 chunk = next(self.chunk_iterator)
                 chunk = next(self.chunk_iterator)
-                self.chunk = memoryview(chunk.data)
+                self.chunk = memoryview(chunk)
             except StopIteration:
             except StopIteration:
                 self.exhausted = True
                 self.exhausted = True
                 return 0  # EOF
                 return 0  # EOF
@@ -2127,73 +2089,6 @@ def clean_lines(lines, lstrip=None, rstrip=None, remove_empty=True, remove_comme
         yield line
         yield line
 
 
 
 
-class CompressionDecider1:
-    def __init__(self, compression, compression_files):
-        """
-        Initialize a CompressionDecider instance (and read config files, if needed).
-
-        :param compression: default CompressionSpec (e.g. from --compression option)
-        :param compression_files: list of compression config files (e.g. from --compression-from) or
-                                  a list of other line iterators
-        """
-        self.compression = compression
-        if not compression_files:
-            self.matcher = None
-        else:
-            self.matcher = PatternMatcher(fallback=compression)
-            for file in compression_files:
-                try:
-                    for line in clean_lines(file):
-                        try:
-                            compr_spec, fn_pattern = line.split(':', 1)
-                        except:
-                            continue
-                        self.matcher.add([parse_pattern(fn_pattern)], CompressionSpec(compr_spec))
-                finally:
-                    if hasattr(file, 'close'):
-                        file.close()
-
-    def decide(self, path):
-        if self.matcher is not None:
-            return self.matcher.match(path)
-        return self.compression
-
-
-class CompressionDecider2:
-    logger = create_logger('borg.debug.file-compression')
-
-    def __init__(self, compression):
-        self.compression = compression
-
-    def decide(self, chunk):
-        # nothing fancy here yet: we either use what the metadata says or the default
-        # later, we can decide based on the chunk data also.
-        # if we compress the data here to decide, we can even update the chunk data
-        # and modify the metadata as desired.
-        compr_spec = chunk.meta.get('compress', self.compression)
-        if compr_spec.name == 'auto':
-            # we did not decide yet, use heuristic:
-            compr_spec, chunk = self.heuristic_lz4(compr_spec, chunk)
-        return compr_spec, chunk
-
-    def heuristic_lz4(self, compr_args, chunk):
-        from .compress import get_compressor
-        meta, data = chunk
-        lz4 = get_compressor('lz4')
-        cdata = lz4.compress(data)
-        data_len = len(data)
-        cdata_len = len(cdata)
-        if cdata_len < 0.97 * data_len:
-            compr_spec = compr_args.spec
-        else:
-            # uncompressible - we could have a special "uncompressible compressor"
-            # that marks such data as uncompressible via compression-type metadata.
-            compr_spec = CompressionSpec('none')
-        self.logger.debug("len(data) == %d, len(lz4(data)) == %d, ratio == %.3f, choosing %s", data_len, cdata_len, cdata_len/data_len, compr_spec)
-        meta['compress'] = compr_spec
-        return compr_spec, Chunk(data, **meta)
-
-
 class ErrorIgnoringTextIOWrapper(io.TextIOWrapper):
 class ErrorIgnoringTextIOWrapper(io.TextIOWrapper):
     def read(self, n):
     def read(self, n):
         if not self.closed:
         if not self.closed:

+ 22 - 28
src/borg/key.py

@@ -13,14 +13,13 @@ from .logger import create_logger
 logger = create_logger()
 logger = create_logger()
 
 
 from .constants import *  # NOQA
 from .constants import *  # NOQA
-from .compress import Compressor, get_compressor
+from .compress import Compressor
 from .crypto import AES, bytes_to_long, bytes_to_int, num_aes_blocks, hmac_sha256, blake2b_256, hkdf_hmac_sha512
 from .crypto import AES, bytes_to_long, bytes_to_int, num_aes_blocks, hmac_sha256, blake2b_256, hkdf_hmac_sha512
-from .helpers import Chunk, StableDict
+from .helpers import StableDict
 from .helpers import Error, IntegrityError
 from .helpers import Error, IntegrityError
 from .helpers import yes
 from .helpers import yes
 from .helpers import get_keys_dir, get_security_dir
 from .helpers import get_keys_dir, get_security_dir
 from .helpers import bin_to_hex
 from .helpers import bin_to_hex
-from .helpers import CompressionDecider2, CompressionSpec
 from .item import Key, EncryptedKey
 from .item import Key, EncryptedKey
 from .platform import SaveFile
 from .platform import SaveFile
 from .nonces import NonceManager
 from .nonces import NonceManager
@@ -143,21 +142,16 @@ class KeyBase:
         self.TYPE_STR = bytes([self.TYPE])
         self.TYPE_STR = bytes([self.TYPE])
         self.repository = repository
         self.repository = repository
         self.target = None  # key location file path / repo obj
         self.target = None  # key location file path / repo obj
-        self.compression_decider2 = CompressionDecider2(CompressionSpec('none'))
-        self.compressor = Compressor('none')  # for decompression
+        # Some commands write new chunks (e.g. rename) but don't take a --compression argument. This duplicates
+        # the default used by those commands who do take a --compression argument.
+        self.compressor = Compressor('lz4')
+        self.decompress = self.compressor.decompress
         self.tam_required = True
         self.tam_required = True
 
 
     def id_hash(self, data):
     def id_hash(self, data):
         """Return HMAC hash using the "id" HMAC key
         """Return HMAC hash using the "id" HMAC key
         """
         """
 
 
-    def compress(self, chunk):
-        compr_args, chunk = self.compression_decider2.decide(chunk)
-        compressor = Compressor(name=compr_args.name, level=compr_args.spec)
-        meta, data = chunk
-        data = compressor.compress(data)
-        return Chunk(data, **meta)
-
     def encrypt(self, chunk):
     def encrypt(self, chunk):
         pass
         pass
 
 
@@ -258,8 +252,8 @@ class PlaintextKey(KeyBase):
         return sha256(data).digest()
         return sha256(data).digest()
 
 
     def encrypt(self, chunk):
     def encrypt(self, chunk):
-        chunk = self.compress(chunk)
-        return b''.join([self.TYPE_STR, chunk.data])
+        data = self.compressor.compress(chunk)
+        return b''.join([self.TYPE_STR, data])
 
 
     def decrypt(self, id, data, decompress=True):
     def decrypt(self, id, data, decompress=True):
         if data[0] != self.TYPE:
         if data[0] != self.TYPE:
@@ -267,10 +261,10 @@ class PlaintextKey(KeyBase):
             raise IntegrityError('Chunk %s: Invalid encryption envelope' % id_str)
             raise IntegrityError('Chunk %s: Invalid encryption envelope' % id_str)
         payload = memoryview(data)[1:]
         payload = memoryview(data)[1:]
         if not decompress:
         if not decompress:
-            return Chunk(payload)
-        data = self.compressor.decompress(payload)
+            return payload
+        data = self.decompress(payload)
         self.assert_id(id, data)
         self.assert_id(id, data)
-        return Chunk(data)
+        return data
 
 
     def _tam_key(self, salt, context):
     def _tam_key(self, salt, context):
         return salt + context
         return salt + context
@@ -336,10 +330,10 @@ class AESKeyBase(KeyBase):
     MAC = hmac_sha256
     MAC = hmac_sha256
 
 
     def encrypt(self, chunk):
     def encrypt(self, chunk):
-        chunk = self.compress(chunk)
-        self.nonce_manager.ensure_reservation(num_aes_blocks(len(chunk.data)))
+        data = self.compressor.compress(chunk)
+        self.nonce_manager.ensure_reservation(num_aes_blocks(len(data)))
         self.enc_cipher.reset()
         self.enc_cipher.reset()
-        data = b''.join((self.enc_cipher.iv[8:], self.enc_cipher.encrypt(chunk.data)))
+        data = b''.join((self.enc_cipher.iv[8:], self.enc_cipher.encrypt(data)))
         assert (self.MAC is blake2b_256 and len(self.enc_hmac_key) == 128 or
         assert (self.MAC is blake2b_256 and len(self.enc_hmac_key) == 128 or
                 self.MAC is hmac_sha256 and len(self.enc_hmac_key) == 32)
                 self.MAC is hmac_sha256 and len(self.enc_hmac_key) == 32)
         hmac = self.MAC(self.enc_hmac_key, data)
         hmac = self.MAC(self.enc_hmac_key, data)
@@ -361,10 +355,10 @@ class AESKeyBase(KeyBase):
         self.dec_cipher.reset(iv=PREFIX + data[33:41])
         self.dec_cipher.reset(iv=PREFIX + data[33:41])
         payload = self.dec_cipher.decrypt(data_view[41:])
         payload = self.dec_cipher.decrypt(data_view[41:])
         if not decompress:
         if not decompress:
-            return Chunk(payload)
-        data = self.compressor.decompress(payload)
+            return payload
+        data = self.decompress(payload)
         self.assert_id(id, data)
         self.assert_id(id, data)
-        return Chunk(data)
+        return data
 
 
     def extract_nonce(self, payload):
     def extract_nonce(self, payload):
         if not (payload[0] == self.TYPE or
         if not (payload[0] == self.TYPE or
@@ -748,18 +742,18 @@ class AuthenticatedKey(ID_BLAKE2b_256, RepoKey):
     STORAGE = KeyBlobStorage.REPO
     STORAGE = KeyBlobStorage.REPO
 
 
     def encrypt(self, chunk):
     def encrypt(self, chunk):
-        chunk = self.compress(chunk)
-        return b''.join([self.TYPE_STR, chunk.data])
+        data = self.compressor.compress(chunk)
+        return b''.join([self.TYPE_STR, data])
 
 
     def decrypt(self, id, data, decompress=True):
     def decrypt(self, id, data, decompress=True):
         if data[0] != self.TYPE:
         if data[0] != self.TYPE:
             raise IntegrityError('Chunk %s: Invalid envelope' % bin_to_hex(id))
             raise IntegrityError('Chunk %s: Invalid envelope' % bin_to_hex(id))
         payload = memoryview(data)[1:]
         payload = memoryview(data)[1:]
         if not decompress:
         if not decompress:
-            return Chunk(payload)
-        data = self.compressor.decompress(payload)
+            return payload
+        data = self.decompress(payload)
         self.assert_id(id, data)
         self.assert_id(id, data)
-        return Chunk(data)
+        return data
 
 
 
 
 AVAILABLE_KEY_TYPES = (
 AVAILABLE_KEY_TYPES = (

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

@@ -72,8 +72,8 @@ class MockCache:
         self.repository = self.MockRepo()
         self.repository = self.MockRepo()
 
 
     def add_chunk(self, id, chunk, stats=None, wait=True):
     def add_chunk(self, id, chunk, stats=None, wait=True):
-        self.objects[id] = chunk.data
-        return id, len(chunk.data), len(chunk.data)
+        self.objects[id] = chunk
+        return id, len(chunk), len(chunk)
 
 
 
 
 class ArchiveTimestampTestCase(BaseTestCase):
 class ArchiveTimestampTestCase(BaseTestCase):

+ 10 - 10
src/borg/testsuite/archiver.py

@@ -34,7 +34,7 @@ from ..cache import Cache
 from ..constants import *  # NOQA
 from ..constants import *  # NOQA
 from ..crypto import bytes_to_long, num_aes_blocks
 from ..crypto import bytes_to_long, num_aes_blocks
 from ..helpers import PatternMatcher, parse_pattern, Location, get_security_dir
 from ..helpers import PatternMatcher, parse_pattern, Location, get_security_dir
-from ..helpers import Chunk, Manifest
+from ..helpers import Manifest
 from ..helpers import EXIT_SUCCESS, EXIT_WARNING, EXIT_ERROR
 from ..helpers import EXIT_SUCCESS, EXIT_WARNING, EXIT_ERROR
 from ..helpers import bin_to_hex
 from ..helpers import bin_to_hex
 from ..item import Item
 from ..item import Item
@@ -2449,7 +2449,7 @@ class ArchiverCheckTestCase(ArchiverTestCaseBase):
                 'version': 1,
                 'version': 1,
             })
             })
             archive_id = key.id_hash(archive)
             archive_id = key.id_hash(archive)
-            repository.put(archive_id, key.encrypt(Chunk(archive)))
+            repository.put(archive_id, key.encrypt(archive))
             repository.commit()
             repository.commit()
         self.cmd('check', self.repository_location, exit_code=1)
         self.cmd('check', self.repository_location, exit_code=1)
         self.cmd('check', '--repair', self.repository_location, exit_code=0)
         self.cmd('check', '--repair', self.repository_location, exit_code=0)
@@ -2537,12 +2537,12 @@ class ManifestAuthenticationTest(ArchiverTestCaseBase):
     def spoof_manifest(self, repository):
     def spoof_manifest(self, repository):
         with repository:
         with repository:
             _, key = Manifest.load(repository)
             _, key = Manifest.load(repository)
-            repository.put(Manifest.MANIFEST_ID, key.encrypt(Chunk(msgpack.packb({
+            repository.put(Manifest.MANIFEST_ID, key.encrypt(msgpack.packb({
                 'version': 1,
                 'version': 1,
                 'archives': {},
                 'archives': {},
                 'config': {},
                 'config': {},
                 'timestamp': (datetime.utcnow() + timedelta(days=1)).isoformat(),
                 'timestamp': (datetime.utcnow() + timedelta(days=1)).isoformat(),
-            }))))
+            })))
             repository.commit()
             repository.commit()
 
 
     def test_fresh_init_tam_required(self):
     def test_fresh_init_tam_required(self):
@@ -2550,11 +2550,11 @@ class ManifestAuthenticationTest(ArchiverTestCaseBase):
         repository = Repository(self.repository_path, exclusive=True)
         repository = Repository(self.repository_path, exclusive=True)
         with repository:
         with repository:
             manifest, key = Manifest.load(repository)
             manifest, key = Manifest.load(repository)
-            repository.put(Manifest.MANIFEST_ID, key.encrypt(Chunk(msgpack.packb({
+            repository.put(Manifest.MANIFEST_ID, key.encrypt(msgpack.packb({
                 'version': 1,
                 'version': 1,
                 'archives': {},
                 'archives': {},
                 'timestamp': (datetime.utcnow() + timedelta(days=1)).isoformat(),
                 'timestamp': (datetime.utcnow() + timedelta(days=1)).isoformat(),
-            }))))
+            })))
             repository.commit()
             repository.commit()
 
 
         with pytest.raises(TAMRequiredError):
         with pytest.raises(TAMRequiredError):
@@ -2570,9 +2570,9 @@ class ManifestAuthenticationTest(ArchiverTestCaseBase):
             key.tam_required = False
             key.tam_required = False
             key.change_passphrase(key._passphrase)
             key.change_passphrase(key._passphrase)
 
 
-            manifest = msgpack.unpackb(key.decrypt(None, repository.get(Manifest.MANIFEST_ID)).data)
+            manifest = msgpack.unpackb(key.decrypt(None, repository.get(Manifest.MANIFEST_ID)))
             del manifest[b'tam']
             del manifest[b'tam']
-            repository.put(Manifest.MANIFEST_ID, key.encrypt(Chunk(msgpack.packb(manifest))))
+            repository.put(Manifest.MANIFEST_ID, key.encrypt(msgpack.packb(manifest)))
             repository.commit()
             repository.commit()
         output = self.cmd('list', '--debug', self.repository_location)
         output = self.cmd('list', '--debug', self.repository_location)
         assert 'archive1234' in output
         assert 'archive1234' in output
@@ -2844,8 +2844,8 @@ def test_get_args():
 
 
 def test_compare_chunk_contents():
 def test_compare_chunk_contents():
     def ccc(a, b):
     def ccc(a, b):
-        chunks_a = [Chunk(data) for data in a]
-        chunks_b = [Chunk(data) for data in b]
+        chunks_a = [data for data in a]
+        chunks_b = [data for data in b]
         compare1 = Archiver.compare_chunk_contents(iter(chunks_a), iter(chunks_b))
         compare1 = Archiver.compare_chunk_contents(iter(chunks_a), iter(chunks_b))
         compare2 = Archiver.compare_chunk_contents(iter(chunks_b), iter(chunks_a))
         compare2 = Archiver.compare_chunk_contents(iter(chunks_b), iter(chunks_a))
         assert compare1 == compare2
         assert compare1 == compare2

+ 46 - 1
src/borg/testsuite/compress.py

@@ -7,7 +7,7 @@ except ImportError:
 
 
 import pytest
 import pytest
 
 
-from ..compress import get_compressor, Compressor, CNONE, ZLIB, LZ4
+from ..compress import get_compressor, Compressor, CompressionSpec, CNONE, ZLIB, LZ4, LZMA, Auto
 
 
 
 
 buffer = bytes(2**16)
 buffer = bytes(2**16)
@@ -107,3 +107,48 @@ def test_compressor():
     for params in params_list:
     for params in params_list:
         c = Compressor(**params)
         c = Compressor(**params)
         assert data == c.decompress(c.compress(data))
         assert data == c.decompress(c.compress(data))
+
+
+def test_auto():
+    compressor = CompressionSpec('auto,zlib,9').compressor
+
+    compressed = compressor.compress(bytes(500))
+    assert Compressor.detect(compressed) == ZLIB
+
+    compressed = compressor.compress(b'\x00\xb8\xa3\xa2-O\xe1i\xb6\x12\x03\xc21\xf3\x8a\xf78\\\x01\xa5b\x07\x95\xbeE\xf8\xa3\x9ahm\xb1~')
+    assert Compressor.detect(compressed) == CNONE
+
+
+def test_compression_specs():
+    with pytest.raises(ValueError):
+        CompressionSpec('')
+
+    assert isinstance(CompressionSpec('none').compressor, CNONE)
+    assert isinstance(CompressionSpec('lz4').compressor, LZ4)
+
+    zlib = CompressionSpec('zlib').compressor
+    assert isinstance(zlib, ZLIB)
+    assert zlib.level == 6
+    zlib = CompressionSpec('zlib,0').compressor
+    assert isinstance(zlib, ZLIB)
+    assert zlib.level == 0
+    zlib = CompressionSpec('zlib,9').compressor
+    assert isinstance(zlib, ZLIB)
+    assert zlib.level == 9
+    with pytest.raises(ValueError):
+        CompressionSpec('zlib,9,invalid')
+
+    lzma = CompressionSpec('lzma').compressor
+    assert isinstance(lzma, LZMA)
+    assert lzma.level == 6
+    lzma = CompressionSpec('lzma,0').compressor
+    assert isinstance(lzma, LZMA)
+    assert lzma.level == 0
+    lzma = CompressionSpec('lzma,9').compressor
+    assert isinstance(lzma, LZMA)
+    assert lzma.level == 9
+
+    with pytest.raises(ValueError):
+        CompressionSpec('lzma,9,invalid')
+    with pytest.raises(ValueError):
+        CompressionSpec('invalid')

+ 2 - 54
src/borg/testsuite/helpers.py

@@ -21,10 +21,9 @@ from ..helpers import get_cache_dir, get_keys_dir, get_security_dir
 from ..helpers import is_slow_msgpack
 from ..helpers import is_slow_msgpack
 from ..helpers import yes, TRUISH, FALSISH, DEFAULTISH
 from ..helpers import yes, TRUISH, FALSISH, DEFAULTISH
 from ..helpers import StableDict, int_to_bigint, bigint_to_int, bin_to_hex
 from ..helpers import StableDict, int_to_bigint, bigint_to_int, bin_to_hex
-from ..helpers import parse_timestamp, ChunkIteratorFileWrapper, ChunkerParams, Chunk
+from ..helpers import parse_timestamp, ChunkIteratorFileWrapper, ChunkerParams
 from ..helpers import ProgressIndicatorPercent, ProgressIndicatorEndless
 from ..helpers import ProgressIndicatorPercent, ProgressIndicatorEndless
 from ..helpers import load_exclude_file, load_pattern_file
 from ..helpers import load_exclude_file, load_pattern_file
-from ..helpers import CompressionSpec, ComprSpec, CompressionDecider1, CompressionDecider2
 from ..helpers import parse_pattern, PatternMatcher
 from ..helpers import parse_pattern, PatternMatcher
 from ..helpers import PathFullPattern, PathPrefixPattern, FnmatchPattern, ShellPattern, RegexPattern
 from ..helpers import PathFullPattern, PathPrefixPattern, FnmatchPattern, ShellPattern, RegexPattern
 from ..helpers import swidth_slice
 from ..helpers import swidth_slice
@@ -698,25 +697,6 @@ def test_pattern_matcher():
     assert PatternMatcher(fallback="hey!").fallback == "hey!"
     assert PatternMatcher(fallback="hey!").fallback == "hey!"
 
 
 
 
-def test_compression_specs():
-    with pytest.raises(ValueError):
-        CompressionSpec('')
-    assert CompressionSpec('none') == ComprSpec(name='none', spec=None)
-    assert CompressionSpec('lz4') == ComprSpec(name='lz4', spec=None)
-    assert CompressionSpec('zlib') == ComprSpec(name='zlib', spec=6)
-    assert CompressionSpec('zlib,0') == ComprSpec(name='zlib', spec=0)
-    assert CompressionSpec('zlib,9') == ComprSpec(name='zlib', spec=9)
-    with pytest.raises(ValueError):
-        CompressionSpec('zlib,9,invalid')
-    assert CompressionSpec('lzma') == ComprSpec(name='lzma', spec=6)
-    assert CompressionSpec('lzma,0') == ComprSpec(name='lzma', spec=0)
-    assert CompressionSpec('lzma,9') == ComprSpec(name='lzma', spec=9)
-    with pytest.raises(ValueError):
-        CompressionSpec('lzma,9,invalid')
-    with pytest.raises(ValueError):
-        CompressionSpec('invalid')
-
-
 def test_chunkerparams():
 def test_chunkerparams():
     assert ChunkerParams('19,23,21,4095') == (19, 23, 21, 4095)
     assert ChunkerParams('19,23,21,4095') == (19, 23, 21, 4095)
     assert ChunkerParams('10,23,16,4095') == (10, 23, 16, 4095)
     assert ChunkerParams('10,23,16,4095') == (10, 23, 16, 4095)
@@ -1178,7 +1158,7 @@ def test_partial_format():
 
 
 
 
 def test_chunk_file_wrapper():
 def test_chunk_file_wrapper():
-    cfw = ChunkIteratorFileWrapper(iter([Chunk(b'abc'), Chunk(b'def')]))
+    cfw = ChunkIteratorFileWrapper(iter([b'abc', b'def']))
     assert cfw.read(2) == b'ab'
     assert cfw.read(2) == b'ab'
     assert cfw.read(50) == b'cdef'
     assert cfw.read(50) == b'cdef'
     assert cfw.exhausted
     assert cfw.exhausted
@@ -1220,38 +1200,6 @@ data2
     assert list(clean_lines(conf, remove_comments=False)) == ['#comment', 'data1 #data1', 'data2', 'data3', ]
     assert list(clean_lines(conf, remove_comments=False)) == ['#comment', 'data1 #data1', 'data2', 'data3', ]
 
 
 
 
-def test_compression_decider1():
-    default = CompressionSpec('zlib')
-    conf = """
-# use super-fast lz4 compression on huge VM files in this path:
-lz4:/srv/vm_disks
-
-# jpeg or zip files do not compress:
-none:*.jpeg
-none:*.zip
-""".splitlines()
-
-    cd = CompressionDecider1(default, [])  # no conf, always use default
-    assert cd.decide('/srv/vm_disks/linux').name == 'zlib'
-    assert cd.decide('test.zip').name == 'zlib'
-    assert cd.decide('test').name == 'zlib'
-
-    cd = CompressionDecider1(default, [conf, ])
-    assert cd.decide('/srv/vm_disks/linux').name == 'lz4'
-    assert cd.decide('test.zip').name == 'none'
-    assert cd.decide('test').name == 'zlib'  # no match in conf, use default
-
-
-def test_compression_decider2():
-    default = CompressionSpec('zlib')
-
-    cd = CompressionDecider2(default)
-    compr_spec, chunk = cd.decide(Chunk(None))
-    assert compr_spec.name == 'zlib'
-    compr_spec, chunk = cd.decide(Chunk(None, compress=CompressionSpec('lzma')))
-    assert compr_spec.name == 'lzma'
-
-
 def test_format_line():
 def test_format_line():
     data = dict(foo='bar baz')
     data = dict(foo='bar baz')
     assert format_line('', data) == ''
     assert format_line('', data) == ''

+ 25 - 24
src/borg/testsuite/key.py

@@ -9,7 +9,7 @@ import msgpack
 
 
 from ..crypto import bytes_to_long, num_aes_blocks
 from ..crypto import bytes_to_long, num_aes_blocks
 from ..helpers import Location
 from ..helpers import Location
-from ..helpers import Chunk, StableDict
+from ..helpers import StableDict
 from ..helpers import IntegrityError
 from ..helpers import IntegrityError
 from ..helpers import get_security_dir
 from ..helpers import get_security_dir
 from ..key import PlaintextKey, PassphraseKey, KeyfileKey, RepoKey, Blake2KeyfileKey, Blake2RepoKey, AuthenticatedKey
 from ..key import PlaintextKey, PassphraseKey, KeyfileKey, RepoKey, Blake2KeyfileKey, Blake2RepoKey, AuthenticatedKey
@@ -104,17 +104,17 @@ class TestKey:
 
 
     def test_plaintext(self):
     def test_plaintext(self):
         key = PlaintextKey.create(None, None)
         key = PlaintextKey.create(None, None)
-        chunk = Chunk(b'foo')
-        assert hexlify(key.id_hash(chunk.data)) == b'2c26b46b68ffc68ff99b453c1d30413413422d706483bfa0f98a5e886266e7ae'
-        assert chunk == key.decrypt(key.id_hash(chunk.data), key.encrypt(chunk))
+        chunk = b'foo'
+        assert hexlify(key.id_hash(chunk)) == b'2c26b46b68ffc68ff99b453c1d30413413422d706483bfa0f98a5e886266e7ae'
+        assert chunk == key.decrypt(key.id_hash(chunk), key.encrypt(chunk))
 
 
     def test_keyfile(self, monkeypatch, keys_dir):
     def test_keyfile(self, monkeypatch, keys_dir):
         monkeypatch.setenv('BORG_PASSPHRASE', 'test')
         monkeypatch.setenv('BORG_PASSPHRASE', 'test')
         key = KeyfileKey.create(self.MockRepository(), self.MockArgs())
         key = KeyfileKey.create(self.MockRepository(), self.MockArgs())
         assert bytes_to_long(key.enc_cipher.iv, 8) == 0
         assert bytes_to_long(key.enc_cipher.iv, 8) == 0
-        manifest = key.encrypt(Chunk(b'ABC'))
+        manifest = key.encrypt(b'ABC')
         assert key.extract_nonce(manifest) == 0
         assert key.extract_nonce(manifest) == 0
-        manifest2 = key.encrypt(Chunk(b'ABC'))
+        manifest2 = key.encrypt(b'ABC')
         assert manifest != manifest2
         assert manifest != manifest2
         assert key.decrypt(None, manifest) == key.decrypt(None, manifest2)
         assert key.decrypt(None, manifest) == key.decrypt(None, manifest2)
         assert key.extract_nonce(manifest2) == 1
         assert key.extract_nonce(manifest2) == 1
@@ -124,8 +124,8 @@ class TestKey:
         # Key data sanity check
         # Key data sanity check
         assert len({key2.id_key, key2.enc_key, key2.enc_hmac_key}) == 3
         assert len({key2.id_key, key2.enc_key, key2.enc_hmac_key}) == 3
         assert key2.chunk_seed != 0
         assert key2.chunk_seed != 0
-        chunk = Chunk(b'foo')
-        assert chunk == key2.decrypt(key.id_hash(chunk.data), key.encrypt(chunk))
+        chunk = b'foo'
+        assert chunk == key2.decrypt(key.id_hash(chunk), key.encrypt(chunk))
 
 
     def test_keyfile_nonce_rollback_protection(self, monkeypatch, keys_dir):
     def test_keyfile_nonce_rollback_protection(self, monkeypatch, keys_dir):
         monkeypatch.setenv('BORG_PASSPHRASE', 'test')
         monkeypatch.setenv('BORG_PASSPHRASE', 'test')
@@ -133,9 +133,9 @@ class TestKey:
         with open(os.path.join(get_security_dir(repository.id_str), 'nonce'), "w") as fd:
         with open(os.path.join(get_security_dir(repository.id_str), 'nonce'), "w") as fd:
             fd.write("0000000000002000")
             fd.write("0000000000002000")
         key = KeyfileKey.create(repository, self.MockArgs())
         key = KeyfileKey.create(repository, self.MockArgs())
-        data = key.encrypt(Chunk(b'ABC'))
+        data = key.encrypt(b'ABC')
         assert key.extract_nonce(data) == 0x2000
         assert key.extract_nonce(data) == 0x2000
-        assert key.decrypt(None, data).data == b'ABC'
+        assert key.decrypt(None, data) == b'ABC'
 
 
     def test_keyfile_kfenv(self, tmpdir, monkeypatch):
     def test_keyfile_kfenv(self, tmpdir, monkeypatch):
         keyfile = tmpdir.join('keyfile')
         keyfile = tmpdir.join('keyfile')
@@ -144,8 +144,8 @@ class TestKey:
         assert not keyfile.exists()
         assert not keyfile.exists()
         key = KeyfileKey.create(self.MockRepository(), self.MockArgs())
         key = KeyfileKey.create(self.MockRepository(), self.MockArgs())
         assert keyfile.exists()
         assert keyfile.exists()
-        chunk = Chunk(b'ABC')
-        chunk_id = key.id_hash(chunk.data)
+        chunk = b'ABC'
+        chunk_id = key.id_hash(chunk)
         chunk_cdata = key.encrypt(chunk)
         chunk_cdata = key.encrypt(chunk)
         key = KeyfileKey.detect(self.MockRepository(), chunk_cdata)
         key = KeyfileKey.detect(self.MockRepository(), chunk_cdata)
         assert chunk == key.decrypt(chunk_id, chunk_cdata)
         assert chunk == key.decrypt(chunk_id, chunk_cdata)
@@ -158,7 +158,7 @@ class TestKey:
             fd.write(self.keyfile2_key_file)
             fd.write(self.keyfile2_key_file)
         monkeypatch.setenv('BORG_PASSPHRASE', 'passphrase')
         monkeypatch.setenv('BORG_PASSPHRASE', 'passphrase')
         key = KeyfileKey.detect(self.MockRepository(), self.keyfile2_cdata)
         key = KeyfileKey.detect(self.MockRepository(), self.keyfile2_cdata)
-        assert key.decrypt(self.keyfile2_id, self.keyfile2_cdata).data == b'payload'
+        assert key.decrypt(self.keyfile2_id, self.keyfile2_cdata) == b'payload'
 
 
     def test_keyfile2_kfenv(self, tmpdir, monkeypatch):
     def test_keyfile2_kfenv(self, tmpdir, monkeypatch):
         keyfile = tmpdir.join('keyfile')
         keyfile = tmpdir.join('keyfile')
@@ -167,14 +167,14 @@ class TestKey:
         monkeypatch.setenv('BORG_KEY_FILE', str(keyfile))
         monkeypatch.setenv('BORG_KEY_FILE', str(keyfile))
         monkeypatch.setenv('BORG_PASSPHRASE', 'passphrase')
         monkeypatch.setenv('BORG_PASSPHRASE', 'passphrase')
         key = KeyfileKey.detect(self.MockRepository(), self.keyfile2_cdata)
         key = KeyfileKey.detect(self.MockRepository(), self.keyfile2_cdata)
-        assert key.decrypt(self.keyfile2_id, self.keyfile2_cdata).data == b'payload'
+        assert key.decrypt(self.keyfile2_id, self.keyfile2_cdata) == b'payload'
 
 
     def test_keyfile_blake2(self, monkeypatch, keys_dir):
     def test_keyfile_blake2(self, monkeypatch, keys_dir):
         with keys_dir.join('keyfile').open('w') as fd:
         with keys_dir.join('keyfile').open('w') as fd:
             fd.write(self.keyfile_blake2_key_file)
             fd.write(self.keyfile_blake2_key_file)
         monkeypatch.setenv('BORG_PASSPHRASE', 'passphrase')
         monkeypatch.setenv('BORG_PASSPHRASE', 'passphrase')
         key = Blake2KeyfileKey.detect(self.MockRepository(), self.keyfile_blake2_cdata)
         key = Blake2KeyfileKey.detect(self.MockRepository(), self.keyfile_blake2_cdata)
-        assert key.decrypt(self.keyfile_blake2_id, self.keyfile_blake2_cdata).data == b'payload'
+        assert key.decrypt(self.keyfile_blake2_id, self.keyfile_blake2_cdata) == b'payload'
 
 
     def test_passphrase(self, keys_dir, monkeypatch):
     def test_passphrase(self, keys_dir, monkeypatch):
         monkeypatch.setenv('BORG_PASSPHRASE', 'test')
         monkeypatch.setenv('BORG_PASSPHRASE', 'test')
@@ -184,9 +184,9 @@ class TestKey:
         assert hexlify(key.enc_hmac_key) == b'b885a05d329a086627412a6142aaeb9f6c54ab7950f996dd65587251f6bc0901'
         assert hexlify(key.enc_hmac_key) == b'b885a05d329a086627412a6142aaeb9f6c54ab7950f996dd65587251f6bc0901'
         assert hexlify(key.enc_key) == b'2ff3654c6daf7381dbbe718d2b20b4f1ea1e34caa6cc65f6bb3ac376b93fed2a'
         assert hexlify(key.enc_key) == b'2ff3654c6daf7381dbbe718d2b20b4f1ea1e34caa6cc65f6bb3ac376b93fed2a'
         assert key.chunk_seed == -775740477
         assert key.chunk_seed == -775740477
-        manifest = key.encrypt(Chunk(b'ABC'))
+        manifest = key.encrypt(b'ABC')
         assert key.extract_nonce(manifest) == 0
         assert key.extract_nonce(manifest) == 0
-        manifest2 = key.encrypt(Chunk(b'ABC'))
+        manifest2 = key.encrypt(b'ABC')
         assert manifest != manifest2
         assert manifest != manifest2
         assert key.decrypt(None, manifest) == key.decrypt(None, manifest2)
         assert key.decrypt(None, manifest) == key.decrypt(None, manifest2)
         assert key.extract_nonce(manifest2) == 1
         assert key.extract_nonce(manifest2) == 1
@@ -197,9 +197,9 @@ class TestKey:
         assert key.enc_hmac_key == key2.enc_hmac_key
         assert key.enc_hmac_key == key2.enc_hmac_key
         assert key.enc_key == key2.enc_key
         assert key.enc_key == key2.enc_key
         assert key.chunk_seed == key2.chunk_seed
         assert key.chunk_seed == key2.chunk_seed
-        chunk = Chunk(b'foo')
-        assert hexlify(key.id_hash(chunk.data)) == b'818217cf07d37efad3860766dcdf1d21e401650fed2d76ed1d797d3aae925990'
-        assert chunk == key2.decrypt(key2.id_hash(chunk.data), key.encrypt(chunk))
+        chunk = b'foo'
+        assert hexlify(key.id_hash(chunk)) == b'818217cf07d37efad3860766dcdf1d21e401650fed2d76ed1d797d3aae925990'
+        assert chunk == key2.decrypt(key2.id_hash(chunk), key.encrypt(chunk))
 
 
     def _corrupt_byte(self, key, data, offset):
     def _corrupt_byte(self, key, data, offset):
         data = bytearray(data)
         data = bytearray(data)
@@ -224,7 +224,7 @@ class TestKey:
             key.decrypt(id, data)
             key.decrypt(id, data)
 
 
     def test_decrypt_decompress(self, key):
     def test_decrypt_decompress(self, key):
-        plaintext = Chunk(b'123456789')
+        plaintext = b'123456789'
         encrypted = key.encrypt(plaintext)
         encrypted = key.encrypt(plaintext)
         assert key.decrypt(None, encrypted, decompress=False) != plaintext
         assert key.decrypt(None, encrypted, decompress=False) != plaintext
         assert key.decrypt(None, encrypted) == plaintext
         assert key.decrypt(None, encrypted) == plaintext
@@ -244,10 +244,11 @@ class TestKey:
     def test_authenticated_encrypt(self, monkeypatch):
     def test_authenticated_encrypt(self, monkeypatch):
         monkeypatch.setenv('BORG_PASSPHRASE', 'test')
         monkeypatch.setenv('BORG_PASSPHRASE', 'test')
         key = AuthenticatedKey.create(self.MockRepository(), self.MockArgs())
         key = AuthenticatedKey.create(self.MockRepository(), self.MockArgs())
-        plaintext = Chunk(b'123456789')
+        plaintext = b'123456789'
         authenticated = key.encrypt(plaintext)
         authenticated = key.encrypt(plaintext)
-        # 0x06 is the key TYPE, 0x0000 identifies CNONE compression
-        assert authenticated == b'\x06\x00\x00' + plaintext.data
+        # 0x06 is the key TYPE, 0x0100 identifies LZ4 compression, 0x90 is part of LZ4 and means that an uncompressed
+        # block of length nine follows (the plaintext).
+        assert authenticated == b'\x06\x01\x00\x90' + plaintext
 
 
 
 
 class TestPassphrase:
 class TestPassphrase: