5824: add (*KeepClient)CollectionFileReader()
authorTom Clegg <tom@curoverse.com>
Thu, 25 Jun 2015 03:33:08 +0000 (23:33 -0400)
committerTom Clegg <tom@curoverse.com>
Thu, 15 Oct 2015 21:06:11 +0000 (17:06 -0400)
sdk/go/arvadostest/fixtures.go [new file with mode: 0644]
sdk/go/keepclient/collectionreader.go [new file with mode: 0644]
sdk/go/keepclient/collectionreader_test.go [new file with mode: 0644]
sdk/go/manifest/manifest.go
sdk/go/manifest/manifest_test.go
services/keep-web/handler.go
services/keep-web/server_test.go

diff --git a/sdk/go/arvadostest/fixtures.go b/sdk/go/arvadostest/fixtures.go
new file mode 100644 (file)
index 0000000..87b28f8
--- /dev/null
@@ -0,0 +1,17 @@
+package arvadostest
+
+const (
+       SpectatorToken        = "zw2f4gwx8hw8cjre7yp6v1zylhrhn3m5gvjq73rtpwhmknrybu"
+       ActiveToken           = "3kg6k6lzmp9kj5cpkcoxie963cmvjahbt2fod9zru30k1jqdmi"
+       AnonymousToken        = "4kg6k6lzmp9kj4cpkcoxie964cmvjahbt4fod9zru44k4jqdmi"
+       FooCollection         = "zzzzz-4zz18-fy296fx3hot09f7"
+       NonexistentCollection = "zzzzz-4zz18-totallynotexist"
+       HelloWorldCollection  = "zzzzz-4zz18-4en62shvi99lxd4"
+       PathologicalManifest  = ". acbd18db4cc2f85cedef654fccc4a4d8+3 37b51d194a7513e45b56f6524f2d51f2+3 73feffa4b7f6bb68e44cf984c85f6e88+3+Z+K@xyzzy acbd18db4cc2f85cedef654fccc4a4d8+3 0:0:zero@0 0:1:f 1:0:zero@1 1:4:ooba 4:0:zero@4 5:1:r 5:4:rbaz 9:0:zero@9\n" +
+               "./overlapReverse acbd18db4cc2f85cedef654fccc4a4d8+3 acbd18db4cc2f85cedef654fccc4a4d8+3 5:1:o 4:2:oo 2:4:ofoo\n" +
+               "./segmented acbd18db4cc2f85cedef654fccc4a4d8+3 37b51d194a7513e45b56f6524f2d51f2+3 0:1:frob 5:1:frob 1:1:frob 1:2:oof 0:1:oof 5:0:frob 3:1:frob\n" +
+               `./foo\040b\141r acbd18db4cc2f85cedef654fccc4a4d8+3 0:3:baz` + "\n" +
+               `./foo\040b\141r acbd18db4cc2f85cedef654fccc4a4d8+3 0:3:b\141z\040w\141z` + "\n" +
+               "./foo acbd18db4cc2f85cedef654fccc4a4d8+3 0:0:zero 0:3:foo\n" +
+               ". acbd18db4cc2f85cedef654fccc4a4d8+3 0:0:foo/zero 0:3:foo/foo\n"
+)
diff --git a/sdk/go/keepclient/collectionreader.go b/sdk/go/keepclient/collectionreader.go
new file mode 100644 (file)
index 0000000..5db944c
--- /dev/null
@@ -0,0 +1,159 @@
+package keepclient
+
+import (
+       "errors"
+       "io"
+       "os"
+
+       "git.curoverse.com/arvados.git/sdk/go/manifest"
+)
+
+// ErrNoManifest indicates the given collection has no manifest
+// information (e.g., manifest_text was excluded by a "select"
+// parameter when retrieving the collection record).
+var ErrNoManifest = errors.New("Collection has no manifest")
+
+// CollectionFileReader returns an io.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) (*cfReader, error) {
+       mText, ok := collection["manifest_text"].(string)
+       if !ok {
+               return nil, ErrNoManifest
+       }
+       m := manifest.Manifest{Text: mText}
+       rdrChan := make(chan *cfReader)
+       go func() {
+               // 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 whatever it's ready to receive.
+                       Q: for len(q) > 0 {
+                               select {
+                               case r.toGet <- q[0]:
+                                       q = q[1:]
+                               default:
+                                       break Q
+                               }
+                       }
+               }
+               if r == nil {
+                       // File not found
+                       rdrChan <- nil
+                       return
+               }
+               close(r.countDone)
+               for _, seg := range q {
+                       r.toGet <- seg
+               }
+               close(r.toGet)
+       }()
+       // Before returning a reader, wait until we know whether the
+       // file exists here:
+       r := <-rdrChan
+       if r == nil {
+               return nil, os.ErrNotExist
+       }
+       return r, nil
+}
+
+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       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
+}
+
+func (r *cfReader) Read(outbuf []byte) (n int, err error) {
+       if r.err != nil {
+               return 0, r.err
+       }
+       for r.buf == nil || len(r.buf) == 0 {
+               var ok bool
+               r.buf, ok = <-r.toRead
+               if r.err != nil {
+                       return 0, r.err
+               } else if !ok {
+                       return 0, io.EOF
+               }
+       }
+       if len(r.buf) > len(outbuf) {
+               n = len(outbuf)
+       } else {
+               n = len(r.buf)
+       }
+       copy(outbuf[:n], r.buf[:n])
+       r.buf = r.buf[n:]
+       return
+}
+
+func (r *cfReader) Close() error {
+       _, _ = <-r.countDone
+       for _ = range r.toGet {
+       }
+       for _ = range r.toRead {
+       }
+       return r.err
+}
+
+func (r *cfReader) Len() uint64 {
+       // Wait for all segments to be counted
+       _, _ = <-r.countDone
+       return r.totalSize
+}
+
+func (r *cfReader) doGet() {
+       defer close(r.toRead)
+       for fs := range r.toGet {
+               rdr, _, _, err := r.keepClient.Get(fs.Locator)
+               if err != nil {
+                       r.err = err
+                       return
+               }
+               var buf = make([]byte, fs.Offset+fs.Len)
+               _, err = io.ReadFull(rdr, buf)
+               if err != nil {
+                       r.err = err
+                       return
+               }
+               for bOff, bLen := fs.Offset, 1<<20; bOff <= fs.Offset+fs.Len && bLen > 0; bOff += bLen {
+                       if bOff+bLen > fs.Offset+fs.Len {
+                               bLen = fs.Offset + fs.Len - bOff
+                       }
+                       r.toRead <- buf[bOff : bOff+bLen]
+               }
+       }
+}
+
+func newCFReader(kc *KeepClient) (r *cfReader) {
+       r = new(cfReader)
+       r.keepClient = kc
+       r.toGet = make(chan *manifest.FileSegment, 2)
+       r.toRead = make(chan []byte)
+       r.countDone = make(chan struct{})
+       go r.doGet()
+       return
+}
diff --git a/sdk/go/keepclient/collectionreader_test.go b/sdk/go/keepclient/collectionreader_test.go
new file mode 100644 (file)
index 0000000..f271208
--- /dev/null
@@ -0,0 +1,123 @@
+package keepclient
+
+import (
+       "crypto/md5"
+       "fmt"
+       "io"
+       "io/ioutil"
+       "net/http"
+       "os"
+
+       "git.curoverse.com/arvados.git/sdk/go/arvadosclient"
+       "git.curoverse.com/arvados.git/sdk/go/arvadostest"
+       check "gopkg.in/check.v1"
+)
+
+var _ = check.Suite(&IntegrationSuite{})
+
+// IntegrationSuite tests need an API server
+type IntegrationSuite struct{}
+
+type SuccessHandler struct {
+       disk map[string][]byte
+       lock chan struct{}
+}
+
+func (h SuccessHandler) ServeHTTP(resp http.ResponseWriter, req *http.Request) {
+       switch req.Method {
+       case "PUT":
+               buf, err := ioutil.ReadAll(req.Body)
+               if err != nil {
+                       resp.WriteHeader(500)
+                       return
+               }
+               pdh := fmt.Sprintf("%x+%d", md5.Sum(buf), len(buf))
+               h.lock <- struct{}{}
+               h.disk[pdh] = buf
+               <- h.lock
+               resp.Write([]byte(pdh))
+       case "GET":
+               pdh := req.URL.Path[1:]
+               h.lock <- struct{}{}
+               buf, ok := h.disk[pdh]
+               <- h.lock
+               if !ok {
+                       resp.WriteHeader(http.StatusNotFound)
+               } else {
+                       resp.Write(buf)
+               }
+       default:
+               resp.WriteHeader(http.StatusMethodNotAllowed)
+       }
+}
+
+type rdrTest struct {
+       mt   string      // manifest text
+       f    string      // filename
+       want interface{} // error or string to expect
+}
+
+func (s *ServerRequiredSuite) TestCollectionReaderContent(c *check.C) {
+       arv, err := arvadosclient.MakeArvadosClient()
+       c.Assert(err, check.IsNil)
+       arv.ApiToken = arvadostest.ActiveToken
+
+       kc, err := MakeKeepClient(&arv)
+       c.Assert(err, check.IsNil)
+
+       {
+               localRoots := make(map[string]string)
+               h := SuccessHandler{disk: make(map[string][]byte), lock: make(chan struct{}, 1)}
+               for i, k := range RunSomeFakeKeepServers(h, 4) {
+                       localRoots[fmt.Sprintf("zzzzz-bi6l4-fakefakefake%03d", i)] = k.url
+               }
+               kc.SetServiceRoots(localRoots, localRoots, nil)
+               kc.PutB([]byte("foo"))
+               kc.PutB([]byte("bar"))
+               kc.PutB([]byte("Hello world\n"))
+               kc.PutB([]byte(""))
+       }
+
+       mt := arvadostest.PathologicalManifest
+
+       for _, testCase := range []rdrTest{
+               {mt: mt, f: "zzzz", want: os.ErrNotExist},
+               {mt: mt, f: "frob", want: os.ErrNotExist},
+               {mt: mt, f: "/segmented/frob", want: os.ErrNotExist},
+               {mt: mt, f: "./segmented/frob", want: os.ErrNotExist},
+               {mt: mt, f: "/f", want: os.ErrNotExist},
+               {mt: mt, f: "./f", want: os.ErrNotExist},
+               {mt: mt, f: "foo bar//baz", want: os.ErrNotExist},
+               {mt: mt, f: "foo/zero", want: ""},
+               {mt: mt, f: "zero@0", want: ""},
+               {mt: mt, f: "zero@1", want: ""},
+               {mt: mt, f: "zero@4", want: ""},
+               {mt: mt, f: "zero@9", want: ""},
+               {mt: mt, f: "f", want: "f"},
+               {mt: mt, f: "ooba", want: "ooba"},
+               {mt: mt, f: "overlapReverse/o", want: "o"},
+               {mt: mt, f: "overlapReverse/oo", want: "oo"},
+               {mt: mt, f: "overlapReverse/ofoo", want: "ofoo"},
+               {mt: mt, f: "foo bar/baz", want: "foo"},
+               {mt: mt, f: "segmented/frob", want: "frob"},
+               {mt: mt, f: "segmented/oof", want: "oof"},
+       } {
+               rdr, err := kc.CollectionFileReader(map[string]interface{}{"manifest_text": testCase.mt}, testCase.f)
+               switch want := testCase.want.(type) {
+               case error:
+                       c.Check(rdr, check.IsNil)
+                       c.Check(err, check.Equals, want)
+               case string:
+                       buf := make([]byte, len(want))
+                       n, err := io.ReadFull(rdr, buf)
+                       c.Check(err, check.IsNil)
+                       for i := 0; i < 4; i++ {
+                               c.Check(string(buf), check.Equals, want)
+                               n, err = rdr.Read(buf)
+                               c.Check(n, check.Equals, 0)
+                               c.Check(err, check.Equals, io.EOF)
+                       }
+                       c.Check(rdr.Close(), check.Equals, nil)
+               }
+       }
+}
index 4e816cd73b30abbb7afce9f4b51d3767a897cfa8..f104d9a1035127c3f75502854e176c72a4017e1d 100644 (file)
 package manifest
 
 import (
+       "errors"
+       "fmt"
        "git.curoverse.com/arvados.git/sdk/go/blockdigest"
        "log"
+       "regexp"
+       "strconv"
        "strings"
 )
 
