Merge pull request #642 from ijc/readdir

Add `ReadDir` to the gateway interface
docker-18.09
Tõnis Tiigi 2018-10-04 14:10:38 -07:00 committed by GitHub
commit ad7e104d02
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
41 changed files with 1732 additions and 435 deletions

View File

@ -19,6 +19,7 @@ import (
digest "github.com/opencontainers/go-digest"
"github.com/pkg/errors"
"github.com/tonistiigi/fsutil"
fstypes "github.com/tonistiigi/fsutil/types"
)
var errNotFound = errors.Errorf("not found")
@ -274,7 +275,7 @@ func (cc *cacheContext) HandleChange(kind fsutil.ChangeKind, p string, fi os.Fil
return
}
stat, ok := fi.Sys().(*fsutil.Stat)
stat, ok := fi.Sys().(*fstypes.Stat)
if !ok {
return errors.Errorf("%s invalid change without stat information", p)
}

View File

@ -20,6 +20,7 @@ import (
"github.com/pkg/errors"
"github.com/stretchr/testify/require"
"github.com/tonistiigi/fsutil"
fstypes "github.com/tonistiigi/fsutil/types"
)
const (
@ -610,7 +611,7 @@ func parseChange(str string) *change {
panic(errStr)
}
c.path = f[1]
st := &fsutil.Stat{}
st := &fstypes.Stat{}
switch f[2] {
case "file":
if len(f) > 3 {
@ -640,7 +641,7 @@ func parseChange(str string) *change {
func emit(fn fsutil.HandleChangeFn, inp []*change) error {
for _, c := range inp {
stat, ok := c.fi.Sys().(*fsutil.Stat)
stat, ok := c.fi.Sys().(*fstypes.Stat)
if !ok {
return errors.Errorf("invalid non-stat change %s", c.fi.Name())
}
@ -675,7 +676,7 @@ func writeChanges(p string, inp []*change) error {
for _, c := range inp {
if c.kind == fsutil.ChangeKindAdd {
p := filepath.Join(p, c.path)
stat, ok := c.fi.Sys().(*fsutil.Stat)
stat, ok := c.fi.Sys().(*fstypes.Stat)
if !ok {
return errors.Errorf("invalid non-stat change %s", p)
}

View File

@ -8,7 +8,7 @@ import (
"path/filepath"
"time"
"github.com/tonistiigi/fsutil"
fstypes "github.com/tonistiigi/fsutil/types"
)
// NewFileHash returns new hash that is used for the builder cache keys
@ -22,7 +22,7 @@ func NewFileHash(path string, fi os.FileInfo) (hash.Hash, error) {
}
}
stat := &fsutil.Stat{
stat := &fstypes.Stat{
Mode: uint32(fi.Mode()),
Size_: fi.Size(),
ModTime: fi.ModTime().UnixNano(),
@ -39,7 +39,7 @@ func NewFileHash(path string, fi os.FileInfo) (hash.Hash, error) {
return NewFromStat(stat)
}
func NewFromStat(stat *fsutil.Stat) (hash.Hash, error) {
func NewFromStat(stat *fstypes.Stat) (hash.Hash, error) {
fi := &statInfo{stat}
hdr, err := tar.FileInfoHeader(fi, stat.Linkname)
if err != nil {
@ -75,7 +75,7 @@ func (tsh *tarsumHash) Reset() {
}
type statInfo struct {
*fsutil.Stat
*fstypes.Stat
}
func (s *statInfo) Name() string {

View File

@ -7,7 +7,7 @@ import (
"syscall"
"github.com/containerd/continuity/sysx"
"github.com/tonistiigi/fsutil"
fstypes "github.com/tonistiigi/fsutil/types"
"golang.org/x/sys/unix"
)
@ -16,7 +16,7 @@ func chmodWindowsTarEntry(perm os.FileMode) os.FileMode {
return perm
}
func setUnixOpt(path string, fi os.FileInfo, stat *fsutil.Stat) error {
func setUnixOpt(path string, fi os.FileInfo, stat *fstypes.Stat) error {
s := fi.Sys().(*syscall.Stat_t)
stat.Uid = s.Uid

View File

@ -5,7 +5,7 @@ package contenthash
import (
"os"
"github.com/tonistiigi/fsutil"
fstypes "github.com/tonistiigi/fsutil/types"
)
// chmodWindowsTarEntry is used to adjust the file permissions used in tar
@ -18,6 +18,6 @@ func chmodWindowsTarEntry(perm os.FileMode) os.FileMode {
return perm
}
func setUnixOpt(path string, fi os.FileInfo, stat *fsutil.Stat) error {
func setUnixOpt(path string, fi os.FileInfo, stat *fstypes.Stat) error {
return nil
}

71
cache/fsutil.go vendored
View File

@ -1,71 +0,0 @@
package cache
import (
"context"
"io"
"io/ioutil"
"os"
"github.com/containerd/continuity/fs"
"github.com/moby/buildkit/snapshot"
)
type ReadRequest struct {
Filename string
Range *FileRange
}
type FileRange struct {
Offset int
Length int
}
func ReadFile(ctx context.Context, ref ImmutableRef, req ReadRequest) ([]byte, error) {
mount, err := ref.Mount(ctx, true)
if err != nil {
return nil, err
}
lm := snapshot.LocalMounter(mount)
root, err := lm.Mount()
if err != nil {
return nil, err
}
defer func() {
if lm != nil {
lm.Unmount()
}
}()
fp, err := fs.RootPath(root, req.Filename)
if err != nil {
return nil, err
}
var dt []byte
if req.Range == nil {
dt, err = ioutil.ReadFile(fp)
if err != nil {
return nil, err
}
} else {
f, err := os.Open(fp)
if err != nil {
return nil, err
}
dt, err = ioutil.ReadAll(io.NewSectionReader(f, int64(req.Range.Offset), int64(req.Range.Length)))
f.Close()
if err != nil {
return nil, err
}
}
if err := lm.Unmount(); err != nil {
return nil, err
}
lm = nil
return dt, err
}

139
cache/util/fsutil.go vendored Normal file
View File

@ -0,0 +1,139 @@
package util
import (
"context"
"io"
"io/ioutil"
"os"
"path/filepath"
"github.com/containerd/continuity/fs"
"github.com/moby/buildkit/cache"
"github.com/moby/buildkit/snapshot"
"github.com/pkg/errors"
"github.com/tonistiigi/fsutil"
fstypes "github.com/tonistiigi/fsutil/types"
)
type ReadRequest struct {
Filename string
Range *FileRange
}
type FileRange struct {
Offset int
Length int
}
func withMount(ctx context.Context, ref cache.ImmutableRef, cb func(string) error) error {
mount, err := ref.Mount(ctx, true)
if err != nil {
return err
}
lm := snapshot.LocalMounter(mount)
root, err := lm.Mount()
if err != nil {
return err
}
defer func() {
if lm != nil {
lm.Unmount()
}
}()
if err := cb(root); err != nil {
return err
}
if err := lm.Unmount(); err != nil {
return err
}
lm = nil
return nil
}
func ReadFile(ctx context.Context, ref cache.ImmutableRef, req ReadRequest) ([]byte, error) {
var dt []byte
err := withMount(ctx, ref, func(root string) error {
fp, err := fs.RootPath(root, req.Filename)
if err != nil {
return err
}
if req.Range == nil {
dt, err = ioutil.ReadFile(fp)
if err != nil {
return err
}
} else {
f, err := os.Open(fp)
if err != nil {
return err
}
dt, err = ioutil.ReadAll(io.NewSectionReader(f, int64(req.Range.Offset), int64(req.Range.Length)))
f.Close()
if err != nil {
return err
}
}
return nil
})
return dt, err
}
type ReadDirRequest struct {
Path string
IncludePattern string
}
func ReadDir(ctx context.Context, ref cache.ImmutableRef, req ReadDirRequest) ([]*fstypes.Stat, error) {
var (
rd []*fstypes.Stat
wo fsutil.WalkOpt
)
if req.IncludePattern != "" {
wo.IncludePatterns = append(wo.IncludePatterns, req.IncludePattern)
}
err := withMount(ctx, ref, func(root string) error {
fp, err := fs.RootPath(root, req.Path)
if err != nil {
return err
}
return fsutil.Walk(ctx, fp, &wo, func(path string, info os.FileInfo, err error) error {
if err != nil {
return errors.Wrapf(err, "walking %q", root)
}
stat, ok := info.Sys().(*fstypes.Stat)
if !ok {
// This "can't happen(tm)".
return errors.Errorf("expected a *fsutil.Stat but got %T", info.Sys())
}
rd = append(rd, stat)
if info.IsDir() {
return filepath.SkipDir
}
return nil
})
})
return rd, err
}
func StatFile(ctx context.Context, ref cache.ImmutableRef, path string) (*fstypes.Stat, error) {
var st *fstypes.Stat
err := withMount(ctx, ref, func(root string) error {
fp, err := fs.RootPath(root, path)
if err != nil {
return err
}
if st, err = fsutil.Stat(fp); err != nil {
return err
}
return nil
})
return st, err
}

View File

@ -84,6 +84,16 @@ func (g *gatewayClientForBuild) ReadFile(ctx context.Context, in *gatewayapi.Rea
return g.gateway.ReadFile(ctx, in, opts...)
}
func (g *gatewayClientForBuild) ReadDir(ctx context.Context, in *gatewayapi.ReadDirRequest, opts ...grpc.CallOption) (*gatewayapi.ReadDirResponse, error) {
ctx = buildid.AppendToOutgoingContext(ctx, g.buildID)
return g.gateway.ReadDir(ctx, in, opts...)
}
func (g *gatewayClientForBuild) StatFile(ctx context.Context, in *gatewayapi.StatFileRequest, opts ...grpc.CallOption) (*gatewayapi.StatFileResponse, error) {
ctx = buildid.AppendToOutgoingContext(ctx, g.buildID)
return g.gateway.StatFile(ctx, in, opts...)
}
func (g *gatewayClientForBuild) Ping(ctx context.Context, in *gatewayapi.PingRequest, opts ...grpc.CallOption) (*gatewayapi.PongResponse, error) {
ctx = buildid.AppendToOutgoingContext(ctx, g.buildID)
return g.gateway.Ping(ctx, in, opts...)

View File

@ -19,7 +19,7 @@ import (
opentracing "github.com/opentracing/opentracing-go"
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
"github.com/tonistiigi/fsutil"
fstypes "github.com/tonistiigi/fsutil/types"
"golang.org/x/sync/errgroup"
)
@ -256,7 +256,7 @@ func prepareSyncedDirs(def *llb.Definition, localDirs map[string]string) ([]file
return nil, errors.Errorf("%s not a directory", d)
}
}
resetUIDAndGID := func(st *fsutil.Stat) bool {
resetUIDAndGID := func(st *fstypes.Stat) bool {
st.Uid = 0
st.Gid = 0
return true

View File

@ -125,3 +125,19 @@ func (gwf *GatewayForwarder) Return(ctx context.Context, req *gwapi.ReturnReques
res, err := fwd.Return(ctx, req)
return res, err
}
func (gwf *GatewayForwarder) ReadDir(ctx context.Context, req *gwapi.ReadDirRequest) (*gwapi.ReadDirResponse, error) {
fwd, err := gwf.lookupForwarder(ctx)
if err != nil {
return nil, errors.Wrap(err, "forwarding ReadDir")
}
return fwd.ReadDir(ctx, req)
}
func (gwf *GatewayForwarder) StatFile(ctx context.Context, req *gwapi.StatFileRequest) (*gwapi.StatFileResponse, error) {
fwd, err := gwf.lookupForwarder(ctx)
if err != nil {
return nil, errors.Wrap(err, "forwarding StatFile")
}
return fwd.StatFile(ctx, req)
}

View File

@ -15,6 +15,7 @@ import (
"github.com/moby/buildkit/util/progress"
"github.com/pkg/errors"
"github.com/tonistiigi/fsutil"
fstypes "github.com/tonistiigi/fsutil/types"
"golang.org/x/sync/errgroup"
"golang.org/x/time/rate"
)
@ -92,7 +93,7 @@ func (e *localExporterInstance) Export(ctx context.Context, inp exporter.Source)
lbl := "copying files"
if isMap {
lbl += " " + k
fs = fsutil.SubDirFS(fs, fsutil.Stat{
fs = fsutil.SubDirFS(fs, fstypes.Stat{
Mode: uint32(os.ModeDir | 0755),
Path: strings.Replace(k, "/", "_", -1),
})

280
frontend/frontend_test.go Normal file
View File

@ -0,0 +1,280 @@
package frontend
import (
"context"
"io/ioutil"
"os"
"path/filepath"
"testing"
"github.com/containerd/continuity/fs/fstest"
"github.com/moby/buildkit/client"
"github.com/moby/buildkit/client/llb"
gateway "github.com/moby/buildkit/frontend/gateway/client"
"github.com/moby/buildkit/util/testutil/integration"
"github.com/pkg/errors"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/tonistiigi/fsutil"
fstypes "github.com/tonistiigi/fsutil/types"
)
var (
errFailed = errors.New("test failed")
)
func TestFrontendIntegration(t *testing.T) {
integration.Run(t, []integration.Test{
testRefReadFile,
testRefReadDir,
testRefStatFile,
})
}
func testRefReadFile(t *testing.T, sb integration.Sandbox) {
t.Parallel()
ctx := context.TODO()
c, err := client.New(ctx, sb.Address())
require.NoError(t, err)
defer c.Close()
testcontent := []byte(`foobar`)
dir, err := tmpdir(
fstest.CreateFile("test", testcontent, 0666),
)
require.NoError(t, err)
defer os.RemoveAll(dir)
frontend := func(ctx context.Context, c gateway.Client) (*gateway.Result, error) {
def, err := llb.Local("mylocal").Marshal()
if err != nil {
return nil, err
}
res, err := c.Solve(ctx, gateway.SolveRequest{
Definition: def.ToPB(),
})
if err != nil {
return nil, err
}
ref, err := res.SingleRef()
if err != nil {
return nil, err
}
for _, tc := range []struct {
name string
exp []byte
r *gateway.FileRange
}{
{"fullfile", testcontent, nil},
{"prefix", []byte(`foo`), &gateway.FileRange{Offset: 0, Length: 3}},
{"suffix", []byte(`ar`), &gateway.FileRange{Offset: 4, Length: 2}},
{"mid", []byte(`oba`), &gateway.FileRange{Offset: 2, Length: 3}},
{"overrun", []byte(`bar`), &gateway.FileRange{Offset: 3, Length: 10}},
} {
t.Run(tc.name, func(t *testing.T) {
r, err := ref.ReadFile(ctx, gateway.ReadRequest{
Filename: "test",
Range: tc.r,
})
require.NoError(t, err)
assert.Equal(t, tc.exp, r)
})
}
return gateway.NewResult(), nil
}
_, err = c.Build(ctx, client.SolveOpt{
LocalDirs: map[string]string{
"mylocal": dir,
},
}, "", frontend, nil)
require.NoError(t, err)
}
func testRefReadDir(t *testing.T, sb integration.Sandbox) {
t.Parallel()
ctx := context.TODO()
c, err := client.New(ctx, sb.Address())
require.NoError(t, err)
defer c.Close()
dir, err := tmpdir(
fstest.CreateDir("somedir", 0777),
fstest.CreateFile("somedir/foo1.txt", []byte(`foo1`), 0666),
fstest.CreateFile("somedir/foo2.txt", []byte{}, 0666),
fstest.CreateFile("somedir/bar.log", []byte(`somethingsomething`), 0666),
fstest.Symlink("bar.log", "somedir/link.log"),
fstest.CreateDir("somedir/baz.dir", 0777),
)
require.NoError(t, err)
defer os.RemoveAll(dir)
expMap := make(map[string]*fstypes.Stat)
fsutil.Walk(ctx, dir, nil, func(path string, info os.FileInfo, err error) error {
require.NoError(t, err)
stat, ok := info.Sys().(*fstypes.Stat)
require.True(t, ok)
stat.ModTime = 0 // this will inevitably differ, we clear it during the tests below too
stat.Path = filepath.Base(stat.Path) // we are only testing reading a single directory here
expMap[path] = stat
return nil
})
frontend := func(ctx context.Context, c gateway.Client) (*gateway.Result, error) {
def, err := llb.Local("mylocal").Marshal()
if err != nil {
return nil, err
}
res, err := c.Solve(ctx, gateway.SolveRequest{
Definition: def.ToPB(),
})
if err != nil {
return nil, err
}
ref, err := res.SingleRef()
if err != nil {
return nil, err
}
for _, tc := range []struct {
name string
req gateway.ReadDirRequest
exp []*fstypes.Stat
}{
{
name: "toplevel",
req: gateway.ReadDirRequest{Path: "/"},
exp: []*fstypes.Stat{
expMap["somedir"],
},
},
{
name: "subdir",
req: gateway.ReadDirRequest{Path: "/somedir"},
exp: []*fstypes.Stat{
expMap["somedir/bar.log"],
expMap["somedir/baz.dir"],
expMap["somedir/foo1.txt"],
expMap["somedir/foo2.txt"],
expMap["somedir/link.log"],
},
},
{
name: "globtxt",
req: gateway.ReadDirRequest{Path: "/somedir", IncludePattern: "*.txt"},
exp: []*fstypes.Stat{
expMap["somedir/foo1.txt"],
expMap["somedir/foo2.txt"],
},
},
{
name: "globlog",
req: gateway.ReadDirRequest{Path: "/somedir", IncludePattern: "*.log"},
exp: []*fstypes.Stat{
expMap["somedir/bar.log"],
expMap["somedir/link.log"],
},
},
{
name: "subsubdir",
req: gateway.ReadDirRequest{Path: "/somedir", IncludePattern: "*.dir"},
exp: []*fstypes.Stat{
expMap["somedir/baz.dir"],
},
},
} {
t.Run(tc.name, func(t *testing.T) {
dirents, err := ref.ReadDir(ctx, tc.req)
require.NoError(t, err)
for _, s := range dirents {
s.ModTime = 0 // this will inevitably differ, we cleared it in the expected versions above.
}
assert.Equal(t, tc.exp, dirents)
})
}
return gateway.NewResult(), nil
}
_, err = c.Build(ctx, client.SolveOpt{
LocalDirs: map[string]string{
"mylocal": dir,
},
}, "", frontend, nil)
require.NoError(t, err)
}
func testRefStatFile(t *testing.T, sb integration.Sandbox) {
t.Parallel()
ctx := context.TODO()
c, err := client.New(ctx, sb.Address())
require.NoError(t, err)
defer c.Close()
testcontent := []byte(`foobar`)
dir, err := tmpdir(
fstest.CreateFile("test", testcontent, 0666),
)
require.NoError(t, err)
defer os.RemoveAll(dir)
exp, err := fsutil.Stat(filepath.Join(dir, "test"))
require.NoError(t, err)
frontend := func(ctx context.Context, c gateway.Client) (*gateway.Result, error) {
def, err := llb.Local("mylocal").Marshal()
if err != nil {
return nil, err
}
res, err := c.Solve(ctx, gateway.SolveRequest{
Definition: def.ToPB(),
})
if err != nil {
return nil, err
}
ref, err := res.SingleRef()
if err != nil {
return nil, err
}
st, err := ref.StatFile(ctx, gateway.StatRequest{
Path: "test",
})
require.NoError(t, err)
require.NotNil(t, st)
assert.Equal(t, exp, st)
return gateway.NewResult(), nil
}
_, err = c.Build(ctx, client.SolveOpt{
LocalDirs: map[string]string{
"mylocal": dir,
},
}, "", frontend, nil)
require.NoError(t, err)
}
func tmpdir(appliers ...fstest.Applier) (string, error) {
tmpdir, err := ioutil.TempDir("", "buildkit-frontend")
if err != nil {
return "", err
}
if err := fstest.Apply(appliers...).Apply(tmpdir); err != nil {
return "", err
}
return tmpdir, nil
}

View File

@ -7,6 +7,7 @@ import (
"github.com/moby/buildkit/util/apicaps"
digest "github.com/opencontainers/go-digest"
specs "github.com/opencontainers/image-spec/specs-go/v1"
fstypes "github.com/tonistiigi/fsutil/types"
)
type Client interface {
@ -17,8 +18,8 @@ type Client interface {
type Reference interface {
ReadFile(ctx context.Context, req ReadRequest) ([]byte, error)
// StatFile(ctx context.Context, req StatRequest) (*StatResponse, error)
// ReadDir(ctx context.Context, req ReadDirRequest) ([]*StatResponse, error)
StatFile(ctx context.Context, req StatRequest) (*fstypes.Stat, error)
ReadDir(ctx context.Context, req ReadDirRequest) ([]*fstypes.Stat, error)
}
type ReadRequest struct {
@ -31,6 +32,15 @@ type FileRange struct {
Length int
}
type ReadDirRequest struct {
Path string
IncludePattern string
}
type StatRequest struct {
Path string
}
// SolveRequest is same as frontend.SolveRequest but avoiding dependency
type SolveRequest struct {
Definition *pb.Definition

View File

@ -5,6 +5,7 @@ import (
"sync"
"github.com/moby/buildkit/cache"
cacheutil "github.com/moby/buildkit/cache/util"
clienttypes "github.com/moby/buildkit/client"
"github.com/moby/buildkit/frontend"
"github.com/moby/buildkit/frontend/gateway/client"
@ -15,6 +16,7 @@ import (
"github.com/moby/buildkit/util/apicaps"
"github.com/moby/buildkit/worker"
"github.com/pkg/errors"
fstypes "github.com/tonistiigi/fsutil/types"
)
func llbBridgeToGatewayClient(ctx context.Context, llbBridge frontend.FrontendLLBBridge, opts map[string]string, workerInfos []clienttypes.WorkerInfo) (*bridgeClient, error) {
@ -136,16 +138,36 @@ func (r *ref) ReadFile(ctx context.Context, req client.ReadRequest) ([]byte, err
if err != nil {
return nil, err
}
newReq := cache.ReadRequest{
newReq := cacheutil.ReadRequest{
Filename: req.Filename,
}
if r := req.Range; r != nil {
newReq.Range = &cache.FileRange{
newReq.Range = &cacheutil.FileRange{
Offset: r.Offset,
Length: r.Length,
}
}
return cache.ReadFile(ctx, ref, newReq)
return cacheutil.ReadFile(ctx, ref, newReq)
}
func (r *ref) ReadDir(ctx context.Context, req client.ReadDirRequest) ([]*fstypes.Stat, error) {
ref, err := r.getImmutableRef()
if err != nil {
return nil, err
}
newReq := cacheutil.ReadDirRequest{
Path: req.Path,
IncludePattern: req.IncludePattern,
}
return cacheutil.ReadDir(ctx, ref, newReq)
}
func (r *ref) StatFile(ctx context.Context, req client.StatRequest) (*fstypes.Stat, error) {
ref, err := r.getImmutableRef()
if err != nil {
return nil, err
}
return cacheutil.StatFile(ctx, ref, req.Path)
}
func (r *ref) getImmutableRef() (cache.ImmutableRef, error) {

View File

@ -14,6 +14,7 @@ import (
"github.com/docker/distribution/reference"
apitypes "github.com/moby/buildkit/api/types"
"github.com/moby/buildkit/cache"
cacheutil "github.com/moby/buildkit/cache/util"
"github.com/moby/buildkit/client"
"github.com/moby/buildkit/client/llb"
"github.com/moby/buildkit/executor"
@ -479,24 +480,24 @@ func (lbf *llbBridgeForwarder) ReadFile(ctx context.Context, req *pb.ReadFileReq
return nil, errors.Errorf("no such ref: %v", req.Ref)
}
if ref == nil {
return nil, errors.Wrapf(os.ErrNotExist, "%s no found", req.FilePath)
return nil, errors.Wrapf(os.ErrNotExist, "%s not found", req.FilePath)
}
workerRef, ok := ref.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid ref: %T", ref.Sys())
}
newReq := cache.ReadRequest{
newReq := cacheutil.ReadRequest{
Filename: req.FilePath,
}
if r := req.Range; r != nil {
newReq.Range = &cache.FileRange{
newReq.Range = &cacheutil.FileRange{
Offset: int(r.Offset),
Length: int(r.Length),
}
}
dt, err := cache.ReadFile(ctx, workerRef.ImmutableRef, newReq)
dt, err := cacheutil.ReadFile(ctx, workerRef.ImmutableRef, newReq)
if err != nil {
return nil, err
}
@ -504,6 +505,58 @@ func (lbf *llbBridgeForwarder) ReadFile(ctx context.Context, req *pb.ReadFileReq
return &pb.ReadFileResponse{Data: dt}, nil
}
func (lbf *llbBridgeForwarder) ReadDir(ctx context.Context, req *pb.ReadDirRequest) (*pb.ReadDirResponse, error) {
ctx = tracing.ContextWithSpanFromContext(ctx, lbf.callCtx)
lbf.mu.Lock()
ref, ok := lbf.refs[req.Ref]
lbf.mu.Unlock()
if !ok {
return nil, errors.Errorf("no such ref: %v", req.Ref)
}
if ref == nil {
return nil, errors.Wrapf(os.ErrNotExist, "%s not found", req.DirPath)
}
workerRef, ok := ref.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid ref: %T", ref.Sys())
}
newReq := cacheutil.ReadDirRequest{
Path: req.DirPath,
IncludePattern: req.IncludePattern,
}
entries, err := cacheutil.ReadDir(ctx, workerRef.ImmutableRef, newReq)
if err != nil {
return nil, err
}
return &pb.ReadDirResponse{Entries: entries}, nil
}
func (lbf *llbBridgeForwarder) StatFile(ctx context.Context, req *pb.StatFileRequest) (*pb.StatFileResponse, error) {
ctx = tracing.ContextWithSpanFromContext(ctx, lbf.callCtx)
lbf.mu.Lock()
ref, ok := lbf.refs[req.Ref]
lbf.mu.Unlock()
if !ok {
return nil, errors.Errorf("no such ref: %v", req.Ref)
}
if ref == nil {
return nil, errors.Wrapf(os.ErrNotExist, "%s not found", req.Path)
}
workerRef, ok := ref.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid ref: %T", ref.Sys())
}
st, err := cacheutil.StatFile(ctx, workerRef.ImmutableRef, req.Path)
if err != nil {
return nil, err
}
return &pb.StatFileResponse{Stat: st}, nil
}
func (lbf *llbBridgeForwarder) Ping(context.Context, *pb.PingRequest) (*pb.PongResponse, error) {
workers := lbf.workers.WorkerInfos()

View File

@ -16,6 +16,7 @@ import (
"github.com/moby/buildkit/util/apicaps"
digest "github.com/opencontainers/go-digest"
"github.com/pkg/errors"
fstypes "github.com/tonistiigi/fsutil/types"
"google.golang.org/grpc"
"google.golang.org/grpc/status"
)
@ -354,6 +355,31 @@ func (r *reference) ReadFile(ctx context.Context, req client.ReadRequest) ([]byt
return resp.Data, nil
}
func (r *reference) ReadDir(ctx context.Context, req client.ReadDirRequest) ([]*fstypes.Stat, error) {
rdr := &pb.ReadDirRequest{
DirPath: req.Path,
IncludePattern: req.IncludePattern,
Ref: r.id,
}
resp, err := r.c.client.ReadDir(ctx, rdr)
if err != nil {
return nil, err
}
return resp.Entries, nil
}
func (r *reference) StatFile(ctx context.Context, req client.StatRequest) (*fstypes.Stat, error) {
rdr := &pb.StatFileRequest{
Path: req.Path,
Ref: r.id,
}
resp, err := r.c.client.StatFile(ctx, rdr)
if err != nil {
return nil, err
}
return resp.Stat, nil
}
func grpcClientConn(ctx context.Context) (context.Context, *grpc.ClientConn, error) {
dialOpt := grpc.WithDialer(func(addr string, d time.Duration) (net.Conn, error) {
return stdioConn(), nil

View File

@ -16,6 +16,8 @@ const (
CapReadFile apicaps.CapID = "readfile"
CapReturnResult apicaps.CapID = "return"
CapReturnMap apicaps.CapID = "returnmap"
CapReadDir apicaps.CapID = "readdir"
CapStatFile apicaps.CapID = "statfile"
)
func init() {
@ -69,4 +71,17 @@ func init() {
Status: apicaps.CapStatusExperimental,
})
Caps.Init(apicaps.Cap{
ID: CapReadDir,
Name: "read static directory",
Enabled: true,
Status: apicaps.CapStatusExperimental,
})
Caps.Init(apicaps.Cap{
ID: CapStatFile,
Name: "stat a file",
Enabled: true,
Status: apicaps.CapStatusExperimental,
})
}

View File

@ -19,6 +19,10 @@
ReadFileRequest
FileRange
ReadFileResponse
ReadDirRequest
ReadDirResponse
StatFileRequest
StatFileResponse
PingRequest
PongResponse
*/
@ -32,6 +36,7 @@ import google_rpc "github.com/gogo/googleapis/google/rpc"
import pb "github.com/moby/buildkit/solver/pb"
import moby_buildkit_v1_types "github.com/moby/buildkit/api/types"
import moby_buildkit_v1_apicaps "github.com/moby/buildkit/util/apicaps/pb"
import fsutil_types "github.com/tonistiigi/fsutil/types"
import github_com_opencontainers_go_digest "github.com/opencontainers/go-digest"
@ -448,13 +453,101 @@ func (m *ReadFileResponse) GetData() []byte {
return nil
}
type ReadDirRequest struct {
Ref string `protobuf:"bytes,1,opt,name=Ref,proto3" json:"Ref,omitempty"`
DirPath string `protobuf:"bytes,2,opt,name=DirPath,proto3" json:"DirPath,omitempty"`
IncludePattern string `protobuf:"bytes,3,opt,name=IncludePattern,proto3" json:"IncludePattern,omitempty"`
}
func (m *ReadDirRequest) Reset() { *m = ReadDirRequest{} }
func (m *ReadDirRequest) String() string { return proto.CompactTextString(m) }
func (*ReadDirRequest) ProtoMessage() {}
func (*ReadDirRequest) Descriptor() ([]byte, []int) { return fileDescriptorGateway, []int{11} }
func (m *ReadDirRequest) GetRef() string {
if m != nil {
return m.Ref
}
return ""
}
func (m *ReadDirRequest) GetDirPath() string {
if m != nil {
return m.DirPath
}
return ""
}
func (m *ReadDirRequest) GetIncludePattern() string {
if m != nil {
return m.IncludePattern
}
return ""
}
type ReadDirResponse struct {
Entries []*fsutil_types.Stat `protobuf:"bytes,1,rep,name=entries" json:"entries,omitempty"`
}
func (m *ReadDirResponse) Reset() { *m = ReadDirResponse{} }
func (m *ReadDirResponse) String() string { return proto.CompactTextString(m) }
func (*ReadDirResponse) ProtoMessage() {}
func (*ReadDirResponse) Descriptor() ([]byte, []int) { return fileDescriptorGateway, []int{12} }
func (m *ReadDirResponse) GetEntries() []*fsutil_types.Stat {
if m != nil {
return m.Entries
}
return nil
}
type StatFileRequest struct {
Ref string `protobuf:"bytes,1,opt,name=Ref,proto3" json:"Ref,omitempty"`
Path string `protobuf:"bytes,2,opt,name=Path,proto3" json:"Path,omitempty"`
}
func (m *StatFileRequest) Reset() { *m = StatFileRequest{} }
func (m *StatFileRequest) String() string { return proto.CompactTextString(m) }
func (*StatFileRequest) ProtoMessage() {}
func (*StatFileRequest) Descriptor() ([]byte, []int) { return fileDescriptorGateway, []int{13} }
func (m *StatFileRequest) GetRef() string {
if m != nil {
return m.Ref
}
return ""
}
func (m *StatFileRequest) GetPath() string {
if m != nil {
return m.Path
}
return ""
}
type StatFileResponse struct {
Stat *fsutil_types.Stat `protobuf:"bytes,1,opt,name=stat" json:"stat,omitempty"`
}
func (m *StatFileResponse) Reset() { *m = StatFileResponse{} }
func (m *StatFileResponse) String() string { return proto.CompactTextString(m) }
func (*StatFileResponse) ProtoMessage() {}
func (*StatFileResponse) Descriptor() ([]byte, []int) { return fileDescriptorGateway, []int{14} }
func (m *StatFileResponse) GetStat() *fsutil_types.Stat {
if m != nil {
return m.Stat
}
return nil
}
type PingRequest struct {
}
func (m *PingRequest) Reset() { *m = PingRequest{} }
func (m *PingRequest) String() string { return proto.CompactTextString(m) }
func (*PingRequest) ProtoMessage() {}
func (*PingRequest) Descriptor() ([]byte, []int) { return fileDescriptorGateway, []int{11} }
func (*PingRequest) Descriptor() ([]byte, []int) { return fileDescriptorGateway, []int{15} }
type PongResponse struct {
FrontendAPICaps []moby_buildkit_v1_apicaps.APICap `protobuf:"bytes,1,rep,name=FrontendAPICaps" json:"FrontendAPICaps"`
@ -465,7 +558,7 @@ type PongResponse struct {
func (m *PongResponse) Reset() { *m = PongResponse{} }
func (m *PongResponse) String() string { return proto.CompactTextString(m) }
func (*PongResponse) ProtoMessage() {}
func (*PongResponse) Descriptor() ([]byte, []int) { return fileDescriptorGateway, []int{12} }
func (*PongResponse) Descriptor() ([]byte, []int) { return fileDescriptorGateway, []int{16} }
func (m *PongResponse) GetFrontendAPICaps() []moby_buildkit_v1_apicaps.APICap {
if m != nil {
@ -500,6 +593,10 @@ func init() {
proto.RegisterType((*ReadFileRequest)(nil), "moby.buildkit.v1.frontend.ReadFileRequest")
proto.RegisterType((*FileRange)(nil), "moby.buildkit.v1.frontend.FileRange")
proto.RegisterType((*ReadFileResponse)(nil), "moby.buildkit.v1.frontend.ReadFileResponse")
proto.RegisterType((*ReadDirRequest)(nil), "moby.buildkit.v1.frontend.ReadDirRequest")
proto.RegisterType((*ReadDirResponse)(nil), "moby.buildkit.v1.frontend.ReadDirResponse")
proto.RegisterType((*StatFileRequest)(nil), "moby.buildkit.v1.frontend.StatFileRequest")
proto.RegisterType((*StatFileResponse)(nil), "moby.buildkit.v1.frontend.StatFileResponse")
proto.RegisterType((*PingRequest)(nil), "moby.buildkit.v1.frontend.PingRequest")
proto.RegisterType((*PongResponse)(nil), "moby.buildkit.v1.frontend.PongResponse")
}
@ -521,6 +618,10 @@ type LLBBridgeClient interface {
Solve(ctx context.Context, in *SolveRequest, opts ...grpc.CallOption) (*SolveResponse, error)
// apicaps:CapReadFile
ReadFile(ctx context.Context, in *ReadFileRequest, opts ...grpc.CallOption) (*ReadFileResponse, error)
// apicaps:CapReadDir
ReadDir(ctx context.Context, in *ReadDirRequest, opts ...grpc.CallOption) (*ReadDirResponse, error)
// apicaps:CapStatFile
StatFile(ctx context.Context, in *StatFileRequest, opts ...grpc.CallOption) (*StatFileResponse, error)
Ping(ctx context.Context, in *PingRequest, opts ...grpc.CallOption) (*PongResponse, error)
Return(ctx context.Context, in *ReturnRequest, opts ...grpc.CallOption) (*ReturnResponse, error)
}
@ -560,6 +661,24 @@ func (c *lLBBridgeClient) ReadFile(ctx context.Context, in *ReadFileRequest, opt
return out, nil
}
func (c *lLBBridgeClient) ReadDir(ctx context.Context, in *ReadDirRequest, opts ...grpc.CallOption) (*ReadDirResponse, error) {
out := new(ReadDirResponse)
err := grpc.Invoke(ctx, "/moby.buildkit.v1.frontend.LLBBridge/ReadDir", in, out, c.cc, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *lLBBridgeClient) StatFile(ctx context.Context, in *StatFileRequest, opts ...grpc.CallOption) (*StatFileResponse, error) {
out := new(StatFileResponse)
err := grpc.Invoke(ctx, "/moby.buildkit.v1.frontend.LLBBridge/StatFile", in, out, c.cc, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *lLBBridgeClient) Ping(ctx context.Context, in *PingRequest, opts ...grpc.CallOption) (*PongResponse, error) {
out := new(PongResponse)
err := grpc.Invoke(ctx, "/moby.buildkit.v1.frontend.LLBBridge/Ping", in, out, c.cc, opts...)
@ -587,6 +706,10 @@ type LLBBridgeServer interface {
Solve(context.Context, *SolveRequest) (*SolveResponse, error)
// apicaps:CapReadFile
ReadFile(context.Context, *ReadFileRequest) (*ReadFileResponse, error)
// apicaps:CapReadDir
ReadDir(context.Context, *ReadDirRequest) (*ReadDirResponse, error)
// apicaps:CapStatFile
StatFile(context.Context, *StatFileRequest) (*StatFileResponse, error)
Ping(context.Context, *PingRequest) (*PongResponse, error)
Return(context.Context, *ReturnRequest) (*ReturnResponse, error)
}
@ -649,6 +772,42 @@ func _LLBBridge_ReadFile_Handler(srv interface{}, ctx context.Context, dec func(
return interceptor(ctx, in, info, handler)
}
func _LLBBridge_ReadDir_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(ReadDirRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(LLBBridgeServer).ReadDir(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/moby.buildkit.v1.frontend.LLBBridge/ReadDir",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(LLBBridgeServer).ReadDir(ctx, req.(*ReadDirRequest))
}
return interceptor(ctx, in, info, handler)
}
func _LLBBridge_StatFile_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(StatFileRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(LLBBridgeServer).StatFile(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/moby.buildkit.v1.frontend.LLBBridge/StatFile",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(LLBBridgeServer).StatFile(ctx, req.(*StatFileRequest))
}
return interceptor(ctx, in, info, handler)
}
func _LLBBridge_Ping_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(PingRequest)
if err := dec(in); err != nil {
@ -701,6 +860,14 @@ var _LLBBridge_serviceDesc = grpc.ServiceDesc{
MethodName: "ReadFile",
Handler: _LLBBridge_ReadFile_Handler,
},
{
MethodName: "ReadDir",
Handler: _LLBBridge_ReadDir_Handler,
},
{
MethodName: "StatFile",
Handler: _LLBBridge_StatFile_Handler,
},
{
MethodName: "Ping",
Handler: _LLBBridge_Ping_Handler,
@ -1169,6 +1336,130 @@ func (m *ReadFileResponse) MarshalTo(dAtA []byte) (int, error) {
return i, nil
}
func (m *ReadDirRequest) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalTo(dAtA)
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *ReadDirRequest) MarshalTo(dAtA []byte) (int, error) {
var i int
_ = i
var l int
_ = l
if len(m.Ref) > 0 {
dAtA[i] = 0xa
i++
i = encodeVarintGateway(dAtA, i, uint64(len(m.Ref)))
i += copy(dAtA[i:], m.Ref)
}
if len(m.DirPath) > 0 {
dAtA[i] = 0x12
i++
i = encodeVarintGateway(dAtA, i, uint64(len(m.DirPath)))
i += copy(dAtA[i:], m.DirPath)
}
if len(m.IncludePattern) > 0 {
dAtA[i] = 0x1a
i++
i = encodeVarintGateway(dAtA, i, uint64(len(m.IncludePattern)))
i += copy(dAtA[i:], m.IncludePattern)
}
return i, nil
}
func (m *ReadDirResponse) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalTo(dAtA)
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *ReadDirResponse) MarshalTo(dAtA []byte) (int, error) {
var i int
_ = i
var l int
_ = l
if len(m.Entries) > 0 {
for _, msg := range m.Entries {
dAtA[i] = 0xa
i++
i = encodeVarintGateway(dAtA, i, uint64(msg.Size()))
n, err := msg.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
i += n
}
}
return i, nil
}
func (m *StatFileRequest) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalTo(dAtA)
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *StatFileRequest) MarshalTo(dAtA []byte) (int, error) {
var i int
_ = i
var l int
_ = l
if len(m.Ref) > 0 {
dAtA[i] = 0xa
i++
i = encodeVarintGateway(dAtA, i, uint64(len(m.Ref)))
i += copy(dAtA[i:], m.Ref)
}
if len(m.Path) > 0 {
dAtA[i] = 0x12
i++
i = encodeVarintGateway(dAtA, i, uint64(len(m.Path)))
i += copy(dAtA[i:], m.Path)
}
return i, nil
}
func (m *StatFileResponse) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalTo(dAtA)
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *StatFileResponse) MarshalTo(dAtA []byte) (int, error) {
var i int
_ = i
var l int
_ = l
if m.Stat != nil {
dAtA[i] = 0xa
i++
i = encodeVarintGateway(dAtA, i, uint64(m.Stat.Size()))
n9, err := m.Stat.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
i += n9
}
return i, nil
}
func (m *PingRequest) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
@ -1449,6 +1740,60 @@ func (m *ReadFileResponse) Size() (n int) {
return n
}
func (m *ReadDirRequest) Size() (n int) {
var l int
_ = l
l = len(m.Ref)
if l > 0 {
n += 1 + l + sovGateway(uint64(l))
}
l = len(m.DirPath)
if l > 0 {
n += 1 + l + sovGateway(uint64(l))
}
l = len(m.IncludePattern)
if l > 0 {
n += 1 + l + sovGateway(uint64(l))
}
return n
}
func (m *ReadDirResponse) Size() (n int) {
var l int
_ = l
if len(m.Entries) > 0 {
for _, e := range m.Entries {
l = e.Size()
n += 1 + l + sovGateway(uint64(l))
}
}
return n
}
func (m *StatFileRequest) Size() (n int) {
var l int
_ = l
l = len(m.Ref)
if l > 0 {
n += 1 + l + sovGateway(uint64(l))
}
l = len(m.Path)
if l > 0 {
n += 1 + l + sovGateway(uint64(l))
}
return n
}
func (m *StatFileResponse) Size() (n int) {
var l int
_ = l
if m.Stat != nil {
l = m.Stat.Size()
n += 1 + l + sovGateway(uint64(l))
}
return n
}
func (m *PingRequest) Size() (n int) {
var l int
_ = l
@ -3088,6 +3433,415 @@ func (m *ReadFileResponse) Unmarshal(dAtA []byte) error {
}
return nil
}
func (m *ReadDirRequest) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
preIndex := iNdEx
var wire uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowGateway
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: ReadDirRequest: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: ReadDirRequest: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Ref", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowGateway
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
stringLen |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
intStringLen := int(stringLen)
if intStringLen < 0 {
return ErrInvalidLengthGateway
}
postIndex := iNdEx + intStringLen
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Ref = string(dAtA[iNdEx:postIndex])
iNdEx = postIndex
case 2:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field DirPath", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowGateway
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
stringLen |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
intStringLen := int(stringLen)
if intStringLen < 0 {
return ErrInvalidLengthGateway
}
postIndex := iNdEx + intStringLen
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.DirPath = string(dAtA[iNdEx:postIndex])
iNdEx = postIndex
case 3:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field IncludePattern", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowGateway
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
stringLen |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
intStringLen := int(stringLen)
if intStringLen < 0 {
return ErrInvalidLengthGateway
}
postIndex := iNdEx + intStringLen
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.IncludePattern = string(dAtA[iNdEx:postIndex])
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipGateway(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthGateway
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func (m *ReadDirResponse) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
preIndex := iNdEx
var wire uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowGateway
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: ReadDirResponse: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: ReadDirResponse: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Entries", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowGateway
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= (int(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthGateway
}
postIndex := iNdEx + msglen
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Entries = append(m.Entries, &fsutil_types.Stat{})
if err := m.Entries[len(m.Entries)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipGateway(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthGateway
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func (m *StatFileRequest) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
preIndex := iNdEx
var wire uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowGateway
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: StatFileRequest: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: StatFileRequest: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Ref", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowGateway
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
stringLen |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
intStringLen := int(stringLen)
if intStringLen < 0 {
return ErrInvalidLengthGateway
}
postIndex := iNdEx + intStringLen
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Ref = string(dAtA[iNdEx:postIndex])
iNdEx = postIndex
case 2:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Path", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowGateway
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
stringLen |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
intStringLen := int(stringLen)
if intStringLen < 0 {
return ErrInvalidLengthGateway
}
postIndex := iNdEx + intStringLen
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Path = string(dAtA[iNdEx:postIndex])
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipGateway(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthGateway
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func (m *StatFileResponse) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
preIndex := iNdEx
var wire uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowGateway
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: StatFileResponse: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: StatFileResponse: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Stat", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowGateway
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= (int(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthGateway
}
postIndex := iNdEx + msglen
if postIndex > l {
return io.ErrUnexpectedEOF
}
if m.Stat == nil {
m.Stat = &fsutil_types.Stat{}
}
if err := m.Stat.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipGateway(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthGateway
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func (m *PingRequest) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
@ -3389,68 +4143,77 @@ var (
func init() { proto.RegisterFile("gateway.proto", fileDescriptorGateway) }
var fileDescriptorGateway = []byte{
// 999 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0xcf, 0x6f, 0xdb, 0x36,
0x14, 0x8e, 0x22, 0xdb, 0xb1, 0x9f, 0xed, 0xc6, 0x23, 0x86, 0x41, 0xd5, 0x21, 0xf5, 0x84, 0xa1,
0xd3, 0xfa, 0x43, 0xc2, 0xdc, 0x0d, 0xed, 0x5a, 0xa0, 0x5b, 0x9d, 0x34, 0x68, 0x36, 0x67, 0x35,
0xd8, 0x43, 0x81, 0x62, 0x3b, 0xd0, 0x36, 0xa5, 0x08, 0x91, 0x45, 0x8d, 0xa2, 0x93, 0x19, 0xbb,
0x6c, 0x3b, 0xf5, 0xbe, 0x7f, 0xaa, 0xc7, 0x9d, 0x77, 0x08, 0x86, 0xdc, 0xf6, 0x5f, 0x0c, 0xa4,
0x28, 0x47, 0xf9, 0xe5, 0x24, 0x27, 0xf3, 0x51, 0xef, 0x7b, 0xef, 0xe3, 0x7b, 0xdf, 0x23, 0x0d,
0xed, 0x90, 0x08, 0x7a, 0x48, 0xe6, 0x5e, 0xca, 0x99, 0x60, 0xe8, 0xf6, 0x94, 0x8d, 0xe6, 0xde,
0x68, 0x16, 0xc5, 0x93, 0xfd, 0x48, 0x78, 0x07, 0x5f, 0x7a, 0x01, 0x67, 0x89, 0xa0, 0xc9, 0xc4,
0x7e, 0x18, 0x46, 0x62, 0x6f, 0x36, 0xf2, 0xc6, 0x6c, 0xea, 0x87, 0x2c, 0x64, 0xbe, 0x42, 0x8c,
0x66, 0x81, 0xb2, 0x94, 0xa1, 0x56, 0x79, 0x24, 0xbb, 0x77, 0xd6, 0x3d, 0x64, 0x2c, 0x8c, 0x29,
0x49, 0xa3, 0x4c, 0x2f, 0x7d, 0x9e, 0x8e, 0xfd, 0x4c, 0x10, 0x31, 0xcb, 0x34, 0xe6, 0x41, 0x09,
0x23, 0x89, 0xf8, 0x05, 0x11, 0x3f, 0x63, 0xf1, 0x01, 0xe5, 0x7e, 0x3a, 0xf2, 0x59, 0x5a, 0x78,
0xfb, 0x97, 0x7a, 0x93, 0x34, 0xf2, 0xc5, 0x3c, 0xa5, 0x99, 0x7f, 0xc8, 0xf8, 0x3e, 0xe5, 0x1a,
0xf0, 0xe8, 0x52, 0xc0, 0x4c, 0x44, 0xb1, 0x44, 0x8d, 0x49, 0x9a, 0xc9, 0x24, 0xf2, 0x37, 0x07,
0x39, 0xff, 0x19, 0x50, 0xc3, 0x34, 0x9b, 0xc5, 0x02, 0x21, 0x30, 0x39, 0x0d, 0x2c, 0xa3, 0x6b,
0xb8, 0x8d, 0x57, 0x2b, 0x58, 0x1a, 0xe8, 0x31, 0x54, 0x38, 0x0d, 0x32, 0x6b, 0xb5, 0x6b, 0xb8,
0xcd, 0xde, 0xa7, 0xde, 0xa5, 0xf5, 0xf3, 0x30, 0x0d, 0x76, 0x49, 0xfa, 0x6a, 0x05, 0x2b, 0x00,
0xfa, 0x01, 0xea, 0x53, 0x2a, 0xc8, 0x84, 0x08, 0x62, 0x41, 0xd7, 0x74, 0x9b, 0x3d, 0x7f, 0x29,
0x58, 0x32, 0xf0, 0x76, 0x35, 0xe2, 0x65, 0x22, 0xf8, 0x1c, 0x2f, 0x02, 0xd8, 0xcf, 0xa0, 0x7d,
0xea, 0x13, 0xea, 0x80, 0xb9, 0x4f, 0xe7, 0x39, 0x55, 0x2c, 0x97, 0xe8, 0x63, 0xa8, 0x1e, 0x90,
0x78, 0x46, 0x15, 0xd3, 0x16, 0xce, 0x8d, 0xa7, 0xab, 0x4f, 0x8c, 0x7e, 0x1d, 0x6a, 0x5c, 0x85,
0x77, 0xfe, 0x54, 0x67, 0x95, 0x34, 0xd1, 0xb7, 0xfa, 0x5c, 0x86, 0xa2, 0x76, 0xff, 0xca, 0x73,
0xc9, 0x9f, 0x2c, 0xa7, 0xa5, 0x80, 0xf6, 0x63, 0x68, 0x2c, 0xb6, 0xae, 0xa2, 0xd3, 0x28, 0xd1,
0x71, 0x04, 0xb4, 0x31, 0x15, 0x33, 0x9e, 0x60, 0xfa, 0xcb, 0x8c, 0x66, 0x02, 0x7d, 0x53, 0xf0,
0x53, 0xf8, 0xab, 0x8a, 0x2c, 0x1d, 0xb1, 0x06, 0x20, 0x17, 0xaa, 0x94, 0x73, 0xc6, 0x75, 0x7b,
0x90, 0x97, 0x2b, 0xcf, 0xe3, 0xe9, 0xd8, 0x7b, 0xa3, 0x94, 0x87, 0x73, 0x07, 0xa7, 0x03, 0xb7,
0x8a, 0xac, 0x59, 0xca, 0x92, 0x8c, 0x3a, 0x7f, 0x19, 0x70, 0x1b, 0x53, 0x25, 0xbc, 0x9d, 0x29,
0x09, 0xe9, 0x26, 0x4b, 0x82, 0x28, 0x2c, 0x48, 0x75, 0xc0, 0xc4, 0x85, 0x16, 0xb0, 0x5c, 0x22,
0x17, 0xea, 0xc3, 0x98, 0x88, 0x80, 0xf1, 0xa9, 0x4e, 0xd7, 0xf2, 0xd2, 0x91, 0x57, 0xec, 0xe1,
0xc5, 0x57, 0xd4, 0x85, 0xa6, 0x0e, 0xbc, 0xcb, 0x26, 0xd4, 0x32, 0x55, 0x8c, 0xf2, 0x16, 0xb2,
0x60, 0x6d, 0xc0, 0xc2, 0x1f, 0xc9, 0x94, 0x5a, 0x15, 0xf5, 0xb5, 0x30, 0x9d, 0xdf, 0x0d, 0xb0,
0x2f, 0x62, 0x95, 0x93, 0x46, 0xdf, 0x43, 0x6d, 0x2b, 0x0a, 0x69, 0x96, 0xd7, 0xaa, 0xd1, 0xef,
0x7d, 0x38, 0xba, 0xb3, 0xf2, 0xcf, 0xd1, 0x9d, 0x7b, 0x25, 0xe9, 0xb3, 0x94, 0x26, 0x63, 0x96,
0x08, 0x12, 0x25, 0x94, 0xcb, 0x61, 0x7c, 0x38, 0x51, 0x10, 0x2f, 0x47, 0x62, 0x1d, 0x01, 0x7d,
0x02, 0xb5, 0x3c, 0xba, 0x96, 0x8c, 0xb6, 0x9c, 0xf7, 0x26, 0xb4, 0xde, 0x48, 0x02, 0x45, 0x2d,
0x3c, 0x80, 0x2d, 0x1a, 0x44, 0x49, 0x24, 0x22, 0x96, 0xe8, 0x26, 0xdd, 0x92, 0x67, 0x3f, 0xd9,
0xc5, 0x25, 0x0f, 0x64, 0x43, 0x7d, 0x5b, 0x37, 0x4c, 0xb7, 0x7f, 0x61, 0xa3, 0x77, 0xd0, 0x2c,
0xd6, 0xaf, 0x53, 0x61, 0x99, 0x4a, 0x7e, 0x4f, 0x96, 0x74, 0xbc, 0xcc, 0xc4, 0x2b, 0x41, 0x73,
0x2d, 0x96, 0x83, 0x21, 0x17, 0xd6, 0x77, 0xa6, 0x29, 0xe3, 0x62, 0x93, 0x8c, 0xf7, 0xa8, 0x54,
0xa7, 0x55, 0xe9, 0x9a, 0x6e, 0x03, 0x9f, 0xdd, 0x46, 0x0f, 0xe0, 0x23, 0x12, 0xc7, 0xec, 0x50,
0xcb, 0x49, 0x09, 0xc3, 0xaa, 0x76, 0x0d, 0xb7, 0x8e, 0xcf, 0x7f, 0x90, 0x5a, 0xde, 0x8e, 0x12,
0x12, 0x5b, 0xa0, 0x3c, 0x72, 0x03, 0x39, 0xd0, 0x7a, 0xf9, 0xab, 0x0c, 0x4b, 0xf9, 0x0b, 0x21,
0xb8, 0xd5, 0x54, 0x45, 0x3c, 0xb5, 0x67, 0x3f, 0x87, 0xce, 0x59, 0xca, 0x37, 0x9a, 0x95, 0x9f,
0xa0, 0xad, 0xcf, 0xaf, 0xfb, 0xdf, 0x29, 0x5d, 0x51, 0xf9, 0x05, 0x75, 0x32, 0x3d, 0xe6, 0x0d,
0xa7, 0xc7, 0xf9, 0x0d, 0xd6, 0x31, 0x25, 0x93, 0xed, 0x28, 0xa6, 0x97, 0xcb, 0x5e, 0x36, 0x33,
0x8a, 0xe9, 0x90, 0x88, 0xbd, 0x45, 0x33, 0xb5, 0x8d, 0x9e, 0x42, 0x15, 0x93, 0x24, 0xa4, 0x3a,
0xf5, 0x67, 0x4b, 0x52, 0xab, 0x24, 0xd2, 0x17, 0xe7, 0x10, 0xe7, 0x19, 0x34, 0x16, 0x7b, 0x52,
0x8a, 0xaf, 0x83, 0x20, 0xa3, 0xb9, 0xac, 0x4d, 0xac, 0x2d, 0xb9, 0x3f, 0xa0, 0x49, 0xa8, 0x53,
0x9b, 0x58, 0x5b, 0xce, 0x5d, 0xe8, 0x9c, 0x30, 0xd7, 0xa5, 0x41, 0x50, 0xd9, 0x92, 0x97, 0xad,
0xa1, 0xfa, 0xa0, 0xd6, 0x4e, 0x1b, 0x9a, 0xc3, 0x28, 0x29, 0x86, 0xda, 0x39, 0x36, 0xa0, 0x35,
0x64, 0xc9, 0xc9, 0x38, 0x0d, 0x61, 0xbd, 0xe8, 0xcf, 0x8b, 0xe1, 0xce, 0x26, 0x49, 0x8b, 0x0b,
0xb1, 0x7b, 0xfe, 0x28, 0xfa, 0xf9, 0xf0, 0x72, 0xc7, 0x7e, 0x45, 0x4e, 0x1e, 0x3e, 0x0b, 0x47,
0xdf, 0xc1, 0xda, 0x60, 0xd0, 0x57, 0x91, 0x56, 0x6f, 0x14, 0xa9, 0x80, 0xa1, 0xe7, 0xb0, 0xf6,
0x56, 0xbd, 0x6a, 0x99, 0x9e, 0x8e, 0x0b, 0xca, 0xaa, 0x1e, 0x3f, 0x2f, 0x77, 0xc3, 0x74, 0xcc,
0xf8, 0x04, 0x17, 0xa0, 0xde, 0xfb, 0x0a, 0x34, 0x06, 0x83, 0x7e, 0x9f, 0x47, 0x93, 0x90, 0xa2,
0x3f, 0x0c, 0x40, 0xe7, 0xef, 0x13, 0xf4, 0xd5, 0x72, 0x95, 0x5c, 0x7c, 0x29, 0xda, 0x5f, 0xdf,
0x10, 0xa5, 0xab, 0xfc, 0x0e, 0xaa, 0x4a, 0xc5, 0xe8, 0xf3, 0x6b, 0xce, 0xb9, 0xed, 0x5e, 0xed,
0xa8, 0x63, 0x8f, 0xa1, 0x5e, 0x28, 0x01, 0xdd, 0x5b, 0x4a, 0xef, 0x94, 0xd0, 0xed, 0xfb, 0xd7,
0xf2, 0xd5, 0x49, 0xde, 0x42, 0x45, 0xca, 0x08, 0xdd, 0x5d, 0x02, 0x2a, 0xe9, 0xcc, 0x5e, 0x76,
0xce, 0x53, 0xfa, 0xfb, 0x59, 0xbe, 0xc7, 0xea, 0x8e, 0x71, 0x97, 0xf2, 0x29, 0x3d, 0x97, 0xf6,
0x17, 0xd7, 0xf0, 0xcc, 0xc3, 0xf7, 0x5b, 0x1f, 0x8e, 0x37, 0x8c, 0xbf, 0x8f, 0x37, 0x8c, 0x7f,
0x8f, 0x37, 0x8c, 0x51, 0x4d, 0xfd, 0xe1, 0x79, 0xf4, 0x7f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x0f,
0xfd, 0x24, 0x08, 0x13, 0x0a, 0x00, 0x00,
// 1144 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0xcf, 0x4f, 0x1b, 0xc7,
0x17, 0x67, 0xb1, 0x8d, 0xed, 0x67, 0x03, 0xfe, 0x8e, 0xbe, 0xaa, 0x36, 0x7b, 0x20, 0xee, 0xaa,
0xa2, 0x0e, 0x21, 0xbb, 0x2a, 0x69, 0x45, 0x4a, 0xa4, 0xa4, 0x31, 0x04, 0x85, 0xd6, 0x34, 0xd6,
0xe4, 0x10, 0x29, 0x6a, 0xa5, 0xae, 0xed, 0xf1, 0x32, 0x62, 0xbd, 0xb3, 0x9d, 0x1d, 0x43, 0x51,
0x2f, 0x6d, 0x4f, 0xbd, 0xf7, 0x9f, 0xca, 0xad, 0x3d, 0xf7, 0x10, 0x55, 0xdc, 0xfa, 0x5f, 0x54,
0xf3, 0x63, 0xed, 0xc5, 0x80, 0x81, 0xd3, 0xce, 0x9b, 0x79, 0x9f, 0xf7, 0x3e, 0x6f, 0xde, 0x8f,
0x59, 0x58, 0x0e, 0x03, 0x41, 0x4e, 0x83, 0x33, 0x2f, 0xe1, 0x4c, 0x30, 0x74, 0x6f, 0xc4, 0x7a,
0x67, 0x5e, 0x6f, 0x4c, 0xa3, 0xc1, 0x31, 0x15, 0xde, 0xc9, 0x67, 0xde, 0x90, 0xb3, 0x58, 0x90,
0x78, 0xe0, 0x3c, 0x0a, 0xa9, 0x38, 0x1a, 0xf7, 0xbc, 0x3e, 0x1b, 0xf9, 0x21, 0x0b, 0x99, 0xaf,
0x10, 0xbd, 0xf1, 0x50, 0x49, 0x4a, 0x50, 0x2b, 0x6d, 0xc9, 0xd9, 0x9a, 0x55, 0x0f, 0x19, 0x0b,
0x23, 0x12, 0x24, 0x34, 0x35, 0x4b, 0x9f, 0x27, 0x7d, 0x3f, 0x15, 0x81, 0x18, 0xa7, 0x06, 0xb3,
0x99, 0xc3, 0x48, 0x22, 0x7e, 0x46, 0xc4, 0x4f, 0x59, 0x74, 0x42, 0xb8, 0x9f, 0xf4, 0x7c, 0x96,
0x64, 0xda, 0xfe, 0xb5, 0xda, 0x41, 0x42, 0x7d, 0x71, 0x96, 0x90, 0xd4, 0x3f, 0x65, 0xfc, 0x98,
0x70, 0x03, 0x78, 0x7c, 0x2d, 0x60, 0x2c, 0x68, 0x24, 0x51, 0xfd, 0x20, 0x49, 0xa5, 0x13, 0xf9,
0x35, 0xa0, 0x7c, 0xd8, 0x82, 0xc5, 0x34, 0x15, 0x94, 0x86, 0xd4, 0x1f, 0xa6, 0x0a, 0xa3, 0xbd,
0xc8, 0x20, 0xb4, 0xba, 0xfb, 0xaf, 0x05, 0x4b, 0x98, 0xa4, 0xe3, 0x48, 0x20, 0x04, 0x05, 0x4e,
0x86, 0xb6, 0xd5, 0xb4, 0x5a, 0xd5, 0x57, 0x0b, 0x58, 0x0a, 0x68, 0x1b, 0x8a, 0x9c, 0x0c, 0x53,
0x7b, 0xb1, 0x69, 0xb5, 0x6a, 0x5b, 0x1f, 0x7b, 0xd7, 0x5e, 0xb7, 0x87, 0xc9, 0xf0, 0x30, 0x48,
0x5e, 0x2d, 0x60, 0x05, 0x40, 0xdf, 0x40, 0x65, 0x44, 0x44, 0x30, 0x08, 0x44, 0x60, 0x43, 0xb3,
0xd0, 0xaa, 0x6d, 0xf9, 0x73, 0xc1, 0x92, 0x81, 0x77, 0x68, 0x10, 0x2f, 0x63, 0xc1, 0xcf, 0xf0,
0xc4, 0x80, 0xf3, 0x14, 0x96, 0x2f, 0x1c, 0xa1, 0x06, 0x14, 0x8e, 0xc9, 0x99, 0xa6, 0x8a, 0xe5,
0x12, 0xfd, 0x1f, 0x4a, 0x27, 0x41, 0x34, 0x26, 0x8a, 0x69, 0x1d, 0x6b, 0x61, 0x67, 0xf1, 0x89,
0xd5, 0xae, 0xc0, 0x12, 0x57, 0xe6, 0xdd, 0xdf, 0x54, 0xac, 0x92, 0x26, 0x7a, 0x6e, 0xe2, 0xb2,
0x14, 0xb5, 0x87, 0x37, 0xc6, 0x25, 0x3f, 0xa9, 0xa6, 0xa5, 0x80, 0xce, 0x36, 0x54, 0x27, 0x5b,
0x37, 0xd1, 0xa9, 0xe6, 0xe8, 0xb8, 0x02, 0x96, 0x31, 0x11, 0x63, 0x1e, 0x63, 0xf2, 0xe3, 0x98,
0xa4, 0x02, 0x7d, 0x99, 0xf1, 0x53, 0xf8, 0x9b, 0x2e, 0x59, 0x2a, 0x62, 0x03, 0x40, 0x2d, 0x28,
0x11, 0xce, 0x19, 0x37, 0xe9, 0x41, 0x9e, 0x2e, 0x54, 0x8f, 0x27, 0x7d, 0xef, 0x8d, 0x2a, 0x54,
0xac, 0x15, 0xdc, 0x06, 0xac, 0x64, 0x5e, 0xd3, 0x84, 0xc5, 0x29, 0x71, 0xff, 0xb0, 0xe0, 0x1e,
0x26, 0xaa, 0x4e, 0x0f, 0x46, 0x41, 0x48, 0x76, 0x59, 0x3c, 0xa4, 0x61, 0x46, 0xaa, 0x01, 0x05,
0x9c, 0xd5, 0x02, 0x96, 0x4b, 0xd4, 0x82, 0x4a, 0x37, 0x0a, 0xc4, 0x90, 0xf1, 0x91, 0x71, 0x57,
0xf7, 0x92, 0x9e, 0x97, 0xed, 0xe1, 0xc9, 0x29, 0x6a, 0x42, 0xcd, 0x18, 0x3e, 0x64, 0x03, 0x62,
0x17, 0x94, 0x8d, 0xfc, 0x16, 0xb2, 0xa1, 0xdc, 0x61, 0xe1, 0xb7, 0xc1, 0x88, 0xd8, 0x45, 0x75,
0x9a, 0x89, 0xee, 0x2f, 0x16, 0x38, 0x57, 0xb1, 0xd2, 0xa4, 0xd1, 0xd7, 0xb0, 0xb4, 0x47, 0x43,
0x92, 0xea, 0xbb, 0xaa, 0xb6, 0xb7, 0xde, 0x7f, 0xb8, 0xbf, 0xf0, 0xf7, 0x87, 0xfb, 0x1b, 0xb9,
0xa2, 0x67, 0x09, 0x89, 0xfb, 0x2c, 0x16, 0x01, 0x8d, 0x09, 0x97, 0xbd, 0xfb, 0x68, 0xa0, 0x20,
0x9e, 0x46, 0x62, 0x63, 0x01, 0x7d, 0x04, 0x4b, 0xda, 0xba, 0x29, 0x19, 0x23, 0xb9, 0xbf, 0x17,
0xa0, 0xfe, 0x46, 0x12, 0xc8, 0xee, 0xc2, 0x03, 0xd8, 0x23, 0x43, 0x1a, 0x53, 0x41, 0x59, 0x6c,
0x92, 0xb4, 0x22, 0x63, 0x9f, 0xee, 0xe2, 0x9c, 0x06, 0x72, 0xa0, 0xb2, 0x6f, 0x12, 0x66, 0xd2,
0x3f, 0x91, 0xd1, 0x3b, 0xa8, 0x65, 0xeb, 0xd7, 0x89, 0xb0, 0x0b, 0xaa, 0xfc, 0x9e, 0xcc, 0xc9,
0x78, 0x9e, 0x89, 0x97, 0x83, 0xea, 0x5a, 0xcc, 0x1b, 0x43, 0x2d, 0x58, 0x3d, 0x18, 0x25, 0x8c,
0x8b, 0xdd, 0xa0, 0x7f, 0x44, 0x64, 0x75, 0xda, 0xc5, 0x66, 0xa1, 0x55, 0xc5, 0xb3, 0xdb, 0x68,
0x13, 0xfe, 0x17, 0x44, 0x11, 0x3b, 0x35, 0xe5, 0xa4, 0x0a, 0xc3, 0x2e, 0x35, 0xad, 0x56, 0x05,
0x5f, 0x3e, 0x90, 0xb5, 0xbc, 0x4f, 0xe3, 0x20, 0xb2, 0x41, 0x69, 0x68, 0x01, 0xb9, 0x50, 0x7f,
0xf9, 0x93, 0x34, 0x4b, 0xf8, 0x0b, 0x21, 0xb8, 0x5d, 0x53, 0x97, 0x78, 0x61, 0xcf, 0x79, 0x06,
0x8d, 0x59, 0xca, 0x77, 0xea, 0x95, 0xef, 0x60, 0xd9, 0xc4, 0x6f, 0xf2, 0xdf, 0xc8, 0x8d, 0x28,
0x3d, 0xa0, 0xa6, 0xdd, 0x53, 0xb8, 0x63, 0xf7, 0xb8, 0x3f, 0xc3, 0x2a, 0x26, 0xc1, 0x60, 0x9f,
0x46, 0xe4, 0xfa, 0xb2, 0x97, 0xc9, 0xa4, 0x11, 0xe9, 0x06, 0xe2, 0x68, 0x92, 0x4c, 0x23, 0xa3,
0x1d, 0x28, 0xe1, 0x20, 0x0e, 0x89, 0x71, 0xfd, 0xc9, 0x1c, 0xd7, 0xca, 0x89, 0xd4, 0xc5, 0x1a,
0xe2, 0x3e, 0x85, 0xea, 0x64, 0x4f, 0x96, 0xe2, 0xeb, 0xe1, 0x30, 0x25, 0xba, 0xac, 0x0b, 0xd8,
0x48, 0x72, 0xbf, 0x43, 0xe2, 0xd0, 0xb8, 0x2e, 0x60, 0x23, 0xb9, 0xeb, 0xd0, 0x98, 0x32, 0x37,
0x57, 0x83, 0xa0, 0xb8, 0x27, 0x87, 0xad, 0xa5, 0xf2, 0xa0, 0xd6, 0xee, 0x40, 0x76, 0x7d, 0x30,
0xd8, 0xa3, 0xfc, 0xfa, 0x00, 0x6d, 0x28, 0xef, 0x51, 0x9e, 0x8b, 0x2f, 0x13, 0xd1, 0x3a, 0xac,
0x1c, 0xc4, 0xfd, 0x68, 0x3c, 0x90, 0xd1, 0x0a, 0xc2, 0x63, 0xd3, 0xca, 0x33, 0xbb, 0xee, 0x73,
0x7d, 0x8f, 0xca, 0x8b, 0x21, 0xb3, 0x09, 0x65, 0x12, 0x0b, 0x4e, 0x49, 0x36, 0x61, 0x91, 0xa7,
0x1f, 0x20, 0x4f, 0x3d, 0x40, 0x6a, 0x38, 0xe1, 0x4c, 0xc5, 0xdd, 0x86, 0x55, 0xb9, 0x31, 0x3f,
0x11, 0x08, 0x8a, 0x39, 0x92, 0x6a, 0xed, 0xee, 0x40, 0x63, 0x0a, 0x34, 0xae, 0xd7, 0xa1, 0x28,
0x9f, 0x37, 0xd3, 0xa7, 0x57, 0xf9, 0x55, 0xe7, 0xee, 0x32, 0xd4, 0xba, 0x34, 0xce, 0x06, 0x9e,
0x7b, 0x6e, 0x41, 0xbd, 0xcb, 0xe2, 0xe9, 0xa8, 0xe9, 0xc2, 0x6a, 0x56, 0xbb, 0x2f, 0xba, 0x07,
0xbb, 0x41, 0x92, 0x85, 0xd2, 0xbc, 0x9c, 0x66, 0xf3, 0x12, 0x7b, 0x5a, 0xb1, 0x5d, 0x94, 0x53,
0x09, 0xcf, 0xc2, 0xd1, 0x57, 0x50, 0xee, 0x74, 0xda, 0xca, 0xd2, 0xe2, 0x9d, 0x2c, 0x65, 0x30,
0xf4, 0x0c, 0xca, 0x6f, 0xd5, 0x0f, 0x42, 0x6a, 0x26, 0xc7, 0x15, 0x25, 0xa7, 0x03, 0xd5, 0x6a,
0x98, 0xf4, 0x19, 0x1f, 0xe0, 0x0c, 0xb4, 0xf5, 0x67, 0x09, 0xaa, 0x9d, 0x4e, 0xbb, 0xcd, 0xe9,
0x20, 0x24, 0xe8, 0x57, 0x0b, 0xd0, 0xe5, 0x59, 0x8b, 0x3e, 0x9f, 0xdf, 0x41, 0x57, 0x3f, 0x18,
0xce, 0x17, 0x77, 0x44, 0x99, 0x5b, 0x7e, 0x07, 0x25, 0xd5, 0xe1, 0xe8, 0xd3, 0x5b, 0xce, 0x40,
0xa7, 0x75, 0xb3, 0xa2, 0xb1, 0xdd, 0x87, 0x4a, 0xd6, 0x25, 0x68, 0x63, 0x2e, 0xbd, 0x0b, 0x43,
0xc0, 0x79, 0x78, 0x2b, 0x5d, 0xe3, 0xe4, 0x07, 0x28, 0x9b, 0xe2, 0x47, 0x0f, 0x6e, 0xc0, 0x4d,
0xdb, 0xd0, 0xd9, 0xb8, 0x8d, 0xea, 0x34, 0x8c, 0xac, 0xc8, 0xe7, 0x86, 0x31, 0xd3, 0x42, 0x73,
0xc3, 0xb8, 0xd4, 0x35, 0x6f, 0xa1, 0x28, 0xbb, 0x01, 0xad, 0xcf, 0x01, 0xe5, 0xda, 0xc5, 0x99,
0x97, 0xae, 0x0b, 0x6d, 0xf4, 0xbd, 0xfc, 0xe5, 0x52, 0xcf, 0x48, 0x6b, 0x6e, 0xcc, 0xb9, 0x3f,
0x22, 0xe7, 0xc1, 0x2d, 0x34, 0xb5, 0xf9, 0x76, 0xfd, 0xfd, 0xf9, 0x9a, 0xf5, 0xd7, 0xf9, 0x9a,
0xf5, 0xcf, 0xf9, 0x9a, 0xd5, 0x5b, 0x52, 0xff, 0xb4, 0x8f, 0xff, 0x0b, 0x00, 0x00, 0xff, 0xff,
0x80, 0x7e, 0xd2, 0xb5, 0x25, 0x0c, 0x00, 0x00,
}

View File

@ -7,6 +7,7 @@ import "github.com/gogo/googleapis/google/rpc/status.proto";
import "github.com/moby/buildkit/solver/pb/ops.proto";
import "github.com/moby/buildkit/api/types/worker.proto";
import "github.com/moby/buildkit/util/apicaps/pb/caps.proto";
import "github.com/tonistiigi/fsutil/types/stat.proto";
option (gogoproto.sizer_all) = true;
option (gogoproto.marshaler_all) = true;
@ -19,6 +20,10 @@ service LLBBridge {
rpc Solve(SolveRequest) returns (SolveResponse);
// apicaps:CapReadFile
rpc ReadFile(ReadFileRequest) returns (ReadFileResponse);
// apicaps:CapReadDir
rpc ReadDir(ReadDirRequest) returns (ReadDirResponse);
// apicaps:CapStatFile
rpc StatFile(StatFileRequest) returns (StatFileResponse);
rpc Ping(PingRequest) returns (PongResponse);
rpc Return(ReturnRequest) returns (ReturnResponse);
}
@ -92,6 +97,25 @@ message ReadFileResponse {
bytes Data = 1;
}
message ReadDirRequest {
string Ref = 1;
string DirPath = 2;
string IncludePattern = 3;
}
message ReadDirResponse {
repeated fsutil.types.Stat entries = 1;
}
message StatFileRequest {
string Ref = 1;
string Path = 2;
}
message StatFileResponse {
fsutil.types.Stat stat = 1;
}
message PingRequest{
}
message PongResponse{

View File

@ -9,6 +9,7 @@ import (
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
"github.com/tonistiigi/fsutil"
fstypes "github.com/tonistiigi/fsutil/types"
"google.golang.org/grpc"
)
@ -81,10 +82,10 @@ func syncTargetDiffCopy(ds grpc.Stream, dest string) error {
}
return fsutil.Receive(ds.Context(), ds, dest, fsutil.ReceiveOpt{
Merge: true,
Filter: func() func(*fsutil.Stat) bool {
Filter: func() func(*fstypes.Stat) bool {
uid := os.Getuid()
gid := os.Getgid()
return func(st *fsutil.Stat) bool {
return func(st *fstypes.Stat) bool {
st.Uid = uint32(uid)
st.Gid = uint32(gid)
return true

View File

@ -10,6 +10,7 @@ import (
"github.com/moby/buildkit/session"
"github.com/pkg/errors"
"github.com/tonistiigi/fsutil"
fstypes "github.com/tonistiigi/fsutil/types"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/metadata"
@ -34,7 +35,7 @@ type SyncedDir struct {
Name string
Dir string
Excludes []string
Map func(*fsutil.Stat) bool
Map func(*fstypes.Stat) bool
}
// NewFSSyncProvider creates a new provider for sending files from client

View File

@ -40,7 +40,7 @@ golang.org/x/time f51c12702a4d776e4c1fa9b0fabab841babae631
github.com/docker/docker 71cd53e4a197b303c6ba086bd584ffd67a884281
github.com/pkg/profile 5b67d428864e92711fcbd2f8629456121a56d91f
github.com/tonistiigi/fsutil 7e391b0e788f9b925f22bd3cf88e0210d1643673
github.com/tonistiigi/fsutil f567071bed2416e4d87d260d3162722651182317
github.com/hashicorp/go-immutable-radix 826af9ccf0feeee615d546d69b11f8e98da8c8f1 git://github.com/tonistiigi/go-immutable-radix.git
github.com/hashicorp/golang-lru a0d98a5f288019575c6d1f4bb1573fef2d1fcdc4
github.com/mitchellh/hashstructure 2bca23e0e452137f789efbc8610126fd8b94f73b

View File

@ -4,6 +4,8 @@ import (
"context"
"hash"
"os"
"github.com/tonistiigi/fsutil/types"
)
type walkerFn func(ctx context.Context, pathC chan<- *currentPath) error
@ -14,7 +16,7 @@ func Changes(ctx context.Context, a, b walkerFn, changeFn ChangeFunc) error {
type HandleChangeFn func(ChangeKind, string, os.FileInfo, error) error
type ContentHasher func(*Stat) (hash.Hash, error)
type ContentHasher func(*types.Stat) (hash.Hash, error)
func GetWalkerFn(root string) walkerFn {
return func(ctx context.Context, pathC chan<- *currentPath) error {

View File

@ -5,6 +5,7 @@ import (
"os"
"strings"
"github.com/tonistiigi/fsutil/types"
"golang.org/x/sync/errgroup"
)
@ -170,11 +171,11 @@ func sameFile(f1, f2 *currentPath) (same bool, retErr error) {
}
}
ls1, ok := f1.f.Sys().(*Stat)
ls1, ok := f1.f.Sys().(*types.Stat)
if !ok {
return false, nil
}
ls2, ok := f2.f.Sys().(*Stat)
ls2, ok := f2.f.Sys().(*types.Stat)
if !ok {
return false, nil
}
@ -185,7 +186,7 @@ func sameFile(f1, f2 *currentPath) (same bool, retErr error) {
// compareStat returns whether the stats are equivalent,
// whether the files are considered the same file, and
// an error
func compareStat(ls1, ls2 *Stat) (bool, error) {
func compareStat(ls1, ls2 *types.Stat) (bool, error) {
return ls1.Mode == ls2.Mode && ls1.Uid == ls2.Uid && ls1.Gid == ls2.Gid && ls1.Devmajor == ls2.Devmajor && ls1.Devminor == ls2.Devminor && ls1.Linkname == ls2.Linkname, nil
}

View File

@ -12,6 +12,7 @@ import (
"github.com/opencontainers/go-digest"
"github.com/pkg/errors"
"github.com/tonistiigi/fsutil/types"
"golang.org/x/sync/errgroup"
)
@ -25,7 +26,7 @@ type DiskWriterOpt struct {
Filter FilterFunc
}
type FilterFunc func(*Stat) bool
type FilterFunc func(*types.Stat) bool
type DiskWriter struct {
opt DiskWriterOpt
@ -95,7 +96,7 @@ func (dw *DiskWriter) HandleChange(kind ChangeKind, p string, fi os.FileInfo, er
return nil
}
stat, ok := fi.Sys().(*Stat)
stat, ok := fi.Sys().(*types.Stat)
if !ok {
return errors.Errorf("%s invalid change without stat information", p)
}
@ -246,7 +247,7 @@ type hashedWriter struct {
}
func newHashWriter(ch ContentHasher, fi os.FileInfo, w io.WriteCloser) (*hashedWriter, error) {
stat, ok := fi.Sys().(*Stat)
stat, ok := fi.Sys().(*types.Stat)
if !ok {
return nil, errors.Errorf("invalid change without stat information")
}

View File

@ -8,9 +8,10 @@ import (
"github.com/containerd/continuity/sysx"
"github.com/pkg/errors"
"github.com/tonistiigi/fsutil/types"
)
func rewriteMetadata(p string, stat *Stat) error {
func rewriteMetadata(p string, stat *types.Stat) error {
for key, value := range stat.Xattrs {
sysx.Setxattr(p, key, value, 0)
}
@ -34,7 +35,7 @@ func rewriteMetadata(p string, stat *Stat) error {
// handleTarTypeBlockCharFifo is an OS-specific helper function used by
// createTarFile to handle the following types of header: Block; Char; Fifo
func handleTarTypeBlockCharFifo(path string, stat *Stat) error {
func handleTarTypeBlockCharFifo(path string, stat *types.Stat) error {
mode := uint32(stat.Mode & 07777)
if os.FileMode(stat.Mode)&os.ModeCharDevice != 0 {
mode |= syscall.S_IFCHR

View File

@ -4,14 +4,15 @@ package fsutil
import (
"github.com/pkg/errors"
"github.com/tonistiigi/fsutil/types"
)
func rewriteMetadata(p string, stat *Stat) error {
func rewriteMetadata(p string, stat *types.Stat) error {
return chtimes(p, stat.ModTime)
}
// handleTarTypeBlockCharFifo is an OS-specific helper function used by
// createTarFile to handle the following types of header: Block; Char; Fifo
func handleTarTypeBlockCharFifo(path string, stat *Stat) error {
func handleTarTypeBlockCharFifo(path string, stat *types.Stat) error {
return errors.New("Not implemented on windows")
}

View File

@ -9,6 +9,7 @@ import (
"strings"
"github.com/pkg/errors"
"github.com/tonistiigi/fsutil/types"
)
type FS interface {
@ -36,13 +37,13 @@ func (fs *fs) Open(p string) (io.ReadCloser, error) {
return os.Open(filepath.Join(fs.root, p))
}
func SubDirFS(fs FS, stat Stat) FS {
func SubDirFS(fs FS, stat types.Stat) FS {
return &subDirFS{fs: fs, stat: stat}
}
type subDirFS struct {
fs FS
stat Stat
stat types.Stat
}
func (fs *subDirFS) Walk(ctx context.Context, fn filepath.WalkFunc) error {
@ -57,7 +58,7 @@ func (fs *subDirFS) Walk(ctx context.Context, fn filepath.WalkFunc) error {
return err
}
return fs.fs.Walk(ctx, func(p string, fi os.FileInfo, err error) error {
stat, ok := fi.Sys().(*Stat)
stat, ok := fi.Sys().(*types.Stat)
if !ok {
return errors.Wrapf(err, "invalid fileinfo without stat info: %s", p)
}

View File

@ -4,6 +4,7 @@ import (
"os"
"github.com/pkg/errors"
"github.com/tonistiigi/fsutil/types"
)
// Hardlinks validates that all targets for links were part of the changes
@ -25,7 +26,7 @@ func (v *Hardlinks) HandleChange(kind ChangeKind, p string, fi os.FileInfo, err
return nil
}
stat, ok := fi.Sys().(*Stat)
stat, ok := fi.Sys().(*types.Stat)
if !ok {
return errors.Errorf("invalid change without stat info: %s", p)
}

View File

@ -7,6 +7,7 @@ import (
"sync"
"github.com/pkg/errors"
"github.com/tonistiigi/fsutil/types"
"golang.org/x/sync/errgroup"
)
@ -119,7 +120,7 @@ func (r *receiver) run(ctx context.Context) error {
g.Go(func() (retErr error) {
defer func() {
if retErr != nil {
r.conn.SendMsg(&Packet{Type: PACKET_ERR, Data: []byte(retErr.Error())})
r.conn.SendMsg(&types.Packet{Type: types.PACKET_ERR, Data: []byte(retErr.Error())})
}
}()
destWalker := emptyWalker
@ -133,7 +134,7 @@ func (r *receiver) run(ctx context.Context) error {
if err := dw.Wait(ctx); err != nil {
return err
}
r.conn.SendMsg(&Packet{Type: PACKET_FIN})
r.conn.SendMsg(&types.Packet{Type: types.PACKET_FIN})
return nil
})
@ -146,9 +147,9 @@ func (r *receiver) run(ctx context.Context) error {
r.progressCb(size, true)
}()
}
var p Packet
var p types.Packet
for {
p = Packet{Data: p.Data[:0]}
p = types.Packet{Data: p.Data[:0]}
if err := r.conn.RecvMsg(&p); err != nil {
return err
}
@ -158,9 +159,9 @@ func (r *receiver) run(ctx context.Context) error {
}
switch p.Type {
case PACKET_ERR:
case types.PACKET_ERR:
return errors.Errorf("error from sender: %s", p.Data)
case PACKET_STAT:
case types.PACKET_STAT:
if p.Stat == nil {
if err := w.update(nil); err != nil {
return err
@ -183,12 +184,12 @@ func (r *receiver) run(ctx context.Context) error {
if err := w.update(cp); err != nil {
return err
}
case PACKET_DATA:
case types.PACKET_DATA:
r.muPipes.Lock()
pw, ok := r.pipes[p.ID]
r.muPipes.Unlock()
if !ok {
return errors.Errorf("invalid file request %s", p.ID)
return errors.Errorf("invalid file request %d", p.ID)
}
if len(p.Data) == 0 {
if err := pw.Close(); err != nil {
@ -199,9 +200,9 @@ func (r *receiver) run(ctx context.Context) error {
return err
}
}
case PACKET_FIN:
case types.PACKET_FIN:
for {
var p Packet
var p types.Packet
if err := r.conn.RecvMsg(&p); err != nil {
if err == io.EOF {
return nil
@ -229,7 +230,7 @@ func (r *receiver) asyncDataFunc(ctx context.Context, p string, wc io.WriteClose
r.muPipes.Lock()
r.pipes[id] = wwc
r.muPipes.Unlock()
if err := r.conn.SendMsg(&Packet{Type: PACKET_REQ, ID: id}); err != nil {
if err := r.conn.SendMsg(&types.Packet{Type: types.PACKET_REQ, ID: id}); err != nil {
return err
}
err := wwc.Wait(ctx)

View File

@ -7,6 +7,7 @@ import (
"sync"
"github.com/pkg/errors"
"github.com/tonistiigi/fsutil/types"
"golang.org/x/sync/errgroup"
)
@ -56,7 +57,7 @@ func (s *sender) run(ctx context.Context) error {
g.Go(func() error {
err := s.walk(ctx)
if err != nil {
s.conn.SendMsg(&Packet{Type: PACKET_ERR, Data: []byte(err.Error())})
s.conn.SendMsg(&types.Packet{Type: types.PACKET_ERR, Data: []byte(err.Error())})
}
return err
})
@ -86,19 +87,19 @@ func (s *sender) run(ctx context.Context) error {
return ctx.Err()
default:
}
var p Packet
var p types.Packet
if err := s.conn.RecvMsg(&p); err != nil {
return err
}
switch p.Type {
case PACKET_ERR:
case types.PACKET_ERR:
return errors.Errorf("error from receiver: %s", p.Data)
case PACKET_REQ:
case types.PACKET_REQ:
if err := s.queue(p.ID); err != nil {
return err
}
case PACKET_FIN:
return s.conn.SendMsg(&Packet{Type: PACKET_FIN})
case types.PACKET_FIN:
return s.conn.SendMsg(&types.Packet{Type: types.PACKET_FIN})
}
}
})
@ -136,7 +137,7 @@ func (s *sender) sendFile(h *sendHandle) error {
return err
}
}
return s.conn.SendMsg(&Packet{ID: h.id, Type: PACKET_DATA})
return s.conn.SendMsg(&types.Packet{ID: h.id, Type: types.PACKET_DATA})
}
func (s *sender) walk(ctx context.Context) error {
@ -145,13 +146,13 @@ func (s *sender) walk(ctx context.Context) error {
if err != nil {
return err
}
stat, ok := fi.Sys().(*Stat)
stat, ok := fi.Sys().(*types.Stat)
if !ok {
return errors.Wrapf(err, "invalid fileinfo without stat info: %s", path)
}
p := &Packet{
Type: PACKET_STAT,
p := &types.Packet{
Type: types.PACKET_STAT,
Stat: stat,
}
if fileCanRequestData(os.FileMode(stat.Mode)) {
@ -166,7 +167,7 @@ func (s *sender) walk(ctx context.Context) error {
if err != nil {
return err
}
return errors.Wrapf(s.conn.SendMsg(&Packet{Type: PACKET_STAT}), "failed to send last stat")
return errors.Wrapf(s.conn.SendMsg(&types.Packet{Type: types.PACKET_STAT}), "failed to send last stat")
}
func fileCanRequestData(m os.FileMode) bool {
@ -184,7 +185,7 @@ func (fs *fileSender) Write(dt []byte) (int, error) {
if len(dt) == 0 {
return 0, nil
}
p := &Packet{Type: PACKET_DATA, ID: fs.id, Data: dt}
p := &types.Packet{Type: types.PACKET_DATA, ID: fs.id, Data: dt}
if err := fs.sender.conn.SendMsg(p); err != nil {
return 0, err
}

61
vendor/github.com/tonistiigi/fsutil/stat.go generated vendored Normal file
View File

@ -0,0 +1,61 @@
package fsutil
import (
"os"
"path/filepath"
"runtime"
"github.com/pkg/errors"
"github.com/tonistiigi/fsutil/types"
)
// constructs a Stat object. path is where the path can be found right
// now, relpath is the desired path to be recorded in the stat (so
// relative to whatever base dir is relevant). fi is the os.Stat
// info. inodemap is used to calculate hardlinks over a series of
// mkstat calls and maps inode to the canonical (aka "first") path for
// a set of hardlinks to that inode.
func mkstat(path, relpath string, fi os.FileInfo, inodemap map[uint64]string) (*types.Stat, error) {
relpath = filepath.ToSlash(relpath)
stat := &types.Stat{
Path: relpath,
Mode: uint32(fi.Mode()),
ModTime: fi.ModTime().UnixNano(),
}
setUnixOpt(fi, stat, relpath, inodemap)
if !fi.IsDir() {
stat.Size_ = fi.Size()
if fi.Mode()&os.ModeSymlink != 0 {
link, err := os.Readlink(path)
if err != nil {
return nil, errors.Wrapf(err, "failed to readlink %s", path)
}
stat.Linkname = link
}
}
if err := loadXattr(path, stat); err != nil {
return nil, errors.Wrapf(err, "failed to xattr %s", relpath)
}
if runtime.GOOS == "windows" {
permPart := stat.Mode & uint32(os.ModePerm)
noPermPart := stat.Mode &^ uint32(os.ModePerm)
// Add the x bit: make everything +x from windows
permPart |= 0111
permPart &= 0755
stat.Mode = noPermPart | permPart
}
return stat, nil
}
func Stat(path string) (*types.Stat, error) {
fi, err := os.Lstat(path)
if err != nil {
return nil, errors.Wrap(err, "os stat")
}
return mkstat(path, filepath.Base(path), fi, nil)
}

View File

@ -8,9 +8,10 @@ import (
"github.com/containerd/continuity/sysx"
"github.com/pkg/errors"
"github.com/tonistiigi/fsutil/types"
)
func loadXattr(origpath string, stat *Stat) error {
func loadXattr(origpath string, stat *types.Stat) error {
xattrs, err := sysx.LListxattr(origpath)
if err != nil {
if errors.Cause(err) == syscall.ENOTSUP {
@ -31,7 +32,7 @@ func loadXattr(origpath string, stat *Stat) error {
return nil
}
func setUnixOpt(fi os.FileInfo, stat *Stat, path string, seenFiles map[uint64]string) {
func setUnixOpt(fi os.FileInfo, stat *types.Stat, path string, seenFiles map[uint64]string) {
s := fi.Sys().(*syscall.Stat_t)
stat.Uid = s.Uid
@ -45,6 +46,7 @@ func setUnixOpt(fi os.FileInfo, stat *Stat, path string, seenFiles map[uint64]st
}
ino := s.Ino
if seenFiles != nil {
if s.Nlink > 1 {
if oldpath, ok := seenFiles[ino]; ok {
stat.Linkname = oldpath
@ -53,6 +55,7 @@ func setUnixOpt(fi os.FileInfo, stat *Stat, path string, seenFiles map[uint64]st
}
seenFiles[ino] = path
}
}
}
func major(device uint64) uint64 {

16
vendor/github.com/tonistiigi/fsutil/stat_windows.go generated vendored Normal file
View File

@ -0,0 +1,16 @@
// +build windows
package fsutil
import (
"os"
"github.com/tonistiigi/fsutil/types"
)
func loadXattr(_ string, _ *types.Stat) error {
return nil
}
func setUnixOpt(_ os.FileInfo, _ *types.Stat, _ string, _ map[uint64]string) {
}

View File

@ -1,3 +1,3 @@
package fsutil
package types
//go:generate protoc --gogoslick_out=. stat.proto wire.proto

View File

@ -1,9 +1,8 @@
// Code generated by protoc-gen-gogo.
// Code generated by protoc-gen-gogo. DO NOT EDIT.
// source: stat.proto
// DO NOT EDIT!
/*
Package fsutil is a generated protocol buffer package.
Package types is a generated protocol buffer package.
It is generated from these files:
stat.proto
@ -13,7 +12,7 @@
Stat
Packet
*/
package fsutil
package types
import proto "github.com/gogo/protobuf/proto"
import fmt "fmt"
@ -23,7 +22,7 @@ import bytes "bytes"
import strings "strings"
import reflect "reflect"
import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys"
import sortkeys "github.com/gogo/protobuf/sortkeys"
import io "io"
@ -127,14 +126,11 @@ func (m *Stat) GetXattrs() map[string][]byte {
}
func init() {
proto.RegisterType((*Stat)(nil), "fsutil.Stat")
proto.RegisterType((*Stat)(nil), "fsutil.types.Stat")
}
func (this *Stat) Equal(that interface{}) bool {
if that == nil {
if this == nil {
return true
}
return false
return this == nil
}
that1, ok := that.(*Stat)
@ -147,10 +143,7 @@ func (this *Stat) Equal(that interface{}) bool {
}
}
if that1 == nil {
if this == nil {
return true
}
return false
return this == nil
} else if this == nil {
return false
}
@ -196,7 +189,7 @@ func (this *Stat) GoString() string {
return "nil"
}
s := make([]string, 0, 14)
s = append(s, "&fsutil.Stat{")
s = append(s, "&types.Stat{")
s = append(s, "Path: "+fmt.Sprintf("%#v", this.Path)+",\n")
s = append(s, "Mode: "+fmt.Sprintf("%#v", this.Mode)+",\n")
s = append(s, "Uid: "+fmt.Sprintf("%#v", this.Uid)+",\n")
@ -210,7 +203,7 @@ func (this *Stat) GoString() string {
for k, _ := range this.Xattrs {
keysForXattrs = append(keysForXattrs, k)
}
github_com_gogo_protobuf_sortkeys.Strings(keysForXattrs)
sortkeys.Strings(keysForXattrs)
mapStringForXattrs := "map[string][]byte{"
for _, k := range keysForXattrs {
mapStringForXattrs += fmt.Sprintf("%#v: %#v,", k, this.Xattrs[k])
@ -318,24 +311,6 @@ func (m *Stat) MarshalTo(dAtA []byte) (int, error) {
return i, nil
}
func encodeFixed64Stat(dAtA []byte, offset int, v uint64) int {
dAtA[offset] = uint8(v)
dAtA[offset+1] = uint8(v >> 8)
dAtA[offset+2] = uint8(v >> 16)
dAtA[offset+3] = uint8(v >> 24)
dAtA[offset+4] = uint8(v >> 32)
dAtA[offset+5] = uint8(v >> 40)
dAtA[offset+6] = uint8(v >> 48)
dAtA[offset+7] = uint8(v >> 56)
return offset + 8
}
func encodeFixed32Stat(dAtA []byte, offset int, v uint32) int {
dAtA[offset] = uint8(v)
dAtA[offset+1] = uint8(v >> 8)
dAtA[offset+2] = uint8(v >> 16)
dAtA[offset+3] = uint8(v >> 24)
return offset + 4
}
func encodeVarintStat(dAtA []byte, offset int, v uint64) int {
for v >= 1<<7 {
dAtA[offset] = uint8(v&0x7f | 0x80)
@ -413,7 +388,7 @@ func (this *Stat) String() string {
for k, _ := range this.Xattrs {
keysForXattrs = append(keysForXattrs, k)
}
github_com_gogo_protobuf_sortkeys.Strings(keysForXattrs)
sortkeys.Strings(keysForXattrs)
mapStringForXattrs := "map[string][]byte{"
for _, k := range keysForXattrs {
mapStringForXattrs += fmt.Sprintf("%v: %v,", k, this.Xattrs[k])
@ -688,7 +663,14 @@ func (m *Stat) Unmarshal(dAtA []byte) error {
if postIndex > l {
return io.ErrUnexpectedEOF
}
var keykey uint64
if m.Xattrs == nil {
m.Xattrs = make(map[string][]byte)
}
var mapkey string
mapvalue := []byte{}
for iNdEx < postIndex {
entryPreIndex := iNdEx
var wire uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowStat
@ -698,11 +680,13 @@ func (m *Stat) Unmarshal(dAtA []byte) error {
}
b := dAtA[iNdEx]
iNdEx++
keykey |= (uint64(b) & 0x7F) << shift
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
if fieldNum == 1 {
var stringLenmapkey uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
@ -726,27 +710,9 @@ func (m *Stat) Unmarshal(dAtA []byte) error {
if postStringIndexmapkey > l {
return io.ErrUnexpectedEOF
}
mapkey := string(dAtA[iNdEx:postStringIndexmapkey])
mapkey = string(dAtA[iNdEx:postStringIndexmapkey])
iNdEx = postStringIndexmapkey
if m.Xattrs == nil {
m.Xattrs = make(map[string][]byte)
}
if iNdEx < postIndex {
var valuekey uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowStat
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
valuekey |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
} else if fieldNum == 2 {
var mapbyteLen uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
@ -770,14 +736,25 @@ func (m *Stat) Unmarshal(dAtA []byte) error {
if postbytesIndex > l {
return io.ErrUnexpectedEOF
}
mapvalue := make([]byte, mapbyteLen)
mapvalue = make([]byte, mapbyteLen)
copy(mapvalue, dAtA[iNdEx:postbytesIndex])
iNdEx = postbytesIndex
m.Xattrs[mapkey] = mapvalue
} else {
var mapvalue []byte
m.Xattrs[mapkey] = mapvalue
iNdEx = entryPreIndex
skippy, err := skipStat(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthStat
}
if (iNdEx + skippy) > postIndex {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
m.Xattrs[mapkey] = mapvalue
iNdEx = postIndex
default:
iNdEx = preIndex
@ -908,24 +885,25 @@ var (
func init() { proto.RegisterFile("stat.proto", fileDescriptorStat) }
var fileDescriptorStat = []byte{
// 303 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x4c, 0x91, 0xb1, 0x4e, 0xf3, 0x30,
0x14, 0x85, 0x73, 0x9b, 0x36, 0x6d, 0xdd, 0xff, 0x97, 0x90, 0xc5, 0x70, 0xd5, 0xc1, 0x8a, 0x98,
0x32, 0xa0, 0x08, 0xc1, 0x02, 0x8c, 0x48, 0xbc, 0x40, 0x60, 0x60, 0x35, 0xb2, 0x29, 0xa6, 0x4d,
0x5c, 0x25, 0x4e, 0x45, 0x99, 0x78, 0x04, 0x1e, 0x83, 0xd7, 0x60, 0x63, 0xec, 0xc8, 0x48, 0xcc,
0xc2, 0xd8, 0x47, 0x40, 0x76, 0xda, 0xc2, 0x76, 0xce, 0x77, 0x7c, 0x65, 0x9d, 0x7b, 0x09, 0xa9,
0x0c, 0x37, 0xe9, 0xbc, 0xd4, 0x46, 0xd3, 0xe8, 0xae, 0xaa, 0x8d, 0x9a, 0x1d, 0xbc, 0x75, 0x48,
0xf7, 0xca, 0x70, 0x43, 0x29, 0xe9, 0xce, 0xb9, 0xb9, 0x47, 0x88, 0x21, 0x19, 0x66, 0x5e, 0x3b,
0x96, 0x6b, 0x21, 0xb1, 0x13, 0x43, 0xf2, 0x3f, 0xf3, 0x9a, 0xee, 0x91, 0xb0, 0x56, 0x02, 0x43,
0x8f, 0x9c, 0x74, 0x64, 0xa2, 0x04, 0x76, 0x5b, 0x32, 0x51, 0xc2, 0xcd, 0x55, 0xea, 0x49, 0x62,
0x2f, 0x86, 0x24, 0xcc, 0xbc, 0xa6, 0x48, 0xfa, 0xb9, 0x16, 0xd7, 0x2a, 0x97, 0x18, 0x79, 0xbc,
0xb5, 0x74, 0x4c, 0x06, 0x33, 0x55, 0x4c, 0x0b, 0x9e, 0x4b, 0xec, 0xfb, 0xdf, 0x77, 0xde, 0x65,
0x42, 0x2e, 0x72, 0xfe, 0xa0, 0x4b, 0x1c, 0xf8, 0xb1, 0x9d, 0xdf, 0x66, 0xaa, 0xd0, 0x25, 0x0e,
0x7f, 0x33, 0xe7, 0xe9, 0x11, 0x89, 0x1e, 0xb9, 0x31, 0x65, 0x85, 0x24, 0x0e, 0x93, 0xd1, 0x31,
0xa6, 0x6d, 0xdf, 0xd4, 0x75, 0x4d, 0x6f, 0x7c, 0x74, 0x59, 0x98, 0x72, 0x99, 0x6d, 0xde, 0x8d,
0xcf, 0xc8, 0xe8, 0x0f, 0x76, 0xa5, 0xa6, 0x72, 0xb9, 0xd9, 0x86, 0x93, 0x74, 0x9f, 0xf4, 0x16,
0x7c, 0x56, 0xb7, 0xdb, 0xf8, 0x97, 0xb5, 0xe6, 0xbc, 0x73, 0x0a, 0x17, 0x87, 0xab, 0x86, 0x05,
0x1f, 0x0d, 0x0b, 0xd6, 0x0d, 0x83, 0x67, 0xcb, 0xe0, 0xd5, 0x32, 0x78, 0xb7, 0x0c, 0x56, 0x96,
0xc1, 0xa7, 0x65, 0xf0, 0x6d, 0x59, 0xb0, 0xb6, 0x0c, 0x5e, 0xbe, 0x58, 0x70, 0x1b, 0xf9, 0x03,
0x9c, 0xfc, 0x04, 0x00, 0x00, 0xff, 0xff, 0x19, 0x97, 0x14, 0xf4, 0x8e, 0x01, 0x00, 0x00,
// 310 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x4c, 0x91, 0xbd, 0x4e, 0xc3, 0x30,
0x14, 0x85, 0x73, 0x9b, 0xfe, 0xba, 0x45, 0x42, 0x16, 0x83, 0xd5, 0xe1, 0x2a, 0x62, 0xca, 0x94,
0x81, 0x4a, 0x08, 0x18, 0x91, 0x78, 0x81, 0xc0, 0x80, 0xd8, 0x8c, 0x6c, 0x8a, 0x69, 0x13, 0x57,
0x89, 0x5b, 0x51, 0x26, 0x1e, 0x81, 0xc7, 0xe0, 0x4d, 0x60, 0xec, 0xc8, 0x48, 0xcc, 0xc2, 0xd8,
0x47, 0x40, 0x76, 0xda, 0xd2, 0xed, 0x9c, 0xef, 0xdc, 0xab, 0xe4, 0x5c, 0x13, 0x52, 0x1a, 0x6e,
0x92, 0x59, 0xa1, 0x8d, 0xa6, 0x83, 0x87, 0x72, 0x6e, 0xd4, 0x34, 0x31, 0xcb, 0x99, 0x2c, 0x8f,
0x3f, 0x1a, 0xa4, 0x79, 0x6d, 0xb8, 0xa1, 0x94, 0x34, 0x67, 0xdc, 0x3c, 0x32, 0x88, 0x20, 0xee,
0xa5, 0x5e, 0x3b, 0x96, 0x69, 0x21, 0x59, 0x23, 0x82, 0xf8, 0x20, 0xf5, 0x9a, 0x1e, 0x92, 0x70,
0xae, 0x04, 0x0b, 0x3d, 0x72, 0xd2, 0x91, 0xb1, 0x12, 0xac, 0x59, 0x93, 0xb1, 0x12, 0x6e, 0xaf,
0x54, 0x2f, 0x92, 0xb5, 0x22, 0x88, 0xc3, 0xd4, 0x6b, 0xca, 0x48, 0x27, 0xd3, 0xe2, 0x46, 0x65,
0x92, 0xb5, 0x3d, 0xde, 0x5a, 0x3a, 0x24, 0xdd, 0xa9, 0xca, 0x27, 0x39, 0xcf, 0x24, 0xeb, 0xf8,
0xaf, 0xef, 0xbc, 0xcb, 0x84, 0x5c, 0x64, 0xfc, 0x49, 0x17, 0xac, 0xeb, 0xd7, 0x76, 0x7e, 0x9b,
0xa9, 0x5c, 0x17, 0xac, 0xf7, 0x9f, 0x39, 0x4f, 0x4f, 0x49, 0xfb, 0x99, 0x1b, 0x53, 0x94, 0x8c,
0x44, 0x61, 0xdc, 0x3f, 0xc1, 0x64, 0xbf, 0x75, 0xe2, 0x1a, 0x27, 0xb7, 0x7e, 0xe0, 0x2a, 0x37,
0xc5, 0x32, 0xdd, 0x4c, 0x0f, 0xcf, 0x49, 0x7f, 0x0f, 0xbb, 0x6a, 0x13, 0xb9, 0xdc, 0xdc, 0xc4,
0x49, 0x7a, 0x44, 0x5a, 0x0b, 0x3e, 0x9d, 0xd7, 0x37, 0x19, 0xa4, 0xb5, 0xb9, 0x68, 0x9c, 0xc1,
0xe5, 0x68, 0x55, 0x61, 0xf0, 0x55, 0x61, 0xb0, 0xae, 0x10, 0x5e, 0x2d, 0xc2, 0xbb, 0x45, 0xf8,
0xb4, 0x08, 0x2b, 0x8b, 0xf0, 0x6d, 0x11, 0x7e, 0x2d, 0x06, 0x6b, 0x8b, 0xf0, 0xf6, 0x83, 0xc1,
0x5d, 0xcb, 0xff, 0xc8, 0x7d, 0xdb, 0xbf, 0xc9, 0xe8, 0x2f, 0x00, 0x00, 0xff, 0xff, 0x90, 0xc2,
0xcf, 0x79, 0xa1, 0x01, 0x00, 0x00,
}

View File

@ -1,6 +1,8 @@
syntax = "proto3";
package fsutil;
package fsutil.types;
option go_package = "types";
message Stat {
string path = 1;

View File

@ -1,8 +1,7 @@
// Code generated by protoc-gen-gogo.
// Code generated by protoc-gen-gogo. DO NOT EDIT.
// source: wire.proto
// DO NOT EDIT!
package fsutil
package types
import proto "github.com/gogo/protobuf/proto"
import fmt "fmt"
@ -50,7 +49,7 @@ var Packet_PacketType_value = map[string]int32{
func (Packet_PacketType) EnumDescriptor() ([]byte, []int) { return fileDescriptorWire, []int{0, 0} }
type Packet struct {
Type Packet_PacketType `protobuf:"varint,1,opt,name=type,proto3,enum=fsutil.Packet_PacketType" json:"type,omitempty"`
Type Packet_PacketType `protobuf:"varint,1,opt,name=type,proto3,enum=fsutil.types.Packet_PacketType" json:"type,omitempty"`
Stat *Stat `protobuf:"bytes,2,opt,name=stat" json:"stat,omitempty"`
ID uint32 `protobuf:"varint,3,opt,name=ID,proto3" json:"ID,omitempty"`
Data []byte `protobuf:"bytes,4,opt,name=data,proto3" json:"data,omitempty"`
@ -89,8 +88,8 @@ func (m *Packet) GetData() []byte {
}
func init() {
proto.RegisterType((*Packet)(nil), "fsutil.Packet")
proto.RegisterEnum("fsutil.Packet_PacketType", Packet_PacketType_name, Packet_PacketType_value)
proto.RegisterType((*Packet)(nil), "fsutil.types.Packet")
proto.RegisterEnum("fsutil.types.Packet_PacketType", Packet_PacketType_name, Packet_PacketType_value)
}
func (x Packet_PacketType) String() string {
s, ok := Packet_PacketType_name[int32(x)]
@ -101,10 +100,7 @@ func (x Packet_PacketType) String() string {
}
func (this *Packet) Equal(that interface{}) bool {
if that == nil {
if this == nil {
return true
}
return false
return this == nil
}
that1, ok := that.(*Packet)
@ -117,10 +113,7 @@ func (this *Packet) Equal(that interface{}) bool {
}
}
if that1 == nil {
if this == nil {
return true
}
return false
return this == nil
} else if this == nil {
return false
}
@ -143,7 +136,7 @@ func (this *Packet) GoString() string {
return "nil"
}
s := make([]string, 0, 8)
s = append(s, "&fsutil.Packet{")
s = append(s, "&types.Packet{")
s = append(s, "Type: "+fmt.Sprintf("%#v", this.Type)+",\n")
if this.Stat != nil {
s = append(s, "Stat: "+fmt.Sprintf("%#v", this.Stat)+",\n")
@ -205,24 +198,6 @@ func (m *Packet) MarshalTo(dAtA []byte) (int, error) {
return i, nil
}
func encodeFixed64Wire(dAtA []byte, offset int, v uint64) int {
dAtA[offset] = uint8(v)
dAtA[offset+1] = uint8(v >> 8)
dAtA[offset+2] = uint8(v >> 16)
dAtA[offset+3] = uint8(v >> 24)
dAtA[offset+4] = uint8(v >> 32)
dAtA[offset+5] = uint8(v >> 40)
dAtA[offset+6] = uint8(v >> 48)
dAtA[offset+7] = uint8(v >> 56)
return offset + 8
}
func encodeFixed32Wire(dAtA []byte, offset int, v uint32) int {
dAtA[offset] = uint8(v)
dAtA[offset+1] = uint8(v >> 8)
dAtA[offset+2] = uint8(v >> 16)
dAtA[offset+3] = uint8(v >> 24)
return offset + 4
}
func encodeVarintWire(dAtA []byte, offset int, v uint64) int {
for v >= 1<<7 {
dAtA[offset] = uint8(v&0x7f | 0x80)
@ -546,22 +521,22 @@ var (
func init() { proto.RegisterFile("wire.proto", fileDescriptorWire) }
var fileDescriptorWire = []byte{
// 259 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xe2, 0xe2, 0x2a, 0xcf, 0x2c, 0x4a,
0xd5, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x62, 0x4b, 0x2b, 0x2e, 0x2d, 0xc9, 0xcc, 0x91, 0xe2,
0x2a, 0x2e, 0x49, 0x2c, 0x81, 0x88, 0x29, 0xdd, 0x65, 0xe4, 0x62, 0x0b, 0x48, 0x4c, 0xce, 0x4e,
0x2d, 0x11, 0xd2, 0xe5, 0x62, 0x29, 0xa9, 0x2c, 0x48, 0x95, 0x60, 0x54, 0x60, 0xd4, 0xe0, 0x33,
0x92, 0xd4, 0x83, 0xa8, 0xd6, 0x83, 0xc8, 0x42, 0xa9, 0x90, 0xca, 0x82, 0xd4, 0x20, 0xb0, 0x32,
0x21, 0x05, 0x2e, 0x16, 0x90, 0x39, 0x12, 0x4c, 0x0a, 0x8c, 0x1a, 0xdc, 0x46, 0x3c, 0x30, 0xe5,
0xc1, 0x25, 0x89, 0x25, 0x41, 0x60, 0x19, 0x21, 0x3e, 0x2e, 0x26, 0x4f, 0x17, 0x09, 0x66, 0x05,
0x46, 0x0d, 0xde, 0x20, 0x26, 0x4f, 0x17, 0x21, 0x21, 0x2e, 0x96, 0x94, 0xc4, 0x92, 0x44, 0x09,
0x16, 0x05, 0x46, 0x0d, 0x9e, 0x20, 0x30, 0x5b, 0x29, 0x8e, 0x8b, 0x0b, 0x61, 0xb2, 0x10, 0x3f,
0x17, 0x77, 0x80, 0xa3, 0xb3, 0xb7, 0x6b, 0x48, 0x7c, 0x70, 0x88, 0x63, 0x88, 0x00, 0x83, 0x10,
0x1f, 0x17, 0x17, 0x54, 0x20, 0xc8, 0x35, 0x50, 0x80, 0x11, 0x49, 0x81, 0x8b, 0x63, 0x88, 0xa3,
0x00, 0x13, 0x92, 0x02, 0x37, 0x4f, 0x3f, 0x01, 0x66, 0x24, 0xbe, 0x6b, 0x50, 0x90, 0x00, 0x8b,
0x93, 0xce, 0x85, 0x87, 0x72, 0x0c, 0x37, 0x1e, 0xca, 0x31, 0x7c, 0x78, 0x28, 0xc7, 0xd8, 0xf0,
0x48, 0x8e, 0x71, 0xc5, 0x23, 0x39, 0xc6, 0x13, 0x8f, 0xe4, 0x18, 0x2f, 0x3c, 0x92, 0x63, 0x7c,
0xf0, 0x48, 0x8e, 0xf1, 0xc5, 0x23, 0x39, 0x86, 0x0f, 0x8f, 0xe4, 0x18, 0x27, 0x3c, 0x96, 0x63,
0x48, 0x62, 0x03, 0x07, 0x8a, 0x31, 0x20, 0x00, 0x00, 0xff, 0xff, 0x8b, 0xce, 0x55, 0x3b, 0x36,
0x01, 0x00, 0x00,
// 268 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0xe2, 0x2a, 0xcf, 0x2c, 0x4a,
0xd5, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0xe2, 0x49, 0x2b, 0x2e, 0x2d, 0xc9, 0xcc, 0xd1, 0x2b,
0xa9, 0x2c, 0x48, 0x2d, 0x96, 0xe2, 0x2a, 0x2e, 0x49, 0x2c, 0x81, 0xc8, 0x28, 0xbd, 0x64, 0xe4,
0x62, 0x0b, 0x48, 0x4c, 0xce, 0x4e, 0x2d, 0x11, 0x32, 0xe6, 0x62, 0x01, 0xc9, 0x4b, 0x30, 0x2a,
0x30, 0x6a, 0xf0, 0x19, 0xc9, 0xeb, 0x21, 0xeb, 0xd1, 0x83, 0xa8, 0x81, 0x52, 0x21, 0x95, 0x05,
0xa9, 0x41, 0x60, 0xc5, 0x42, 0x6a, 0x5c, 0x2c, 0x20, 0xd3, 0x24, 0x98, 0x14, 0x18, 0x35, 0xb8,
0x8d, 0x84, 0x50, 0x35, 0x05, 0x97, 0x24, 0x96, 0x04, 0x81, 0xe5, 0x85, 0xf8, 0xb8, 0x98, 0x3c,
0x5d, 0x24, 0x98, 0x15, 0x18, 0x35, 0x78, 0x83, 0x98, 0x3c, 0x5d, 0x84, 0x84, 0xb8, 0x58, 0x52,
0x12, 0x4b, 0x12, 0x25, 0x58, 0x14, 0x18, 0x35, 0x78, 0x82, 0xc0, 0x6c, 0xa5, 0x38, 0x2e, 0x2e,
0x84, 0xf9, 0x42, 0xfc, 0x5c, 0xdc, 0x01, 0x8e, 0xce, 0xde, 0xae, 0x21, 0xf1, 0xc1, 0x21, 0x8e,
0x21, 0x02, 0x0c, 0x42, 0x7c, 0x5c, 0x5c, 0x50, 0x81, 0x20, 0xd7, 0x40, 0x01, 0x46, 0x24, 0x05,
0x2e, 0x8e, 0x21, 0x8e, 0x02, 0x4c, 0x48, 0x0a, 0xdc, 0x3c, 0xfd, 0x04, 0x98, 0x91, 0xf8, 0xae,
0x41, 0x41, 0x02, 0x2c, 0x4e, 0xc6, 0x17, 0x1e, 0xca, 0x31, 0xdc, 0x78, 0x28, 0xc7, 0xf0, 0xe1,
0xa1, 0x1c, 0x63, 0xc3, 0x23, 0x39, 0xc6, 0x15, 0x8f, 0xe4, 0x18, 0x4f, 0x3c, 0x92, 0x63, 0xbc,
0xf0, 0x48, 0x8e, 0xf1, 0xc1, 0x23, 0x39, 0xc6, 0x17, 0x8f, 0xe4, 0x18, 0x3e, 0x3c, 0x92, 0x63,
0x9c, 0xf0, 0x58, 0x8e, 0x21, 0x8a, 0x15, 0xec, 0x87, 0x24, 0x36, 0x70, 0x38, 0x19, 0x03, 0x02,
0x00, 0x00, 0xff, 0xff, 0xce, 0x0f, 0xe2, 0x94, 0x4f, 0x01, 0x00, 0x00,
}

View File

@ -1,6 +1,8 @@
syntax = "proto3";
package fsutil;
package fsutil.types;
option go_package = "types";
import "stat.proto";

View File

@ -4,12 +4,12 @@ import (
"context"
"os"
"path/filepath"
"runtime"
"strings"
"time"
"github.com/docker/docker/pkg/fileutils"
"github.com/pkg/errors"
"github.com/tonistiigi/fsutil/types"
)
type WalkOpt struct {
@ -18,7 +18,7 @@ type WalkOpt struct {
// FollowPaths contains symlinks that are resolved into include patterns
// before performing the fs walk
FollowPaths []string
Map func(*Stat) bool
Map func(*types.Stat) bool
}
func Walk(ctx context.Context, p string, opt *WalkOpt, fn filepath.WalkFunc) error {
@ -146,37 +146,9 @@ func Walk(ctx context.Context, p string, opt *WalkOpt, fn filepath.WalkFunc) err
}
passedFilter:
path = filepath.ToSlash(path)
stat := &Stat{
Path: path,
Mode: uint32(fi.Mode()),
ModTime: fi.ModTime().UnixNano(),
}
setUnixOpt(fi, stat, path, seenFiles)
if !fi.IsDir() {
stat.Size_ = fi.Size()
if fi.Mode()&os.ModeSymlink != 0 {
link, err := os.Readlink(origpath)
stat, err := mkstat(origpath, path, fi, seenFiles)
if err != nil {
return errors.Wrapf(err, "failed to readlink %s", origpath)
}
stat.Linkname = link
}
}
if err := loadXattr(origpath, stat); err != nil {
return errors.Wrapf(err, "failed to xattr %s", path)
}
if runtime.GOOS == "windows" {
permPart := stat.Mode & uint32(os.ModePerm)
noPermPart := stat.Mode &^ uint32(os.ModePerm)
// Add the x bit: make everything +x from windows
permPart |= 0111
permPart &= 0755
stat.Mode = noPermPart | permPart
return err
}
select {
@ -197,7 +169,7 @@ func Walk(ctx context.Context, p string, opt *WalkOpt, fn filepath.WalkFunc) err
}
type StatInfo struct {
*Stat
*types.Stat
}
func (s *StatInfo) Name() string {

View File

@ -1,14 +0,0 @@
// +build windows
package fsutil
import (
"os"
)
func loadXattr(_ string, _ *Stat) error {
return nil
}
func setUnixOpt(_ os.FileInfo, _ *Stat, _ string, _ map[uint64]string) {
}