| 
					
				 | 
			
			
				@@ -4,9 +4,11 @@ from itertools import groupby 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				 import errno 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				 import shutil 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				 import tempfile 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+import threading 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				 from .key import key_factory 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				 from .remote import cache_if_remote 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				 import msgpack 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+from multiprocessing import cpu_count 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				 import os 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				 import socket 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				 import stat 
			 | 
		
	
	
		
			
				| 
					
				 | 
			
			
				@@ -18,7 +20,8 @@ from .platform import acl_get, acl_set 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				 from .chunker import Chunker 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				 from .hashindex import ChunkIndex 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				 from .helpers import parse_timestamp, Error, uid2user, user2uid, gid2group, group2gid, \ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-    Manifest, Statistics, decode_dict, st_mtime_ns, make_path_safe, StableDict, int_to_bigint, bigint_to_int 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+    Manifest, Statistics, decode_dict, st_mtime_ns, make_path_safe, StableDict, int_to_bigint, bigint_to_int, \ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+    make_queue, TerminatedQueue 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				  
			 | 
		
	
		
			
				 | 
				 | 
			
			
				 ITEMS_BUFFER = 1024 * 1024 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				 CHUNK_MIN = 1024 
			 | 
		
	
	
		
			
				| 
					
				 | 
			
			
				@@ -114,6 +117,193 @@ class CacheChunkBuffer(ChunkBuffer): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         return id_ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				  
			 | 
		
	
		
			
				 | 
				 | 
			
			
				  
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+class ParallelProcessor: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+    def __init__(self, archive, ncrypters=None): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.archive = archive 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        if ncrypters is None: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            # note: cpu_count for 2 cores with HT is 4 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            # put load on all logical cores and avoid idle cores 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            ncrypters = cpu_count() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.ncrypters = ncrypters 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.start_threads() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+    def reader(self): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        while True: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            elem = self.reader_queue.get() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            if elem is None: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                self.reader_queue.task_done() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                break 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            item = elem 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            n = 0 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            # Only chunkify the file if needed 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            if b'chunks' in item and item[b'chunks'] is None: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                fd, fh = item.pop(b'fd', None), -1 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                if fd is None: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    fh = Archive._open_rb(item.pop(b'path_name'), item[b'st']) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    fd = os.fdopen(fh, 'rb') 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                with fd: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    for chunk in self.archive.chunker.chunkify(fd, fh): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                        # important: chunk is a memoryview - make a copy or it will 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                        # have changed when we use it! 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                        chunk = bytes(chunk) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                        self.crypter_queue.put((item, n, chunk)) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                        n += 1 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            self.writer_queue.put((item, n, None, None, None, None))  # signal EOF via id == None , give number of chunks 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            self.reader_queue.task_done() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+    def crypter(self): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        while True: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            elem = self.crypter_queue.get() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            if elem is None: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                self.crypter_queue.task_done() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                break 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            item, n, chunk = elem 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            size = len(chunk) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            id = self.archive.key.id_hash(chunk) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            seen = self.archive.cache.seen_or_announce_chunk(id, size) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            if not seen: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                # we have never seen this id before, so we need to process it 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                # TODO check if this creates duplicate IV/CTR values for AES 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                cchunk = self.archive.key.encrypt(chunk) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                csize = len(cchunk) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            else: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                cchunk, csize = None, None 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            self.writer_queue.put((item, n, cchunk, id, size, csize)) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            self.crypter_queue.task_done() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+    def writer(self): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        item_infos = {}  # item path -> info dict 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        size_infos = {}  # chunk id -> sizes 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        dying = False 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        while True: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            elem = self.writer_queue.get() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            if elem is None: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                if not dying: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    # received poison from stop_threads, start dying, 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    # but still do work the delayer thread might give us. 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    dying = True 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    # give poison to the delayer thread 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    self.delayer_queue.put(None) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    self.writer_queue.task_done() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    continue 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                else: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    # we received the final poison from the dying delayer 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    self.writer_queue.task_done() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    # we are dead now 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    break 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            item, n, cchunk, id, size, csize = elem 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            path = item[b'path'] 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            info = item_infos.setdefault(path, dict(count=None, chunks=[])) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            if id is None: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                if n is not None:  # note: n == None is a retry 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    # EOF signalled, n is the total count of chunks 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    info['count'] = n 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            else: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                size, csize, new_chunk = self.archive.cache.add_chunk_nostats(cchunk, id, size, csize) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                info['chunks'].append((n, id, new_chunk)) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                if csize != 0: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    size_infos[id] = (size, csize) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            if len(info['chunks']) == info['count']: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                # we have processed all chunks or no chunks needed processing 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                if b'chunks' in item: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    chunks = item[b'chunks'] 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    if chunks is None: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                        # we want chunks, but we have no chunk id list yet, compute them 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                        try: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                            chunks = self.archive.cache.postprocess_results( 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                                size_infos, info['chunks'], self.archive.stats) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                        except self.archive.cache.ChunkSizeNotReady: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                            # we looked up a chunk id, but do not have the size info yet. retry later. 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                            self.delayer_queue.put((item, None, None, None, None, None)) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                            self.writer_queue.task_done() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                            continue 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    else: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                        # we have a chunk id list already, increase the ref counters, compute sizes 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                        chunks = [self.archive.cache.chunk_incref(id_, self.archive.stats) for id_ in chunks] 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    item[b'chunks'] = chunks 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                path_hash = item.pop(b'path_hash', None) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                if path_hash and chunks is not None:  # a fs object (not stdin) and a regular file 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    st = item.pop(b'st', None) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    self.archive.cache.memorize_file(path_hash, st, [c[0] for c in chunks]) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                del item_infos[path] 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                self.archive.stats.nfiles += 1 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                self.archive.add_item(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            self.writer_queue.task_done() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+    def delayer(self): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        # it is a pain that we need the compressed size for the chunks cache as it is not 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        # available for duplicate chunks until the original chunk has finished processing. 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        # this loop of (writer, delayer) with pipes connecting them is a hack to address 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        # this, but it makes thread teardown complicated. Rather get rid of csize? 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        while True: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            elem = self.delayer_queue.get() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            if elem is None: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                # we received poison from dying writer thread, kill the writer, too. 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                self.writer_queue.put(None) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                self.delayer_queue.task_done() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                # we are dead now 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                break 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            time.sleep(0.001)  # reschedule, avoid data circulating too fast 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            self.writer_queue.put(elem) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            self.delayer_queue.task_done() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+    def start_threads(self): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        def run_thread(func, name=None, daemon=False): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            t = threading.Thread(target=func, name=name) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            t.daemon = daemon 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            t.start() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            return t 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        # max. memory usage of a queue with chunk data is about queue_len * CHUNK_MAX 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        queue_len = min(max(self.ncrypters, 4), 8) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.reader_queue = make_queue('reader', queue_len * 10)  # small items (no chunk data) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.crypter_queue = make_queue('crypter', queue_len) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.writer_queue = make_queue('writer', queue_len) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.delayer_queue = make_queue('delay', queue_len) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.reader_thread = run_thread(self.reader, 'reader') 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.crypter_threads = [] 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        for i in range(self.ncrypters): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            self.crypter_threads.append(run_thread(self.crypter, name='crypter-%d' % i)) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.delayer_thread = run_thread(self.delayer, name='delayer') 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.writer_thread = run_thread(self.writer, name='writer') 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+    def wait_finish(self): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.reader_queue.join() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.crypter_queue.join() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.writer_queue.join() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.delayer_queue.join() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.writer_queue.join() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+    def stop_threads(self): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        count_before = threading.active_count() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        # for every thread: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        #   put poison pill into its queue, 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        #   wait until queue is processed (and thread has terminated itself) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        #   make queue unusable 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.reader_queue.put(None) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.reader_queue.join() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.reader_thread.join() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.reader_queue = TerminatedQueue() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        for i in range(self.ncrypters): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            self.crypter_queue.put(None) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.crypter_queue.join() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        for t in self.crypter_threads: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            t.join() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.crypter_queue = TerminatedQueue() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.writer_queue.put(None)  # the writer will poison the delayer first 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.delayer_thread.join() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.delayer_queue = TerminatedQueue() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.writer_thread.join() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.writer_queue = TerminatedQueue() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        count_after = threading.active_count() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        assert count_before - 3 - self.ncrypters == count_after 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        if count_after > 1: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            print('They are alive!') 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            tl = [t.name for t in threading.enumerate()] 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            tl.remove('MainThread') 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            assert tl == [] 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				 class Archive: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				  
			 | 
		
	
		
			
				 | 
				 | 
			
			
				     class DoesNotExist(Error): 
			 | 
		
	
	
		
			
				| 
					
				 | 
			
			
				@@ -142,6 +332,7 @@ class Archive: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         self.numeric_owner = numeric_owner 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         self.pipeline = DownloadPipeline(self.repository, self.key) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         if create: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            self.pp = ParallelProcessor(self) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             self.items_buffer = CacheChunkBuffer(self.cache, self.key, self.stats) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             self.chunker = Chunker(WINDOW_SIZE, CHUNK_MASK, CHUNK_MIN, CHUNK_MAX, self.key.chunk_seed) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             if name in manifest.archives: 
			 | 
		
	
	
		
			
				| 
					
				 | 
			
			
				@@ -154,11 +345,16 @@ class Archive: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                     break 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                 i += 1 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         else: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            self.pp = None 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             if name not in self.manifest.archives: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                 raise self.DoesNotExist(name) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             info = self.manifest.archives[name] 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             self.load(info[b'id']) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				  
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+    def close(self): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        if self.pp: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            self.pp.stop_threads() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				     def _load_meta(self, id): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         data = self.key.decrypt(id, self.repository.get(id)) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         metadata = msgpack.unpackb(data) 
			 | 
		
	
	
		
			
				| 
					
				 | 
			
			
				@@ -185,6 +381,9 @@ class Archive: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         for item in self.pipeline.unpack_many(self.metadata[b'items'], filter=filter, preload=preload): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             yield item 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				  
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+    def add_item_queued(self, item): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.pp.reader_queue.put(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				     def add_item(self, item): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         if self.show_progress and time.time() - self.last_progress > 0.2: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             self.stats.show_progress(item=item) 
			 | 
		
	
	
		
			
				| 
					
				 | 
			
			
				@@ -200,6 +399,7 @@ class Archive: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         self.cache.chunk_decref(self.id, self.stats) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				  
			 | 
		
	
		
			
				 | 
				 | 
			
			
				     def save(self, name=None, timestamp=None): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.pp.wait_finish() 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         name = name or self.name 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         if name in self.manifest.archives: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             raise self.AlreadyExists(name) 
			 | 
		
	
	
		
			
				| 
					
				 | 
			
			
				@@ -267,7 +467,9 @@ class Archive: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             if stat.S_ISDIR(st.st_mode): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                 os.rmdir(path) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             else: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-                os.unlink(path) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                # XXX do not remove a regular file, it could be the "source" 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                # of a hardlink - a still empty inode that needs to be filled. 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                pass 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         except UnicodeEncodeError: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             raise self.IncompatibleFilesystemEncodingError(path, sys.getfilesystemencoding()) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         except OSError: 
			 | 
		
	
	
		
			
				| 
					
				 | 
			
			
				@@ -286,6 +488,14 @@ class Archive: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                 source = os.path.join(dest, item[b'source']) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                 if os.path.exists(path): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                     os.unlink(path) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                if not os.path.exists(source): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    # due to multithreaded nature and different processing time, 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    # the hardlink (without file content) often is in the archive 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    # BEFORE the "source" file (with content). 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    # we create an empty file that is filled with content when 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    # the "source" item is extracted: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                    with open(source, 'wb') as fd: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                        pass 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                 os.link(source, path) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             else: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                 with open(path, 'wb') as fd: 
			 | 
		
	
	
		
			
				| 
					
				 | 
			
			
				@@ -408,19 +618,19 @@ class Archive: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				     def process_dir(self, path, st): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         item = {b'path': make_path_safe(path)} 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         item.update(self.stat_attrs(st, path)) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-        self.add_item(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.add_item_queued(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         return 'd'  # directory 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				  
			 | 
		
	
		
			
				 | 
				 | 
			
			
				     def process_fifo(self, path, st): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         item = {b'path': make_path_safe(path)} 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         item.update(self.stat_attrs(st, path)) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-        self.add_item(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.add_item_queued(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         return 'f'  # fifo 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				  
			 | 
		
	
		
			
				 | 
				 | 
			
			
				     def process_dev(self, path, st): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         item = {b'path': make_path_safe(path), b'rdev': st.st_rdev} 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         item.update(self.stat_attrs(st, path)) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-        self.add_item(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.add_item_queued(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         if stat.S_ISCHR(st.st_mode): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             return 'c'  # char device 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         elif stat.S_ISBLK(st.st_mode): 
			 | 
		
	
	
		
			
				| 
					
				 | 
			
			
				@@ -430,25 +640,21 @@ class Archive: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         source = os.readlink(path) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         item = {b'path': make_path_safe(path), b'source': source} 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         item.update(self.stat_attrs(st, path)) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-        self.add_item(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.add_item_queued(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         return 's'  # symlink 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				  
			 | 
		
	
		
			
				 | 
				 | 
			
			
				     def process_stdin(self, path, cache): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         uid, gid = 0, 0 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-        fd = sys.stdin.buffer  # binary 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-        chunks = [] 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-        for chunk in self.chunker.chunkify(fd): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-            chunks.append(cache.add_chunk(self.key.id_hash(chunk), chunk, self.stats)) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-        self.stats.nfiles += 1 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         item = { 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             b'path': path, 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-            b'chunks': chunks, 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            b'fd': sys.stdin.buffer,  # binary 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             b'mode': 0o100660,  # regular file, ug=rw 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             b'uid': uid, b'user': uid2user(uid), 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             b'gid': gid, b'group': gid2group(gid), 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             b'mtime': int_to_bigint(int(time.time()) * 1000000000) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         } 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-        self.add_item(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.add_item_queued(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        return 'A' 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				  
			 | 
		
	
		
			
				 | 
				 | 
			
			
				     def process_file(self, path, st, cache): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         status = None 
			 | 
		
	
	
		
			
				| 
					
				 | 
			
			
				@@ -459,7 +665,7 @@ class Archive: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             if (st.st_ino, st.st_dev) in self.hard_links: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                 item = self.stat_attrs(st, path) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                 item.update({b'path': safe_path, b'source': source}) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-                self.add_item(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                self.add_item_queued(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                 status = 'h'  # regular file, hardlink (to already seen inodes) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                 return status 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             else: 
			 | 
		
	
	
		
			
				| 
					
				 | 
			
			
				@@ -473,23 +679,23 @@ class Archive: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                 if not cache.seen_chunk(id_): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                     break 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             else: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-                chunks = [cache.chunk_incref(id_, self.stats) for id_ in ids] 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+                chunks = ids 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				                 status = 'U'  # regular file, unchanged 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         else: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             status = 'A'  # regular file, added 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-        # Only chunkify the file if needed 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         if chunks is None: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-            fh = Archive._open_rb(path, st) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-            with os.fdopen(fh, 'rb') as fd: 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-                chunks = [] 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-                for chunk in self.chunker.chunkify(fd, fh): 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-                    chunks.append(cache.add_chunk(self.key.id_hash(chunk), chunk, self.stats)) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-            cache.memorize_file(path_hash, st, [c[0] for c in chunks]) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				             status = status or 'M'  # regular file, modified (if not 'A' already) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-        item = {b'path': safe_path, b'chunks': chunks} 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        item = { 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            b'path': safe_path, 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            b'path_name': path, 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            b'path_hash': path_hash, 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            b'chunks': chunks, 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+            b'st': st, 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        } 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         item.update(self.stat_attrs(st, path)) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-        self.stats.nfiles += 1 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				-        self.add_item(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+        self.add_item_queued(item) 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				+ 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				         return status 
			 | 
		
	
		
			
				 | 
				 | 
			
			
				  
			 | 
		
	
		
			
				 | 
				 | 
			
			
				     @staticmethod 
			 |