19192: Add a few bytes to MemorySize to account for data structures.
[arvados.git] / sdk / go / arvados / fs_collection_test.go
index 7fd03b120a7f34240393f884f88992b885499e1f..c2cac3c6ce2e963b36b7654729e56524ba9bc2db 100644 (file)
@@ -6,8 +6,8 @@ package arvados
 
 import (
        "bytes"
+       "context"
        "crypto/md5"
-       "crypto/sha1"
        "errors"
        "fmt"
        "io"
@@ -17,6 +17,7 @@ import (
        "os"
        "regexp"
        "runtime"
+       "runtime/pprof"
        "strings"
        "sync"
        "sync/atomic"
@@ -31,15 +32,25 @@ var _ = check.Suite(&CollectionFSSuite{})
 type keepClientStub struct {
        blocks      map[string][]byte
        refreshable map[string]bool
-       onPut       func(bufcopy []byte) // called from PutB, before acquiring lock
+       reads       []string             // locators from ReadAt() calls
+       onWrite     func(bufcopy []byte) // called from WriteBlock, before acquiring lock
+       authToken   string               // client's auth token (used for signing locators)
+       sigkey      string               // blob signing key
+       sigttl      time.Duration        // blob signing ttl
        sync.RWMutex
 }
 
 var errStub404 = errors.New("404 block not found")
 
 func (kcs *keepClientStub) ReadAt(locator string, p []byte, off int) (int, error) {
+       kcs.Lock()
+       kcs.reads = append(kcs.reads, locator)
+       kcs.Unlock()
        kcs.RLock()
        defer kcs.RUnlock()
+       if err := VerifySignature(locator, kcs.authToken, kcs.sigttl, []byte(kcs.sigkey)); err != nil {
+               return 0, err
+       }
        buf := kcs.blocks[locator[:32]]
        if buf == nil {
                return 0, errStub404
@@ -47,22 +58,33 @@ func (kcs *keepClientStub) ReadAt(locator string, p []byte, off int) (int, error
        return copy(p, buf[off:]), nil
 }
 
-func (kcs *keepClientStub) PutB(p []byte) (string, int, error) {
-       locator := fmt.Sprintf("%x+%d+A12345@abcde", md5.Sum(p), len(p))
-       buf := make([]byte, len(p))
-       copy(buf, p)
-       if kcs.onPut != nil {
-               kcs.onPut(buf)
+func (kcs *keepClientStub) BlockWrite(_ context.Context, opts BlockWriteOptions) (BlockWriteResponse, error) {
+       if opts.Data == nil {
+               panic("oops, stub is not made for this")
+       }
+       locator := SignLocator(fmt.Sprintf("%x+%d", md5.Sum(opts.Data), len(opts.Data)), kcs.authToken, time.Now().Add(kcs.sigttl), kcs.sigttl, []byte(kcs.sigkey))
+       buf := make([]byte, len(opts.Data))
+       copy(buf, opts.Data)
+       if kcs.onWrite != nil {
+               kcs.onWrite(buf)
+       }
+       for _, sc := range opts.StorageClasses {
+               if sc != "default" {
+                       return BlockWriteResponse{}, fmt.Errorf("stub does not write storage class %q", sc)
+               }
        }
        kcs.Lock()
        defer kcs.Unlock()
        kcs.blocks[locator[:32]] = buf
-       return locator, 1, nil
+       return BlockWriteResponse{Locator: locator, Replicas: 1}, nil
 }
 
-var localOrRemoteSignature = regexp.MustCompile(`\+[AR][^+]*`)
+var reRemoteSignature = regexp.MustCompile(`\+[AR][^+]*`)
 
 func (kcs *keepClientStub) LocalLocator(locator string) (string, error) {
+       if strings.Contains(locator, "+A") {
+               return locator, nil
+       }
        kcs.Lock()
        defer kcs.Unlock()
        if strings.Contains(locator, "+R") {
@@ -73,8 +95,9 @@ func (kcs *keepClientStub) LocalLocator(locator string) (string, error) {
                        return "", fmt.Errorf("kcs.refreshable[%q]==false", locator)
                }
        }
-       fakeSig := fmt.Sprintf("+A%x@%x", sha1.Sum(nil), time.Now().Add(time.Hour*24*14).Unix())
-       return localOrRemoteSignature.ReplaceAllLiteralString(locator, fakeSig), nil
+       locator = reRemoteSignature.ReplaceAllLiteralString(locator, "")
+       locator = SignLocator(locator, kcs.authToken, time.Now().Add(kcs.sigttl), kcs.sigttl, []byte(kcs.sigkey))
+       return locator, nil
 }
 
 type CollectionFSSuite struct {
@@ -86,12 +109,17 @@ type CollectionFSSuite struct {
 
 func (s *CollectionFSSuite) SetUpTest(c *check.C) {
        s.client = NewClientFromEnv()
+       s.client.AuthToken = fixtureActiveToken
        err := s.client.RequestAndDecode(&s.coll, "GET", "arvados/v1/collections/"+fixtureFooAndBarFilesInDirUUID, nil, nil)
        c.Assert(err, check.IsNil)
        s.kc = &keepClientStub{
                blocks: map[string][]byte{
                        "3858f62230ac3c915f300c664312c63f": []byte("foobar"),
-               }}
+               },
+               sigkey:    fixtureBlobSigningKey,
+               sigttl:    fixtureBlobSigningTTL,
+               authToken: fixtureActiveToken,
+       }
        s.fs, err = s.coll.FileSystem(s.client, s.kc)
        c.Assert(err, check.IsNil)
 }
@@ -101,6 +129,22 @@ func (s *CollectionFSSuite) TestHttpFileSystemInterface(c *check.C) {
        c.Check(ok, check.Equals, true)
 }
 
+func (s *CollectionFSSuite) TestUnattainableStorageClasses(c *check.C) {
+       fs, err := (&Collection{
+               StorageClassesDesired: []string{"unobtainium"},
+       }).FileSystem(s.client, s.kc)
+       c.Assert(err, check.IsNil)
+
+       f, err := fs.OpenFile("/foo", os.O_CREATE|os.O_WRONLY, 0777)
+       c.Assert(err, check.IsNil)
+       _, err = f.Write([]byte("food"))
+       c.Assert(err, check.IsNil)
+       err = f.Close()
+       c.Assert(err, check.IsNil)
+       _, err = fs.MarshalManifest(".")
+       c.Assert(err, check.ErrorMatches, `.*stub does not write storage class \"unobtainium\"`)
+}
+
 func (s *CollectionFSSuite) TestColonInFilename(c *check.C) {
        fs, err := (&Collection{
                ManifestText: "./foo:foo 3858f62230ac3c915f300c664312c63f+3 0:3:bar:bar\n",
@@ -535,7 +579,7 @@ func (s *CollectionFSSuite) TestConcurrentWriters(c *check.C) {
        }
 
        maxBlockSize = 8
-       defer func() { maxBlockSize = 2 << 26 }()
+       defer func() { maxBlockSize = 1 << 26 }()
 
        var wg sync.WaitGroup
        for n := 0; n < 128; n++ {
@@ -1039,18 +1083,18 @@ func (s *CollectionFSSuite) TestOpenFileFlags(c *check.C) {
        c.Check(err, check.ErrorMatches, `invalid flag.*`)
 }
 
-func (s *CollectionFSSuite) TestFlushFullBlocks(c *check.C) {
-       defer func(wab, mbs int) {
-               writeAheadBlocks = wab
+func (s *CollectionFSSuite) TestFlushFullBlocksWritingLongFile(c *check.C) {
+       defer func(cw, mbs int) {
+               concurrentWriters = cw
                maxBlockSize = mbs
-       }(writeAheadBlocks, maxBlockSize)
-       writeAheadBlocks = 2
+       }(concurrentWriters, maxBlockSize)
+       concurrentWriters = 2
        maxBlockSize = 1024
 
        proceed := make(chan struct{})
        var started, concurrent int32
        blk2done := false
-       s.kc.onPut = func([]byte) {
+       s.kc.onWrite = func([]byte) {
                atomic.AddInt32(&concurrent, 1)
                switch atomic.AddInt32(&started, 1) {
                case 1:
@@ -1069,7 +1113,7 @@ func (s *CollectionFSSuite) TestFlushFullBlocks(c *check.C) {
                default:
                        time.Sleep(time.Millisecond)
                }
-               c.Check(atomic.AddInt32(&concurrent, -1) < int32(writeAheadBlocks), check.Equals, true)
+               c.Check(atomic.AddInt32(&concurrent, -1) < int32(concurrentWriters), check.Equals, true)
        }
 
        fs, err := (&Collection{}).FileSystem(s.client, s.kc)
@@ -1105,6 +1149,250 @@ func (s *CollectionFSSuite) TestFlushFullBlocks(c *check.C) {
        c.Check(currentMemExtents(), check.HasLen, 0)
 }
 
+// Ensure blocks get flushed to disk if a lot of data is written to
+// small files/directories without calling sync().
+//
+// Write four 512KiB files into each of 256 top-level dirs (total
+// 512MiB), calling Flush() every 8 dirs. Ensure memory usage never
+// exceeds 24MiB (4 concurrentWriters * 2MiB + 8 unflushed dirs *
+// 2MiB).
+func (s *CollectionFSSuite) TestFlushAll(c *check.C) {
+       fs, err := (&Collection{}).FileSystem(s.client, s.kc)
+       c.Assert(err, check.IsNil)
+
+       s.kc.onWrite = func([]byte) {
+               // discard flushed data -- otherwise the stub will use
+               // unlimited memory
+               time.Sleep(time.Millisecond)
+               s.kc.Lock()
+               defer s.kc.Unlock()
+               s.kc.blocks = map[string][]byte{}
+       }
+       for i := 0; i < 256; i++ {
+               buf := bytes.NewBuffer(make([]byte, 524288))
+               fmt.Fprintf(buf, "test file in dir%d", i)
+
+               dir := fmt.Sprintf("dir%d", i)
+               fs.Mkdir(dir, 0755)
+               for j := 0; j < 2; j++ {
+                       f, err := fs.OpenFile(fmt.Sprintf("%s/file%d", dir, j), os.O_WRONLY|os.O_CREATE, 0)
+                       c.Assert(err, check.IsNil)
+                       defer f.Close()
+                       _, err = io.Copy(f, buf)
+                       c.Assert(err, check.IsNil)
+               }
+
+               if i%8 == 0 {
+                       fs.Flush("", true)
+               }
+
+               size := fs.MemorySize()
+               if !c.Check(size <= 1<<24, check.Equals, true) {
+                       c.Logf("at dir%d fs.MemorySize()=%d", i, size)
+                       return
+               }
+       }
+}
+
+// Ensure short blocks at the end of a stream don't get flushed by
+// Flush(false).
+//
+// Write 67x 1MiB files to each of 8 dirs, and check that 8 full 64MiB
+// blocks have been flushed while 8x 3MiB is still buffered in memory.
+func (s *CollectionFSSuite) TestFlushFullBlocksOnly(c *check.C) {
+       fs, err := (&Collection{}).FileSystem(s.client, s.kc)
+       c.Assert(err, check.IsNil)
+
+       var flushed int64
+       s.kc.onWrite = func(p []byte) {
+               atomic.AddInt64(&flushed, int64(len(p)))
+       }
+
+       nDirs := int64(8)
+       megabyte := make([]byte, 1<<20)
+       for i := int64(0); i < nDirs; i++ {
+               dir := fmt.Sprintf("dir%d", i)
+               fs.Mkdir(dir, 0755)
+               for j := 0; j < 67; j++ {
+                       f, err := fs.OpenFile(fmt.Sprintf("%s/file%d", dir, j), os.O_WRONLY|os.O_CREATE, 0)
+                       c.Assert(err, check.IsNil)
+                       defer f.Close()
+                       _, err = f.Write(megabyte)
+                       c.Assert(err, check.IsNil)
+               }
+       }
+       inodebytes := int64((nDirs*(67*2+1) + 1) * 64)
+       c.Check(fs.MemorySize(), check.Equals, int64(nDirs*67<<20)+inodebytes)
+       c.Check(flushed, check.Equals, int64(0))
+
+       waitForFlush := func(expectUnflushed, expectFlushed int64) {
+               for deadline := time.Now().Add(5 * time.Second); fs.MemorySize() > expectUnflushed && time.Now().Before(deadline); time.Sleep(10 * time.Millisecond) {
+               }
+               c.Check(fs.MemorySize(), check.Equals, expectUnflushed)
+               c.Check(flushed, check.Equals, expectFlushed)
+       }
+
+       // Nothing flushed yet
+       waitForFlush((nDirs*67)<<20+inodebytes, 0)
+
+       // Flushing a non-empty dir "/" is non-recursive and there are
+       // no top-level files, so this has no effect
+       fs.Flush("/", false)
+       waitForFlush((nDirs*67)<<20+inodebytes, 0)
+
+       // Flush the full block in dir0
+       fs.Flush("dir0", false)
+       waitForFlush((nDirs*67-64)<<20+inodebytes, 64<<20)
+
+       err = fs.Flush("dir-does-not-exist", false)
+       c.Check(err, check.NotNil)
+
+       // Flush full blocks in all dirs
+       fs.Flush("", false)
+       waitForFlush(nDirs*3<<20+inodebytes, nDirs*64<<20)
+
+       // Flush non-full blocks, too
+       fs.Flush("", true)
+       waitForFlush(inodebytes, nDirs*67<<20)
+}
+
+// Even when writing lots of files/dirs from different goroutines, as
+// long as Flush(dir,false) is called after writing each file,
+// unflushed data should be limited to one full block per
+// concurrentWriter, plus one nearly-full block at the end of each
+// dir/stream.
+func (s *CollectionFSSuite) TestMaxUnflushed(c *check.C) {
+       nDirs := int64(8)
+       maxUnflushed := (int64(concurrentWriters) + nDirs) << 26
+
+       fs, err := (&Collection{}).FileSystem(s.client, s.kc)
+       c.Assert(err, check.IsNil)
+
+       release := make(chan struct{})
+       timeout := make(chan struct{})
+       time.AfterFunc(10*time.Second, func() { close(timeout) })
+       var putCount, concurrency int64
+       var unflushed int64
+       s.kc.onWrite = func(p []byte) {
+               defer atomic.AddInt64(&unflushed, -int64(len(p)))
+               cur := atomic.AddInt64(&concurrency, 1)
+               defer atomic.AddInt64(&concurrency, -1)
+               pc := atomic.AddInt64(&putCount, 1)
+               if pc < int64(concurrentWriters) {
+                       // Block until we reach concurrentWriters, to
+                       // make sure we're really accepting concurrent
+                       // writes.
+                       select {
+                       case <-release:
+                       case <-timeout:
+                               c.Error("timeout")
+                       }
+               } else if pc == int64(concurrentWriters) {
+                       // Unblock the first N-1 PUT reqs.
+                       close(release)
+               }
+               c.Assert(cur <= int64(concurrentWriters), check.Equals, true)
+               c.Assert(atomic.LoadInt64(&unflushed) <= maxUnflushed, check.Equals, true)
+       }
+
+       var owg sync.WaitGroup
+       megabyte := make([]byte, 1<<20)
+       for i := int64(0); i < nDirs; i++ {
+               dir := fmt.Sprintf("dir%d", i)
+               fs.Mkdir(dir, 0755)
+               owg.Add(1)
+               go func() {
+                       defer owg.Done()
+                       defer fs.Flush(dir, true)
+                       var iwg sync.WaitGroup
+                       defer iwg.Wait()
+                       for j := 0; j < 67; j++ {
+                               iwg.Add(1)
+                               go func(j int) {
+                                       defer iwg.Done()
+                                       f, err := fs.OpenFile(fmt.Sprintf("%s/file%d", dir, j), os.O_WRONLY|os.O_CREATE, 0)
+                                       c.Assert(err, check.IsNil)
+                                       defer f.Close()
+                                       n, err := f.Write(megabyte)
+                                       c.Assert(err, check.IsNil)
+                                       atomic.AddInt64(&unflushed, int64(n))
+                                       fs.Flush(dir, false)
+                               }(j)
+                       }
+               }()
+       }
+       owg.Wait()
+       fs.Flush("", true)
+}
+
+func (s *CollectionFSSuite) TestFlushStress(c *check.C) {
+       done := false
+       defer func() { done = true }()
+       time.AfterFunc(10*time.Second, func() {
+               if !done {
+                       pprof.Lookup("goroutine").WriteTo(os.Stderr, 1)
+                       panic("timeout")
+               }
+       })
+
+       wrote := 0
+       s.kc.onWrite = func(p []byte) {
+               s.kc.Lock()
+               s.kc.blocks = map[string][]byte{}
+               wrote++
+               defer c.Logf("wrote block %d, %d bytes", wrote, len(p))
+               s.kc.Unlock()
+               time.Sleep(20 * time.Millisecond)
+       }
+
+       fs, err := (&Collection{}).FileSystem(s.client, s.kc)
+       c.Assert(err, check.IsNil)
+
+       data := make([]byte, 1<<20)
+       for i := 0; i < 3; i++ {
+               dir := fmt.Sprintf("dir%d", i)
+               fs.Mkdir(dir, 0755)
+               for j := 0; j < 200; j++ {
+                       data[0] = byte(j)
+                       f, err := fs.OpenFile(fmt.Sprintf("%s/file%d", dir, j), os.O_WRONLY|os.O_CREATE, 0)
+                       c.Assert(err, check.IsNil)
+                       _, err = f.Write(data)
+                       c.Assert(err, check.IsNil)
+                       f.Close()
+                       fs.Flush(dir, false)
+               }
+               _, err := fs.MarshalManifest(".")
+               c.Check(err, check.IsNil)
+       }
+}
+
+func (s *CollectionFSSuite) TestFlushShort(c *check.C) {
+       s.kc.onWrite = func([]byte) {
+               s.kc.Lock()
+               s.kc.blocks = map[string][]byte{}
+               s.kc.Unlock()
+       }
+       fs, err := (&Collection{}).FileSystem(s.client, s.kc)
+       c.Assert(err, check.IsNil)
+       for _, blocksize := range []int{8, 1000000} {
+               dir := fmt.Sprintf("dir%d", blocksize)
+               err = fs.Mkdir(dir, 0755)
+               c.Assert(err, check.IsNil)
+               data := make([]byte, blocksize)
+               for i := 0; i < 100; i++ {
+                       f, err := fs.OpenFile(fmt.Sprintf("%s/file%d", dir, i), os.O_WRONLY|os.O_CREATE, 0)
+                       c.Assert(err, check.IsNil)
+                       _, err = f.Write(data)
+                       c.Assert(err, check.IsNil)
+                       f.Close()
+                       fs.Flush(dir, false)
+               }
+               fs.Flush(dir, true)
+               _, err := fs.MarshalManifest(".")
+               c.Check(err, check.IsNil)
+       }
+}
+
 func (s *CollectionFSSuite) TestBrokenManifests(c *check.C) {
        for _, txt := range []string{
                "\n",
@@ -1154,6 +1442,152 @@ func (s *CollectionFSSuite) TestEdgeCaseManifests(c *check.C) {
        }
 }
 
+func (s *CollectionFSSuite) TestSnapshotSplice(c *check.C) {
+       filedata1 := "hello snapshot+splice world\n"
+       fs, err := (&Collection{}).FileSystem(s.client, s.kc)
+       c.Assert(err, check.IsNil)
+       {
+               f, err := fs.OpenFile("file1", os.O_CREATE|os.O_RDWR, 0700)
+               c.Assert(err, check.IsNil)
+               _, err = f.Write([]byte(filedata1))
+               c.Assert(err, check.IsNil)
+               err = f.Close()
+               c.Assert(err, check.IsNil)
+       }
+
+       snap, err := Snapshot(fs, "/")
+       c.Assert(err, check.IsNil)
+       err = Splice(fs, "dir1", snap)
+       c.Assert(err, check.IsNil)
+       f, err := fs.Open("dir1/file1")
+       c.Assert(err, check.IsNil)
+       buf, err := io.ReadAll(f)
+       c.Assert(err, check.IsNil)
+       c.Check(string(buf), check.Equals, filedata1)
+}
+
+func (s *CollectionFSSuite) TestRefreshSignatures(c *check.C) {
+       filedata1 := "hello refresh signatures world\n"
+       fs, err := (&Collection{}).FileSystem(s.client, s.kc)
+       c.Assert(err, check.IsNil)
+       fs.Mkdir("d1", 0700)
+       f, err := fs.OpenFile("d1/file1", os.O_CREATE|os.O_RDWR, 0700)
+       c.Assert(err, check.IsNil)
+       _, err = f.Write([]byte(filedata1))
+       c.Assert(err, check.IsNil)
+       err = f.Close()
+       c.Assert(err, check.IsNil)
+
+       filedata2 := "hello refresh signatures universe\n"
+       fs.Mkdir("d2", 0700)
+       f, err = fs.OpenFile("d2/file2", os.O_CREATE|os.O_RDWR, 0700)
+       c.Assert(err, check.IsNil)
+       _, err = f.Write([]byte(filedata2))
+       c.Assert(err, check.IsNil)
+       err = f.Close()
+       c.Assert(err, check.IsNil)
+       txt, err := fs.MarshalManifest(".")
+       c.Assert(err, check.IsNil)
+       var saved Collection
+       err = s.client.RequestAndDecode(&saved, "POST", "arvados/v1/collections", nil, map[string]interface{}{
+               "select": []string{"manifest_text", "uuid", "portable_data_hash"},
+               "collection": map[string]interface{}{
+                       "manifest_text": txt,
+               },
+       })
+       c.Assert(err, check.IsNil)
+
+       // Update signatures synchronously if they are already expired
+       // when Read() is called.
+       {
+               saved.ManifestText = SignManifest(saved.ManifestText, s.kc.authToken, time.Now().Add(-2*time.Second), s.kc.sigttl, []byte(s.kc.sigkey))
+               fs, err := saved.FileSystem(s.client, s.kc)
+               c.Assert(err, check.IsNil)
+               f, err := fs.OpenFile("d1/file1", os.O_RDONLY, 0)
+               c.Assert(err, check.IsNil)
+               buf, err := ioutil.ReadAll(f)
+               c.Check(err, check.IsNil)
+               c.Check(string(buf), check.Equals, filedata1)
+       }
+
+       // Update signatures asynchronously if we're more than half
+       // way to TTL when Read() is called.
+       {
+               exp := time.Now().Add(2 * time.Minute)
+               saved.ManifestText = SignManifest(saved.ManifestText, s.kc.authToken, exp, s.kc.sigttl, []byte(s.kc.sigkey))
+               fs, err := saved.FileSystem(s.client, s.kc)
+               c.Assert(err, check.IsNil)
+               f1, err := fs.OpenFile("d1/file1", os.O_RDONLY, 0)
+               c.Assert(err, check.IsNil)
+               f2, err := fs.OpenFile("d2/file2", os.O_RDONLY, 0)
+               c.Assert(err, check.IsNil)
+               buf, err := ioutil.ReadAll(f1)
+               c.Check(err, check.IsNil)
+               c.Check(string(buf), check.Equals, filedata1)
+
+               // Ensure fs treats the 2-minute TTL as less than half
+               // the server's signing TTL. If we don't do this,
+               // collectionfs will guess the signature is fresh,
+               // i.e., signing TTL is 2 minutes, and won't do an
+               // async refresh.
+               fs.(*collectionFileSystem).guessSignatureTTL = time.Hour
+
+               refreshed := false
+               for deadline := time.Now().Add(time.Second * 10); time.Now().Before(deadline) && !refreshed; time.Sleep(time.Second / 10) {
+                       _, err = f1.Seek(0, io.SeekStart)
+                       c.Assert(err, check.IsNil)
+                       buf, err = ioutil.ReadAll(f1)
+                       c.Assert(err, check.IsNil)
+                       c.Assert(string(buf), check.Equals, filedata1)
+                       loc := s.kc.reads[len(s.kc.reads)-1]
+                       t, err := signatureExpiryTime(loc)
+                       c.Assert(err, check.IsNil)
+                       c.Logf("last read block %s had signature expiry time %v", loc, t)
+                       if t.Sub(time.Now()) > time.Hour {
+                               refreshed = true
+                       }
+               }
+               c.Check(refreshed, check.Equals, true)
+
+               // Second locator should have been updated at the same
+               // time.
+               buf, err = ioutil.ReadAll(f2)
+               c.Assert(err, check.IsNil)
+               c.Assert(string(buf), check.Equals, filedata2)
+               loc := s.kc.reads[len(s.kc.reads)-1]
+               c.Check(loc, check.Not(check.Equals), s.kc.reads[len(s.kc.reads)-2])
+               t, err := signatureExpiryTime(s.kc.reads[len(s.kc.reads)-1])
+               c.Assert(err, check.IsNil)
+               c.Logf("last read block %s had signature expiry time %v", loc, t)
+               c.Check(t.Sub(time.Now()) > time.Hour, check.Equals, true)
+       }
+}
+
+var bigmanifest = func() string {
+       var buf bytes.Buffer
+       for i := 0; i < 2000; i++ {
+               fmt.Fprintf(&buf, "./dir%d", i)
+               for i := 0; i < 100; i++ {
+                       fmt.Fprintf(&buf, " d41d8cd98f00b204e9800998ecf8427e+99999")
+               }
+               for i := 0; i < 2000; i++ {
+                       fmt.Fprintf(&buf, " 1200000:300000:file%d", i)
+               }
+               fmt.Fprintf(&buf, "\n")
+       }
+       return buf.String()
+}()
+
+func (s *CollectionFSSuite) BenchmarkParseManifest(c *check.C) {
+       DebugLocksPanicMode = false
+       c.Logf("test manifest is %d bytes", len(bigmanifest))
+       for i := 0; i < c.N; i++ {
+               fs, err := (&Collection{ManifestText: bigmanifest}).FileSystem(s.client, s.kc)
+               c.Check(err, check.IsNil)
+               c.Check(fs, check.NotNil)
+       }
+}
+
 func (s *CollectionFSSuite) checkMemSize(c *check.C, f File) {
        fn := f.(*filehandle).inode.(*filenode)
        var memsize int64