10990: Support Range headers with non-zero start offset.
authorTom Clegg <tom@curoverse.com>
Fri, 27 Jan 2017 03:23:22 +0000 (22:23 -0500)
committerTom Clegg <tom@curoverse.com>
Fri, 27 Jan 2017 03:23:22 +0000 (22:23 -0500)
sdk/go/keepclient/block_cache.go [new file with mode: 0644]
sdk/go/keepclient/collectionreader.go
sdk/go/keepclient/keepclient.go
services/crunch-run/crunchrun.go
services/crunch-run/crunchrun_test.go
services/keep-web/handler.go
services/keep-web/handler_test.go
services/keep-web/ranges_test.go

diff --git a/sdk/go/keepclient/block_cache.go b/sdk/go/keepclient/block_cache.go
new file mode 100644 (file)
index 0000000..7d03b68
--- /dev/null
@@ -0,0 +1,104 @@
+package keepclient
+
+import (
+       "io/ioutil"
+       "sort"
+       "sync"
+       "time"
+)
+
+var DefaultBlockCache = &BlockCache{}
+
+type BlockCache struct {
+       // Maximum number of blocks to keep in the cache. If 0, a
+       // default size (currently 4) is used instead.
+       MaxBlocks int
+
+       cache     map[string]*cacheBlock
+       mtx       sync.Mutex
+       setupOnce sync.Once
+}
+
+const defaultMaxBlocks = 4
+
+// Sweep deletes the least recently used blocks from the cache until
+// there are no more than MaxBlocks left.
+func (c *BlockCache) Sweep() {
+       max := c.MaxBlocks
+       if max < defaultMaxBlocks {
+               max = defaultMaxBlocks
+       }
+       c.mtx.Lock()
+       defer c.mtx.Unlock()
+       if len(c.cache) <= max {
+               return
+       }
+       lru := make([]time.Time, 0, len(c.cache))
+       for _, b := range c.cache {
+               lru = append(lru, b.lastUse)
+       }
+       sort.Sort(sort.Reverse(timeSlice(lru)))
+       threshold := lru[max]
+       for loc, b := range c.cache {
+               if !b.lastUse.After(threshold) {
+                       delete(c.cache, loc)
+               }
+       }
+}
+
+// Get returns data from the cache, first retrieving it from Keep if
+// necessary.
+func (c *BlockCache) Get(kc *KeepClient, locator string) ([]byte, error) {
+       c.setupOnce.Do(c.setup)
+       cacheKey := locator[:32]
+       c.mtx.Lock()
+       b, ok := c.cache[cacheKey]
+       if !ok || b.err != nil {
+               b = &cacheBlock{
+                       fetched: make(chan struct{}),
+                       lastUse: time.Now(),
+               }
+               c.cache[cacheKey] = b
+               go func() {
+                       rdr, _, _, err := kc.Get(locator)
+                       var data []byte
+                       if err == nil {
+                               data, err = ioutil.ReadAll(rdr)
+                       }
+                       c.mtx.Lock()
+                       b.data, b.err = data, err
+                       c.mtx.Unlock()
+                       close(b.fetched)
+                       go c.Sweep()
+               }()
+       }
+       c.mtx.Unlock()
+
+       // Wait (with mtx unlocked) for the fetch goroutine to finish,
+       // in case it hasn't already.
+       <-b.fetched
+
+       c.mtx.Lock()
+       b.lastUse = time.Now()
+       c.mtx.Unlock()
+       return b.data, b.err
+}
+
+func (c *BlockCache) setup() {
+       c.cache = make(map[string]*cacheBlock)
+}
+
+type timeSlice []time.Time
+
+func (ts timeSlice) Len() int { return len(ts) }
+
+func (ts timeSlice) Less(i, j int) bool { return ts[i].Before(ts[j]) }
+
+func (ts timeSlice) Swap(i, j int) { ts[i], ts[j] = ts[j], ts[i] }
+
+type cacheBlock struct {
+       data    []byte
+       err     error
+       fetched chan struct{}
+       lastUse time.Time
+}
index 33bb58710e0c94e1cfa562b8bd1c56afff62a4d7..ca53ee52a26ea7930460d563d3a49e36feed0d06 100644 (file)
@@ -2,16 +2,20 @@ package keepclient
 
 import (
        "errors"
+       "fmt"
        "io"
        "os"
 
        "git.curoverse.com/arvados.git/sdk/go/manifest"
 )
 
