Merge branch 'master' into 3198-writable-fuse
[arvados.git] / sdk / python / arvados / arvfile.py
index 4edecf7b38ec74c1436702bb722b6df613c98f3c..3009a90fa6099a7ba06ed6d8e988569f57ab64c3 100644 (file)
@@ -2,21 +2,33 @@ import functools
 import os
 import zlib
 import bz2
-from .ranges import *
-from arvados.retry import retry_method
 import config
 import hashlib
-import hashlib
 import threading
 import Queue
 import copy
 import errno
+import re
+import logging
+
+from .errors import KeepWriteError, AssertionError, ArgumentError
+from .keep import KeepLocator
+from ._normalize_stream import normalize_stream
+from ._ranges import locators_and_ranges, replace_range, Range
+from .retry import retry_method
+
+MOD = "mod"
+WRITE = "write"
+
+_logger = logging.getLogger('arvados.arvfile')
 
 def split(path):
     """split(path) -> streamname, filename
 
-    Separate the stream name and file name in a /-separated stream path.
-    If no stream name is available, assume '.'.
+    Separate the stream name and file name in a /-separated stream path and
+    return a tuple (stream_name, file_name).  If no stream name is available,
+    assume '.'.
+
     """
     try:
         stream_name, file_name = path.rsplit('/', 1)
@@ -24,7 +36,7 @@ def split(path):
         stream_name, file_name = '.', path
     return stream_name, file_name
 
-class ArvadosFileBase(object):
+class _FileLikeObjectBase(object):
     def __init__(self, name, mode):
         self.name = name
         self.mode = mode
@@ -33,11 +45,11 @@ class ArvadosFileBase(object):
     @staticmethod
     def _before_close(orig_func):
         @functools.wraps(orig_func)
-        def wrapper(self, *args, **kwargs):
+        def before_close_wrapper(self, *args, **kwargs):
             if self.closed:
                 raise ValueError("I/O operation on closed stream file")
             return orig_func(self, *args, **kwargs)
-        return wrapper
+        return before_close_wrapper
 
     def __enter__(self):
         return self
@@ -53,16 +65,9 @@ class ArvadosFileBase(object):
         self.closed = True
 
 
-class ArvadosFileReaderBase(ArvadosFileBase):
-    class _NameAttribute(str):
-        # The Python file API provides a plain .name attribute.
-        # Older SDK provided a name() method.
-        # This class provides both, for maximum compatibility.
-        def __call__(self):
-            return self
-
+class ArvadosFileReaderBase(_FileLikeObjectBase):
     def __init__(self, name, mode, num_retries=None):
-        super(ArvadosFileReaderBase, self).__init__(self._NameAttribute(name), mode)
+        super(ArvadosFileReaderBase, self).__init__(name, mode)
         self._filepos = 0L
         self.num_retries = num_retries
         self._readline_cache = (None, None)
@@ -77,8 +82,8 @@ class ArvadosFileReaderBase(ArvadosFileBase):
     def decompressed_name(self):
         return re.sub('\.(bz2|gz)$', '', self.name)
 
-    @ArvadosFileBase._before_close
-    def seek(self, pos, whence=os.SEEK_CUR):
+    @_FileLikeObjectBase._before_close
+    def seek(self, pos, whence=os.SEEK_SET):
         if whence == os.SEEK_CUR:
             pos += self._filepos
         elif whence == os.SEEK_END:
@@ -88,7 +93,7 @@ class ArvadosFileReaderBase(ArvadosFileBase):
     def tell(self):
         return self._filepos
 
-    @ArvadosFileBase._before_close
+    @_FileLikeObjectBase._before_close
     @retry_method
     def readall(self, size=2**20, num_retries=None):
         while True:
@@ -97,7 +102,7 @@ class ArvadosFileReaderBase(ArvadosFileBase):
                 break
             yield data
 
-    @ArvadosFileBase._before_close
+    @_FileLikeObjectBase._before_close
     @retry_method
     def readline(self, size=float('inf'), num_retries=None):
         cache_pos, cache_data = self._readline_cache
@@ -121,7 +126,7 @@ class ArvadosFileReaderBase(ArvadosFileBase):
         self._readline_cache = (self.tell(), data[nextline_index:])
         return data[:nextline_index]
 
-    @ArvadosFileBase._before_close
+    @_FileLikeObjectBase._before_close
     @retry_method
     def decompress(self, decompress, size, num_retries=None):
         for segment in self.readall(size, num_retries):
@@ -129,7 +134,7 @@ class ArvadosFileReaderBase(ArvadosFileBase):
             if data:
                 yield data
 
-    @ArvadosFileBase._before_close
+    @_FileLikeObjectBase._before_close
     @retry_method
     def readall_decompressed(self, size=2**20, num_retries=None):
         self.seek(0)
@@ -144,7 +149,7 @@ class ArvadosFileReaderBase(ArvadosFileBase):
         else:
             return self.readall(size, num_retries=num_retries)
 
-    @ArvadosFileBase._before_close
+    @_FileLikeObjectBase._before_close
     @retry_method
     def readlines(self, sizehint=float('inf'), num_retries=None):
         data = []
@@ -167,8 +172,15 @@ class ArvadosFileReaderBase(ArvadosFileBase):
 
 
 class StreamFileReader(ArvadosFileReaderBase):