+var ErrInvalidToken = errors.New("Invalid token")
+
+var LocatorPattern = regexp.MustCompile(
+       "^[0-9a-fA-F]{32}\\+[0-9]+(\\+[A-Z][A-Za-z0-9@_-]+)*$")
+
 type Manifest struct {
        Text string
 }
 
+type BlockLocator struct {
+       Digest blockdigest.BlockDigest
+       Size   int
+       Hints  []string
+}
+
+type DataSegment struct {
+       BlockLocator
+       Locator      string
+       StreamOffset uint64
+}
+
+// FileSegment is a portion of a file that is contained within a
+// single block.
+type FileSegment struct {
+       Locator string
+       // Offset (within this block) of this data segment
+       Offset int
+       Len    int
+}
+
 // Represents a single line from a manifest.
 type ManifestStream struct {
        StreamName string
        Blocks     []string
-       Files      []string
+       FileTokens []string
+}
+
+var escapeSeq = regexp.MustCompile(`\\([0-9]{3}|\\)`)
+
+func unescapeSeq(seq string) string {
+       if seq == `\\` {
+               return `\`
+       }
+       i, err := strconv.ParseUint(seq[1:], 8, 8)
+       if err != nil {
+               // Invalid escape sequence: can't unescape.
+               return seq
+       }
+       return string([]byte{byte(i)})
+}
+
+func UnescapeName(s string) string {
+       return escapeSeq.ReplaceAllStringFunc(s, unescapeSeq)
+}
+
+func ParseBlockLocator(s string) (b BlockLocator, err error) {
+       if !LocatorPattern.MatchString(s) {
+               err = fmt.Errorf("String \"%s\" does not match BlockLocator pattern "+
+                       "\"%s\".",
+                       s,
+                       LocatorPattern.String())
+       } else {
+               tokens := strings.Split(s, "+")
+               var blockSize int64
+               var blockDigest blockdigest.BlockDigest
+               // We expect both of the following to succeed since LocatorPattern
+               // restricts the strings appropriately.
+               blockDigest, err = blockdigest.FromString(tokens[0])
+               if err != nil {
+                       return
+               }
+               blockSize, err = strconv.ParseInt(tokens[1], 10, 0)
+               if err != nil {
+                       return
+               }
+               b.Digest = blockDigest
+               b.Size = int(blockSize)
+               b.Hints = tokens[2:]
+       }
+       return
+}
+
+func parseFileToken(tok string) (segPos, segLen uint64, name string, err error) {
+       parts := strings.SplitN(tok, ":", 3)
+       if len(parts) != 3 {
+               err = ErrInvalidToken
+               return
+       }
+       segPos, err = strconv.ParseUint(parts[0], 10, 64)
+       if err != nil {
+               return
+       }
+       segLen, err = strconv.ParseUint(parts[1], 10, 64)
+       if err != nil {
+               return
+       }
+       name = UnescapeName(parts[2])
+       return
+}
+
+func (s *ManifestStream) FileSegmentIterByName(filepath string) <-chan *FileSegment {
+       ch := make(chan *FileSegment)
+       go func() {
+               s.sendFileSegmentIterByName(filepath, ch)
+               close(ch)
+       }()
+       return ch
+}
+
+func (s *ManifestStream) sendFileSegmentIterByName(filepath string, ch chan<- *FileSegment) {
+       blockLens := make([]int, 0, len(s.Blocks))
+       // This is what streamName+"/"+fileName will look like:
+       target := "./" + filepath
+       for _, fTok := range s.FileTokens {
+               wantPos, wantLen, name, err := parseFileToken(fTok)
+               if err != nil {
+                       // Skip (!) invalid file tokens.
+                       continue
+               }
+               if s.StreamName+"/"+name != target {
+                       continue
+               }
+               if wantLen == 0 {
+                       ch <- &FileSegment{Locator: "d41d8cd98f00b204e9800998ecf8427e+0", Offset: 0, Len: 0}
+                       continue
+               }
+               // Linear search for blocks containing data for this
+               // file
+               var blockPos uint64 = 0 // position of block in stream
+               for i, loc := range s.Blocks {
+                       if blockPos >= wantPos+wantLen {
+                               break
+                       }
+                       if len(blockLens) <= i {
+                               blockLens = blockLens[:i+1]
+                               b, err := ParseBlockLocator(loc)
+                               if err != nil {
+                                       // Unparseable locator -> unusable
+                                       // stream.
+                                       ch <- nil
+                                       return
+                               }
+                               blockLens[i] = b.Size
+                       }
+                       blockLen := uint64(blockLens[i])
+                       if blockPos+blockLen <= wantPos {
+                               blockPos += blockLen
+                               continue
+                       }
+                       fseg := FileSegment{
+                               Locator: loc,
+                               Offset:  0,
+                               Len:     blockLens[i],
+                       }
+                       if blockPos < wantPos {
+                               fseg.Offset = int(wantPos - blockPos)
+                               fseg.Len -= fseg.Offset
+                       }
+                       if blockPos+blockLen > wantPos+wantLen {
+                               fseg.Len = int(wantPos+wantLen-blockPos) - fseg.Offset
+                       }
+                       ch <- &fseg
+                       blockPos += blockLen
+               }
+       }
 }
 
 func parseManifestStream(s string) (m ManifestStream) {
        tokens := strings.Split(s, " ")
-       m.StreamName = tokens[0]
+       m.StreamName = UnescapeName(tokens[0])
        tokens = tokens[1:]
        var i int
        for i = range tokens {
@@ -32,7 +192,7 @@ func parseManifestStream(s string) (m ManifestStream) {
                }
        }
        m.Blocks = tokens[:i]
-       m.Files = tokens[i:]
+       m.FileTokens = tokens[i:]
        return
 }
 
@@ -58,6 +218,20 @@ func (m *Manifest) StreamIter() <-chan ManifestStream {
        return ch
 }
 
+func (m *Manifest) FileSegmentIterByName(filepath string) <-chan *FileSegment {
+       ch := make(chan *FileSegment)
+       go func() {
+               for stream := range m.StreamIter() {
+                       if !strings.HasPrefix("./"+filepath, stream.StreamName+"/") {
+                               continue
+                       }
+                       stream.sendFileSegmentIterByName(filepath, ch)
+               }
+               close(ch)
+       }()
+       return ch
+}
+
 // Blocks may appear mulitple times within the same manifest if they
 // are used by multiple files. In that case this Iterator will output
 // the same block multiple times.
index 8cfe3d907721e4f7d33048dfa16ef91e38dec12d..364648d643cea64dadc706eb1f5c42e227ccbb39 100644 (file)
@@ -1,10 +1,13 @@
 package manifest
 
 import (
-       "git.curoverse.com/arvados.git/sdk/go/blockdigest"
        "io/ioutil"
+       "reflect"
        "runtime"
        "testing"
+
+       "git.curoverse.com/arvados.git/sdk/go/arvadostest"
+       "git.curoverse.com/arvados.git/sdk/go/blockdigest"
 )
 
 func getStackTrace() string {
@@ -60,7 +63,7 @@ func expectStringSlicesEqual(t *testing.T, actual []string, expected []string) {
 func expectManifestStream(t *testing.T, actual ManifestStream, expected ManifestStream) {
        expectEqual(t, actual.StreamName, expected.StreamName)
        expectStringSlicesEqual(t, actual.Blocks, expected.Blocks)
-       expectStringSlicesEqual(t, actual.Files, expected.Files)
+       expectStringSlicesEqual(t, actual.FileTokens, expected.FileTokens)
 }
 
 func expectBlockLocator(t *testing.T, actual blockdigest.BlockLocator, expected blockdigest.BlockLocator) {
@@ -72,8 +75,19 @@ func expectBlockLocator(t *testing.T, actual blockdigest.BlockLocator, expected
 func TestParseManifestStreamSimple(t *testing.T) {
        m := parseManifestStream(". 365f83f5f808896ec834c8b595288735+2310+K@qr1hi+Af0c9a66381f3b028677411926f0be1c6282fe67c@542b5ddf 0:2310:qr1hi-8i9sb-ienvmpve1a0vpoi.log.txt")
        expectManifestStream(t, m, ManifestStream{StreamName: ".",
-               Blocks: []string{"365f83f5f808896ec834c8b595288735+2310+K@qr1hi+Af0c9a66381f3b028677411926f0be1c6282fe67c@542b5ddf"},
-               Files:  []string{"0:2310:qr1hi-8i9sb-ienvmpve1a0vpoi.log.txt"}})
+               Blocks:     []string{"365f83f5f808896ec834c8b595288735+2310+K@qr1hi+Af0c9a66381f3b028677411926f0be1c6282fe67c@542b5ddf"},
+               FileTokens: []string{"0:2310:qr1hi-8i9sb-ienvmpve1a0vpoi.log.txt"}})
+}
+
+func TestParseBlockLocatorSimple(t *testing.T) {
+       b, err := ParseBlockLocator("365f83f5f808896ec834c8b595288735+2310+K@qr1hi+Af0c9a66381f3b028677411926f0be1c6282fe67c@542b5ddf")
+       if err != nil {
+               t.Fatalf("Unexpected error parsing block locator: %v", err)
+       }
+       expectBlockLocator(t, b, BlockLocator{Digest: blockdigest.AssertFromString("365f83f5f808896ec834c8b595288735"),
+               Size: 2310,
+               Hints: []string{"K@qr1hi",
+                       "Af0c9a66381f3b028677411926f0be1c6282fe67c@542b5ddf"}})
 }
 
 func TestStreamIterShortManifestWithBlankStreams(t *testing.T) {
@@ -88,8 +102,8 @@ func TestStreamIterShortManifestWithBlankStreams(t *testing.T) {
        expectManifestStream(t,
                firstStream,
                ManifestStream{StreamName: ".",
-                       Blocks: []string{"b746e3d2104645f2f64cd3cc69dd895d+15693477+E2866e643690156651c03d876e638e674dcd79475@5441920c"},
-                       Files:  []string{"0:15893477:chr10_band0_s0_e3000000.fj"}})
+                       Blocks:     []string{"b746e3d2104645f2f64cd3cc69dd895d+15693477+E2866e643690156651c03d876e638e674dcd79475@5441920c"},
+                       FileTokens: []string{"0:15893477:chr10_band0_s0_e3000000.fj"}})
 
        received, ok := <-streamIter
        if ok {
@@ -126,3 +140,58 @@ func TestBlockIterLongManifest(t *testing.T) {
                        Size:  31367794,
                        Hints: []string{"E53f903684239bcc114f7bf8ff9bd6089f33058db@5441920c"}})
 }
+
+func TestUnescape(t *testing.T) {
+       for _, testCase := range [][]string{
+               {`\040`, ` `},
+               {`\009`, `\009`},
+               {`\\\040\\`, `\ \`},
+               {`\\040\`, `\040\`},
+       } {
+               in := testCase[0]
+               expect := testCase[1]
+               got := UnescapeName(in)
+               if expect != got {
+                       t.Errorf("For '%s' got '%s' instead of '%s'", in, got, expect)
+               }
+       }
+}
+
+type fsegtest struct {
+       mt   string        // manifest text
+       f    string        // filename
+       want []FileSegment // segments should be received on channel
+}
+
+func TestFileSegmentIterByName(t *testing.T) {
+       mt := arvadostest.PathologicalManifest
+       for _, testCase := range []fsegtest{
+               {mt: mt, f: "zzzz", want: nil},
+               // This case is too sensitive: it would be acceptable
+               // (even preferable) to return only one empty segment.
+               {mt: mt, f: "foo/zero", want: []FileSegment{{"d41d8cd98f00b204e9800998ecf8427e+0", 0, 0}, {"d41d8cd98f00b204e9800998ecf8427e+0", 0, 0}}},
+               {mt: mt, f: "zero@0", want: []FileSegment{{"d41d8cd98f00b204e9800998ecf8427e+0", 0, 0}}},
+               {mt: mt, f: "zero@1", want: []FileSegment{{"d41d8cd98f00b204e9800998ecf8427e+0", 0, 0}}},
+               {mt: mt, f: "zero@4", want: []FileSegment{{"d41d8cd98f00b204e9800998ecf8427e+0", 0, 0}}},
+               {mt: mt, f: "zero@9", want: []FileSegment{{"d41d8cd98f00b204e9800998ecf8427e+0", 0, 0}}},
+               {mt: mt, f: "f", want: []FileSegment{{"acbd18db4cc2f85cedef654fccc4a4d8+3", 0, 1}}},
+               {mt: mt, f: "ooba", want: []FileSegment{{"acbd18db4cc2f85cedef654fccc4a4d8+3", 1, 2}, {"37b51d194a7513e45b56f6524f2d51f2+3", 0, 2}}},
+               {mt: mt, f: "overlapReverse/o", want: []FileSegment{{"acbd18db4cc2f85cedef654fccc4a4d8+3", 2, 1}}},
+               {mt: mt, f: "overlapReverse/oo", want: []FileSegment{{"acbd18db4cc2f85cedef654fccc4a4d8+3", 1, 2}}},
+               {mt: mt, f: "overlapReverse/ofoo", want: []FileSegment{{"acbd18db4cc2f85cedef654fccc4a4d8+3", 2, 1}, {"acbd18db4cc2f85cedef654fccc4a4d8+3", 0, 3}}},
+               {mt: mt, f: "foo bar/baz", want: []FileSegment{{"acbd18db4cc2f85cedef654fccc4a4d8+3", 0, 3}}},
+               // This case is too sensitive: it would be better to
+               // omit the empty segment.
+               {mt: mt, f: "segmented/frob", want: []FileSegment{{"acbd18db4cc2f85cedef654fccc4a4d8+3", 0, 1}, {"37b51d194a7513e45b56f6524f2d51f2+3", 2, 1}, {"acbd18db4cc2f85cedef654fccc4a4d8+3", 1, 1}, {"d41d8cd98f00b204e9800998ecf8427e+0", 0, 0}, {"37b51d194a7513e45b56f6524f2d51f2+3", 0, 1}}},
+               {mt: mt, f: "segmented/oof", want: []FileSegment{{"acbd18db4cc2f85cedef654fccc4a4d8+3", 1, 2}, {"acbd18db4cc2f85cedef654fccc4a4d8+3", 0, 1}}},
+       } {
+               m := Manifest{Text: testCase.mt}
+               var got []FileSegment
+               for fs := range m.FileSegmentIterByName(testCase.f) {
+                       got = append(got, *fs)
+               }
+               if !reflect.DeepEqual(got, testCase.want) {
+                       t.Errorf("For %#v:\n got  %#v\n want %#v", testCase.f, got, testCase.want)
+               }
+       }
+}
index 48e364091d34d87bf527567a6f8d12aad80a3f36..04af920b9e868d5db68fae78be9f365e98481f41 100644 (file)
@@ -11,6 +11,7 @@ import (
        "git.curoverse.com/arvados.git/sdk/go/arvadosclient"
        "git.curoverse.com/arvados.git/sdk/go/auth"
        "git.curoverse.com/arvados.git/sdk/go/httpserver"
+       "git.curoverse.com/arvados.git/sdk/go/keepclient"
 )
 
 var clientPool = arvadosclient.MakeClientPool()
@@ -136,17 +137,20 @@ func (h *handler) ServeHTTP(wOrig http.ResponseWriter, r *http.Request) {
        }
 
        filename := strings.Join(targetPath, "/")
-       rdr, err := arvadosclient.CollectionFileReader(collection, filename)
+       kc, err := keepclient.MakeKeepClient(arv)
+       if err != nil {
+               statusCode, statusText = http.StatusInternalServerError, err.Error()
+               return
+       }
+       rdr, err := kc.CollectionFileReader(collection, filename)
        if os.IsNotExist(err) {
                statusCode = http.StatusNotFound
                return
-       } else if err == arvadosclient.ErrNotImplemented {
-               statusCode = http.StatusNotImplemented
-               return
        } else if err != nil {
                statusCode, statusText = http.StatusBadGateway, err.Error()
                return
        }
+       defer rdr.Close()
 
        // One or both of these can be -1 if not found:
        basenamePos := strings.LastIndex(filename, "/")
index 66c6812001a198d827d5856dfc396a72c38132a2..b4d6d17a3c252a11d757d34d5a55228544808d09 100644 (file)
@@ -15,16 +15,7 @@ import (
 
 var _ = check.Suite(&IntegrationSuite{})
 
-const (
-       spectatorToken  = "zw2f4gwx8hw8cjre7yp6v1zylhrhn3m5gvjq73rtpwhmknrybu"
-       activeToken     = "3kg6k6lzmp9kj5cpkcoxie963cmvjahbt2fod9zru30k1jqdmi"
-       anonymousToken  = "4kg6k6lzmp9kj4cpkcoxie964cmvjahbt4fod9zru44k4jqdmi"
-       fooCollection   = "zzzzz-4zz18-fy296fx3hot09f7"
-       bogusCollection = "zzzzz-4zz18-totallynotexist"
-       hwCollection    = "zzzzz-4zz18-4en62shvi99lxd4"
-)
-
-// IntegrationSuite tests need an API server and an arv-git-httpd server
+// IntegrationSuite tests need an API server and a keep-web server
 type IntegrationSuite struct {
        testServer *server
 }
@@ -34,12 +25,12 @@ func (s *IntegrationSuite) TestNoToken(c *check.C) {
                "",
                "bogustoken",
        } {
-               hdr, body := s.runCurl(c, token, "/collections/"+fooCollection+"/foo")
+               hdr, body := s.runCurl(c, token, "/collections/"+arvadostest.FooCollection+"/foo")
                c.Check(hdr, check.Matches, `(?s)HTTP/1.1 401 Unauthorized\r\n.*`)
                c.Check(body, check.Equals, "")
 
                if token != "" {
-                       hdr, body = s.runCurl(c, token, "/collections/download/"+fooCollection+"/"+token+"/foo")
+                       hdr, body = s.runCurl(c, token, "/collections/download/"+arvadostest.FooCollection+"/"+token+"/foo")
                        c.Check(hdr, check.Matches, `(?s)HTTP/1.1 404 Not Found\r\n.*`)
                        c.Check(body, check.Equals, "")
                }
@@ -62,46 +53,46 @@ func (s *IntegrationSuite) Test404(c *check.C) {
                "/download",
                "/collections",
                "/collections/",
-               "/collections/" + fooCollection,
-               "/collections/" + fooCollection + "/",
+               "/collections/" + arvadostest.FooCollection,
+               "/collections/" + arvadostest.FooCollection + "/",
                // Non-existent file in collection
-               "/collections/" + fooCollection + "/theperthcountyconspiracy",
-               "/collections/download/" + fooCollection + "/" + activeToken + "/theperthcountyconspiracy",
+               "/collections/" + arvadostest.FooCollection + "/theperthcountyconspiracy",
+               "/collections/download/" + arvadostest.FooCollection + "/" + arvadostest.ActiveToken + "/theperthcountyconspiracy",
                // Non-existent collection
-               "/collections/" + bogusCollection,
-               "/collections/" + bogusCollection + "/",
-               "/collections/" + bogusCollection + "/theperthcountyconspiracy",
-               "/collections/download/" + bogusCollection + "/" + activeToken + "/theperthcountyconspiracy",
+               "/collections/" + arvadostest.NonexistentCollection,
+               "/collections/" + arvadostest.NonexistentCollection + "/",
+               "/collections/" + arvadostest.NonexistentCollection + "/theperthcountyconspiracy",
+               "/collections/download/" + arvadostest.NonexistentCollection + "/" + arvadostest.ActiveToken + "/theperthcountyconspiracy",
        } {
-               hdr, body := s.runCurl(c, activeToken, uri)
+               hdr, body := s.runCurl(c, arvadostest.ActiveToken, uri)
                c.Check(hdr, check.Matches, "(?s)HTTP/1.1 404 Not Found\r\n.*")
                c.Check(body, check.Equals, "")
        }
 }
 
 func (s *IntegrationSuite) Test200(c *check.C) {
-       anonymousTokens = []string{anonymousToken}
+       anonymousTokens = []string{arvadostest.AnonymousToken}
        arv, err := arvadosclient.MakeArvadosClient()
        c.Assert(err, check.Equals, nil)
-       arv.ApiToken = activeToken
+       arv.ApiToken = arvadostest.ActiveToken
        kc, err := keepclient.MakeKeepClient(&arv)
        c.Assert(err, check.Equals, nil)
        kc.PutB([]byte("Hello world\n"))
        kc.PutB([]byte("foo"))
        for _, spec := range [][]string{
                // My collection
-               {activeToken, "/collections/" + fooCollection + "/foo", "acbd18db4cc2f85cedef654fccc4a4d8"},
-               {"", "/collections/download/" + fooCollection + "/" + activeToken + "/foo", "acbd18db4cc2f85cedef654fccc4a4d8"},
-               {"tokensobogus", "/collections/download/" + fooCollection + "/" + activeToken + "/foo", "acbd18db4cc2f85cedef654fccc4a4d8"},
-               {activeToken, "/collections/download/" + fooCollection + "/" + activeToken + "/foo", "acbd18db4cc2f85cedef654fccc4a4d8"},
-               {anonymousToken, "/collections/download/" + fooCollection + "/" + activeToken + "/foo", "acbd18db4cc2f85cedef654fccc4a4d8"},
+               {arvadostest.ActiveToken, "/collections/" + arvadostest.FooCollection + "/foo", "acbd18db4cc2f85cedef654fccc4a4d8"},
+               {"", "/collections/download/" + arvadostest.FooCollection + "/" + arvadostest.ActiveToken + "/foo", "acbd18db4cc2f85cedef654fccc4a4d8"},
+               {"tokensobogus", "/collections/download/" + arvadostest.FooCollection + "/" + arvadostest.ActiveToken + "/foo", "acbd18db4cc2f85cedef654fccc4a4d8"},
+               {arvadostest.ActiveToken, "/collections/download/" + arvadostest.FooCollection + "/" + arvadostest.ActiveToken + "/foo", "acbd18db4cc2f85cedef654fccc4a4d8"},
+               {arvadostest.AnonymousToken, "/collections/download/" + arvadostest.FooCollection + "/" + arvadostest.ActiveToken + "/foo", "acbd18db4cc2f85cedef654fccc4a4d8"},
                // Anonymously accessible user agreement. These should
                // start working when CollectionFileReader provides
                // real data instead of fake/stub data.
-               {"", "/collections/"+hwCollection+"/Hello%20world.txt", "f0ef7081e1539ac00ef5b761b4fb01b3"},
-               {activeToken, "/collections/"+hwCollection+"/Hello%20world.txt", "f0ef7081e1539ac00ef5b761b4fb01b3"},
-               {spectatorToken, "/collections/"+hwCollection+"/Hello%20world.txt", "f0ef7081e1539ac00ef5b761b4fb01b3"},
-               {spectatorToken, "/collections/download/"+hwCollection+"/"+spectatorToken+"/Hello%20world.txt", "f0ef7081e1539ac00ef5b761b4fb01b3"},
+               {"", "/collections/" + arvadostest.HelloWorldCollection + "/Hello%20world.txt", "f0ef7081e1539ac00ef5b761b4fb01b3"},
+               {arvadostest.ActiveToken, "/collections/" + arvadostest.HelloWorldCollection + "/Hello%20world.txt", "f0ef7081e1539ac00ef5b761b4fb01b3"},
+               {arvadostest.SpectatorToken, "/collections/" + arvadostest.HelloWorldCollection + "/Hello%20world.txt", "f0ef7081e1539ac00ef5b761b4fb01b3"},
+               {arvadostest.SpectatorToken, "/collections/download/" + arvadostest.HelloWorldCollection + "/" + arvadostest.SpectatorToken + "/Hello%20world.txt", "f0ef7081e1539ac00ef5b761b4fb01b3"},
        } {
                hdr, body := s.runCurl(c, spec[0], spec[1])
                if strings.HasPrefix(hdr, "HTTP/1.1 501 Not Implemented\r\n") && body == "" {