-// ReadCloserWithLen extends io.ReadCloser with a Len() method that
-// returns the total number of bytes available to read.
-type ReadCloserWithLen interface {
-       io.ReadCloser
+// A Reader implements, io.Reader, io.Seeker, and io.Closer, and has a
+// Len() method that returns the total number of bytes available to
+// read.
+type Reader interface {
+       io.Reader
+       io.Seeker
+       io.Closer
        Len() uint64
 }
 
@@ -31,10 +35,10 @@ const (
 // parameter when retrieving the collection record).
 var ErrNoManifest = errors.New("Collection has no manifest")
 
-// CollectionFileReader returns a ReadCloserWithLen that reads file
-// content from a collection. The filename must be given relative to
-// the root of the collection, without a leading "./".
-func (kc *KeepClient) CollectionFileReader(collection map[string]interface{}, filename string) (ReadCloserWithLen, error) {
+// CollectionFileReader returns a Reader that reads file content from
+// a collection. The filename must be given relative to the root of
+// the collection, without a leading "./".
+func (kc *KeepClient) CollectionFileReader(collection map[string]interface{}, filename string) (Reader, error) {
        mText, ok := collection["manifest_text"].(string)
        if !ok {
                return nil, ErrNoManifest
@@ -43,218 +47,126 @@ func (kc *KeepClient) CollectionFileReader(collection map[string]interface{}, fi
        return kc.ManifestFileReader(m, filename)
 }
 
-func (kc *KeepClient) ManifestFileReader(m manifest.Manifest, filename string) (ReadCloserWithLen, error) {
-       rdrChan := make(chan *cfReader)
-       go kc.queueSegmentsToGet(m, filename, rdrChan)
-       r, ok := <-rdrChan
-       if !ok {
-               return nil, os.ErrNotExist
+func (kc *KeepClient) ManifestFileReader(m manifest.Manifest, filename string) (Reader, error) {
+       f := &file{
+               kc: kc,
        }
-       return r, nil
+       return f, f.load(m, filename)
 }
 
-// Send segments for the specified file to r.toGet. Send a *cfReader
-// to rdrChan if the specified file is found (even if it's empty).
-// Then, close rdrChan.
-func (kc *KeepClient) queueSegmentsToGet(m manifest.Manifest, filename string, rdrChan chan *cfReader) {
-       defer close(rdrChan)
-
-       // q is a queue of FileSegments that we have received but
-       // haven't yet been able to send to toGet.
-       var q []*manifest.FileSegment
-       var r *cfReader
-       for seg := range m.FileSegmentIterByName(filename) {
-               if r == nil {
-                       // We've just discovered that the requested
-                       // filename does appear in the manifest, so we
-                       // can return a real reader (not nil) from
-                       // CollectionFileReader().
-                       r = newCFReader(kc)
-                       rdrChan <- r
-               }
-               q = append(q, seg)
-               r.totalSize += uint64(seg.Len)
-               // Send toGet as many segments as we can until it
-               // blocks.
-       Q:
-               for len(q) > 0 {
-                       select {
-                       case r.toGet <- q[0]:
-                               q = q[1:]
-                       default:
-                               break Q
-                       }
-               }
-       }
-       if r == nil {
-               // File not found.
-               return
-       }
-       close(r.countDone)
-       for _, seg := range q {
-               r.toGet <- seg
-       }
-       close(r.toGet)
+type file struct {
+       kc       *KeepClient
+       segments []*manifest.FileSegment
+       size     int64 // total file size
+       offset   int64 // current read offset
+
+       // current/latest segment accessed -- might or might not match pos
+       seg           *manifest.FileSegment
+       segStart      int64 // position of segment relative to file
+       segData       []byte
+       segNext       []*manifest.FileSegment
+       readaheadDone bool
 }
 
-type cfReader struct {
-       keepClient *KeepClient
-
-       // doGet() reads FileSegments from toGet, gets the data from
-       // Keep, and sends byte slices to toRead to be consumed by
-       // Read().
-       toGet chan *manifest.FileSegment
-
-       // toRead is a buffered channel, sized to fit one full Keep
-       // block. This lets us verify checksums without having a
-       // store-and-forward delay between blocks: by the time the
-       // caller starts receiving data from block N, cfReader is
-       // starting to fetch block N+1. A larger buffer would be
-       // useful for a caller whose read speed varies a lot.
-       toRead chan []byte
-
-       // bytes ready to send next time someone calls Read()
-       buf []byte
-
-       // Total size of the file being read. Not safe to read this
-       // until countDone is closed.
-       totalSize uint64
-       countDone chan struct{}
-
-       // First error encountered.
-       err error
-
-       // errNotNil is closed IFF err contains a non-nil error.
-       // Receiving from it will block until an error occurs.
-       errNotNil chan struct{}
-
-       // rdrClosed is closed IFF the reader's Close() method has
-       // been called. Any goroutines associated with the reader will
-       // stop and free up resources when they notice this channel is
-       // closed.
-       rdrClosed chan struct{}
+func (f *file) Close() error {
+       f.kc = nil
+       f.segments = nil
+       f.segData = nil
+       return nil
 }
 
-func (r *cfReader) Read(outbuf []byte) (int, error) {
-       if r.Error() != nil {
-               // Short circuit: the caller might as well find out
-               // now that we hit an error, even if there's buffered
-               // data we could return.
-               return 0, r.Error()
+func (f *file) Read(buf []byte) (int, error) {
+       if f.seg == nil || f.offset < f.segStart || f.offset >= f.segStart+int64(f.seg.Len) {
+               // f.seg does not cover the current read offset
+               // (f.pos).  Iterate over f.segments to find the one
+               // that does.
+               f.seg = nil
+               f.segStart = 0
+               f.segData = nil
+               f.segNext = f.segments
+               for len(f.segNext) > 0 {
+                       seg := f.segNext[0]
+                       f.segNext = f.segNext[1:]
+                       segEnd := f.segStart + int64(seg.Len)
+                       if segEnd > f.offset {
+                               f.seg = seg
+                               break
+                       }
+                       f.segStart = segEnd
+               }
+               f.readaheadDone = false
        }
-       for len(r.buf) == 0 {
-               // Private buffer was emptied out by the last Read()
-               // (or this is the first Read() and r.buf is nil).
-               // Read from r.toRead until we get a non-empty slice
-               // or hit an error.
-               var ok bool
-               r.buf, ok = <-r.toRead
-               if r.Error() != nil {
-                       // Error encountered while waiting for bytes
-                       return 0, r.Error()
-               } else if !ok {
-                       // No more bytes to read, no error encountered
-                       return 0, io.EOF
+       if f.seg == nil {
+               return 0, io.EOF
+       }
+       if f.segData == nil {
+               data, err := f.kc.cache().Get(f.kc, f.seg.Locator)
+               if err != nil {
+                       return 0, err
                }
+               if len(data) < f.seg.Offset+f.seg.Len {
+                       return 0, fmt.Errorf("invalid segment (offset %d len %d) in %d-byte block %s", f.seg.Offset, f.seg.Len, len(data), f.seg.Locator)
+               }
+               f.segData = data[f.seg.Offset : f.seg.Offset+f.seg.Len]
        }
-       // Copy as much as possible from our private buffer to the
-       // caller's buffer
-       n := len(r.buf)
-       if len(r.buf) > len(outbuf) {
-               n = len(outbuf)
+       dataOff := int(f.offset - f.segStart)
+       dataLen := f.seg.Len - dataOff
+
+       if !f.readaheadDone && len(f.segNext) > 0 && f.offset >= 1048576 && dataOff+dataLen >= 1048576 {
+               // If we have already read more than just the first
+               // few bytes of this file, and more than just a few
+               // bytes of this block, and there's more data for this
+               // file in the next segment/block ... then there's a
+               // good chance we are going to want the next block
+               // soon. Start getting it into the cache now.
+               go f.kc.cache().Get(f.kc, f.segNext[0].Locator)
+               f.readaheadDone = true
        }
-       copy(outbuf[:n], r.buf[:n])
-
-       // Next call to Read() will continue where we left off
-       r.buf = r.buf[n:]
 
+       n := len(buf)
+       if n > dataLen {
+               n = dataLen
+       }
+       copy(buf[:n], f.segData[dataOff:dataOff+n])
+       f.offset += int64(n)
        return n, nil
 }
 
-// Close releases resources. It returns a non-nil error if an error
-// was encountered by the reader.
-func (r *cfReader) Close() error {
-       close(r.rdrClosed)
-       return r.Error()
-}
-
-// Error returns an error if one has been encountered, otherwise
-// nil. It is safe to call from any goroutine.
-func (r *cfReader) Error() error {
-       select {
-       case <-r.errNotNil:
-               return r.err
+func (f *file) Seek(offset int64, whence int) (int64, error) {
+       var want int64
+       switch whence {
+       case io.SeekStart:
+               want = offset
+       case io.SeekCurrent:
+               want = f.offset + offset
+       case io.SeekEnd:
+               want = f.size + offset
        default:
-               return nil
+               return f.offset, fmt.Errorf("invalid whence %d", whence)
        }
+       if want < 0 {
+               return f.offset, fmt.Errorf("attempted seek to %d", want)
+       }
+       if want > f.size {
+               want = f.size
+       }
+       f.offset = want
+       return f.offset, nil
 }
 
-// Len returns the total number of bytes in the file being read. If
-// necessary, it waits for manifest parsing to finish.
-func (r *cfReader) Len() uint64 {
-       // Wait for all segments to be counted
-       <-r.countDone
-       return r.totalSize
+func (f *file) Len() uint64 {
+       return uint64(f.size)
 }
 
-func (r *cfReader) doGet() {
-       defer close(r.toRead)
-GET:
-       for fs := range r.toGet {
-               rdr, _, _, err := r.keepClient.Get(fs.Locator)
-               if err != nil {
-                       r.err = err
-                       close(r.errNotNil)
-                       return
-               }
-               var buf = make([]byte, fs.Offset+fs.Len)
-               _, err = io.ReadFull(rdr, buf)
-               errClosing := rdr.Close()
-               if err == nil {
-                       err = errClosing
-               }
-               if err != nil {
-                       r.err = err
-                       close(r.errNotNil)
-                       return
-               }
-               for bOff, bLen := fs.Offset, dataSliceSize; bOff < fs.Offset+fs.Len && bLen > 0; bOff += bLen {
-                       if bOff+bLen > fs.Offset+fs.Len {
-                               bLen = fs.Offset + fs.Len - bOff
-                       }
-                       select {
-                       case r.toRead <- buf[bOff : bOff+bLen]:
-                       case <-r.rdrClosed:
-                               // Reader is closed: no point sending
-                               // anything more to toRead.
-                               break GET
-                       }
-               }
-               // It is possible that r.rdrClosed is closed but we
-               // never noticed because r.toRead was also ready in
-               // every select{} above. Here we check before wasting
-               // a keepclient.Get() call.
-               select {
-               case <-r.rdrClosed:
-                       break GET
-               default:
-               }
+func (f *file) load(m manifest.Manifest, path string) error {
+       f.segments = nil
+       f.size = 0
+       for seg := range m.FileSegmentIterByName(path) {
+               f.segments = append(f.segments, seg)
+               f.size += int64(seg.Len)
        }
-       // In case we exited the above loop early: before returning,
-       // drain the toGet channel so its sender doesn't sit around
-       // blocking forever.
-       for range r.toGet {
+       if f.segments == nil {
+               return os.ErrNotExist
        }
-}
-
-func newCFReader(kc *KeepClient) (r *cfReader) {
-       r = new(cfReader)
-       r.keepClient = kc
-       r.rdrClosed = make(chan struct{})
-       r.errNotNil = make(chan struct{})
-       r.toGet = make(chan *manifest.FileSegment, 2)
-       r.toRead = make(chan []byte, (BLOCKSIZE+dataSliceSize-1)/dataSliceSize)
-       r.countDone = make(chan struct{})
-       go r.doGet()
-       return
+       return nil
 }
index baf4bac02444170446c91a61c0b7469813bf308c..4f84afca61a413796fbb222eb5108473c449cfaf 100644 (file)
@@ -72,6 +72,7 @@ type KeepClient struct {
        lock               sync.RWMutex
        Client             *http.Client
        Retries            int
+       BlockCache         *BlockCache
 
        // set to 1 if all writable services are of disk type, otherwise 0
        replicasPerService int
@@ -406,6 +407,14 @@ func (kc *KeepClient) getSortedRoots(locator string) []string {
        return found
 }
 
+func (kc *KeepClient) cache() *BlockCache {
+       if kc.BlockCache != nil {
+               return kc.BlockCache
+       } else {
+               return DefaultBlockCache
+       }
+}
+
 type Locator struct {
        Hash  string
        Size  int      // -1 if data size is not known
index 971cb3a27a246c9fbe1a325496a6da63e6e69ac4..0e979c1a4a35bf99bd78d327221843d8127e0f6e 100644 (file)
@@ -40,7 +40,7 @@ var ErrCancelled = errors.New("Cancelled")
 // IKeepClient is the minimal Keep API methods used by crunch-run.
 type IKeepClient interface {
        PutHB(hash string, buf []byte) (string, int, error)
-       ManifestFileReader(m manifest.Manifest, filename string) (keepclient.ReadCloserWithLen, error)
+       ManifestFileReader(m manifest.Manifest, filename string) (keepclient.Reader, error)
 }
 
 // NewLogWriter is a factory function to create a new log writer.
index b9856aca2964755a7fab0f9baf527084adff8532..e8b45c978d92d6860672af22f61bcaa33071cfd0 100644 (file)
@@ -245,7 +245,11 @@ func (fw FileWrapper) Len() uint64 {
        return fw.len
 }
 
-func (client *KeepTestClient) ManifestFileReader(m manifest.Manifest, filename string) (keepclient.ReadCloserWithLen, error) {
+func (fw FileWrapper) Seek(int64, int) (int64, error) {
+       return 0, errors.New("not implemented")
+}
+
+func (client *KeepTestClient) ManifestFileReader(m manifest.Manifest, filename string) (keepclient.Reader, error) {
        if filename == hwImageId+".tar" {
                rdr := ioutil.NopCloser(&bytes.Buffer{})
                client.Called = true
@@ -324,7 +328,7 @@ func (KeepErrorTestClient) PutHB(hash string, buf []byte) (string, int, error) {
        return "", 0, errors.New("KeepError")
 }
 
-func (KeepErrorTestClient) ManifestFileReader(m manifest.Manifest, filename string) (keepclient.ReadCloserWithLen, error) {
+func (KeepErrorTestClient) ManifestFileReader(m manifest.Manifest, filename string) (keepclient.Reader, error) {
        return nil, errors.New("KeepError")
 }
 
@@ -348,7 +352,11 @@ func (ErrorReader) Len() uint64 {
        return 0
 }
 
-func (KeepReadErrorTestClient) ManifestFileReader(m manifest.Manifest, filename string) (keepclient.ReadCloserWithLen, error) {
+func (ErrorReader) Seek(int64, int) (int64, error) {
+       return 0, errors.New("ErrorReader")
+}
+
+func (KeepReadErrorTestClient) ManifestFileReader(m manifest.Manifest, filename string) (keepclient.Reader, error) {
        return ErrorReader{}, nil
 }
 
index 11d0d96b298de5e4369474418f9f78583634510e..8dee88d485e40003b1c4f70d3f1cf86354a67a9e 100644 (file)
@@ -8,10 +8,11 @@ import (
        "net/http"
        "net/url"
        "os"
-       "regexp"
+       "path"
        "strconv"
        "strings"
        "sync"
+       "time"
 
        "git.curoverse.com/arvados.git/sdk/go/arvadosclient"
        "git.curoverse.com/arvados.git/sdk/go/auth"
@@ -346,40 +347,18 @@ func (h *handler) ServeHTTP(wOrig http.ResponseWriter, r *http.Request) {
                        w.Header().Set("Content-Type", t)
                }
        }
-       if rdr, ok := rdr.(keepclient.ReadCloserWithLen); ok {
+       if rdr, ok := rdr.(keepclient.Reader); ok {
                w.Header().Set("Content-Length", fmt.Sprintf("%d", rdr.Len()))
        }
 
        applyContentDispositionHdr(w, r, filename[basenamePos:], attachment)
-       rangeRdr, statusCode := applyRangeHdr(w, r, rdr)
 
-       w.WriteHeader(statusCode)
-       _, err = io.Copy(w, rangeRdr)
+       modstr, _ := collection["modified_at"].(string)
+       modtime, err := time.Parse(time.RFC3339Nano, modstr)
        if err != nil {
-               statusCode, statusText = http.StatusBadGateway, err.Error()
-       }
-}
-
-var rangeRe = regexp.MustCompile(`^bytes=0-([0-9]*)$`)
-
-func applyRangeHdr(w http.ResponseWriter, r *http.Request, rdr keepclient.ReadCloserWithLen) (io.Reader, int) {
-       w.Header().Set("Accept-Ranges", "bytes")
-       hdr := r.Header.Get("Range")
-       fields := rangeRe.FindStringSubmatch(hdr)
-       if fields == nil {
-               return rdr, http.StatusOK
-       }
-       rangeEnd, err := strconv.ParseInt(fields[1], 10, 64)
-       if err != nil {
-               // Empty or too big for int64 == send entire content
-               return rdr, http.StatusOK
-       }
-       if uint64(rangeEnd) >= rdr.Len() {
-               return rdr, http.StatusOK
+               modtime = time.Now()
        }
-       w.Header().Set("Content-Length", fmt.Sprintf("%d", rangeEnd+1))
-       w.Header().Set("Content-Range", fmt.Sprintf("bytes %d-%d/%d", 0, rangeEnd, rdr.Len()))
-       return &io.LimitedReader{R: rdr, N: rangeEnd + 1}, http.StatusPartialContent
+       http.ServeContent(w, r, path.Base(filename), modtime, rdr)
 }
 
 func applyContentDispositionHdr(w http.ResponseWriter, r *http.Request, filename string, isAttachment bool) {
index b3e17e8b61db0a4f3dd88b48ae82e3c6f6579673..0c960b8c0e323e860b4f79ec032d879de9a92944 100644 (file)
@@ -350,49 +350,6 @@ func (s *IntegrationSuite) TestAnonymousTokenError(c *check.C) {
        )
 }
 
-func (s *IntegrationSuite) TestRange(c *check.C) {
-       s.testServer.Config.AnonymousTokens = []string{arvadostest.AnonymousToken}
-       u, _ := url.Parse("http://example.com/c=" + arvadostest.HelloWorldCollection + "/Hello%20world.txt")
-       req := &http.Request{
-               Method:     "GET",
-               Host:       u.Host,
-               URL:        u,
-               RequestURI: u.RequestURI(),
-               Header:     http.Header{"Range": {"bytes=0-4"}},
-       }
-       resp := httptest.NewRecorder()
-       s.testServer.Handler.ServeHTTP(resp, req)
-       c.Check(resp.Code, check.Equals, http.StatusPartialContent)
-       c.Check(resp.Body.String(), check.Equals, "Hello")
-       c.Check(resp.Header().Get("Content-Length"), check.Equals, "5")
-       c.Check(resp.Header().Get("Content-Range"), check.Equals, "bytes 0-4/12")
-
-       req.Header.Set("Range", "bytes=0-")
-       resp = httptest.NewRecorder()
-       s.testServer.Handler.ServeHTTP(resp, req)
-       // 200 and 206 are both correct:
-       c.Check(resp.Code, check.Equals, http.StatusOK)
-       c.Check(resp.Body.String(), check.Equals, "Hello world\n")
-       c.Check(resp.Header().Get("Content-Length"), check.Equals, "12")
-
-       // Unsupported ranges are ignored
-       for _, hdr := range []string{
-               "bytes=5-5",  // non-zero start byte
-               "bytes=-5",   // last 5 bytes
-               "cubits=0-5", // unsupported unit
-               "bytes=0-340282366920938463463374607431768211456", // 2^128
-       } {
-               req.Header.Set("Range", hdr)
-               resp = httptest.NewRecorder()
-               s.testServer.Handler.ServeHTTP(resp, req)
-               c.Check(resp.Code, check.Equals, http.StatusOK)
-               c.Check(resp.Body.String(), check.Equals, "Hello world\n")
-               c.Check(resp.Header().Get("Content-Length"), check.Equals, "12")
-               c.Check(resp.Header().Get("Content-Range"), check.Equals, "")
-               c.Check(resp.Header().Get("Accept-Ranges"), check.Equals, "bytes")
-       }
-}
-
 // XHRs can't follow redirect-with-cookie so they rely on method=POST
 // and disposition=attachment (telling us it's acceptable to respond
 // with content instead of a redirect) and an Origin header that gets
index 5dfe02fcfa7a57e69048152539620f8b908008db..186306d19ef543187df3a26324af4e6dd197a3fc 100644 (file)
@@ -19,7 +19,7 @@ func (s *IntegrationSuite) TestRanges(c *check.C) {
                for i := 0; i < blocksize; i++ {
                        testdata[i] = byte(' ')
                }
-               copy(testdata[:3], []byte("foo"))
+               copy(testdata[1:4], []byte("foo"))
                arv, err := arvadosclient.MakeArvadosClient()
                c.Assert(err, check.Equals, nil)
                arv.ApiToken = arvadostest.ActiveToken
@@ -27,12 +27,10 @@ func (s *IntegrationSuite) TestRanges(c *check.C) {
                c.Assert(err, check.Equals, nil)
                loc, _, err := kc.PutB(testdata[:])
                c.Assert(err, check.Equals, nil)
+               loc2, _, err := kc.PutB([]byte{'Z'})
+               c.Assert(err, check.Equals, nil)
 
-               mtext := "."
-               for i := 0; i < 4; i++ {
-                       mtext = mtext + " " + loc
-               }
-               mtext = mtext + fmt.Sprintf(" 0:%d:testdata.bin\n", blocksize*4)
+               mtext := fmt.Sprintf(". %s %s %s %s %s 1:%d:testdata.bin 0:1:space.txt\n", loc, loc, loc, loc, loc2, blocksize*4)
                coll := map[string]interface{}{}
                err = arv.Create("collections",
                        map[string]interface{}{
@@ -53,17 +51,18 @@ func (s *IntegrationSuite) TestRanges(c *check.C) {
                expectBody string
        }{
                {"0-2", true, "foo"},
+               {"-2", true, " Z"},
                {"1-4", true, "oo  "},
                {"z-y", false, ""},
                {"1000000-1000003", true, "foo "},
                {"999999-1000003", true, " foo "},
                {"2000000-2000003", true, "foo "},
                {"1999999-2000002", true, " foo"},
-               {"3999998-3999999", true, "  "},
-               {"3999998-4000004", true, "  "},
-               {"3999998-", true, "  "},
+               {"3999998-3999999", true, " Z"},
+               {"3999998-4000004", true, " Z"},
+               {"3999998-", true, " Z"},
        } {
-               c.Logf("%+v", trial)
+               c.Logf("trial: %#v", trial)
                resp := httptest.NewRecorder()
                req := &http.Request{
                        Method:     "GET",
@@ -79,10 +78,13 @@ func (s *IntegrationSuite) TestRanges(c *check.C) {
                if trial.expectObey {
                        c.Check(resp.Code, check.Equals, http.StatusPartialContent)
                        c.Check(resp.Body.Len(), check.Equals, len(trial.expectBody))
-                       c.Check(resp.Body.String()[:len(trial.expectBody)], check.Equals, trial.expectBody)
+                       if resp.Body.Len() > 1000 {
+                               c.Check(resp.Body.String()[:1000]+"[...]", check.Equals, trial.expectBody)
+                       } else {
+                               c.Check(resp.Body.String(), check.Equals, trial.expectBody)
+                       }
                } else {
-                       c.Check(resp.Code, check.Equals, http.StatusOK)
-                       c.Check(resp.Body.Len(), check.Equals, blocksize*4)
+                       c.Check(resp.Code, check.Equals, http.StatusRequestedRangeNotSatisfiable)
                }
        }
 }