+    class _NameAttribute(str):
+        # The Python file API provides a plain .name attribute.
+        # Older SDK provided a name() method.
+        # This class provides both, for maximum compatibility.
+        def __call__(self):
+            return self
+
     def __init__(self, stream, segments, name):
-        super(StreamFileReader, self).__init__(name, 'rb', num_retries=stream.num_retries)
+        super(StreamFileReader, self).__init__(self._NameAttribute(name), 'rb', num_retries=stream.num_retries)
         self._stream = stream
         self.segments = segments
 
@@ -179,7 +191,7 @@ class StreamFileReader(ArvadosFileReaderBase):
         n = self.segments[-1]
         return n.range_start + n.range_size
 
-    @ArvadosFileBase._before_close
+    @_FileLikeObjectBase._before_close
     @retry_method
     def read(self, size, num_retries=None):
         """Read up to 'size' bytes from the stream, starting at the current file position"""
@@ -190,14 +202,14 @@ class StreamFileReader(ArvadosFileReaderBase):
         available_chunks = locators_and_ranges(self.segments, self._filepos, size)
         if available_chunks:
             lr = available_chunks[0]
-            data = self._stream._readfrom(lr.locator+lr.segment_offset,
+            data = self._stream.readfrom(lr.locator+lr.segment_offset,
                                           lr.segment_size,
                                           num_retries=num_retries)
 
         self._filepos += len(data)
         return data
 
-    @ArvadosFileBase._before_close
+    @_FileLikeObjectBase._before_close
     @retry_method
     def readfrom(self, start, size, num_retries=None):
         """Read up to 'size' bytes from the stream, starting at 'start'"""
@@ -206,23 +218,28 @@ class StreamFileReader(ArvadosFileReaderBase):
 
         data = []
         for lr in locators_and_ranges(self.segments, start, size):
-            data.append(self._stream._readfrom(lr.locator+lr.segment_offset, lr.segment_size,
+            data.append(self._stream.readfrom(lr.locator+lr.segment_offset, lr.segment_size,
                                               num_retries=num_retries))
         return ''.join(data)
 
     def as_manifest(self):
-        from stream import normalize_stream
         segs = []
         for r in self.segments:
             segs.extend(self._stream.locators_and_ranges(r.locator, r.range_size))
         return " ".join(normalize_stream(".", {self.name: segs})) + "\n"
 
 
-class BufferBlock(object):
-    """
-    A BufferBlock is a stand-in for a Keep block that is in the process of being
-    written.  Writers can append to it, get the size, and compute the Keep locator.
+def synchronized(orig_func):
+    @functools.wraps(orig_func)
+    def synchronized_wrapper(self, *args, **kwargs):
+        with self.lock:
+            return orig_func(self, *args, **kwargs)
+    return synchronized_wrapper
 
+class _BufferBlock(object):
+    """A stand-in for a Keep block that is in the process of being written.
+
+    Writers can append to it, get the size, and compute the Keep locator.
     There are three valid states:
 
     WRITABLE
@@ -236,7 +253,9 @@ class BufferBlock(object):
       released, fetching the block will fetch it via keep client (since we
       discarded the internal copy), and identifiers referring to the BufferBlock
       can be replaced with the block locator.
+
     """
+
     WRITABLE = 0
     PENDING = 1
     COMMITTED = 2
@@ -251,22 +270,26 @@ class BufferBlock(object):
 
         :owner:
           ArvadosFile that owns this block
+
         """
         self.blockid = blockid
         self.buffer_block = bytearray(starting_capacity)
         self.buffer_view = memoryview(self.buffer_block)
         self.write_pointer = 0
-        self.state = BufferBlock.WRITABLE
+        self._state = _BufferBlock.WRITABLE
         self._locator = None
         self.owner = owner
+        self.lock = threading.Lock()
 
+    @synchronized
     def append(self, data):
+        """Append some data to the buffer.
+
+        Only valid if the block is in WRITABLE state.  Implements an expanding
+        buffer, doubling capacity as needed to accomdate all the data.
+
         """
-        Append some data to the buffer.  Only valid if the block is in WRITABLE
-        state.  Implements an expanding buffer, doubling capacity as needed to
-        accomdate all the data.
-        """
-        if self.state == BufferBlock.WRITABLE:
+        if self._state == _BufferBlock.WRITABLE:
             while (self.write_pointer+len(data)) > len(self.buffer_block):
                 new_buffer_block = bytearray(len(self.buffer_block) * 2)
                 new_buffer_block[0:self.write_pointer] = self.buffer_block[0:self.write_pointer]
@@ -278,34 +301,47 @@ class BufferBlock(object):
         else:
             raise AssertionError("Buffer block is not writable")
 
+    @synchronized
+    def set_state(self, nextstate, loc=None):
+        if ((self._state == _BufferBlock.WRITABLE and nextstate == _BufferBlock.PENDING) or
+            (self._state == _BufferBlock.PENDING and nextstate == _BufferBlock.COMMITTED)):
+            self._state = nextstate
+            if self._state == _BufferBlock.COMMITTED:
+                self._locator = loc
+                self.buffer_view = None
+                self.buffer_block = None
+        else:
+            raise AssertionError("Invalid state change from %s to %s" % (self.state, nextstate))
+
+    @synchronized
+    def state(self):
+        return self._state
+
     def size(self):
-        """Amount of data written to the buffer"""
+        """The amount of data written to the buffer."""
         return self.write_pointer
 
+    @synchronized
     def locator(self):
         """The Keep locator for this buffer's contents."""
         if self._locator is None:
             self._locator = "%s+%i" % (hashlib.md5(self.buffer_view[0:self.write_pointer]).hexdigest(), self.size())
         return self._locator
 
+    @synchronized
+    def clone(self, new_blockid, owner):
+        if self._state == _BufferBlock.COMMITTED:
+            raise AssertionError("Can only duplicate a writable or pending buffer block")
+        bufferblock = _BufferBlock(new_blockid, self.size(), owner)
+        bufferblock.append(self.buffer_view[0:self.size()])
+        return bufferblock
 
-class AsyncKeepWriteErrors(Exception):
-    """
-    Roll up one or more Keep write exceptions (generated by background
-    threads) into a single one.
-    """
-    def __init__(self, errors):
-        self.errors = errors
-
-    def __repr__(self):
-        return "\n".join(self.errors)
+    @synchronized
+    def clear(self):
+        self.owner = None
+        self.buffer_block = None
+        self.buffer_view = None
 
-def synchronized(orig_func):
-    @functools.wraps(orig_func)
-    def wrapper(self, *args, **kwargs):
-        with self.lock:
-            return orig_func(self, *args, **kwargs)
-    return wrapper
 
 class NoopLock(object):
     def __enter__(self):
@@ -320,23 +356,22 @@ class NoopLock(object):
     def release(self):
         pass
 
-SYNC_READONLY = 1
-SYNC_EXPLICIT = 2
-SYNC_LIVE = 3
 
 def must_be_writable(orig_func):
     @functools.wraps(orig_func)
-    def wrapper(self, *args, **kwargs):
-        if self.sync_mode() == SYNC_READONLY:
-            raise IOError((errno.EROFS, "Collection is read only"))
+    def must_be_writable_wrapper(self, *args, **kwargs):
+        if not self.writable():
+            raise IOError(errno.EROFS, "Collection must be writable.")
         return orig_func(self, *args, **kwargs)
-    return wrapper
+    return must_be_writable_wrapper
 
 
-class BlockManager(object):
-    """
-    BlockManager handles buffer blocks, background block uploads, and
-    background block prefetch for a Collection of ArvadosFiles.
+class _BlockManager(object):
+    """BlockManager handles buffer blocks.
+
+    Also handles background block uploads, and background block prefetch for a
+    Collection of ArvadosFiles.
+
     """
     def __init__(self, keep):
         """keep: KeepClient object to use"""
@@ -354,8 +389,7 @@ class BlockManager(object):
 
     @synchronized
     def alloc_bufferblock(self, blockid=None, starting_capacity=2**14, owner=None):
-        """
-        Allocate a new, empty bufferblock in WRITABLE state and return it.
+        """Allocate a new, empty bufferblock in WRITABLE state and return it.
 
         :blockid:
           optional block identifier, otherwise one will be automatically assigned
@@ -365,40 +399,38 @@ class BlockManager(object):
 
         :owner:
           ArvadosFile that owns this block
+
         """
         if blockid is None:
             blockid = "bufferblock%i" % len(self._bufferblocks)
-        bb = BufferBlock(blockid, starting_capacity=starting_capacity, owner=owner)
-        self._bufferblocks[bb.blockid] = bb
-        return bb
+        bufferblock = _BufferBlock(blockid, starting_capacity=starting_capacity, owner=owner)
+        self._bufferblocks[bufferblock.blockid] = bufferblock
+        return bufferblock
 
     @synchronized
-    def dup_block(self, blockid, owner):
-        """
-        Create a new bufferblock in WRITABLE state, initialized with the content of an existing bufferblock.
+    def dup_block(self, block, owner):
+        """Create a new bufferblock initialized with the content of an existing bufferblock.
 
-        :blockid:
-          the block to copy.  May be an existing buffer block id.
+        :block:
+          the buffer block to copy.
 
         :owner:
           ArvadosFile that owns the new block
+
         """
         new_blockid = "bufferblock%i" % len(self._bufferblocks)
-        block = self._bufferblocks[blockid]
-        bb = BufferBlock(new_blockid, len(block), owner)
-        bb.append(block)
-        self._bufferblocks[bb.blockid] = bb
-        return bb
+        bufferblock = block.clone(new_blockid, owner)
+        self._bufferblocks[bufferblock.blockid] = bufferblock
+        return bufferblock
 
     @synchronized
-    def is_bufferblock(self, id):
-        return id in self._bufferblocks
+    def is_bufferblock(self, locator):
+        return locator in self._bufferblocks
 
     @synchronized
     def stop_threads(self):
-        """
-        Shut down and wait for background upload and download threads to finish.
-        """
+        """Shut down and wait for background upload and download threads to finish."""
+
         if self._put_threads is not None:
             for t in self._put_threads:
                 self._put_queue.put(None)
@@ -416,119 +448,159 @@ class BlockManager(object):
         self._prefetch_threads = None
         self._prefetch_queue = None
 
-    def commit_bufferblock(self, block):
-        """
-        Initiate a background upload of a bufferblock.  This will block if the
-        upload queue is at capacity, otherwise it will return immediately.
+    def commit_bufferblock(self, block, wait):
+        """Initiate a background upload of a bufferblock.
+
+        :block:
+          The block object to upload
+
+        :wait:
+          If `wait` is True, upload the block synchronously.
+          If `wait` is False, upload the block asynchronously.  This will
+          return immediately unless if the upload queue is at capacity, in
+          which case it will wait on an upload queue slot.
+
         """
 
-        def worker(self):
-            """
-            Background uploader thread.
-            """
+        def commit_bufferblock_worker(self):
+            """Background uploader thread."""
+
             while True:
                 try:
-                    b = self._put_queue.get()
-                    if b is None:
+                    bufferblock = self._put_queue.get()
+                    if bufferblock is None:
                         return
-                    b._locator = self._keep.put(b.buffer_view[0:b.write_pointer].tobytes())
-                    b.state = BufferBlock.COMMITTED
-                    b.buffer_view = None
-                    b.buffer_block = None
+
+                    loc = self._keep.put(bufferblock.buffer_view[0:bufferblock.write_pointer].tobytes())
+                    bufferblock.set_state(_BufferBlock.COMMITTED, loc)
+
                 except Exception as e:
-                    print e
-                    self._put_errors.put(e)
+                    self._put_errors.put((bufferblock.locator(), e))
                 finally:
                     if self._put_queue is not None:
                         self._put_queue.task_done()
 
-        with self.lock:
-            if self._put_threads is None:
-                # Start uploader threads.
-
-                # If we don't limit the Queue size, the upload queue can quickly
-                # grow to take up gigabytes of RAM if the writing process is
-                # generating data more quickly than it can be send to the Keep
-                # servers.
-                #
-                # With two upload threads and a queue size of 2, this means up to 4
-                # blocks pending.  If they are full 64 MiB blocks, that means up to
-                # 256 MiB of internal buffering, which is the same size as the
-                # default download block cache in KeepClient.
-                self._put_queue = Queue.Queue(maxsize=2)
-                self._put_errors = Queue.Queue()
-
-                self._put_threads = []
-                for i in xrange(0, self.num_put_threads):
-                    t = threading.Thread(target=worker, args=(self,))
-                    self._put_threads.append(t)
-                    t.daemon = True
-                    t.start()
-
-        # Mark the block as PENDING so to disallow any more appends.
-        block.state = BufferBlock.PENDING
-        self._put_queue.put(block)
-
-    def get_block(self, locator, num_retries, cache_only=False):
-        """
-        Fetch a block.  First checks to see if the locator is a BufferBlock and
-        return that, if not, passes the request through to KeepClient.get().
+        if block.state() != _BufferBlock.WRITABLE:
+            return
+
+        if wait:
+            block.set_state(_BufferBlock.PENDING)
+            loc = self._keep.put(block.buffer_view[0:block.write_pointer].tobytes())
+            block.set_state(_BufferBlock.COMMITTED, loc)
+        else:
+            with self.lock:
+                if self._put_threads is None:
+                    # Start uploader threads.
+
+                    # If we don't limit the Queue size, the upload queue can quickly
+                    # grow to take up gigabytes of RAM if the writing process is
+                    # generating data more quickly than it can be send to the Keep
+                    # servers.
+                    #
+                    # With two upload threads and a queue size of 2, this means up to 4
+                    # blocks pending.  If they are full 64 MiB blocks, that means up to
+                    # 256 MiB of internal buffering, which is the same size as the
+                    # default download block cache in KeepClient.
+                    self._put_queue = Queue.Queue(maxsize=2)
+                    self._put_errors = Queue.Queue()
+
+                    self._put_threads = []
+                    for i in xrange(0, self.num_put_threads):
+                        thread = threading.Thread(target=commit_bufferblock_worker, args=(self,))
+                        self._put_threads.append(thread)
+                        thread.daemon = True
+                        thread.start()
+
+            # Mark the block as PENDING so to disallow any more appends.
+            block.set_state(_BufferBlock.PENDING)
+            self._put_queue.put(block)
+
+    @synchronized
+    def get_bufferblock(self, locator):
+        return self._bufferblocks.get(locator)
+
+    @synchronized
+    def delete_bufferblock(self, locator):
+        bb = self._bufferblocks[locator]
+        bb.clear()
+        del self._bufferblocks[locator]
+
+    def get_block_contents(self, locator, num_retries, cache_only=False):
+        """Fetch a block.
+
+        First checks to see if the locator is a BufferBlock and return that, if
+        not, passes the request through to KeepClient.get().
+
         """
         with self.lock:
             if locator in self._bufferblocks:
-                bb = self._bufferblocks[locator]
-                if bb.state != BufferBlock.COMMITTED:
-                    return bb.buffer_view[0:bb.write_pointer].tobytes()
+                bufferblock = self._bufferblocks[locator]
+                if bufferblock.state() != _BufferBlock.COMMITTED:
+                    return bufferblock.buffer_view[0:bufferblock.write_pointer].tobytes()
                 else:
-                    locator = bb._locator
-        return self._keep.get(locator, num_retries=num_retries, cache_only=cache_only)
+                    locator = bufferblock._locator
+        if cache_only:
+            return self._keep.get_from_cache(locator)
+        else:
+            return self._keep.get(locator, num_retries=num_retries)
 
     def commit_all(self):
-        """
-        Commit all outstanding buffer blocks.  Unlike commit_bufferblock(), this
-        is a synchronous call, and will not return until all buffer blocks are
-        uploaded.  Raises AsyncKeepWriteErrors() if any blocks failed to
-        upload.
+        """Commit all outstanding buffer blocks.
+
+        Unlike commit_bufferblock(), this is a synchronous call, and will not
+        return until all buffer blocks are uploaded.  Raises
+        KeepWriteError() if any blocks failed to upload.
+
         """
         with self.lock:
             items = self._bufferblocks.items()
 
         for k,v in items:
-            if v.state == BufferBlock.WRITABLE:
-                self.commit_bufferblock(v)
+            if v.state() == _BufferBlock.WRITABLE:
+                v.owner.flush(False)
 
         with self.lock:
             if self._put_queue is not None:
                 self._put_queue.join()
+
                 if not self._put_errors.empty():
-                    e = []
+                    err = []
                     try:
                         while True:
-                            e.append(self._put_errors.get(False))
+                            err.append(self._put_errors.get(False))
                     except Queue.Empty:
                         pass
-                    raise AsyncKeepWriteErrors(e)
+                    raise KeepWriteError("Error writing some blocks", err, label="block")
+
+        for k,v in items:
+            # flush again with wait=True to remove committed bufferblocks from
+            # the segments.
+            if v.owner:
+                v.owner.flush(True)
+
 
     def block_prefetch(self, locator):
-        """
-        Initiate a background download of a block.  This assumes that the
-        underlying KeepClient implements a block cache, so repeated requests
-        for the same block will not result in repeated downloads (unless the
-        block is evicted from the cache.)  This method does not block.
+        """Initiate a background download of a block.
+
+        This assumes that the underlying KeepClient implements a block cache,
+        so repeated requests for the same block will not result in repeated
+        downloads (unless the block is evicted from the cache.)  This method
+        does not block.
+
         """
 
         if not self.prefetch_enabled:
             return
 
-        def worker(self):
-            """Background downloader thread."""
+        def block_prefetch_worker(self):
+            """The background downloader thread."""
             while True:
                 try:
                     b = self._prefetch_queue.get()
                     if b is None:
                         return
                     self._keep.get(b)
-                except:
+                except Exception:
                     pass
 
         with self.lock:
@@ -538,21 +610,28 @@ class BlockManager(object):
                 self._prefetch_queue = Queue.Queue()
                 self._prefetch_threads = []
                 for i in xrange(0, self.num_get_threads):
-                    t = threading.Thread(target=worker, args=(self,))
-                    self._prefetch_threads.append(t)
-                    t.daemon = True
-                    t.start()
+                    thread = threading.Thread(target=block_prefetch_worker, args=(self,))
+                    self._prefetch_threads.append(thread)
+                    thread.daemon = True
+                    thread.start()
         self._prefetch_queue.put(locator)
 
 
 class ArvadosFile(object):
-    """
-    ArvadosFile manages the underlying representation of a file in Keep as a sequence of
-    segments spanning a set of blocks, and implements random read/write access.
+    """Represent a file in a Collection.
+
+    ArvadosFile manages the underlying representation of a file in Keep as a
+    sequence of segments spanning a set of blocks, and implements random
+    read/write access.
+
+    This object may be accessed from multiple threads.
+
     """
 
-    def __init__(self, parent, stream=[], segments=[]):
+    def __init__(self, parent, name, stream=[], segments=[]):
         """
+        ArvadosFile constructor.
+
         :stream:
           a list of Range objects representing a block stream
 
@@ -560,42 +639,48 @@ class ArvadosFile(object):
           a list of Range objects representing segments
         """
         self.parent = parent
+        self.name = name
         self._modified = True
         self._segments = []
-        self.lock = parent._root_lock()
+        self.lock = parent.root_collection().lock
         for s in segments:
             self._add_segment(stream, s.locator, s.range_size)
         self._current_bblock = None
 
-    def sync_mode(self):
-        return self.parent.sync_mode()
+    def writable(self):
+        return self.parent.writable()
 
     @synchronized
     def segments(self):
         return copy.copy(self._segments)
 
     @synchronized
-    def clone(self, new_parent):
+    def clone(self, new_parent, new_name):
         """Make a copy of this file."""
-        cp = ArvadosFile(new_parent)
-
-        map_loc = {}
-        for r in self._segments:
-            new_loc = r.locator
-            if self.parent._my_block_manager().is_bufferblock(r.locator):
-                if r.locator not in map_loc:
-                    map_loc[r.locator] = self.parent._my_block_manager().dup_block(r.locator, cp).blockid
-                new_loc = map_loc[r.locator]
-
-            cp._segments.append(Range(new_loc, r.range_start, r.range_size, r.segment_offset))
-
+        cp = ArvadosFile(new_parent, new_name)
+        cp.replace_contents(self)
         return cp
 
     @must_be_writable
     @synchronized
     def replace_contents(self, other):
         """Replace segments of this file with segments from another `ArvadosFile` object."""
-        self._segments = other.segments()
+
+        map_loc = {}
+        self._segments = []
+        for other_segment in other.segments():
+            new_loc = other_segment.locator
+            if other.parent._my_block_manager().is_bufferblock(other_segment.locator):
+                if other_segment.locator not in map_loc:
+                    bufferblock = other.parent._my_block_manager().get_bufferblock(other_segment.locator)
+                    if bufferblock.state() != _BufferBlock.WRITABLE:
+                        map_loc[other_segment.locator] = bufferblock.locator()
+                    else:
+                        map_loc[other_segment.locator] = self.parent._my_block_manager().dup_block(bufferblock, self).blockid
+                new_loc = map_loc[other_segment.locator]
+
+            self._segments.append(Range(new_loc, other_segment.range_start, other_segment.range_size, other_segment.segment_offset))
+
         self._modified = True
 
     def __eq__(self, other):
@@ -604,9 +689,29 @@ class ArvadosFile(object):
         if not isinstance(other, ArvadosFile):
             return False
 
-        s = other.segments()
+        othersegs = other.segments()
         with self.lock:
-            return self._segments == s
+            if len(self._segments) != len(othersegs):
+                return False
+            for i in xrange(0, len(othersegs)):
+                seg1 = self._segments[i]
+                seg2 = othersegs[i]
+                loc1 = seg1.locator
+                loc2 = seg2.locator
+
+                if self.parent._my_block_manager().is_bufferblock(loc1):
+                    loc1 = self.parent._my_block_manager().get_bufferblock(loc1).locator()
+
+                if other.parent._my_block_manager().is_bufferblock(loc2):
+                    loc2 = other.parent._my_block_manager().get_bufferblock(loc2).locator()
+
+                if (KeepLocator(loc1).stripped() != KeepLocator(loc2).stripped() or
+                    seg1.range_start != seg2.range_start or
+                    seg1.range_size != seg2.range_size or
+                    seg1.segment_offset != seg2.segment_offset):
+                    return False
+
+        return True
 
     def __ne__(self, other):
         return not self.__eq__(other)
@@ -624,10 +729,12 @@ class ArvadosFile(object):
     @must_be_writable
     @synchronized
     def truncate(self, size):
-        """
-        Adjust the size of the file.  If `size` is less than the size of the file,
-        the file contents after `size` will be discarded.  If `size` is greater
-        than the current size of the file, an IOError will be raised.
+        """Shrink the size of the file.
+
+        If `size` is less than the size of the file, the file contents after
+        `size` will be discarded.  If `size` is greater than the current size
+        of the file, an IOError will be raised.
+
         """
         if size < self.size():
             new_segs = []
@@ -637,7 +744,7 @@ class ArvadosFile(object):
                     # segment is past the trucate size, all done
                     break
                 elif size < range_end:
-                    nr = Range(r.locator, r.range_start, size - r.range_start)
+                    nr = Range(r.locator, r.range_start, size - r.range_start, 0)
                     nr.segment_offset = r.segment_offset
                     new_segs.append(nr)
                     break
@@ -647,34 +754,42 @@ class ArvadosFile(object):
             self._segments = new_segs
             self._modified = True
         elif size > self.size():
-            raise IOError("truncate() does not support extending the file size")
+            raise IOError(errno.EINVAL, "truncate() does not support extending the file size")
 
-    @synchronized
-    def readfrom(self, offset, size, num_retries):
-        """
-        read upto `size` bytes from the file starting at `offset`.
+    def readfrom(self, offset, size, num_retries, exact=False):
+        """Read up to `size` bytes from the file starting at `offset`.
+
+        :exact:
+         If False (default), return less data than requested if the read
+         crosses a block boundary and the next block isn't cached.  If True,
+         only return less data than requested when hitting EOF.
         """
-        if size == 0 or offset >= self.size():
-            return ''
-        data = []
 
-        for lr in locators_and_ranges(self._segments, offset, size + config.KEEP_BLOCK_SIZE):
+        with self.lock:
+            if size == 0 or offset >= self.size():
+                return ''
+            prefetch = locators_and_ranges(self._segments, offset, size + config.KEEP_BLOCK_SIZE)
+            readsegs = locators_and_ranges(self._segments, offset, size)
+
+        for lr in prefetch:
             self.parent._my_block_manager().block_prefetch(lr.locator)
 
-        for lr in locators_and_ranges(self._segments, offset, size):
-            d = self.parent._my_block_manager().get_block(lr.locator, num_retries=num_retries, cache_only=bool(data))
-            if d:
-                data.append(d[lr.segment_offset:lr.segment_offset+lr.segment_size])
+        data = []
+        for lr in readsegs:
+            block = self.parent._my_block_manager().get_block_contents(lr.locator, num_retries=num_retries, cache_only=(bool(data) and not exact))
+            if block:
+                data.append(block[lr.segment_offset:lr.segment_offset+lr.segment_size])
             else:
                 break
         return ''.join(data)
 
-    def _repack_writes(self):
-        """
-        Test if the buffer block has more data than is referenced by actual segments
-        (this happens when a buffered write over-writes a file range written in
-        a previous buffered write).  Re-pack the buffer block for efficiency
+    def _repack_writes(self, num_retries):
+        """Test if the buffer block has more data than actual segments.
+
+        This happens when a buffered write over-writes a file range written in
+        a previous buffered write.  Re-pack the buffer block for efficiency
         and to avoid leaking information.
+
         """
         segs = self._segments
 
@@ -686,9 +801,10 @@ class ArvadosFile(object):
         if write_total < self._current_bblock.size():
             # There is more data in the buffer block than is actually accounted for by segments, so
             # re-pack into a new buffer by copying over to a new buffer block.
+            contents = self.parent._my_block_manager().get_block_contents(self._current_bblock.blockid, num_retries)
             new_bb = self.parent._my_block_manager().alloc_bufferblock(self._current_bblock.blockid, starting_capacity=write_total, owner=self)
             for t in bufferblock_segs:
-                new_bb.append(self._current_bblock.buffer_view[t.segment_offset:t.segment_offset+t.range_size].tobytes())
+                new_bb.append(contents[t.segment_offset:t.segment_offset+t.range_size])
                 t.segment_offset = new_bb.size() - t.range_size
 
             self._current_bblock = new_bb
@@ -696,9 +812,11 @@ class ArvadosFile(object):
     @must_be_writable
     @synchronized
     def writeto(self, offset, data, num_retries):
-        """
-        Write `data` to the file starting at `offset`.  This will update
-        existing bytes and/or extend the size of the file as necessary.
+        """Write `data` to the file starting at `offset`.
+
+        This will update existing bytes and/or extend the size of the file as
+        necessary.
+
         """
         if len(data) == 0:
             return
@@ -707,71 +825,156 @@ class ArvadosFile(object):
             raise ArgumentError("Offset is past the end of the file")
 
         if len(data) > config.KEEP_BLOCK_SIZE:
-            raise ArgumentError("Please append data in chunks smaller than %i bytes (config.KEEP_BLOCK_SIZE)" % (config.KEEP_BLOCK_SIZE))
+            # Chunk it up into smaller writes
+            n = 0
+            dataview = memoryview(data)
+            while n < len(data):
+                self.writeto(offset+n, dataview[n:n + config.KEEP_BLOCK_SIZE].tobytes(), num_retries)
+                n += config.KEEP_BLOCK_SIZE
+            return
 
         self._modified = True
 
-        if self._current_bblock is None or self._current_bblock.state != BufferBlock.WRITABLE:
+        if self._current_bblock is None or self._current_bblock.state() != _BufferBlock.WRITABLE:
             self._current_bblock = self.parent._my_block_manager().alloc_bufferblock(owner=self)
 
         if (self._current_bblock.size() + len(data)) > config.KEEP_BLOCK_SIZE:
-            self._repack_writes()
+            self._repack_writes(num_retries)
             if (self._current_bblock.size() + len(data)) > config.KEEP_BLOCK_SIZE:
-                self.parent._my_block_manager().commit_bufferblock(self._current_bblock)
+                self.parent._my_block_manager().commit_bufferblock(self._current_bblock, False)
                 self._current_bblock = self.parent._my_block_manager().alloc_bufferblock(owner=self)
 
         self._current_bblock.append(data)
 
         replace_range(self._segments, offset, len(data), self._current_bblock.blockid, self._current_bblock.write_pointer - len(data))
 
+        self.parent.notify(WRITE, self.parent, self.name, (self, self))
+
+        return len(data)
+
+    @synchronized
+    def flush(self, wait=True, num_retries=0):
+        """Flush bufferblocks to Keep."""
+        if self.modified():
+            if self._current_bblock and self._current_bblock.state() == _BufferBlock.WRITABLE:
+                self._repack_writes(num_retries)
+                self.parent._my_block_manager().commit_bufferblock(self._current_bblock, wait)
+            if wait:
+                to_delete = set()
+                for s in self._segments:
+                    bb = self.parent._my_block_manager().get_bufferblock(s.locator)
+                    if bb:
+                        if bb.state() != _BufferBlock.COMMITTED:
+                            _logger.error("bufferblock %s is not committed" % (s.locator))
+                        else:
+                            to_delete.add(s.locator)
+                            s.locator = bb.locator()
+                for s in to_delete:
+                   self.parent._my_block_manager().delete_bufferblock(s)
+
+            self.parent.notify(MOD, self.parent, self.name, (self, self))
+
     @must_be_writable
     @synchronized
     def add_segment(self, blocks, pos, size):
-        """
-        Add a segment to the end of the file, with `pos` and `offset` referencing a
-        section of the stream described by `blocks` (a list of Range objects)
+        """Add a segment to the end of the file.
+
+        `pos` and `offset` reference a section of the stream described by
+        `blocks` (a list of Range objects)
+
         """
         self._add_segment(blocks, pos, size)
 
     def _add_segment(self, blocks, pos, size):
-        """
-        (Internal version.)
-        """
+        """Internal implementation of add_segment."""
         self._modified = True
         for lr in locators_and_ranges(blocks, pos, size):
-            last = self._segments[-1] if self._segments else Range(0, 0, 0)
+            last = self._segments[-1] if self._segments else Range(0, 0, 0, 0)
             r = Range(lr.locator, last.range_start+last.range_size, lr.segment_size, lr.segment_offset)
             self._segments.append(r)
 
     @synchronized
     def size(self):
-        """Get the file size"""
+        """Get the file size."""
         if self._segments:
             n = self._segments[-1]
             return n.range_start + n.range_size
         else:
             return 0
 
+    @synchronized
+    def manifest_text(self, stream_name=".", portable_locators=False, normalize=False):
+        buf = ""
+        filestream = []
+        for segment in self.segments:
+            loc = segment.locator
+            if loc.startswith("bufferblock"):
+                loc = self._bufferblocks[loc].calculate_locator()
+            if portable_locators:
+                loc = KeepLocator(loc).stripped()
+            filestream.append(LocatorAndRange(loc, locator_block_size(loc),
+                                 segment.segment_offset, segment.range_size))
+        buf += ' '.join(normalize_stream(stream_name, {stream_name: filestream}))
+        buf += "\n"
+        return buf
+
+    @must_be_writable
+    @synchronized
+    def _reparent(self, newparent, newname):
+        self._modified = True
+        self.flush()
+        self.parent.remove(self.name)
+        self.parent = newparent
+        self.name = newname
+        self.lock = self.parent.root_collection().lock
+
+
 class ArvadosFileReader(ArvadosFileReaderBase):
-    def __init__(self, arvadosfile, name, mode="r", num_retries=None):
-        super(ArvadosFileReader, self).__init__(name, mode, num_retries=num_retries)
+    """Wraps ArvadosFile in a file-like object supporting reading only.
+
+    Be aware that this class is NOT thread safe as there is no locking around
+    updating file pointer.
+
+    """
+
+    def __init__(self, arvadosfile,  mode="r", num_retries=None):
+        super(ArvadosFileReader, self).__init__(arvadosfile.name, mode, num_retries=num_retries)
         self.arvadosfile = arvadosfile
 
     def size(self):
         return self.arvadosfile.size()
 
-    @ArvadosFileBase._before_close
+    def stream_name(self):
+        return self.arvadosfile.parent.stream_name()
+
+    @_FileLikeObjectBase._before_close
     @retry_method
-    def read(self, size, num_retries=None):
-        """Read up to `size` bytes from the stream, starting at the current file position"""
-        data = self.arvadosfile.readfrom(self._filepos, size, num_retries=num_retries)
-        self._filepos += len(data)
-        return data
+    def read(self, size=None, num_retries=None):
+        """Read up to `size` bytes from the file and return the result.
+
+        Starts at the current file position.  If `size` is None, read the
+        entire remainder of the file.
+        """
+        if size is None:
+            data = []
+            rd = self.arvadosfile.readfrom(self._filepos, config.KEEP_BLOCK_SIZE, num_retries)
+            while rd:
+                data.append(rd)
+                self._filepos += len(rd)
+                rd = self.arvadosfile.readfrom(self._filepos, config.KEEP_BLOCK_SIZE, num_retries)
+            return ''.join(data)
+        else:
+            data = self.arvadosfile.readfrom(self._filepos, size, num_retries, exact=True)
+            self._filepos += len(data)
+            return data
 
-    @ArvadosFileBase._before_close
+    @_FileLikeObjectBase._before_close
     @retry_method
     def readfrom(self, offset, size, num_retries=None):
-        """Read up to `size` bytes from the stream, starting at the current file position"""
+        """Read up to `size` bytes from the stream, starting at the specified file offset.
+
+        This method does not change the file position.
+        """
         return self.arvadosfile.readfrom(offset, size, num_retries)
 
     def flush(self):
@@ -779,27 +982,45 @@ class ArvadosFileReader(ArvadosFileReaderBase):
 
 
 class ArvadosFileWriter(ArvadosFileReader):
-    def __init__(self, arvadosfile, name, mode, num_retries=None):
-        super(ArvadosFileWriter, self).__init__(arvadosfile, name, mode, num_retries=num_retries)
+    """Wraps ArvadosFile in a file-like object supporting both reading and writing.
+
+    Be aware that this class is NOT thread safe as there is no locking around
+    updating file pointer.
+
+    """
 
-    @ArvadosFileBase._before_close
+    def __init__(self, arvadosfile, mode, num_retries=None):
+        super(ArvadosFileWriter, self).__init__(arvadosfile, mode, num_retries=num_retries)
+
+    @_FileLikeObjectBase._before_close
     @retry_method
     def write(self, data, num_retries=None):
         if self.mode[0] == "a":
-            self.arvadosfile.writeto(self.size(), data)
+            self.arvadosfile.writeto(self.size(), data, num_retries)
         else:
             self.arvadosfile.writeto(self._filepos, data, num_retries)
             self._filepos += len(data)
+        return len(data)
 
-    @ArvadosFileBase._before_close
+    @_FileLikeObjectBase._before_close
     @retry_method
     def writelines(self, seq, num_retries=None):
         for s in seq:
-            self.write(s)
+            self.write(s, num_retries)
 
+    @_FileLikeObjectBase._before_close
     def truncate(self, size=None):
         if size is None:
             size = self._filepos
         self.arvadosfile.truncate(size)
         if self._filepos > self.size():
             self._filepos = self.size()
+
+    @_FileLikeObjectBase._before_close
+    def flush(self):
+        self.arvadosfile.flush()
+
+    def close(self):
+        if not self.closed:
+            self.flush()
+            super(ArvadosFileWriter, self).close()