contenthash: initial commit

Signed-off-by: Tonis Tiigi <tonistiigi@gmail.com>
docker-18.09
Tonis Tiigi 2017-07-27 14:46:51 -07:00
parent 39b9dec52f
commit 2c38da5650
57 changed files with 2866 additions and 5366 deletions

498
cache/contenthash/checksum.go vendored Normal file
View File

@ -0,0 +1,498 @@
package contenthash
import (
"bytes"
"context"
"crypto/sha256"
"os"
"path"
"path/filepath"
"sync"
"github.com/docker/docker/pkg/pools"
"github.com/docker/docker/pkg/symlink"
iradix "github.com/hashicorp/go-immutable-radix"
"github.com/moby/buildkit/cache"
"github.com/moby/buildkit/snapshot"
digest "github.com/opencontainers/go-digest"
"github.com/pkg/errors"
"github.com/tonistiigi/fsutil"
)
var errNotFound = errors.Errorf("not found")
var defaultManager = &cacheManager{actives: map[string]*cacheContext{}}
// Layout in the radix tree: Every path is saved by cleaned absolute unix path.
// Directories have 2 records, one contains digest for directory header, other
// the recursive digest for directory contents. "/dir/" is the record for
// header, "/dir" is for contents. For the root node "" (empty string) is the
// key for root, "/" for the root header
func Checksum(ctx context.Context, ref cache.ImmutableRef, path string) (digest.Digest, error) {
return defaultManager.Checksum(ctx, ref, path)
}
// func GetCacheContext(ctx context.Context, ref cache.ImmutableRef) (CacheContext, error) {
//
// }
//
// func SetCacheContext(ctx context.Context, ref cache.ImmutableRef, cc CacheContext) error {
//
// }
type CacheContext interface {
HandleChange(kind fsutil.ChangeKind, p string, fi os.FileInfo, err error)
// Reset(p string)
Marshal() ([]byte, error)
}
type CacheRecord struct {
Type CacheRecordType
Link string
Digest digest.Digest
}
type Hashed interface {
Digest() digest.Digest
}
type CacheRecordType int
const (
CacheRecordFile CacheRecordType = iota
CacheRecordDir
CacheRecordDirHeader
CacheRecordSymlink
)
type cacheManager struct {
mu sync.Mutex
actives map[string]*cacheContext
}
func (cm *cacheManager) Checksum(ctx context.Context, ref cache.ImmutableRef, p string) (digest.Digest, error) {
cm.mu.Lock()
cc, ok := cm.actives[ref.ID()]
if !ok {
cc = newCacheContext(ref)
cm.actives[ref.ID()] = cc
}
cc.refs++
cm.mu.Unlock()
defer func() {
cm.mu.Lock()
cc.refs--
if cc.refs == 0 {
cc.save() // TODO: do this on background, BUT need to unmount before releasing, possibly wrap ref
cc.clean()
delete(cm.actives, ref.ID())
}
cm.mu.Unlock()
}()
return cc.Checksum(ctx, p)
}
type cacheContext struct {
mu sync.RWMutex
mountPath string
unmount func() error
ref cache.ImmutableRef
refs int
tree *iradix.Tree
// isDirty bool
// used in HandleChange
txn *iradix.Txn
node *iradix.Node
dirtyMap map[string]struct{}
}
func newCacheContext(ref cache.ImmutableRef) *cacheContext {
cc := &cacheContext{
ref: ref,
tree: iradix.New(),
dirtyMap: map[string]struct{}{},
}
// cc.Load(md)
return cc
}
func (cc *cacheContext) save() {
// TODO:
}
// HandleChange notifies the source about a modification operation
func (cc *cacheContext) HandleChange(kind fsutil.ChangeKind, p string, fi os.FileInfo, err error) (retErr error) {
p = path.Join("/", filepath.ToSlash(p))
if p == "/" {
p = ""
}
k := []byte(p)
deleteDir := func(cr *CacheRecord) {
if cr.Type == CacheRecordDir {
cc.node.WalkPrefix(append(k, []byte("/")...), func(k []byte, v interface{}) bool {
cc.txn.Delete(k)
return false
})
}
}
cc.mu.Lock()
if cc.txn == nil {
cc.txn = cc.tree.Txn()
cc.node = cc.tree.Root()
}
if kind == fsutil.ChangeKindDelete {
v, ok := cc.txn.Delete(k)
if ok {
deleteDir(v.(*CacheRecord))
}
d := path.Dir(string(k))
if d == "/" {
d = ""
}
cc.dirtyMap[d] = struct{}{}
cc.mu.Unlock()
return
}
stat, ok := fi.Sys().(*fsutil.Stat)
if !ok {
return errors.Errorf("%s invalid change without stat information", p)
}
h, ok := fi.(Hashed)
if !ok {
cc.mu.Unlock()
return errors.Errorf("invalid fileinfo: %s", p)
}
v, ok := cc.node.Get(k)
if ok {
deleteDir(v.(*CacheRecord))
}
cr := &CacheRecord{
Type: CacheRecordFile,
}
if fi.Mode()&os.ModeSymlink != 0 {
cr.Type = CacheRecordSymlink
cr.Link = filepath.ToSlash(stat.Linkname)
}
if fi.IsDir() {
cr.Type = CacheRecordDirHeader
cr2 := &CacheRecord{
Type: CacheRecordDir,
}
cc.txn.Insert(k, cr2)
k = append(k, []byte("/")...)
}
cr.Digest = h.Digest()
cc.txn.Insert(k, cr)
d := path.Dir(string(k))
if d == "/" {
d = ""
}
cc.dirtyMap[d] = struct{}{}
cc.mu.Unlock()
return nil
}
func (cc *cacheContext) Checksum(ctx context.Context, p string) (digest.Digest, error) {
const maxSymlinkLimit = 255
i := 0
for {
if i > maxSymlinkLimit {
return "", errors.Errorf("too many symlinks: %s", p)
}
cr, err := cc.ChecksumNoFollow(ctx, p)
if err != nil {
return "", err
}
if cr.Type == CacheRecordSymlink {
link := cr.Link
if !path.IsAbs(cr.Link) {
link = path.Join(path.Dir(p), link)
}
i++
p = link
} else {
return cr.Digest, nil
}
}
}
func (cc *cacheContext) ChecksumNoFollow(ctx context.Context, p string) (*CacheRecord, error) {
p = path.Join("/", filepath.ToSlash(p))
if p == "/" {
p = ""
}
cc.mu.RLock()
if cc.txn == nil {
root := cc.tree.Root()
cc.mu.RUnlock()
v, ok := root.Get([]byte(p))
if ok {
cr := v.(*CacheRecord)
if cr.Digest != "" {
return cr, nil
}
}
} else {
cc.mu.RUnlock()
}
cc.mu.Lock()
defer cc.mu.Unlock()
if cc.txn != nil {
cc.commitActiveTransaction()
}
return cc.lazyChecksum(ctx, p)
}
func (cc *cacheContext) commitActiveTransaction() {
for d := range cc.dirtyMap {
addParentToMap(d, cc.dirtyMap)
}
for d := range cc.dirtyMap {
cc.txn.Insert([]byte(d), &CacheRecord{Type: CacheRecordDir})
}
cc.tree = cc.txn.Commit()
cc.node = nil
cc.dirtyMap = map[string]struct{}{}
cc.txn = nil
}
func (cc *cacheContext) lazyChecksum(ctx context.Context, p string) (*CacheRecord, error) {
root := cc.tree.Root()
if cc.needsScan(root, p) {
if err := cc.scanPath(ctx, p); err != nil {
return nil, err
}
}
k := []byte(p)
root = cc.tree.Root()
txn := cc.tree.Txn()
cr, err := cc.checksum(ctx, root, txn, k)
if err != nil {
return nil, err
}
cc.tree = txn.Commit()
return cr, err
}
func (cc *cacheContext) checksum(ctx context.Context, root *iradix.Node, txn *iradix.Txn, k []byte) (*CacheRecord, error) {
v, ok := root.Get(k)
if !ok {
return nil, errors.Wrapf(errNotFound, "%s not found", string(k))
}
cr := v.(*CacheRecord)
if cr.Digest != "" {
return cr, nil
}
var dgst digest.Digest
switch cr.Type {
case CacheRecordDir:
h := sha256.New()
iter := root.Iterator()
next := append(k, []byte("/")...)
iter.SeekPrefix(next)
for {
subk, _, ok := iter.Next()
if !ok || bytes.Compare(next, subk) > 0 {
break
}
h.Write(bytes.TrimPrefix(subk, k))
subcr, err := cc.checksum(ctx, root, txn, subk)
if err != nil {
return nil, err
}
h.Write([]byte(subcr.Digest))
if subcr.Type == CacheRecordDir { // skip subfiles
next = append(k, []byte("/\xff")...)
iter.SeekPrefix(next)
}
}
dgst = digest.NewDigest(digest.SHA256, h)
default:
p := string(bytes.TrimSuffix(k, []byte("/")))
target, err := cc.root(ctx)
if err != nil {
return nil, err
}
// no FollowSymlinkInScope because invalid paths should not be inserted
fp := filepath.Join(target, filepath.FromSlash(p))
fi, err := os.Lstat(fp)
if err != nil {
return nil, err
}
dgst, err = prepareDigest(fp, p, fi)
if err != nil {
return nil, err
}
}
cr2 := &CacheRecord{
Digest: dgst,
Type: cr.Type,
Link: cr.Link,
}
txn.Insert(k, cr2)
return cr2, nil
}
func (cc *cacheContext) needsScan(root *iradix.Node, p string) bool {
if p == "/" {
p = ""
}
if _, ok := root.Get([]byte(p)); !ok {
if p == "" {
return true
}
return cc.needsScan(root, path.Clean(path.Dir(p)))
}
return false
}
func (cc *cacheContext) root(ctx context.Context) (string, error) {
if cc.mountPath != "" {
return cc.mountPath, nil
}
mounts, err := cc.ref.Mount(ctx, true)
if err != nil {
return "", err
}
lm := snapshot.LocalMounter(mounts)
mp, err := lm.Mount()
if err != nil {
return "", err
}
cc.mountPath = mp
cc.unmount = lm.Unmount
return mp, nil
}
func (cc *cacheContext) clean() error {
if cc.mountPath != "" {
err := cc.unmount()
cc.mountPath = ""
cc.unmount = nil
return err
}
return nil
}
func (cc *cacheContext) scanPath(ctx context.Context, p string) (retErr error) {
p = path.Join("/", p)
d, _ := path.Split(p)
mp, err := cc.root(ctx)
if err != nil {
return err
}
parentPath, err := symlink.FollowSymlinkInScope(filepath.Join(mp, filepath.FromSlash(d)), mp)
if err != nil {
return err
}
n := cc.tree.Root()
txn := cc.tree.Txn()
err = filepath.Walk(parentPath, func(path string, fi os.FileInfo, err error) error {
if err != nil {
return errors.Wrapf(err, "failed to walk %s", path)
}
rel, err := filepath.Rel(mp, path)
if err != nil {
return err
}
k := []byte(filepath.Join("/", filepath.ToSlash(rel)))
if string(k) == "/" {
k = []byte{}
}
if _, ok := n.Get(k); !ok {
cr := &CacheRecord{
Type: CacheRecordFile,
}
if fi.Mode()&os.ModeSymlink != 0 {
cr.Type = CacheRecordSymlink
link, err := os.Readlink(path)
if err != nil {
return err
}
cr.Link = filepath.ToSlash(link)
}
if fi.IsDir() {
cr.Type = CacheRecordDirHeader
cr2 := &CacheRecord{
Type: CacheRecordDir,
}
txn.Insert(k, cr2)
k = append(k, []byte("/")...)
}
txn.Insert(k, cr)
}
return nil
})
if err != nil {
return err
}
cc.tree = txn.Commit()
return nil
}
func prepareDigest(fp, p string, fi os.FileInfo) (digest.Digest, error) {
h, err := NewFileHash(fp, fi)
if err != nil {
return "", errors.Wrapf(err, "failed to create hash for %s", p)
}
if fi.Mode().IsRegular() && fi.Size() > 0 {
// TODO: would be nice to put the contents to separate hash first
// so it can be cached for hardlinks
f, err := os.Open(fp)
if err != nil {
return "", errors.Wrapf(err, "failed to open %s", p)
}
defer f.Close()
if _, err := pools.Copy(h, f); err != nil {
return "", errors.Wrapf(err, "failed to copy file data for %s", p)
}
}
return digest.NewDigest(digest.SHA256, h), nil
}
func addParentToMap(d string, m map[string]struct{}) {
if d == "" {
return
}
d = path.Dir(d)
if d == "/" {
d = ""
}
m[d] = struct{}{}
addParentToMap(d, m)
}

426
cache/contenthash/checksum_test.go vendored Normal file
View File

@ -0,0 +1,426 @@
package contenthash
import (
"context"
"fmt"
"io"
"io/ioutil"
"os"
"path/filepath"
"strings"
"testing"
"github.com/containerd/containerd/snapshot/naive"
"github.com/moby/buildkit/cache"
"github.com/moby/buildkit/cache/metadata"
"github.com/moby/buildkit/snapshot"
digest "github.com/opencontainers/go-digest"
"github.com/pkg/errors"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/tonistiigi/fsutil"
)
const (
dgstFileData0 = digest.Digest("sha256:5a08b2cd870f1f48a9513629458c2c962ac11a2e44908956b3ec2d269c3e1223")
dgstDirD0 = digest.Digest("sha256:260564ab80256c98f63fde8a6ef289525cc542c723970d1c47d721c9c469f9ea")
dgstDirD0Modified = digest.Digest("sha256:60c901771ca6f68be81fefe1fcb921118458e0de00227e7c9a9676fb4f32946b")
)
func TestChecksumBasicFile(t *testing.T) {
tmpdir, err := ioutil.TempDir("", "buildkit-state")
require.NoError(t, err)
defer os.RemoveAll(tmpdir)
cm := setupCacheManager(t, tmpdir)
defer cm.Close()
ch := []string{
"ADD foo file data0",
"ADD bar file data1",
"ADD d0 dir",
"ADD d0/abc file data0",
"ADD d0/def symlink abc",
"ADD d0/ghi symlink nosuchfile",
}
ref := createRef(t, cm, ch)
// for the digest values, the actual values are not important in development
// phase but consistency is
cc := newCacheContext(ref)
_, err = cc.Checksum(context.TODO(), "nosuch")
assert.Error(t, err)
dgst, err := cc.Checksum(context.TODO(), "foo")
assert.NoError(t, err)
assert.Equal(t, dgstFileData0, dgst)
// second file returns different hash
dgst, err = cc.Checksum(context.TODO(), "bar")
assert.NoError(t, err)
assert.Equal(t, digest.Digest("sha256:cb62966e6dc11e3252ce1a14ed51c6ed0cf112de9c5d23104dc6dcc708f914f1"), dgst)
// same file inside a directory
dgst, err = cc.Checksum(context.TODO(), "d0/abc")
assert.NoError(t, err)
assert.Equal(t, dgstFileData0, dgst)
// repeat because codepath is different
dgst, err = cc.Checksum(context.TODO(), "d0/abc")
assert.NoError(t, err)
assert.Equal(t, dgstFileData0, dgst)
// symlink to the same file is followed, returns same hash
dgst, err = cc.Checksum(context.TODO(), "d0/def")
assert.NoError(t, err)
assert.Equal(t, dgstFileData0, dgst)
_, err = cc.Checksum(context.TODO(), "d0/ghi")
assert.Error(t, err)
assert.Equal(t, errNotFound, errors.Cause(err))
dgst, err = cc.Checksum(context.TODO(), "/")
assert.NoError(t, err)
assert.Equal(t, digest.Digest("sha256:0d87c8c2a606f961483cd4c5dc0350a4136a299b4066eea4a969d6ed756614cd"), dgst)
dgst, err = cc.Checksum(context.TODO(), "d0")
assert.NoError(t, err)
assert.Equal(t, dgstDirD0, dgst)
cc.clean()
err = ref.Release(context.TODO())
require.NoError(t, err)
// this is same directory as previous d0
ch = []string{
"ADD abc file data0",
"ADD def symlink abc",
"ADD ghi symlink nosuchfile",
}
ref = createRef(t, cm, ch)
cc = newCacheContext(ref)
dgst, err = cc.Checksum(context.TODO(), "/")
assert.NoError(t, err)
assert.Equal(t, dgstDirD0, dgst)
cc.clean()
err = ref.Release(context.TODO())
require.NoError(t, err)
// test that removing broken symlink changes hash even though symlink itself can't be checksummed
ch = []string{
"ADD abc file data0",
"ADD def symlink abc",
}
ref = createRef(t, cm, ch)
cc = newCacheContext(ref)
dgst, err = cc.Checksum(context.TODO(), "/")
assert.NoError(t, err)
assert.Equal(t, dgstDirD0Modified, dgst)
assert.NotEqual(t, dgstDirD0, dgst)
cc.clean()
err = ref.Release(context.TODO())
require.NoError(t, err)
// test multiple scans, get checksum of nested file first
ch = []string{
"ADD abc dir",
"ADD abc/aa dir",
"ADD abc/aa/foo file data2",
"ADD d0 dir",
"ADD d0/abc file data0",
"ADD d0/def symlink abc",
"ADD d0/ghi symlink nosuchfile",
}
ref = createRef(t, cm, ch)
cc = newCacheContext(ref)
dgst, err = cc.Checksum(context.TODO(), "abc/aa/foo")
assert.NoError(t, err)
assert.Equal(t, digest.Digest("sha256:e1e22281a1ebb637e46aa0781c7fceaca817f1268dd2047dfbce4a23a6cf50ad"), dgst)
assert.NotEqual(t, dgstDirD0, dgst)
// this will force rescan
dgst, err = cc.Checksum(context.TODO(), "d0")
assert.NoError(t, err)
assert.Equal(t, dgstDirD0, dgst)
cc.clean()
err = ref.Release(context.TODO())
require.NoError(t, err)
}
func TestHandleChange(t *testing.T) {
tmpdir, err := ioutil.TempDir("", "buildkit-state")
require.NoError(t, err)
defer os.RemoveAll(tmpdir)
cm := setupCacheManager(t, tmpdir)
defer cm.Close()
ch := []string{
"ADD foo file data0",
"ADD bar file data1",
"ADD d0 dir",
"ADD d0/abc file data0",
"ADD d0/def symlink abc",
"ADD d0/ghi symlink nosuchfile",
}
ref := createRef(t, cm, nil)
// for the digest values, the actual values are not important in development
// phase but consistency is
cc := newCacheContext(ref)
err = emit(cc.HandleChange, changeStream(ch))
assert.NoError(t, err)
dgstFoo, err := cc.Checksum(context.TODO(), "foo")
assert.NoError(t, err)
assert.Equal(t, dgstFileData0, dgstFoo)
// symlink to the same file is followed, returns same hash
dgst, err := cc.Checksum(context.TODO(), "d0/def")
assert.NoError(t, err)
assert.Equal(t, dgstFoo, dgst)
// symlink to the same file is followed, returns same hash
dgst, err = cc.Checksum(context.TODO(), "d0")
assert.NoError(t, err)
assert.Equal(t, dgstDirD0, dgst)
ch = []string{
"DEL d0/ghi file",
}
err = emit(cc.HandleChange, changeStream(ch))
assert.NoError(t, err)
dgst, err = cc.Checksum(context.TODO(), "d0")
assert.NoError(t, err)
assert.Equal(t, dgstDirD0Modified, dgst)
ch = []string{
"DEL d0 dir",
}
err = emit(cc.HandleChange, changeStream(ch))
assert.NoError(t, err)
_, err = cc.Checksum(context.TODO(), "d0")
assert.Error(t, err)
assert.Equal(t, errNotFound, errors.Cause(err))
_, err = cc.Checksum(context.TODO(), "d0/abc")
assert.Error(t, err)
assert.Equal(t, errNotFound, errors.Cause(err))
cc.clean()
err = ref.Release(context.TODO())
require.NoError(t, err)
}
func createRef(t *testing.T, cm cache.Manager, files []string) cache.ImmutableRef {
mref, err := cm.New(context.TODO(), nil)
require.NoError(t, err)
mounts, err := mref.Mount(context.TODO(), false)
require.NoError(t, err)
lm := snapshot.LocalMounter(mounts)
mp, err := lm.Mount()
require.NoError(t, err)
err = writeChanges(mp, changeStream(files))
lm.Unmount()
require.NoError(t, err)
ref, err := mref.Commit(context.TODO())
require.NoError(t, err)
return ref
}
func setupCacheManager(t *testing.T, tmpdir string) cache.Manager {
snapshotter, err := naive.NewSnapshotter(filepath.Join(tmpdir, "snapshots"))
require.NoError(t, err)
md, err := metadata.NewStore(filepath.Join(tmpdir, "metadata.db"))
require.NoError(t, err)
cm, err := cache.NewManager(cache.ManagerOpt{
Snapshotter: snapshotter,
MetadataStore: md,
})
require.NoError(t, err)
return cm
}
// these test helpers are from tonistiigi/fsutil
type change struct {
kind fsutil.ChangeKind
path string
fi os.FileInfo
data string
}
func changeStream(dt []string) (changes []*change) {
for _, s := range dt {
changes = append(changes, parseChange(s))
}
return
}
func parseChange(str string) *change {
f := strings.Fields(str)
errStr := fmt.Sprintf("invalid change %q", str)
if len(f) < 3 {
panic(errStr)
}
c := &change{}
switch f[0] {
case "ADD":
c.kind = fsutil.ChangeKindAdd
case "CHG":
c.kind = fsutil.ChangeKindModify
case "DEL":
c.kind = fsutil.ChangeKindDelete
default:
panic(errStr)
}
c.path = f[1]
st := &fsutil.Stat{}
switch f[2] {
case "file":
if len(f) > 3 {
if f[3][0] == '>' {
st.Linkname = f[3][1:]
} else {
c.data = f[3]
st.Size_ = int64(len(f[3]))
}
}
st.Mode |= 0644
case "dir":
st.Mode |= uint32(os.ModeDir)
st.Mode |= 0700
case "symlink":
if len(f) < 4 {
panic(errStr)
}
st.Mode |= uint32(os.ModeSymlink)
st.Linkname = f[3]
st.Mode |= 0777
}
c.fi = &fsutil.StatInfo{st}
return c
}
func emit(fn fsutil.HandleChangeFn, inp []*change) error {
for _, c := range inp {
stat, ok := c.fi.Sys().(*fsutil.Stat)
if !ok {
return errors.Errorf("invalid non-stat change %s", c.fi.Name())
}
fi := c.fi
if c.kind != fsutil.ChangeKindDelete {
h, err := NewFromStat(stat)
if err != nil {
return err
}
if _, err := io.Copy(h, strings.NewReader(c.data)); err != nil {
return err
}
fi = &withHash{FileInfo: c.fi, digest: digest.NewDigest(digest.SHA256, h)}
}
if err := fn(c.kind, c.path, fi, nil); err != nil {
return err
}
}
return nil
}
type withHash struct {
digest digest.Digest
os.FileInfo
}
func (wh *withHash) Digest() digest.Digest {
return wh.digest
}
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)
if !ok {
return errors.Errorf("invalid non-stat change %s", p)
}
if c.fi.IsDir() {
if err := os.Mkdir(p, 0700); err != nil {
return err
}
} else if c.fi.Mode()&os.ModeSymlink != 0 {
if err := os.Symlink(stat.Linkname, p); err != nil {
return err
}
} else if len(stat.Linkname) > 0 {
if err := os.Link(filepath.Join(p, stat.Linkname), p); err != nil {
return err
}
} else {
f, err := os.Create(p)
if err != nil {
return err
}
if len(c.data) > 0 {
if _, err := f.Write([]byte(c.data)); err != nil {
return err
}
}
f.Close()
}
}
}
return nil
}

108
cache/contenthash/filehash.go vendored Normal file
View File

@ -0,0 +1,108 @@
package contenthash
import (
"archive/tar"
"crypto/sha256"
"hash"
"os"
"path/filepath"
"time"
"github.com/stevvooe/continuity/sysx"
"github.com/tonistiigi/fsutil"
)
// NewFileHash returns new hash that is used for the builder cache keys
func NewFileHash(path string, fi os.FileInfo) (hash.Hash, error) {
var link string
if fi.Mode()&os.ModeSymlink != 0 {
var err error
link, err = os.Readlink(path)
if err != nil {
return nil, err
}
}
stat := &fsutil.Stat{
Mode: uint32(fi.Mode()),
Size_: fi.Size(),
ModTime: fi.ModTime().UnixNano(),
Linkname: link,
}
setUnixOpt(fi, stat)
attrs, err := sysx.LListxattr(path)
if err != nil {
return nil, err
}
if len(attrs) > 0 {
stat.Xattrs = map[string][]byte{}
for _, attr := range attrs {
v, err := sysx.LGetxattr(path, attr)
if err == nil {
stat.Xattrs[attr] = v
}
}
}
return NewFromStat(stat)
}
func NewFromStat(stat *fsutil.Stat) (hash.Hash, error) {
fi := &statInfo{stat}
hdr, err := tar.FileInfoHeader(fi, stat.Linkname)
if err != nil {
return nil, err
}
hdr.Name = "" // note: empty name is different from current has in docker build. Name is added on recursive directory scan instead
hdr.Mode = int64(chmodWindowsTarEntry(os.FileMode(hdr.Mode)))
hdr.Devmajor = stat.Devmajor
hdr.Devminor = stat.Devminor
if len(stat.Xattrs) > 0 {
hdr.Xattrs = make(map[string]string, len(stat.Xattrs))
for k, v := range stat.Xattrs {
hdr.Xattrs[k] = string(v)
}
}
// fmt.Printf("hdr: %#v\n", hdr)
tsh := &tarsumHash{hdr: hdr, Hash: sha256.New()}
tsh.Reset() // initialize header
return tsh, nil
}
type tarsumHash struct {
hash.Hash
hdr *tar.Header
}
// Reset resets the Hash to its initial state.
func (tsh *tarsumHash) Reset() {
// comply with hash.Hash and reset to the state hash had before any writes
tsh.Hash.Reset()
WriteV1TarsumHeaders(tsh.hdr, tsh.Hash)
}
type statInfo struct {
*fsutil.Stat
}
func (s *statInfo) Name() string {
return filepath.Base(s.Stat.Path)
}
func (s *statInfo) Size() int64 {
return s.Stat.Size_
}
func (s *statInfo) Mode() os.FileMode {
return os.FileMode(s.Stat.Mode)
}
func (s *statInfo) ModTime() time.Time {
return time.Unix(s.Stat.ModTime/1e9, s.Stat.ModTime%1e9)
}
func (s *statInfo) IsDir() bool {
return s.Mode().IsDir()
}
func (s *statInfo) Sys() interface{} {
return s.Stat
}

37
cache/contenthash/filehash_unix.go vendored Normal file
View File

@ -0,0 +1,37 @@
// +build !windows
package contenthash
import (
"os"
"syscall"
"github.com/tonistiigi/fsutil"
)
func chmodWindowsTarEntry(perm os.FileMode) os.FileMode {
return perm
}
func setUnixOpt(fi os.FileInfo, stat *fsutil.Stat) {
s := fi.Sys().(*syscall.Stat_t)
stat.Uid = s.Uid
stat.Gid = s.Gid
if !fi.IsDir() {
if s.Mode&syscall.S_IFBLK != 0 ||
s.Mode&syscall.S_IFCHR != 0 {
stat.Devmajor = int64(major(uint64(s.Rdev)))
stat.Devminor = int64(minor(uint64(s.Rdev)))
}
}
}
func major(device uint64) uint64 {
return (device >> 8) & 0xfff
}
func minor(device uint64) uint64 {
return (device & 0xff) | ((device >> 12) & 0xfff00)
}

22
cache/contenthash/filehash_windows.go vendored Normal file
View File

@ -0,0 +1,22 @@
// +build windows
package contenthash
import (
"os"
"github.com/tonistiigi/fsutil"
)
// chmodWindowsTarEntry is used to adjust the file permissions used in tar
// header based on the platform the archival is done.
func chmodWindowsTarEntry(perm os.FileMode) os.FileMode {
perm &= 0755
// Add the x bit: make everything +x from windows
perm |= 0111
return perm
}
func setUnixOpt(fi os.FileInfo, stat *fsutil.Stat) {
}

60
cache/contenthash/tarsum.go vendored Normal file
View File

@ -0,0 +1,60 @@
package contenthash
import (
"archive/tar"
"io"
"sort"
"strconv"
)
// WriteV1TarsumHeaders writes a tar header to a writer in V1 tarsum format.
func WriteV1TarsumHeaders(h *tar.Header, w io.Writer) {
for _, elem := range v1TarHeaderSelect(h) {
w.Write([]byte(elem[0] + elem[1]))
}
}
// Functions below are from docker legacy tarsum implementation.
// There is no valid technical reason to continue using them.
func v0TarHeaderSelect(h *tar.Header) (orderedHeaders [][2]string) {
return [][2]string{
{"name", h.Name},
{"mode", strconv.FormatInt(h.Mode, 10)},
{"uid", strconv.Itoa(h.Uid)},
{"gid", strconv.Itoa(h.Gid)},
{"size", strconv.FormatInt(h.Size, 10)},
{"mtime", strconv.FormatInt(h.ModTime.UTC().Unix(), 10)},
{"typeflag", string([]byte{h.Typeflag})},
{"linkname", h.Linkname},
{"uname", h.Uname},
{"gname", h.Gname},
{"devmajor", strconv.FormatInt(h.Devmajor, 10)},
{"devminor", strconv.FormatInt(h.Devminor, 10)},
}
}
func v1TarHeaderSelect(h *tar.Header) (orderedHeaders [][2]string) {
// Get extended attributes.
xAttrKeys := make([]string, len(h.Xattrs))
for k := range h.Xattrs {
xAttrKeys = append(xAttrKeys, k)
}
sort.Strings(xAttrKeys)
// Make the slice with enough capacity to hold the 11 basic headers
// we want from the v0 selector plus however many xattrs we have.
orderedHeaders = make([][2]string, 0, 11+len(xAttrKeys))
// Copy all headers from v0 excluding the 'mtime' header (the 5th element).
v0headers := v0TarHeaderSelect(h)
orderedHeaders = append(orderedHeaders, v0headers[0:5]...)
orderedHeaders = append(orderedHeaders, v0headers[6:]...)
// Finally, append the sorted xattrs.
for _, k := range xAttrKeys {
orderedHeaders = append(orderedHeaders, [2]string{k, h.Xattrs[k]})
}
return
}

View File

@ -25,6 +25,6 @@ func recvDiffCopy(ds grpc.Stream, dest string, cu CacheUpdater, progress progres
cu.MarkSupported(true)
cf = cu.HandleChange
}
return fsutil.Receive(ds.Context(), ds, dest, cf, progress)
_ = cf
return fsutil.Receive(ds.Context(), ds, dest, nil, nil, progress)
}

View File

@ -36,6 +36,8 @@ github.com/BurntSushi/locker 392720b78f44e9d0249fcac6c43b111b47a370b8
github.com/docker/docker 6301ac0c27aef52a96820482a01869457ae416f7 https://github.com/mcandre/moby.git
github.com/pkg/profile 5b67d428864e92711fcbd2f8629456121a56d91f
github.com/tonistiigi/fsutil 923ce3f08a66a6677878bbee066406d13eecd139
github.com/tonistiigi/fsutil 195d62bee906e45aa700b8ebeb3417f7b126bb23
github.com/stevvooe/continuity 86cec1535a968310e7532819f699ff2830ed7463
github.com/dmcgowan/go-tar 2e2c51242e8993c50445dab7c03c8e7febddd0cf
github.com/hashicorp/go-immutable-radix 8e8ed81f8f0bf1bdd829593fdd5c29922c1ea990
github.com/hashicorp/golang-lru a0d98a5f288019575c6d1f4bb1573fef2d1fcdc4

View File

@ -1 +0,0 @@
This code provides helper functions for dealing with archive files.

File diff suppressed because it is too large Load Diff

View File

@ -1,92 +0,0 @@
package archive
import (
"archive/tar"
"os"
"path/filepath"
"strings"
"syscall"
"github.com/docker/docker/pkg/system"
)
func getWhiteoutConverter(format WhiteoutFormat) tarWhiteoutConverter {
if format == OverlayWhiteoutFormat {
return overlayWhiteoutConverter{}
}
return nil
}
type overlayWhiteoutConverter struct{}
func (overlayWhiteoutConverter) ConvertWrite(hdr *tar.Header, path string, fi os.FileInfo) (wo *tar.Header, err error) {
// convert whiteouts to AUFS format
if fi.Mode()&os.ModeCharDevice != 0 && hdr.Devmajor == 0 && hdr.Devminor == 0 {
// we just rename the file and make it normal
dir, filename := filepath.Split(hdr.Name)
hdr.Name = filepath.Join(dir, WhiteoutPrefix+filename)
hdr.Mode = 0600
hdr.Typeflag = tar.TypeReg
hdr.Size = 0
}
if fi.Mode()&os.ModeDir != 0 {
// convert opaque dirs to AUFS format by writing an empty file with the prefix
opaque, err := system.Lgetxattr(path, "trusted.overlay.opaque")
if err != nil {
return nil, err
}
if len(opaque) == 1 && opaque[0] == 'y' {
if hdr.Xattrs != nil {
delete(hdr.Xattrs, "trusted.overlay.opaque")
}
// create a header for the whiteout file
// it should inherit some properties from the parent, but be a regular file
wo = &tar.Header{
Typeflag: tar.TypeReg,
Mode: hdr.Mode & int64(os.ModePerm),
Name: filepath.Join(hdr.Name, WhiteoutOpaqueDir),
Size: 0,
Uid: hdr.Uid,
Uname: hdr.Uname,
Gid: hdr.Gid,
Gname: hdr.Gname,
AccessTime: hdr.AccessTime,
ChangeTime: hdr.ChangeTime,
}
}
}
return
}
func (overlayWhiteoutConverter) ConvertRead(hdr *tar.Header, path string) (bool, error) {
base := filepath.Base(path)
dir := filepath.Dir(path)
// if a directory is marked as opaque by the AUFS special file, we need to translate that to overlay
if base == WhiteoutOpaqueDir {
err := syscall.Setxattr(dir, "trusted.overlay.opaque", []byte{'y'}, 0)
// don't write the file itself
return false, err
}
// if a file was deleted and we are using overlay, we need to create a character device
if strings.HasPrefix(base, WhiteoutPrefix) {
originalBase := base[len(WhiteoutPrefix):]
originalPath := filepath.Join(dir, originalBase)
if err := syscall.Mknod(originalPath, syscall.S_IFCHR, 0); err != nil {
return false, err
}
if err := os.Chown(originalPath, hdr.Uid, hdr.Gid); err != nil {
return false, err
}
// don't write the file itself
return false, nil
}
return true, nil
}

View File

@ -1,7 +0,0 @@
// +build !linux
package archive
func getWhiteoutConverter(format WhiteoutFormat) tarWhiteoutConverter {
return nil
}

View File

@ -1,121 +0,0 @@
// +build !windows
package archive
import (
"archive/tar"
"errors"
"os"
"path/filepath"
"syscall"
"github.com/docker/docker/pkg/idtools"
"github.com/docker/docker/pkg/system"
rsystem "github.com/opencontainers/runc/libcontainer/system"
)
// fixVolumePathPrefix does platform specific processing to ensure that if
// the path being passed in is not in a volume path format, convert it to one.
func fixVolumePathPrefix(srcPath string) string {
return srcPath
}
// getWalkRoot calculates the root path when performing a TarWithOptions.
// We use a separate function as this is platform specific. On Linux, we
// can't use filepath.Join(srcPath,include) because this will clean away
// a trailing "." or "/" which may be important.
func getWalkRoot(srcPath string, include string) string {
return srcPath + string(filepath.Separator) + include
}
// CanonicalTarNameForPath returns platform-specific filepath
// to canonical posix-style path for tar archival. p is relative
// path.
func CanonicalTarNameForPath(p string) (string, error) {
return p, nil // already unix-style
}
// chmodTarEntry is used to adjust the file permissions used in tar header based
// on the platform the archival is done.
func chmodTarEntry(perm os.FileMode) os.FileMode {
return perm // noop for unix as golang APIs provide perm bits correctly
}
func setHeaderForSpecialDevice(hdr *tar.Header, name string, stat interface{}) (err error) {
s, ok := stat.(*syscall.Stat_t)
if ok {
// Currently go does not fill in the major/minors
if s.Mode&syscall.S_IFBLK != 0 ||
s.Mode&syscall.S_IFCHR != 0 {
hdr.Devmajor = int64(major(uint64(s.Rdev)))
hdr.Devminor = int64(minor(uint64(s.Rdev)))
}
}
return
}
func getInodeFromStat(stat interface{}) (inode uint64, err error) {
s, ok := stat.(*syscall.Stat_t)
if ok {
inode = uint64(s.Ino)
}
return
}
func getFileUIDGID(stat interface{}) (idtools.IDPair, error) {
s, ok := stat.(*syscall.Stat_t)
if !ok {
return idtools.IDPair{}, errors.New("cannot convert stat value to syscall.Stat_t")
}
return idtools.IDPair{UID: int(s.Uid), GID: int(s.Gid)}, nil
}
func major(device uint64) uint64 {
return (device >> 8) & 0xfff
}
func minor(device uint64) uint64 {
return (device & 0xff) | ((device >> 12) & 0xfff00)
}
// handleTarTypeBlockCharFifo is an OS-specific helper function used by
// createTarFile to handle the following types of header: Block; Char; Fifo
func handleTarTypeBlockCharFifo(hdr *tar.Header, path string) error {
if rsystem.RunningInUserNS() {
// cannot create a device if running in user namespace
return nil
}
mode := uint32(hdr.Mode & 07777)
switch hdr.Typeflag {
case tar.TypeBlock:
mode |= syscall.S_IFBLK
case tar.TypeChar:
mode |= syscall.S_IFCHR
case tar.TypeFifo:
mode |= syscall.S_IFIFO
}
return system.Mknod(path, mode, int(system.Mkdev(hdr.Devmajor, hdr.Devminor)))
}
func handleLChmod(hdr *tar.Header, path string, hdrInfo os.FileInfo) error {
if hdr.Typeflag == tar.TypeLink {
if fi, err := os.Lstat(hdr.Linkname); err == nil && (fi.Mode()&os.ModeSymlink == 0) {
if err := os.Chmod(path, hdrInfo.Mode()); err != nil {
return err
}
}
} else if hdr.Typeflag != tar.TypeSymlink {
if err := os.Chmod(path, hdrInfo.Mode()); err != nil {
return err
}
}
return nil
}

View File

@ -1,79 +0,0 @@
// +build windows
package archive
import (
"archive/tar"
"fmt"
"os"
"path/filepath"
"strings"
"github.com/docker/docker/pkg/idtools"
"github.com/docker/docker/pkg/longpath"
)
// fixVolumePathPrefix does platform specific processing to ensure that if
// the path being passed in is not in a volume path format, convert it to one.
func fixVolumePathPrefix(srcPath string) string {
return longpath.AddPrefix(srcPath)
}
// getWalkRoot calculates the root path when performing a TarWithOptions.
// We use a separate function as this is platform specific.
func getWalkRoot(srcPath string, include string) string {
return filepath.Join(srcPath, include)
}
// CanonicalTarNameForPath returns platform-specific filepath
// to canonical posix-style path for tar archival. p is relative
// path.
func CanonicalTarNameForPath(p string) (string, error) {
// windows: convert windows style relative path with backslashes
// into forward slashes. Since windows does not allow '/' or '\'
// in file names, it is mostly safe to replace however we must
// check just in case
if strings.Contains(p, "/") {
return "", fmt.Errorf("Windows path contains forward slash: %s", p)
}
return strings.Replace(p, string(os.PathSeparator), "/", -1), nil
}
// chmodTarEntry is used to adjust the file permissions used in tar header based
// on the platform the archival is done.
func chmodTarEntry(perm os.FileMode) os.FileMode {
//perm &= 0755 // this 0-ed out tar flags (like link, regular file, directory marker etc.)
permPart := perm & os.ModePerm
noPermPart := perm &^ os.ModePerm
// Add the x bit: make everything +x from windows
permPart |= 0111
permPart &= 0755
return noPermPart | permPart
}
func setHeaderForSpecialDevice(hdr *tar.Header, name string, stat interface{}) (err error) {
// do nothing. no notion of Rdev, Nlink in stat on Windows
return
}
func getInodeFromStat(stat interface{}) (inode uint64, err error) {
// do nothing. no notion of Inode in stat on Windows
return
}
// handleTarTypeBlockCharFifo is an OS-specific helper function used by
// createTarFile to handle the following types of header: Block; Char; Fifo
func handleTarTypeBlockCharFifo(hdr *tar.Header, path string) error {
return nil
}
func handleLChmod(hdr *tar.Header, path string, hdrInfo os.FileInfo) error {
return nil
}
func getFileUIDGID(stat interface{}) (idtools.IDPair, error) {
// no notion of file ownership mapping yet on Windows
return idtools.IDPair{0, 0}, nil
}

View File

@ -1,441 +0,0 @@
package archive
import (
"archive/tar"
"bytes"
"fmt"
"io"
"io/ioutil"
"os"
"path/filepath"
"sort"
"strings"
"syscall"
"time"
"github.com/docker/docker/pkg/idtools"
"github.com/docker/docker/pkg/pools"
"github.com/docker/docker/pkg/system"
"github.com/sirupsen/logrus"
)
// ChangeType represents the change type.
type ChangeType int
const (
// ChangeModify represents the modify operation.
ChangeModify = iota
// ChangeAdd represents the add operation.
ChangeAdd
// ChangeDelete represents the delete operation.
ChangeDelete
)
func (c ChangeType) String() string {
switch c {
case ChangeModify:
return "C"
case ChangeAdd:
return "A"
case ChangeDelete:
return "D"
}
return ""
}
// Change represents a change, it wraps the change type and path.
// It describes changes of the files in the path respect to the
// parent layers. The change could be modify, add, delete.
// This is used for layer diff.
type Change struct {
Path string
Kind ChangeType
}
func (change *Change) String() string {
return fmt.Sprintf("%s %s", change.Kind, change.Path)
}
// for sort.Sort
type changesByPath []Change
func (c changesByPath) Less(i, j int) bool { return c[i].Path < c[j].Path }
func (c changesByPath) Len() int { return len(c) }
func (c changesByPath) Swap(i, j int) { c[j], c[i] = c[i], c[j] }
// Gnu tar and the go tar writer don't have sub-second mtime
// precision, which is problematic when we apply changes via tar
// files, we handle this by comparing for exact times, *or* same
// second count and either a or b having exactly 0 nanoseconds
func sameFsTime(a, b time.Time) bool {
return a == b ||
(a.Unix() == b.Unix() &&
(a.Nanosecond() == 0 || b.Nanosecond() == 0))
}
func sameFsTimeSpec(a, b syscall.Timespec) bool {
return a.Sec == b.Sec &&
(a.Nsec == b.Nsec || a.Nsec == 0 || b.Nsec == 0)
}
// Changes walks the path rw and determines changes for the files in the path,
// with respect to the parent layers
func Changes(layers []string, rw string) ([]Change, error) {
return changes(layers, rw, aufsDeletedFile, aufsMetadataSkip)
}
func aufsMetadataSkip(path string) (skip bool, err error) {
skip, err = filepath.Match(string(os.PathSeparator)+WhiteoutMetaPrefix+"*", path)
if err != nil {
skip = true
}
return
}
func aufsDeletedFile(root, path string, fi os.FileInfo) (string, error) {
f := filepath.Base(path)
// If there is a whiteout, then the file was removed
if strings.HasPrefix(f, WhiteoutPrefix) {
originalFile := f[len(WhiteoutPrefix):]
return filepath.Join(filepath.Dir(path), originalFile), nil
}
return "", nil
}
type skipChange func(string) (bool, error)
type deleteChange func(string, string, os.FileInfo) (string, error)
func changes(layers []string, rw string, dc deleteChange, sc skipChange) ([]Change, error) {
var (
changes []Change
changedDirs = make(map[string]struct{})
)
err := filepath.Walk(rw, func(path string, f os.FileInfo, err error) error {
if err != nil {
return err
}
// Rebase path
path, err = filepath.Rel(rw, path)
if err != nil {
return err
}
// As this runs on the daemon side, file paths are OS specific.
path = filepath.Join(string(os.PathSeparator), path)
// Skip root
if path == string(os.PathSeparator) {
return nil
}
if sc != nil {
if skip, err := sc(path); skip {
return err
}
}
change := Change{
Path: path,
}
deletedFile, err := dc(rw, path, f)
if err != nil {
return err
}
// Find out what kind of modification happened
if deletedFile != "" {
change.Path = deletedFile
change.Kind = ChangeDelete
} else {
// Otherwise, the file was added
change.Kind = ChangeAdd
// ...Unless it already existed in a top layer, in which case, it's a modification
for _, layer := range layers {
stat, err := os.Stat(filepath.Join(layer, path))
if err != nil && !os.IsNotExist(err) {
return err
}
if err == nil {
// The file existed in the top layer, so that's a modification
// However, if it's a directory, maybe it wasn't actually modified.
// If you modify /foo/bar/baz, then /foo will be part of the changed files only because it's the parent of bar
if stat.IsDir() && f.IsDir() {
if f.Size() == stat.Size() && f.Mode() == stat.Mode() && sameFsTime(f.ModTime(), stat.ModTime()) {
// Both directories are the same, don't record the change
return nil
}
}
change.Kind = ChangeModify
break
}
}
}
// If /foo/bar/file.txt is modified, then /foo/bar must be part of the changed files.
// This block is here to ensure the change is recorded even if the
// modify time, mode and size of the parent directory in the rw and ro layers are all equal.
// Check https://github.com/docker/docker/pull/13590 for details.
if f.IsDir() {
changedDirs[path] = struct{}{}
}
if change.Kind == ChangeAdd || change.Kind == ChangeDelete {
parent := filepath.Dir(path)
if _, ok := changedDirs[parent]; !ok && parent != "/" {
changes = append(changes, Change{Path: parent, Kind: ChangeModify})
changedDirs[parent] = struct{}{}
}
}
// Record change
changes = append(changes, change)
return nil
})
if err != nil && !os.IsNotExist(err) {
return nil, err
}
return changes, nil
}
// FileInfo describes the information of a file.
type FileInfo struct {
parent *FileInfo
name string
stat *system.StatT
children map[string]*FileInfo
capability []byte
added bool
}
// LookUp looks up the file information of a file.
func (info *FileInfo) LookUp(path string) *FileInfo {
// As this runs on the daemon side, file paths are OS specific.
parent := info
if path == string(os.PathSeparator) {
return info
}
pathElements := strings.Split(path, string(os.PathSeparator))
for _, elem := range pathElements {
if elem != "" {
child := parent.children[elem]
if child == nil {
return nil
}
parent = child
}
}
return parent
}
func (info *FileInfo) path() string {
if info.parent == nil {
// As this runs on the daemon side, file paths are OS specific.
return string(os.PathSeparator)
}
return filepath.Join(info.parent.path(), info.name)
}
func (info *FileInfo) addChanges(oldInfo *FileInfo, changes *[]Change) {
sizeAtEntry := len(*changes)
if oldInfo == nil {
// add
change := Change{
Path: info.path(),
Kind: ChangeAdd,
}
*changes = append(*changes, change)
info.added = true
}
// We make a copy so we can modify it to detect additions
// also, we only recurse on the old dir if the new info is a directory
// otherwise any previous delete/change is considered recursive
oldChildren := make(map[string]*FileInfo)
if oldInfo != nil && info.isDir() {
for k, v := range oldInfo.children {
oldChildren[k] = v
}
}
for name, newChild := range info.children {
oldChild := oldChildren[name]
if oldChild != nil {
// change?
oldStat := oldChild.stat
newStat := newChild.stat
// Note: We can't compare inode or ctime or blocksize here, because these change
// when copying a file into a container. However, that is not generally a problem
// because any content change will change mtime, and any status change should
// be visible when actually comparing the stat fields. The only time this
// breaks down is if some code intentionally hides a change by setting
// back mtime
if statDifferent(oldStat, newStat) ||
!bytes.Equal(oldChild.capability, newChild.capability) {
change := Change{
Path: newChild.path(),
Kind: ChangeModify,
}
*changes = append(*changes, change)
newChild.added = true
}
// Remove from copy so we can detect deletions
delete(oldChildren, name)
}
newChild.addChanges(oldChild, changes)
}
for _, oldChild := range oldChildren {
// delete
change := Change{
Path: oldChild.path(),
Kind: ChangeDelete,
}
*changes = append(*changes, change)
}
// If there were changes inside this directory, we need to add it, even if the directory
// itself wasn't changed. This is needed to properly save and restore filesystem permissions.
// As this runs on the daemon side, file paths are OS specific.
if len(*changes) > sizeAtEntry && info.isDir() && !info.added && info.path() != string(os.PathSeparator) {
change := Change{
Path: info.path(),
Kind: ChangeModify,
}
// Let's insert the directory entry before the recently added entries located inside this dir
*changes = append(*changes, change) // just to resize the slice, will be overwritten
copy((*changes)[sizeAtEntry+1:], (*changes)[sizeAtEntry:])
(*changes)[sizeAtEntry] = change
}
}
// Changes add changes to file information.
func (info *FileInfo) Changes(oldInfo *FileInfo) []Change {
var changes []Change
info.addChanges(oldInfo, &changes)
return changes
}
func newRootFileInfo() *FileInfo {
// As this runs on the daemon side, file paths are OS specific.
root := &FileInfo{
name: string(os.PathSeparator),
children: make(map[string]*FileInfo),
}
return root
}
// ChangesDirs compares two directories and generates an array of Change objects describing the changes.
// If oldDir is "", then all files in newDir will be Add-Changes.
func ChangesDirs(newDir, oldDir string) ([]Change, error) {
var (
oldRoot, newRoot *FileInfo
)
if oldDir == "" {
emptyDir, err := ioutil.TempDir("", "empty")
if err != nil {
return nil, err
}
defer os.Remove(emptyDir)
oldDir = emptyDir
}
oldRoot, newRoot, err := collectFileInfoForChanges(oldDir, newDir)
if err != nil {
return nil, err
}
return newRoot.Changes(oldRoot), nil
}
// ChangesSize calculates the size in bytes of the provided changes, based on newDir.
func ChangesSize(newDir string, changes []Change) int64 {
var (
size int64
sf = make(map[uint64]struct{})
)
for _, change := range changes {
if change.Kind == ChangeModify || change.Kind == ChangeAdd {
file := filepath.Join(newDir, change.Path)
fileInfo, err := os.Lstat(file)
if err != nil {
logrus.Errorf("Can not stat %q: %s", file, err)
continue
}
if fileInfo != nil && !fileInfo.IsDir() {
if hasHardlinks(fileInfo) {
inode := getIno(fileInfo)
if _, ok := sf[inode]; !ok {
size += fileInfo.Size()
sf[inode] = struct{}{}
}
} else {
size += fileInfo.Size()
}
}
}
}
return size
}
// ExportChanges produces an Archive from the provided changes, relative to dir.
func ExportChanges(dir string, changes []Change, uidMaps, gidMaps []idtools.IDMap) (io.ReadCloser, error) {
reader, writer := io.Pipe()
go func() {
ta := newTarAppender(idtools.NewIDMappingsFromMaps(uidMaps, gidMaps), writer)
// this buffer is needed for the duration of this piped stream
defer pools.BufioWriter32KPool.Put(ta.Buffer)
sort.Sort(changesByPath(changes))
// In general we log errors here but ignore them because
// during e.g. a diff operation the container can continue
// mutating the filesystem and we can see transient errors
// from this
for _, change := range changes {
if change.Kind == ChangeDelete {
whiteOutDir := filepath.Dir(change.Path)
whiteOutBase := filepath.Base(change.Path)
whiteOut := filepath.Join(whiteOutDir, WhiteoutPrefix+whiteOutBase)
timestamp := time.Now()
hdr := &tar.Header{
Name: whiteOut[1:],
Size: 0,
ModTime: timestamp,
AccessTime: timestamp,
ChangeTime: timestamp,
}
if err := ta.TarWriter.WriteHeader(hdr); err != nil {
logrus.Debugf("Can't write whiteout header: %s", err)
}
} else {
path := filepath.Join(dir, change.Path)
if err := ta.addTarFile(path, change.Path[1:]); err != nil {
logrus.Debugf("Can't add file %s to tar: %s", path, err)
}
}
}
// Make sure to check the error on Close.
if err := ta.TarWriter.Close(); err != nil {
logrus.Debugf("Can't close layer: %s", err)
}
if err := writer.Close(); err != nil {
logrus.Debugf("failed close Changes writer: %s", err)
}
}()
return reader, nil
}

View File

@ -1,312 +0,0 @@
package archive
import (
"bytes"
"fmt"
"os"
"path/filepath"
"sort"
"syscall"
"unsafe"
"github.com/docker/docker/pkg/system"
)
// walker is used to implement collectFileInfoForChanges on linux. Where this
// method in general returns the entire contents of two directory trees, we
// optimize some FS calls out on linux. In particular, we take advantage of the
// fact that getdents(2) returns the inode of each file in the directory being
// walked, which, when walking two trees in parallel to generate a list of
// changes, can be used to prune subtrees without ever having to lstat(2) them
// directly. Eliminating stat calls in this way can save up to seconds on large
// images.
type walker struct {
dir1 string
dir2 string
root1 *FileInfo
root2 *FileInfo
}
// collectFileInfoForChanges returns a complete representation of the trees
// rooted at dir1 and dir2, with one important exception: any subtree or
// leaf where the inode and device numbers are an exact match between dir1
// and dir2 will be pruned from the results. This method is *only* to be used
// to generating a list of changes between the two directories, as it does not
// reflect the full contents.
func collectFileInfoForChanges(dir1, dir2 string) (*FileInfo, *FileInfo, error) {
w := &walker{
dir1: dir1,
dir2: dir2,
root1: newRootFileInfo(),
root2: newRootFileInfo(),
}
i1, err := os.Lstat(w.dir1)
if err != nil {
return nil, nil, err
}
i2, err := os.Lstat(w.dir2)
if err != nil {
return nil, nil, err
}
if err := w.walk("/", i1, i2); err != nil {
return nil, nil, err
}
return w.root1, w.root2, nil
}
// Given a FileInfo, its path info, and a reference to the root of the tree
// being constructed, register this file with the tree.
func walkchunk(path string, fi os.FileInfo, dir string, root *FileInfo) error {
if fi == nil {
return nil
}
parent := root.LookUp(filepath.Dir(path))
if parent == nil {
return fmt.Errorf("walkchunk: Unexpectedly no parent for %s", path)
}
info := &FileInfo{
name: filepath.Base(path),
children: make(map[string]*FileInfo),
parent: parent,
}
cpath := filepath.Join(dir, path)
stat, err := system.FromStatT(fi.Sys().(*syscall.Stat_t))
if err != nil {
return err
}
info.stat = stat
info.capability, _ = system.Lgetxattr(cpath, "security.capability") // lgetxattr(2): fs access
parent.children[info.name] = info
return nil
}
// Walk a subtree rooted at the same path in both trees being iterated. For
// example, /docker/overlay/1234/a/b/c/d and /docker/overlay/8888/a/b/c/d
func (w *walker) walk(path string, i1, i2 os.FileInfo) (err error) {
// Register these nodes with the return trees, unless we're still at the
// (already-created) roots:
if path != "/" {
if err := walkchunk(path, i1, w.dir1, w.root1); err != nil {
return err
}
if err := walkchunk(path, i2, w.dir2, w.root2); err != nil {
return err
}
}
is1Dir := i1 != nil && i1.IsDir()
is2Dir := i2 != nil && i2.IsDir()
sameDevice := false
if i1 != nil && i2 != nil {
si1 := i1.Sys().(*syscall.Stat_t)
si2 := i2.Sys().(*syscall.Stat_t)
if si1.Dev == si2.Dev {
sameDevice = true
}
}
// If these files are both non-existent, or leaves (non-dirs), we are done.
if !is1Dir && !is2Dir {
return nil
}
// Fetch the names of all the files contained in both directories being walked:
var names1, names2 []nameIno
if is1Dir {
names1, err = readdirnames(filepath.Join(w.dir1, path)) // getdents(2): fs access
if err != nil {
return err
}
}
if is2Dir {
names2, err = readdirnames(filepath.Join(w.dir2, path)) // getdents(2): fs access
if err != nil {
return err
}
}
// We have lists of the files contained in both parallel directories, sorted
// in the same order. Walk them in parallel, generating a unique merged list
// of all items present in either or both directories.
var names []string
ix1 := 0
ix2 := 0
for {
if ix1 >= len(names1) {
break
}
if ix2 >= len(names2) {
break
}
ni1 := names1[ix1]
ni2 := names2[ix2]
switch bytes.Compare([]byte(ni1.name), []byte(ni2.name)) {
case -1: // ni1 < ni2 -- advance ni1
// we will not encounter ni1 in names2
names = append(names, ni1.name)
ix1++
case 0: // ni1 == ni2
if ni1.ino != ni2.ino || !sameDevice {
names = append(names, ni1.name)
}
ix1++
ix2++
case 1: // ni1 > ni2 -- advance ni2
// we will not encounter ni2 in names1
names = append(names, ni2.name)
ix2++
}
}
for ix1 < len(names1) {
names = append(names, names1[ix1].name)
ix1++
}
for ix2 < len(names2) {
names = append(names, names2[ix2].name)
ix2++
}
// For each of the names present in either or both of the directories being
// iterated, stat the name under each root, and recurse the pair of them:
for _, name := range names {
fname := filepath.Join(path, name)
var cInfo1, cInfo2 os.FileInfo
if is1Dir {
cInfo1, err = os.Lstat(filepath.Join(w.dir1, fname)) // lstat(2): fs access
if err != nil && !os.IsNotExist(err) {
return err
}
}
if is2Dir {
cInfo2, err = os.Lstat(filepath.Join(w.dir2, fname)) // lstat(2): fs access
if err != nil && !os.IsNotExist(err) {
return err
}
}
if err = w.walk(fname, cInfo1, cInfo2); err != nil {
return err
}
}
return nil
}
// {name,inode} pairs used to support the early-pruning logic of the walker type
type nameIno struct {
name string
ino uint64
}
type nameInoSlice []nameIno
func (s nameInoSlice) Len() int { return len(s) }
func (s nameInoSlice) Swap(i, j int) { s[i], s[j] = s[j], s[i] }
func (s nameInoSlice) Less(i, j int) bool { return s[i].name < s[j].name }
// readdirnames is a hacked-apart version of the Go stdlib code, exposing inode
// numbers further up the stack when reading directory contents. Unlike
// os.Readdirnames, which returns a list of filenames, this function returns a
// list of {filename,inode} pairs.
func readdirnames(dirname string) (names []nameIno, err error) {
var (
size = 100
buf = make([]byte, 4096)
nbuf int
bufp int
nb int
)
f, err := os.Open(dirname)
if err != nil {
return nil, err
}
defer f.Close()
names = make([]nameIno, 0, size) // Empty with room to grow.
for {
// Refill the buffer if necessary
if bufp >= nbuf {
bufp = 0
nbuf, err = syscall.ReadDirent(int(f.Fd()), buf) // getdents on linux
if nbuf < 0 {
nbuf = 0
}
if err != nil {
return nil, os.NewSyscallError("readdirent", err)
}
if nbuf <= 0 {
break // EOF
}
}
// Drain the buffer
nb, names = parseDirent(buf[bufp:nbuf], names)
bufp += nb
}
sl := nameInoSlice(names)
sort.Sort(sl)
return sl, nil
}
// parseDirent is a minor modification of syscall.ParseDirent (linux version)
// which returns {name,inode} pairs instead of just names.
func parseDirent(buf []byte, names []nameIno) (consumed int, newnames []nameIno) {
origlen := len(buf)
for len(buf) > 0 {
dirent := (*syscall.Dirent)(unsafe.Pointer(&buf[0]))
buf = buf[dirent.Reclen:]
if dirent.Ino == 0 { // File absent in directory.
continue
}
bytes := (*[10000]byte)(unsafe.Pointer(&dirent.Name[0]))
var name = string(bytes[0:clen(bytes[:])])
if name == "." || name == ".." { // Useless names
continue
}
names = append(names, nameIno{name, dirent.Ino})
}
return origlen - len(buf), names
}
func clen(n []byte) int {
for i := 0; i < len(n); i++ {
if n[i] == 0 {
return i
}
}
return len(n)
}
// OverlayChanges walks the path rw and determines changes for the files in the path,
// with respect to the parent layers
func OverlayChanges(layers []string, rw string) ([]Change, error) {
return changes(layers, rw, overlayDeletedFile, nil)
}
func overlayDeletedFile(root, path string, fi os.FileInfo) (string, error) {
if fi.Mode()&os.ModeCharDevice != 0 {
s := fi.Sys().(*syscall.Stat_t)
if major(uint64(s.Rdev)) == 0 && minor(uint64(s.Rdev)) == 0 {
return path, nil
}
}
if fi.Mode()&os.ModeDir != 0 {
opaque, err := system.Lgetxattr(filepath.Join(root, path), "trusted.overlay.opaque")
if err != nil {
return "", err
}
if len(opaque) == 1 && opaque[0] == 'y' {
return path, nil
}
}
return "", nil
}

View File

@ -1,97 +0,0 @@
// +build !linux
package archive
import (
"fmt"
"os"
"path/filepath"
"runtime"
"strings"
"github.com/docker/docker/pkg/system"
)
func collectFileInfoForChanges(oldDir, newDir string) (*FileInfo, *FileInfo, error) {
var (
oldRoot, newRoot *FileInfo
err1, err2 error
errs = make(chan error, 2)
)
go func() {
oldRoot, err1 = collectFileInfo(oldDir)
errs <- err1
}()
go func() {
newRoot, err2 = collectFileInfo(newDir)
errs <- err2
}()
// block until both routines have returned
for i := 0; i < 2; i++ {
if err := <-errs; err != nil {
return nil, nil, err
}
}
return oldRoot, newRoot, nil
}
func collectFileInfo(sourceDir string) (*FileInfo, error) {
root := newRootFileInfo()
err := filepath.Walk(sourceDir, func(path string, f os.FileInfo, err error) error {
if err != nil {
return err
}
// Rebase path
relPath, err := filepath.Rel(sourceDir, path)
if err != nil {
return err
}
// As this runs on the daemon side, file paths are OS specific.
relPath = filepath.Join(string(os.PathSeparator), relPath)
// See https://github.com/golang/go/issues/9168 - bug in filepath.Join.
// Temporary workaround. If the returned path starts with two backslashes,
// trim it down to a single backslash. Only relevant on Windows.
if runtime.GOOS == "windows" {
if strings.HasPrefix(relPath, `\\`) {
relPath = relPath[1:]
}
}
if relPath == string(os.PathSeparator) {
return nil
}
parent := root.LookUp(filepath.Dir(relPath))
if parent == nil {
return fmt.Errorf("collectFileInfo: Unexpectedly no parent for %s", relPath)
}
info := &FileInfo{
name: filepath.Base(relPath),
children: make(map[string]*FileInfo),
parent: parent,
}
s, err := system.Lstat(path)
if err != nil {
return err
}
info.stat = s
info.capability, _ = system.Lgetxattr(path, "security.capability")
parent.children[info.name] = info
return nil
})
if err != nil {
return nil, err
}
return root, nil
}

View File

@ -1,36 +0,0 @@
// +build !windows
package archive
import (
"os"
"syscall"
"github.com/docker/docker/pkg/system"
)
func statDifferent(oldStat *system.StatT, newStat *system.StatT) bool {
// Don't look at size for dirs, its not a good measure of change
if oldStat.Mode() != newStat.Mode() ||
oldStat.UID() != newStat.UID() ||
oldStat.GID() != newStat.GID() ||
oldStat.Rdev() != newStat.Rdev() ||
// Don't look at size for dirs, its not a good measure of change
(oldStat.Mode()&syscall.S_IFDIR != syscall.S_IFDIR &&
(!sameFsTimeSpec(oldStat.Mtim(), newStat.Mtim()) || (oldStat.Size() != newStat.Size()))) {
return true
}
return false
}
func (info *FileInfo) isDir() bool {
return info.parent == nil || info.stat.Mode()&syscall.S_IFDIR != 0
}
func getIno(fi os.FileInfo) uint64 {
return uint64(fi.Sys().(*syscall.Stat_t).Ino)
}
func hasHardlinks(fi os.FileInfo) bool {
return fi.Sys().(*syscall.Stat_t).Nlink > 1
}

View File

@ -1,30 +0,0 @@
package archive
import (
"os"
"github.com/docker/docker/pkg/system"
)
func statDifferent(oldStat *system.StatT, newStat *system.StatT) bool {
// Don't look at size for dirs, its not a good measure of change
if oldStat.Mtim() != newStat.Mtim() ||
oldStat.Mode() != newStat.Mode() ||
oldStat.Size() != newStat.Size() && !oldStat.Mode().IsDir() {
return true
}
return false
}
func (info *FileInfo) isDir() bool {
return info.parent == nil || info.stat.Mode().IsDir()
}
func getIno(fi os.FileInfo) (inode uint64) {
return
}
func hasHardlinks(fi os.FileInfo) bool {
return false
}

View File

@ -1,458 +0,0 @@
package archive
import (
"archive/tar"
"errors"
"io"
"io/ioutil"
"os"
"path/filepath"
"strings"
"github.com/docker/docker/pkg/system"
"github.com/sirupsen/logrus"
)
// Errors used or returned by this file.
var (
ErrNotDirectory = errors.New("not a directory")
ErrDirNotExists = errors.New("no such directory")
ErrCannotCopyDir = errors.New("cannot copy directory")
ErrInvalidCopySource = errors.New("invalid copy source content")
)
// PreserveTrailingDotOrSeparator returns the given cleaned path (after
// processing using any utility functions from the path or filepath stdlib
// packages) and appends a trailing `/.` or `/` if its corresponding original
// path (from before being processed by utility functions from the path or
// filepath stdlib packages) ends with a trailing `/.` or `/`. If the cleaned
// path already ends in a `.` path segment, then another is not added. If the
// clean path already ends in a path separator, then another is not added.
func PreserveTrailingDotOrSeparator(cleanedPath, originalPath string) string {
// Ensure paths are in platform semantics
cleanedPath = normalizePath(cleanedPath)
originalPath = normalizePath(originalPath)
if !specifiesCurrentDir(cleanedPath) && specifiesCurrentDir(originalPath) {
if !hasTrailingPathSeparator(cleanedPath) {
// Add a separator if it doesn't already end with one (a cleaned
// path would only end in a separator if it is the root).
cleanedPath += string(filepath.Separator)
}
cleanedPath += "."
}
if !hasTrailingPathSeparator(cleanedPath) && hasTrailingPathSeparator(originalPath) {
cleanedPath += string(filepath.Separator)
}
return cleanedPath
}
// assertsDirectory returns whether the given path is
// asserted to be a directory, i.e., the path ends with
// a trailing '/' or `/.`, assuming a path separator of `/`.
func assertsDirectory(path string) bool {
return hasTrailingPathSeparator(path) || specifiesCurrentDir(path)
}
// hasTrailingPathSeparator returns whether the given
// path ends with the system's path separator character.
func hasTrailingPathSeparator(path string) bool {
return len(path) > 0 && os.IsPathSeparator(path[len(path)-1])
}
// specifiesCurrentDir returns whether the given path specifies
// a "current directory", i.e., the last path segment is `.`.
func specifiesCurrentDir(path string) bool {
return filepath.Base(path) == "."
}
// SplitPathDirEntry splits the given path between its directory name and its
// basename by first cleaning the path but preserves a trailing "." if the
// original path specified the current directory.
func SplitPathDirEntry(path string) (dir, base string) {
cleanedPath := filepath.Clean(normalizePath(path))
if specifiesCurrentDir(path) {
cleanedPath += string(filepath.Separator) + "."
}
return filepath.Dir(cleanedPath), filepath.Base(cleanedPath)
}
// TarResource archives the resource described by the given CopyInfo to a Tar
// archive. A non-nil error is returned if sourcePath does not exist or is
// asserted to be a directory but exists as another type of file.
//
// This function acts as a convenient wrapper around TarWithOptions, which
// requires a directory as the source path. TarResource accepts either a
// directory or a file path and correctly sets the Tar options.
func TarResource(sourceInfo CopyInfo) (content io.ReadCloser, err error) {
return TarResourceRebase(sourceInfo.Path, sourceInfo.RebaseName)
}
// TarResourceRebase is like TarResource but renames the first path element of
// items in the resulting tar archive to match the given rebaseName if not "".
func TarResourceRebase(sourcePath, rebaseName string) (content io.ReadCloser, err error) {
sourcePath = normalizePath(sourcePath)
if _, err = os.Lstat(sourcePath); err != nil {
// Catches the case where the source does not exist or is not a
// directory if asserted to be a directory, as this also causes an
// error.
return
}
// Separate the source path between its directory and
// the entry in that directory which we are archiving.
sourceDir, sourceBase := SplitPathDirEntry(sourcePath)
filter := []string{sourceBase}
logrus.Debugf("copying %q from %q", sourceBase, sourceDir)
return TarWithOptions(sourceDir, &TarOptions{
Compression: Uncompressed,
IncludeFiles: filter,
IncludeSourceDir: true,
RebaseNames: map[string]string{
sourceBase: rebaseName,
},
})
}
// CopyInfo holds basic info about the source
// or destination path of a copy operation.
type CopyInfo struct {
Path string
Exists bool
IsDir bool
RebaseName string
}
// CopyInfoSourcePath stats the given path to create a CopyInfo
// struct representing that resource for the source of an archive copy
// operation. The given path should be an absolute local path. A source path
// has all symlinks evaluated that appear before the last path separator ("/"
// on Unix). As it is to be a copy source, the path must exist.
func CopyInfoSourcePath(path string, followLink bool) (CopyInfo, error) {
// normalize the file path and then evaluate the symbol link
// we will use the target file instead of the symbol link if
// followLink is set
path = normalizePath(path)
resolvedPath, rebaseName, err := ResolveHostSourcePath(path, followLink)
if err != nil {
return CopyInfo{}, err
}
stat, err := os.Lstat(resolvedPath)
if err != nil {
return CopyInfo{}, err
}
return CopyInfo{
Path: resolvedPath,
Exists: true,
IsDir: stat.IsDir(),
RebaseName: rebaseName,
}, nil
}
// CopyInfoDestinationPath stats the given path to create a CopyInfo
// struct representing that resource for the destination of an archive copy
// operation. The given path should be an absolute local path.
func CopyInfoDestinationPath(path string) (info CopyInfo, err error) {
maxSymlinkIter := 10 // filepath.EvalSymlinks uses 255, but 10 already seems like a lot.
path = normalizePath(path)
originalPath := path
stat, err := os.Lstat(path)
if err == nil && stat.Mode()&os.ModeSymlink == 0 {
// The path exists and is not a symlink.
return CopyInfo{
Path: path,
Exists: true,
IsDir: stat.IsDir(),
}, nil
}
// While the path is a symlink.
for n := 0; err == nil && stat.Mode()&os.ModeSymlink != 0; n++ {
if n > maxSymlinkIter {
// Don't follow symlinks more than this arbitrary number of times.
return CopyInfo{}, errors.New("too many symlinks in " + originalPath)
}
// The path is a symbolic link. We need to evaluate it so that the
// destination of the copy operation is the link target and not the
// link itself. This is notably different than CopyInfoSourcePath which
// only evaluates symlinks before the last appearing path separator.
// Also note that it is okay if the last path element is a broken
// symlink as the copy operation should create the target.
var linkTarget string
linkTarget, err = os.Readlink(path)
if err != nil {
return CopyInfo{}, err
}
if !system.IsAbs(linkTarget) {
// Join with the parent directory.
dstParent, _ := SplitPathDirEntry(path)
linkTarget = filepath.Join(dstParent, linkTarget)
}
path = linkTarget
stat, err = os.Lstat(path)
}
if err != nil {
// It's okay if the destination path doesn't exist. We can still
// continue the copy operation if the parent directory exists.
if !os.IsNotExist(err) {
return CopyInfo{}, err
}
// Ensure destination parent dir exists.
dstParent, _ := SplitPathDirEntry(path)
parentDirStat, err := os.Lstat(dstParent)
if err != nil {
return CopyInfo{}, err
}
if !parentDirStat.IsDir() {
return CopyInfo{}, ErrNotDirectory
}
return CopyInfo{Path: path}, nil
}
// The path exists after resolving symlinks.
return CopyInfo{
Path: path,
Exists: true,
IsDir: stat.IsDir(),
}, nil
}
// PrepareArchiveCopy prepares the given srcContent archive, which should
// contain the archived resource described by srcInfo, to the destination
// described by dstInfo. Returns the possibly modified content archive along
// with the path to the destination directory which it should be extracted to.
func PrepareArchiveCopy(srcContent io.Reader, srcInfo, dstInfo CopyInfo) (dstDir string, content io.ReadCloser, err error) {
// Ensure in platform semantics
srcInfo.Path = normalizePath(srcInfo.Path)
dstInfo.Path = normalizePath(dstInfo.Path)
// Separate the destination path between its directory and base
// components in case the source archive contents need to be rebased.
dstDir, dstBase := SplitPathDirEntry(dstInfo.Path)
_, srcBase := SplitPathDirEntry(srcInfo.Path)
switch {
case dstInfo.Exists && dstInfo.IsDir:
// The destination exists as a directory. No alteration
// to srcContent is needed as its contents can be
// simply extracted to the destination directory.
return dstInfo.Path, ioutil.NopCloser(srcContent), nil
case dstInfo.Exists && srcInfo.IsDir:
// The destination exists as some type of file and the source
// content is a directory. This is an error condition since
// you cannot copy a directory to an existing file location.
return "", nil, ErrCannotCopyDir
case dstInfo.Exists:
// The destination exists as some type of file and the source content
// is also a file. The source content entry will have to be renamed to
// have a basename which matches the destination path's basename.
if len(srcInfo.RebaseName) != 0 {
srcBase = srcInfo.RebaseName
}
return dstDir, RebaseArchiveEntries(srcContent, srcBase, dstBase), nil
case srcInfo.IsDir:
// The destination does not exist and the source content is an archive
// of a directory. The archive should be extracted to the parent of
// the destination path instead, and when it is, the directory that is
// created as a result should take the name of the destination path.
// The source content entries will have to be renamed to have a
// basename which matches the destination path's basename.
if len(srcInfo.RebaseName) != 0 {
srcBase = srcInfo.RebaseName
}
return dstDir, RebaseArchiveEntries(srcContent, srcBase, dstBase), nil
case assertsDirectory(dstInfo.Path):
// The destination does not exist and is asserted to be created as a
// directory, but the source content is not a directory. This is an
// error condition since you cannot create a directory from a file
// source.
return "", nil, ErrDirNotExists
default:
// The last remaining case is when the destination does not exist, is
// not asserted to be a directory, and the source content is not an
// archive of a directory. It this case, the destination file will need
// to be created when the archive is extracted and the source content
// entry will have to be renamed to have a basename which matches the
// destination path's basename.
if len(srcInfo.RebaseName) != 0 {
srcBase = srcInfo.RebaseName
}
return dstDir, RebaseArchiveEntries(srcContent, srcBase, dstBase), nil
}
}
// RebaseArchiveEntries rewrites the given srcContent archive replacing
// an occurrence of oldBase with newBase at the beginning of entry names.
func RebaseArchiveEntries(srcContent io.Reader, oldBase, newBase string) io.ReadCloser {
if oldBase == string(os.PathSeparator) {
// If oldBase specifies the root directory, use an empty string as
// oldBase instead so that newBase doesn't replace the path separator
// that all paths will start with.
oldBase = ""
}
rebased, w := io.Pipe()
go func() {
srcTar := tar.NewReader(srcContent)
rebasedTar := tar.NewWriter(w)
for {
hdr, err := srcTar.Next()
if err == io.EOF {
// Signals end of archive.
rebasedTar.Close()
w.Close()
return
}
if err != nil {
w.CloseWithError(err)
return
}
hdr.Name = strings.Replace(hdr.Name, oldBase, newBase, 1)
if err = rebasedTar.WriteHeader(hdr); err != nil {
w.CloseWithError(err)
return
}
if _, err = io.Copy(rebasedTar, srcTar); err != nil {
w.CloseWithError(err)
return
}
}
}()
return rebased
}
// CopyResource performs an archive copy from the given source path to the
// given destination path. The source path MUST exist and the destination
// path's parent directory must exist.
func CopyResource(srcPath, dstPath string, followLink bool) error {
var (
srcInfo CopyInfo
err error
)
// Ensure in platform semantics
srcPath = normalizePath(srcPath)
dstPath = normalizePath(dstPath)
// Clean the source and destination paths.
srcPath = PreserveTrailingDotOrSeparator(filepath.Clean(srcPath), srcPath)
dstPath = PreserveTrailingDotOrSeparator(filepath.Clean(dstPath), dstPath)
if srcInfo, err = CopyInfoSourcePath(srcPath, followLink); err != nil {
return err
}
content, err := TarResource(srcInfo)
if err != nil {
return err
}
defer content.Close()
return CopyTo(content, srcInfo, dstPath)
}
// CopyTo handles extracting the given content whose
// entries should be sourced from srcInfo to dstPath.
func CopyTo(content io.Reader, srcInfo CopyInfo, dstPath string) error {
// The destination path need not exist, but CopyInfoDestinationPath will
// ensure that at least the parent directory exists.
dstInfo, err := CopyInfoDestinationPath(normalizePath(dstPath))
if err != nil {
return err
}
dstDir, copyArchive, err := PrepareArchiveCopy(content, srcInfo, dstInfo)
if err != nil {
return err
}
defer copyArchive.Close()
options := &TarOptions{
NoLchown: true,
NoOverwriteDirNonDir: true,
}
return Untar(copyArchive, dstDir, options)
}
// ResolveHostSourcePath decides real path need to be copied with parameters such as
// whether to follow symbol link or not, if followLink is true, resolvedPath will return
// link target of any symbol link file, else it will only resolve symlink of directory
// but return symbol link file itself without resolving.
func ResolveHostSourcePath(path string, followLink bool) (resolvedPath, rebaseName string, err error) {
if followLink {
resolvedPath, err = filepath.EvalSymlinks(path)
if err != nil {
return
}
resolvedPath, rebaseName = GetRebaseName(path, resolvedPath)
} else {
dirPath, basePath := filepath.Split(path)
// if not follow symbol link, then resolve symbol link of parent dir
var resolvedDirPath string
resolvedDirPath, err = filepath.EvalSymlinks(dirPath)
if err != nil {
return
}
// resolvedDirPath will have been cleaned (no trailing path separators) so
// we can manually join it with the base path element.
resolvedPath = resolvedDirPath + string(filepath.Separator) + basePath
if hasTrailingPathSeparator(path) && filepath.Base(path) != filepath.Base(resolvedPath) {
rebaseName = filepath.Base(path)
}
}
return resolvedPath, rebaseName, nil
}
// GetRebaseName normalizes and compares path and resolvedPath,
// return completed resolved path and rebased file name
func GetRebaseName(path, resolvedPath string) (string, string) {
// linkTarget will have been cleaned (no trailing path separators and dot) so
// we can manually join it with them
var rebaseName string
if specifiesCurrentDir(path) && !specifiesCurrentDir(resolvedPath) {
resolvedPath += string(filepath.Separator) + "."
}
if hasTrailingPathSeparator(path) && !hasTrailingPathSeparator(resolvedPath) {
resolvedPath += string(filepath.Separator)
}
if filepath.Base(path) != filepath.Base(resolvedPath) {
// In the case where the path had a trailing separator and a symlink
// evaluation has changed the last path component, we will need to
// rebase the name in the archive that is being copied to match the
// originally requested name.
rebaseName = filepath.Base(path)
}
return resolvedPath, rebaseName
}

View File

@ -1,11 +0,0 @@
// +build !windows
package archive
import (
"path/filepath"
)
func normalizePath(path string) string {
return filepath.ToSlash(path)
}

View File

@ -1,9 +0,0 @@
package archive
import (
"path/filepath"
)
func normalizePath(path string) string {
return filepath.FromSlash(path)
}

View File

@ -1,256 +0,0 @@
package archive
import (
"archive/tar"
"fmt"
"io"
"io/ioutil"
"os"
"path/filepath"
"runtime"
"strings"
"github.com/docker/docker/pkg/idtools"
"github.com/docker/docker/pkg/pools"
"github.com/docker/docker/pkg/system"
"github.com/sirupsen/logrus"
)
// UnpackLayer unpack `layer` to a `dest`. The stream `layer` can be
// compressed or uncompressed.
// Returns the size in bytes of the contents of the layer.
func UnpackLayer(dest string, layer io.Reader, options *TarOptions) (size int64, err error) {
tr := tar.NewReader(layer)
trBuf := pools.BufioReader32KPool.Get(tr)
defer pools.BufioReader32KPool.Put(trBuf)
var dirs []*tar.Header
unpackedPaths := make(map[string]struct{})
if options == nil {
options = &TarOptions{}
}
if options.ExcludePatterns == nil {
options.ExcludePatterns = []string{}
}
idMappings := idtools.NewIDMappingsFromMaps(options.UIDMaps, options.GIDMaps)
aufsTempdir := ""
aufsHardlinks := make(map[string]*tar.Header)
// Iterate through the files in the archive.
for {
hdr, err := tr.Next()
if err == io.EOF {
// end of tar archive
break
}
if err != nil {
return 0, err
}
size += hdr.Size
// Normalize name, for safety and for a simple is-root check
hdr.Name = filepath.Clean(hdr.Name)
// Windows does not support filenames with colons in them. Ignore
// these files. This is not a problem though (although it might
// appear that it is). Let's suppose a client is running docker pull.
// The daemon it points to is Windows. Would it make sense for the
// client to be doing a docker pull Ubuntu for example (which has files
// with colons in the name under /usr/share/man/man3)? No, absolutely
// not as it would really only make sense that they were pulling a
// Windows image. However, for development, it is necessary to be able
// to pull Linux images which are in the repository.
//
// TODO Windows. Once the registry is aware of what images are Windows-
// specific or Linux-specific, this warning should be changed to an error
// to cater for the situation where someone does manage to upload a Linux
// image but have it tagged as Windows inadvertently.
if runtime.GOOS == "windows" {
if strings.Contains(hdr.Name, ":") {
logrus.Warnf("Windows: Ignoring %s (is this a Linux image?)", hdr.Name)
continue
}
}
// Note as these operations are platform specific, so must the slash be.
if !strings.HasSuffix(hdr.Name, string(os.PathSeparator)) {
// Not the root directory, ensure that the parent directory exists.
// This happened in some tests where an image had a tarfile without any
// parent directories.
parent := filepath.Dir(hdr.Name)
parentPath := filepath.Join(dest, parent)
if _, err := os.Lstat(parentPath); err != nil && os.IsNotExist(err) {
err = system.MkdirAll(parentPath, 0600, "")
if err != nil {
return 0, err
}
}
}
// Skip AUFS metadata dirs
if strings.HasPrefix(hdr.Name, WhiteoutMetaPrefix) {
// Regular files inside /.wh..wh.plnk can be used as hardlink targets
// We don't want this directory, but we need the files in them so that
// such hardlinks can be resolved.
if strings.HasPrefix(hdr.Name, WhiteoutLinkDir) && hdr.Typeflag == tar.TypeReg {
basename := filepath.Base(hdr.Name)
aufsHardlinks[basename] = hdr
if aufsTempdir == "" {
if aufsTempdir, err = ioutil.TempDir("", "dockerplnk"); err != nil {
return 0, err
}
defer os.RemoveAll(aufsTempdir)
}
if err := createTarFile(filepath.Join(aufsTempdir, basename), dest, hdr, tr, true, nil, options.InUserNS); err != nil {
return 0, err
}
}
if hdr.Name != WhiteoutOpaqueDir {
continue
}
}
path := filepath.Join(dest, hdr.Name)
rel, err := filepath.Rel(dest, path)
if err != nil {
return 0, err
}
// Note as these operations are platform specific, so must the slash be.
if strings.HasPrefix(rel, ".."+string(os.PathSeparator)) {
return 0, breakoutError(fmt.Errorf("%q is outside of %q", hdr.Name, dest))
}
base := filepath.Base(path)
if strings.HasPrefix(base, WhiteoutPrefix) {
dir := filepath.Dir(path)
if base == WhiteoutOpaqueDir {
_, err := os.Lstat(dir)
if err != nil {
return 0, err
}
err = filepath.Walk(dir, func(path string, info os.FileInfo, err error) error {
if err != nil {
if os.IsNotExist(err) {
err = nil // parent was deleted
}
return err
}
if path == dir {
return nil
}
if _, exists := unpackedPaths[path]; !exists {
err := os.RemoveAll(path)
return err
}
return nil
})
if err != nil {
return 0, err
}
} else {
originalBase := base[len(WhiteoutPrefix):]
originalPath := filepath.Join(dir, originalBase)
if err := os.RemoveAll(originalPath); err != nil {
return 0, err
}
}
} else {
// If path exits we almost always just want to remove and replace it.
// The only exception is when it is a directory *and* the file from
// the layer is also a directory. Then we want to merge them (i.e.
// just apply the metadata from the layer).
if fi, err := os.Lstat(path); err == nil {
if !(fi.IsDir() && hdr.Typeflag == tar.TypeDir) {
if err := os.RemoveAll(path); err != nil {
return 0, err
}
}
}
trBuf.Reset(tr)
srcData := io.Reader(trBuf)
srcHdr := hdr
// Hard links into /.wh..wh.plnk don't work, as we don't extract that directory, so
// we manually retarget these into the temporary files we extracted them into
if hdr.Typeflag == tar.TypeLink && strings.HasPrefix(filepath.Clean(hdr.Linkname), WhiteoutLinkDir) {
linkBasename := filepath.Base(hdr.Linkname)
srcHdr = aufsHardlinks[linkBasename]
if srcHdr == nil {
return 0, fmt.Errorf("Invalid aufs hardlink")
}
tmpFile, err := os.Open(filepath.Join(aufsTempdir, linkBasename))
if err != nil {
return 0, err
}
defer tmpFile.Close()
srcData = tmpFile
}
if err := remapIDs(idMappings, srcHdr); err != nil {
return 0, err
}
if err := createTarFile(path, dest, srcHdr, srcData, true, nil, options.InUserNS); err != nil {
return 0, err
}
// Directory mtimes must be handled at the end to avoid further
// file creation in them to modify the directory mtime
if hdr.Typeflag == tar.TypeDir {
dirs = append(dirs, hdr)
}
unpackedPaths[path] = struct{}{}
}
}
for _, hdr := range dirs {
path := filepath.Join(dest, hdr.Name)
if err := system.Chtimes(path, hdr.AccessTime, hdr.ModTime); err != nil {
return 0, err
}
}
return size, nil
}
// ApplyLayer parses a diff in the standard layer format from `layer`,
// and applies it to the directory `dest`. The stream `layer` can be
// compressed or uncompressed.
// Returns the size in bytes of the contents of the layer.
func ApplyLayer(dest string, layer io.Reader) (int64, error) {
return applyLayerHandler(dest, layer, &TarOptions{}, true)
}
// ApplyUncompressedLayer parses a diff in the standard layer format from
// `layer`, and applies it to the directory `dest`. The stream `layer`
// can only be uncompressed.
// Returns the size in bytes of the contents of the layer.
func ApplyUncompressedLayer(dest string, layer io.Reader, options *TarOptions) (int64, error) {
return applyLayerHandler(dest, layer, options, false)
}
// do the bulk load of ApplyLayer, but allow for not calling DecompressStream
func applyLayerHandler(dest string, layer io.Reader, options *TarOptions, decompress bool) (int64, error) {
dest = filepath.Clean(dest)
// We need to be able to set any perms
oldmask, err := system.Umask(0)
if err != nil {
return 0, err
}
defer system.Umask(oldmask) // ignore err, ErrNotSupportedPlatform
if decompress {
layer, err = DecompressStream(layer)
if err != nil {
return 0, err
}
}
return UnpackLayer(dest, layer, options)
}

View File

@ -1,97 +0,0 @@
// +build ignore
// Simple tool to create an archive stream from an old and new directory
//
// By default it will stream the comparison of two temporary directories with junk files
package main
import (
"flag"
"fmt"
"io"
"io/ioutil"
"os"
"path"
"github.com/docker/docker/pkg/archive"
"github.com/sirupsen/logrus"
)
var (
flDebug = flag.Bool("D", false, "debugging output")
flNewDir = flag.String("newdir", "", "")
flOldDir = flag.String("olddir", "", "")
log = logrus.New()
)
func main() {
flag.Usage = func() {
fmt.Println("Produce a tar from comparing two directory paths. By default a demo tar is created of around 200 files (including hardlinks)")
fmt.Printf("%s [OPTIONS]\n", os.Args[0])
flag.PrintDefaults()
}
flag.Parse()
log.Out = os.Stderr
if (len(os.Getenv("DEBUG")) > 0) || *flDebug {
logrus.SetLevel(logrus.DebugLevel)
}
var newDir, oldDir string
if len(*flNewDir) == 0 {
var err error
newDir, err = ioutil.TempDir("", "docker-test-newDir")
if err != nil {
log.Fatal(err)
}
defer os.RemoveAll(newDir)
if _, err := prepareUntarSourceDirectory(100, newDir, true); err != nil {
log.Fatal(err)
}
} else {
newDir = *flNewDir
}
if len(*flOldDir) == 0 {
oldDir, err := ioutil.TempDir("", "docker-test-oldDir")
if err != nil {
log.Fatal(err)
}
defer os.RemoveAll(oldDir)
} else {
oldDir = *flOldDir
}
changes, err := archive.ChangesDirs(newDir, oldDir)
if err != nil {
log.Fatal(err)
}
a, err := archive.ExportChanges(newDir, changes)
if err != nil {
log.Fatal(err)
}
defer a.Close()
i, err := io.Copy(os.Stdout, a)
if err != nil && err != io.EOF {
log.Fatal(err)
}
fmt.Fprintf(os.Stderr, "wrote archive of %d bytes", i)
}
func prepareUntarSourceDirectory(numberOfFiles int, targetPath string, makeLinks bool) (int, error) {
fileData := []byte("fooo")
for n := 0; n < numberOfFiles; n++ {
fileName := fmt.Sprintf("file-%d", n)
if err := ioutil.WriteFile(path.Join(targetPath, fileName), fileData, 0700); err != nil {
return 0, err
}
if makeLinks {
if err := os.Link(path.Join(targetPath, fileName), path.Join(targetPath, fileName+"-link")); err != nil {
return 0, err
}
}
}
totalSize := numberOfFiles * len(fileData)
return totalSize, nil
}

View File

@ -1,16 +0,0 @@
package archive
import (
"syscall"
"time"
)
func timeToTimespec(time time.Time) (ts syscall.Timespec) {
if time.IsZero() {
// Return UTIME_OMIT special value
ts.Sec = 0
ts.Nsec = ((1 << 30) - 2)
return
}
return syscall.NsecToTimespec(time.UnixNano())
}

View File

@ -1,16 +0,0 @@
// +build !linux
package archive
import (
"syscall"
"time"
)
func timeToTimespec(time time.Time) (ts syscall.Timespec) {
nsec := int64(0)
if !time.IsZero() {
nsec = time.UnixNano()
}
return syscall.NsecToTimespec(nsec)
}

View File

@ -1,23 +0,0 @@
package archive
// Whiteouts are files with a special meaning for the layered filesystem.
// Docker uses AUFS whiteout files inside exported archives. In other
// filesystems these files are generated/handled on tar creation/extraction.
// WhiteoutPrefix prefix means file is a whiteout. If this is followed by a
// filename this means that file has been removed from the base layer.
const WhiteoutPrefix = ".wh."
// WhiteoutMetaPrefix prefix means whiteout has a special meaning and is not
// for removing an actual file. Normally these files are excluded from exported
// archives.
const WhiteoutMetaPrefix = WhiteoutPrefix + WhiteoutPrefix
// WhiteoutLinkDir is a directory AUFS uses for storing hardlink links to other
// layers. Normally these should not go into exported archives and all changed
// hardlinks should be copied to the top layer.
const WhiteoutLinkDir = WhiteoutMetaPrefix + "plnk"
// WhiteoutOpaqueDir file means directory has been made opaque - meaning
// readdir calls to this directory do not follow to lower layers.
const WhiteoutOpaqueDir = WhiteoutMetaPrefix + ".opq"

View File

@ -1,59 +0,0 @@
package archive
import (
"archive/tar"
"bytes"
"io"
)
// Generate generates a new archive from the content provided
// as input.
//
// `files` is a sequence of path/content pairs. A new file is
// added to the archive for each pair.
// If the last pair is incomplete, the file is created with an
// empty content. For example:
//
// Generate("foo.txt", "hello world", "emptyfile")
//
// The above call will return an archive with 2 files:
// * ./foo.txt with content "hello world"
// * ./empty with empty content
//
// FIXME: stream content instead of buffering
// FIXME: specify permissions and other archive metadata
func Generate(input ...string) (io.Reader, error) {
files := parseStringPairs(input...)
buf := new(bytes.Buffer)
tw := tar.NewWriter(buf)
for _, file := range files {
name, content := file[0], file[1]
hdr := &tar.Header{
Name: name,
Size: int64(len(content)),
}
if err := tw.WriteHeader(hdr); err != nil {
return nil, err
}
if _, err := tw.Write([]byte(content)); err != nil {
return nil, err
}
}
if err := tw.Close(); err != nil {
return nil, err
}
return buf, nil
}
func parseStringPairs(input ...string) (output [][2]string) {
output = make([][2]string, 0, len(input)/2+1)
for i := 0; i < len(input); i += 2 {
var pair [2]string
pair[0] = input[i]
if i+1 < len(input) {
pair[1] = input[i+1]
}
output = append(output, pair)
}
return
}

View File

@ -1,279 +0,0 @@
package idtools
import (
"bufio"
"fmt"
"os"
"sort"
"strconv"
"strings"
)
// IDMap contains a single entry for user namespace range remapping. An array
// of IDMap entries represents the structure that will be provided to the Linux
// kernel for creating a user namespace.
type IDMap struct {
ContainerID int `json:"container_id"`
HostID int `json:"host_id"`
Size int `json:"size"`
}
type subIDRange struct {
Start int
Length int
}
type ranges []subIDRange
func (e ranges) Len() int { return len(e) }
func (e ranges) Swap(i, j int) { e[i], e[j] = e[j], e[i] }
func (e ranges) Less(i, j int) bool { return e[i].Start < e[j].Start }
const (
subuidFileName string = "/etc/subuid"
subgidFileName string = "/etc/subgid"
)
// MkdirAllAs creates a directory (include any along the path) and then modifies
// ownership to the requested uid/gid. If the directory already exists, this
// function will still change ownership to the requested uid/gid pair.
// Deprecated: Use MkdirAllAndChown
func MkdirAllAs(path string, mode os.FileMode, ownerUID, ownerGID int) error {
return mkdirAs(path, mode, ownerUID, ownerGID, true, true)
}
// MkdirAs creates a directory and then modifies ownership to the requested uid/gid.
// If the directory already exists, this function still changes ownership
// Deprecated: Use MkdirAndChown with a IDPair
func MkdirAs(path string, mode os.FileMode, ownerUID, ownerGID int) error {
return mkdirAs(path, mode, ownerUID, ownerGID, false, true)
}
// MkdirAllAndChown creates a directory (include any along the path) and then modifies
// ownership to the requested uid/gid. If the directory already exists, this
// function will still change ownership to the requested uid/gid pair.
func MkdirAllAndChown(path string, mode os.FileMode, ids IDPair) error {
return mkdirAs(path, mode, ids.UID, ids.GID, true, true)
}
// MkdirAndChown creates a directory and then modifies ownership to the requested uid/gid.
// If the directory already exists, this function still changes ownership
func MkdirAndChown(path string, mode os.FileMode, ids IDPair) error {
return mkdirAs(path, mode, ids.UID, ids.GID, false, true)
}
// MkdirAllAndChownNew creates a directory (include any along the path) and then modifies
// ownership ONLY of newly created directories to the requested uid/gid. If the
// directories along the path exist, no change of ownership will be performed
func MkdirAllAndChownNew(path string, mode os.FileMode, ids IDPair) error {
return mkdirAs(path, mode, ids.UID, ids.GID, true, false)
}
// GetRootUIDGID retrieves the remapped root uid/gid pair from the set of maps.
// If the maps are empty, then the root uid/gid will default to "real" 0/0
func GetRootUIDGID(uidMap, gidMap []IDMap) (int, int, error) {
uid, err := toHost(0, uidMap)
if err != nil {
return -1, -1, err
}
gid, err := toHost(0, gidMap)
if err != nil {
return -1, -1, err
}
return uid, gid, nil
}
// toContainer takes an id mapping, and uses it to translate a
// host ID to the remapped ID. If no map is provided, then the translation
// assumes a 1-to-1 mapping and returns the passed in id
func toContainer(hostID int, idMap []IDMap) (int, error) {
if idMap == nil {
return hostID, nil
}
for _, m := range idMap {
if (hostID >= m.HostID) && (hostID <= (m.HostID + m.Size - 1)) {
contID := m.ContainerID + (hostID - m.HostID)
return contID, nil
}
}
return -1, fmt.Errorf("Host ID %d cannot be mapped to a container ID", hostID)
}
// toHost takes an id mapping and a remapped ID, and translates the
// ID to the mapped host ID. If no map is provided, then the translation
// assumes a 1-to-1 mapping and returns the passed in id #
func toHost(contID int, idMap []IDMap) (int, error) {
if idMap == nil {
return contID, nil
}
for _, m := range idMap {
if (contID >= m.ContainerID) && (contID <= (m.ContainerID + m.Size - 1)) {
hostID := m.HostID + (contID - m.ContainerID)
return hostID, nil
}
}
return -1, fmt.Errorf("Container ID %d cannot be mapped to a host ID", contID)
}
// IDPair is a UID and GID pair
type IDPair struct {
UID int
GID int
}
// IDMappings contains a mappings of UIDs and GIDs
type IDMappings struct {
uids []IDMap
gids []IDMap
}
// NewIDMappings takes a requested user and group name and
// using the data from /etc/sub{uid,gid} ranges, creates the
// proper uid and gid remapping ranges for that user/group pair
func NewIDMappings(username, groupname string) (*IDMappings, error) {
subuidRanges, err := parseSubuid(username)
if err != nil {
return nil, err
}
subgidRanges, err := parseSubgid(groupname)
if err != nil {
return nil, err
}
if len(subuidRanges) == 0 {
return nil, fmt.Errorf("No subuid ranges found for user %q", username)
}
if len(subgidRanges) == 0 {
return nil, fmt.Errorf("No subgid ranges found for group %q", groupname)
}
return &IDMappings{
uids: createIDMap(subuidRanges),
gids: createIDMap(subgidRanges),
}, nil
}
// NewIDMappingsFromMaps creates a new mapping from two slices
// Deprecated: this is a temporary shim while transitioning to IDMapping
func NewIDMappingsFromMaps(uids []IDMap, gids []IDMap) *IDMappings {
return &IDMappings{uids: uids, gids: gids}
}
// RootPair returns a uid and gid pair for the root user. The error is ignored
// because a root user always exists, and the defaults are correct when the uid
// and gid maps are empty.
func (i *IDMappings) RootPair() IDPair {
uid, gid, _ := GetRootUIDGID(i.uids, i.gids)
return IDPair{UID: uid, GID: gid}
}
// ToHost returns the host UID and GID for the container uid, gid.
// Remapping is only performed if the ids aren't already the remapped root ids
func (i *IDMappings) ToHost(pair IDPair) (IDPair, error) {
var err error
target := i.RootPair()
if pair.UID != target.UID {
target.UID, err = toHost(pair.UID, i.uids)
if err != nil {
return target, err
}
}
if pair.GID != target.GID {
target.GID, err = toHost(pair.GID, i.gids)
}
return target, err
}
// ToContainer returns the container UID and GID for the host uid and gid
func (i *IDMappings) ToContainer(pair IDPair) (int, int, error) {
uid, err := toContainer(pair.UID, i.uids)
if err != nil {
return -1, -1, err
}
gid, err := toContainer(pair.GID, i.gids)
return uid, gid, err
}
// Empty returns true if there are no id mappings
func (i *IDMappings) Empty() bool {
return len(i.uids) == 0 && len(i.gids) == 0
}
// UIDs return the UID mapping
// TODO: remove this once everything has been refactored to use pairs
func (i *IDMappings) UIDs() []IDMap {
return i.uids
}
// GIDs return the UID mapping
// TODO: remove this once everything has been refactored to use pairs
func (i *IDMappings) GIDs() []IDMap {
return i.gids
}
func createIDMap(subidRanges ranges) []IDMap {
idMap := []IDMap{}
// sort the ranges by lowest ID first
sort.Sort(subidRanges)
containerID := 0
for _, idrange := range subidRanges {
idMap = append(idMap, IDMap{
ContainerID: containerID,
HostID: idrange.Start,
Size: idrange.Length,
})
containerID = containerID + idrange.Length
}
return idMap
}
func parseSubuid(username string) (ranges, error) {
return parseSubidFile(subuidFileName, username)
}
func parseSubgid(username string) (ranges, error) {
return parseSubidFile(subgidFileName, username)
}
// parseSubidFile will read the appropriate file (/etc/subuid or /etc/subgid)
// and return all found ranges for a specified username. If the special value
// "ALL" is supplied for username, then all ranges in the file will be returned
func parseSubidFile(path, username string) (ranges, error) {
var rangeList ranges
subidFile, err := os.Open(path)
if err != nil {
return rangeList, err
}
defer subidFile.Close()
s := bufio.NewScanner(subidFile)
for s.Scan() {
if err := s.Err(); err != nil {
return rangeList, err
}
text := strings.TrimSpace(s.Text())
if text == "" || strings.HasPrefix(text, "#") {
continue
}
parts := strings.Split(text, ":")
if len(parts) != 3 {
return rangeList, fmt.Errorf("Cannot parse subuid/gid information: Format not correct for %s file", path)
}
if parts[0] == username || username == "ALL" {
startid, err := strconv.Atoi(parts[1])
if err != nil {
return rangeList, fmt.Errorf("String to int conversion failed during subuid/gid parsing of %s: %v", path, err)
}
length, err := strconv.Atoi(parts[2])
if err != nil {
return rangeList, fmt.Errorf("String to int conversion failed during subuid/gid parsing of %s: %v", path, err)
}
rangeList = append(rangeList, subIDRange{startid, length})
}
}
return rangeList, nil
}

View File

@ -1,204 +0,0 @@
// +build !windows
package idtools
import (
"bytes"
"fmt"
"io"
"os"
"path/filepath"
"strings"
"sync"
"github.com/docker/docker/pkg/system"
"github.com/opencontainers/runc/libcontainer/user"
)
var (
entOnce sync.Once
getentCmd string
)
func mkdirAs(path string, mode os.FileMode, ownerUID, ownerGID int, mkAll, chownExisting bool) error {
// make an array containing the original path asked for, plus (for mkAll == true)
// all path components leading up to the complete path that don't exist before we MkdirAll
// so that we can chown all of them properly at the end. If chownExisting is false, we won't
// chown the full directory path if it exists
var paths []string
if _, err := os.Stat(path); err != nil && os.IsNotExist(err) {
paths = []string{path}
} else if err == nil && chownExisting {
// short-circuit--we were called with an existing directory and chown was requested
return os.Chown(path, ownerUID, ownerGID)
} else if err == nil {
// nothing to do; directory path fully exists already and chown was NOT requested
return nil
}
if mkAll {
// walk back to "/" looking for directories which do not exist
// and add them to the paths array for chown after creation
dirPath := path
for {
dirPath = filepath.Dir(dirPath)
if dirPath == "/" {
break
}
if _, err := os.Stat(dirPath); err != nil && os.IsNotExist(err) {
paths = append(paths, dirPath)
}
}
if err := system.MkdirAll(path, mode, ""); err != nil && !os.IsExist(err) {
return err
}
} else {
if err := os.Mkdir(path, mode); err != nil && !os.IsExist(err) {
return err
}
}
// even if it existed, we will chown the requested path + any subpaths that
// didn't exist when we called MkdirAll
for _, pathComponent := range paths {
if err := os.Chown(pathComponent, ownerUID, ownerGID); err != nil {
return err
}
}
return nil
}
// CanAccess takes a valid (existing) directory and a uid, gid pair and determines
// if that uid, gid pair has access (execute bit) to the directory
func CanAccess(path string, pair IDPair) bool {
statInfo, err := system.Stat(path)
if err != nil {
return false
}
fileMode := os.FileMode(statInfo.Mode())
permBits := fileMode.Perm()
return accessible(statInfo.UID() == uint32(pair.UID),
statInfo.GID() == uint32(pair.GID), permBits)
}
func accessible(isOwner, isGroup bool, perms os.FileMode) bool {
if isOwner && (perms&0100 == 0100) {
return true
}
if isGroup && (perms&0010 == 0010) {
return true
}
if perms&0001 == 0001 {
return true
}
return false
}
// LookupUser uses traditional local system files lookup (from libcontainer/user) on a username,
// followed by a call to `getent` for supporting host configured non-files passwd and group dbs
func LookupUser(username string) (user.User, error) {
// first try a local system files lookup using existing capabilities
usr, err := user.LookupUser(username)
if err == nil {
return usr, nil
}
// local files lookup failed; attempt to call `getent` to query configured passwd dbs
usr, err = getentUser(fmt.Sprintf("%s %s", "passwd", username))
if err != nil {
return user.User{}, err
}
return usr, nil
}
// LookupUID uses traditional local system files lookup (from libcontainer/user) on a uid,
// followed by a call to `getent` for supporting host configured non-files passwd and group dbs
func LookupUID(uid int) (user.User, error) {
// first try a local system files lookup using existing capabilities
usr, err := user.LookupUid(uid)
if err == nil {
return usr, nil
}
// local files lookup failed; attempt to call `getent` to query configured passwd dbs
return getentUser(fmt.Sprintf("%s %d", "passwd", uid))
}
func getentUser(args string) (user.User, error) {
reader, err := callGetent(args)
if err != nil {
return user.User{}, err
}
users, err := user.ParsePasswd(reader)
if err != nil {
return user.User{}, err
}
if len(users) == 0 {
return user.User{}, fmt.Errorf("getent failed to find passwd entry for %q", strings.Split(args, " ")[1])
}
return users[0], nil
}
// LookupGroup uses traditional local system files lookup (from libcontainer/user) on a group name,
// followed by a call to `getent` for supporting host configured non-files passwd and group dbs
func LookupGroup(groupname string) (user.Group, error) {
// first try a local system files lookup using existing capabilities
group, err := user.LookupGroup(groupname)
if err == nil {
return group, nil
}
// local files lookup failed; attempt to call `getent` to query configured group dbs
return getentGroup(fmt.Sprintf("%s %s", "group", groupname))
}
// LookupGID uses traditional local system files lookup (from libcontainer/user) on a group ID,
// followed by a call to `getent` for supporting host configured non-files passwd and group dbs
func LookupGID(gid int) (user.Group, error) {
// first try a local system files lookup using existing capabilities
group, err := user.LookupGid(gid)
if err == nil {
return group, nil
}
// local files lookup failed; attempt to call `getent` to query configured group dbs
return getentGroup(fmt.Sprintf("%s %d", "group", gid))
}
func getentGroup(args string) (user.Group, error) {
reader, err := callGetent(args)
if err != nil {
return user.Group{}, err
}
groups, err := user.ParseGroup(reader)
if err != nil {
return user.Group{}, err
}
if len(groups) == 0 {
return user.Group{}, fmt.Errorf("getent failed to find groups entry for %q", strings.Split(args, " ")[1])
}
return groups[0], nil
}
func callGetent(args string) (io.Reader, error) {
entOnce.Do(func() { getentCmd, _ = resolveBinary("getent") })
// if no `getent` command on host, can't do anything else
if getentCmd == "" {
return nil, fmt.Errorf("")
}
out, err := execCmd(getentCmd, args)
if err != nil {
exitCode, errC := system.GetExitCode(err)
if errC != nil {
return nil, err
}
switch exitCode {
case 1:
return nil, fmt.Errorf("getent reported invalid parameters/database unknown")
case 2:
terms := strings.Split(args, " ")
return nil, fmt.Errorf("getent unable to find entry %q in %s database", terms[1], terms[0])
case 3:
return nil, fmt.Errorf("getent database doesn't support enumeration")
default:
return nil, err
}
}
return bytes.NewReader(out), nil
}

View File

@ -1,25 +0,0 @@
// +build windows
package idtools
import (
"os"
"github.com/docker/docker/pkg/system"
)
// Platforms such as Windows do not support the UID/GID concept. So make this
// just a wrapper around system.MkdirAll.
func mkdirAs(path string, mode os.FileMode, ownerUID, ownerGID int, mkAll, chownExisting bool) error {
if err := system.MkdirAll(path, mode, ""); err != nil && !os.IsExist(err) {
return err
}
return nil
}
// CanAccess takes a valid (existing) directory and a uid, gid pair and determines
// if that uid, gid pair has access (execute bit) to the directory
// Windows does not require/support this function, so always return true
func CanAccess(path string, pair IDPair) bool {
return true
}

View File

@ -1,164 +0,0 @@
package idtools
import (
"fmt"
"regexp"
"sort"
"strconv"
"strings"
"sync"
)
// add a user and/or group to Linux /etc/passwd, /etc/group using standard
// Linux distribution commands:
// adduser --system --shell /bin/false --disabled-login --disabled-password --no-create-home --group <username>
// useradd -r -s /bin/false <username>
var (
once sync.Once
userCommand string
cmdTemplates = map[string]string{
"adduser": "--system --shell /bin/false --no-create-home --disabled-login --disabled-password --group %s",
"useradd": "-r -s /bin/false %s",
"usermod": "-%s %d-%d %s",
}
idOutRegexp = regexp.MustCompile(`uid=([0-9]+).*gid=([0-9]+)`)
// default length for a UID/GID subordinate range
defaultRangeLen = 65536
defaultRangeStart = 100000
userMod = "usermod"
)
// AddNamespaceRangesUser takes a username and uses the standard system
// utility to create a system user/group pair used to hold the
// /etc/sub{uid,gid} ranges which will be used for user namespace
// mapping ranges in containers.
func AddNamespaceRangesUser(name string) (int, int, error) {
if err := addUser(name); err != nil {
return -1, -1, fmt.Errorf("Error adding user %q: %v", name, err)
}
// Query the system for the created uid and gid pair
out, err := execCmd("id", name)
if err != nil {
return -1, -1, fmt.Errorf("Error trying to find uid/gid for new user %q: %v", name, err)
}
matches := idOutRegexp.FindStringSubmatch(strings.TrimSpace(string(out)))
if len(matches) != 3 {
return -1, -1, fmt.Errorf("Can't find uid, gid from `id` output: %q", string(out))
}
uid, err := strconv.Atoi(matches[1])
if err != nil {
return -1, -1, fmt.Errorf("Can't convert found uid (%s) to int: %v", matches[1], err)
}
gid, err := strconv.Atoi(matches[2])
if err != nil {
return -1, -1, fmt.Errorf("Can't convert found gid (%s) to int: %v", matches[2], err)
}
// Now we need to create the subuid/subgid ranges for our new user/group (system users
// do not get auto-created ranges in subuid/subgid)
if err := createSubordinateRanges(name); err != nil {
return -1, -1, fmt.Errorf("Couldn't create subordinate ID ranges: %v", err)
}
return uid, gid, nil
}
func addUser(userName string) error {
once.Do(func() {
// set up which commands are used for adding users/groups dependent on distro
if _, err := resolveBinary("adduser"); err == nil {
userCommand = "adduser"
} else if _, err := resolveBinary("useradd"); err == nil {
userCommand = "useradd"
}
})
if userCommand == "" {
return fmt.Errorf("Cannot add user; no useradd/adduser binary found")
}
args := fmt.Sprintf(cmdTemplates[userCommand], userName)
out, err := execCmd(userCommand, args)
if err != nil {
return fmt.Errorf("Failed to add user with error: %v; output: %q", err, string(out))
}
return nil
}
func createSubordinateRanges(name string) error {
// first, we should verify that ranges weren't automatically created
// by the distro tooling
ranges, err := parseSubuid(name)
if err != nil {
return fmt.Errorf("Error while looking for subuid ranges for user %q: %v", name, err)
}
if len(ranges) == 0 {
// no UID ranges; let's create one
startID, err := findNextUIDRange()
if err != nil {
return fmt.Errorf("Can't find available subuid range: %v", err)
}
out, err := execCmd(userMod, fmt.Sprintf(cmdTemplates[userMod], "v", startID, startID+defaultRangeLen-1, name))
if err != nil {
return fmt.Errorf("Unable to add subuid range to user: %q; output: %s, err: %v", name, out, err)
}
}
ranges, err = parseSubgid(name)
if err != nil {
return fmt.Errorf("Error while looking for subgid ranges for user %q: %v", name, err)
}
if len(ranges) == 0 {
// no GID ranges; let's create one
startID, err := findNextGIDRange()
if err != nil {
return fmt.Errorf("Can't find available subgid range: %v", err)
}
out, err := execCmd(userMod, fmt.Sprintf(cmdTemplates[userMod], "w", startID, startID+defaultRangeLen-1, name))
if err != nil {
return fmt.Errorf("Unable to add subgid range to user: %q; output: %s, err: %v", name, out, err)
}
}
return nil
}
func findNextUIDRange() (int, error) {
ranges, err := parseSubuid("ALL")
if err != nil {
return -1, fmt.Errorf("Couldn't parse all ranges in /etc/subuid file: %v", err)
}
sort.Sort(ranges)
return findNextRangeStart(ranges)
}
func findNextGIDRange() (int, error) {
ranges, err := parseSubgid("ALL")
if err != nil {
return -1, fmt.Errorf("Couldn't parse all ranges in /etc/subgid file: %v", err)
}
sort.Sort(ranges)
return findNextRangeStart(ranges)
}
func findNextRangeStart(rangeList ranges) (int, error) {
startID := defaultRangeStart
for _, arange := range rangeList {
if wouldOverlap(arange, startID) {
startID = arange.Start + arange.Length
}
}
return startID, nil
}
func wouldOverlap(arange subIDRange, ID int) bool {
low := ID
high := ID + defaultRangeLen
if (low >= arange.Start && low <= arange.Start+arange.Length) ||
(high <= arange.Start+arange.Length && high >= arange.Start) {
return true
}
return false
}

View File

@ -1,12 +0,0 @@
// +build !linux
package idtools
import "fmt"
// AddNamespaceRangesUser takes a name and finds an unused uid, gid pair
// and calls the appropriate helper function to add the group and then
// the user to the group in /etc/group and /etc/passwd respectively.
func AddNamespaceRangesUser(name string) (int, int, error) {
return -1, -1, fmt.Errorf("No support for adding users or groups on this OS")
}

View File

@ -1,32 +0,0 @@
// +build !windows
package idtools
import (
"fmt"
"os/exec"
"path/filepath"
"strings"
)
func resolveBinary(binname string) (string, error) {
binaryPath, err := exec.LookPath(binname)
if err != nil {
return "", err
}
resolvedPath, err := filepath.EvalSymlinks(binaryPath)
if err != nil {
return "", err
}
//only return no error if the final resolved binary basename
//matches what was searched for
if filepath.Base(resolvedPath) == binname {
return resolvedPath, nil
}
return "", fmt.Errorf("Binary %q does not resolve to a binary of that name in $PATH (%q)", binname, resolvedPath)
}
func execCmd(cmd, args string) ([]byte, error) {
execCmd := exec.Command(cmd, strings.Split(args, " ")...)
return execCmd.CombinedOutput()
}

View File

@ -1,11 +0,0 @@
package promise
// Go is a basic promise implementation: it wraps calls a function in a goroutine,
// and returns a channel which will later return the function's return value.
func Go(f func() error) chan error {
ch := make(chan error, 1)
go func() {
ch <- f()
}()
return ch
}

View File

@ -1,21 +0,0 @@
package tarsum
// BuilderContext is an interface extending TarSum by adding the Remove method.
// In general there was concern about adding this method to TarSum itself
// so instead it is being added just to "BuilderContext" which will then
// only be used during the .dockerignore file processing
// - see builder/evaluator.go
type BuilderContext interface {
TarSum
Remove(string)
}
func (bc *tarSum) Remove(filename string) {
for i, fis := range bc.sums {
if fis.Name() == filename {
bc.sums = append(bc.sums[:i], bc.sums[i+1:]...)
// Note, we don't just return because there could be
// more than one with this name
}
}
}

View File

@ -1,126 +0,0 @@
package tarsum
import "sort"
// FileInfoSumInterface provides an interface for accessing file checksum
// information within a tar file. This info is accessed through interface
// so the actual name and sum cannot be melded with.
type FileInfoSumInterface interface {
// File name
Name() string
// Checksum of this particular file and its headers
Sum() string
// Position of file in the tar
Pos() int64
}
type fileInfoSum struct {
name string
sum string
pos int64
}
func (fis fileInfoSum) Name() string {
return fis.name
}
func (fis fileInfoSum) Sum() string {
return fis.sum
}
func (fis fileInfoSum) Pos() int64 {
return fis.pos
}
// FileInfoSums provides a list of FileInfoSumInterfaces.
type FileInfoSums []FileInfoSumInterface
// GetFile returns the first FileInfoSumInterface with a matching name.
func (fis FileInfoSums) GetFile(name string) FileInfoSumInterface {
for i := range fis {
if fis[i].Name() == name {
return fis[i]
}
}
return nil
}
// GetAllFile returns a FileInfoSums with all matching names.
func (fis FileInfoSums) GetAllFile(name string) FileInfoSums {
f := FileInfoSums{}
for i := range fis {
if fis[i].Name() == name {
f = append(f, fis[i])
}
}
return f
}
// GetDuplicatePaths returns a FileInfoSums with all duplicated paths.
func (fis FileInfoSums) GetDuplicatePaths() (dups FileInfoSums) {
seen := make(map[string]int, len(fis)) // allocate earl. no need to grow this map.
for i := range fis {
f := fis[i]
if _, ok := seen[f.Name()]; ok {
dups = append(dups, f)
} else {
seen[f.Name()] = 0
}
}
return dups
}
// Len returns the size of the FileInfoSums.
func (fis FileInfoSums) Len() int { return len(fis) }
// Swap swaps two FileInfoSum values if a FileInfoSums list.
func (fis FileInfoSums) Swap(i, j int) { fis[i], fis[j] = fis[j], fis[i] }
// SortByPos sorts FileInfoSums content by position.
func (fis FileInfoSums) SortByPos() {
sort.Sort(byPos{fis})
}
// SortByNames sorts FileInfoSums content by name.
func (fis FileInfoSums) SortByNames() {
sort.Sort(byName{fis})
}
// SortBySums sorts FileInfoSums content by sums.
func (fis FileInfoSums) SortBySums() {
dups := fis.GetDuplicatePaths()
if len(dups) > 0 {
sort.Sort(bySum{fis, dups})
} else {
sort.Sort(bySum{fis, nil})
}
}
// byName is a sort.Sort helper for sorting by file names.
// If names are the same, order them by their appearance in the tar archive
type byName struct{ FileInfoSums }
func (bn byName) Less(i, j int) bool {
if bn.FileInfoSums[i].Name() == bn.FileInfoSums[j].Name() {
return bn.FileInfoSums[i].Pos() < bn.FileInfoSums[j].Pos()
}
return bn.FileInfoSums[i].Name() < bn.FileInfoSums[j].Name()
}
// bySum is a sort.Sort helper for sorting by the sums of all the fileinfos in the tar archive
type bySum struct {
FileInfoSums
dups FileInfoSums
}
func (bs bySum) Less(i, j int) bool {
if bs.dups != nil && bs.FileInfoSums[i].Name() == bs.FileInfoSums[j].Name() {
return bs.FileInfoSums[i].Pos() < bs.FileInfoSums[j].Pos()
}
return bs.FileInfoSums[i].Sum() < bs.FileInfoSums[j].Sum()
}
// byPos is a sort.Sort helper for sorting by the sums of all the fileinfos by their original order
type byPos struct{ FileInfoSums }
func (bp byPos) Less(i, j int) bool {
return bp.FileInfoSums[i].Pos() < bp.FileInfoSums[j].Pos()
}

View File

@ -1,295 +0,0 @@
// Package tarsum provides algorithms to perform checksum calculation on
// filesystem layers.
//
// The transportation of filesystems, regarding Docker, is done with tar(1)
// archives. There are a variety of tar serialization formats [2], and a key
// concern here is ensuring a repeatable checksum given a set of inputs from a
// generic tar archive. Types of transportation include distribution to and from a
// registry endpoint, saving and loading through commands or Docker daemon APIs,
// transferring the build context from client to Docker daemon, and committing the
// filesystem of a container to become an image.
//
// As tar archives are used for transit, but not preserved in many situations, the
// focus of the algorithm is to ensure the integrity of the preserved filesystem,
// while maintaining a deterministic accountability. This includes neither
// constraining the ordering or manipulation of the files during the creation or
// unpacking of the archive, nor include additional metadata state about the file
// system attributes.
package tarsum
import (
"archive/tar"
"bytes"
"compress/gzip"
"crypto"
"crypto/sha256"
"encoding/hex"
"errors"
"fmt"
"hash"
"io"
"path"
"strings"
)
const (
buf8K = 8 * 1024
buf16K = 16 * 1024
buf32K = 32 * 1024
)
// NewTarSum creates a new interface for calculating a fixed time checksum of a
// tar archive.
//
// This is used for calculating checksums of layers of an image, in some cases
// including the byte payload of the image's json metadata as well, and for
// calculating the checksums for buildcache.
func NewTarSum(r io.Reader, dc bool, v Version) (TarSum, error) {
return NewTarSumHash(r, dc, v, DefaultTHash)
}
// NewTarSumHash creates a new TarSum, providing a THash to use rather than
// the DefaultTHash.
func NewTarSumHash(r io.Reader, dc bool, v Version, tHash THash) (TarSum, error) {
headerSelector, err := getTarHeaderSelector(v)
if err != nil {
return nil, err
}
ts := &tarSum{Reader: r, DisableCompression: dc, tarSumVersion: v, headerSelector: headerSelector, tHash: tHash}
err = ts.initTarSum()
return ts, err
}
// NewTarSumForLabel creates a new TarSum using the provided TarSum version+hash label.
func NewTarSumForLabel(r io.Reader, disableCompression bool, label string) (TarSum, error) {
parts := strings.SplitN(label, "+", 2)
if len(parts) != 2 {
return nil, errors.New("tarsum label string should be of the form: {tarsum_version}+{hash_name}")
}
versionName, hashName := parts[0], parts[1]
version, ok := tarSumVersionsByName[versionName]
if !ok {
return nil, fmt.Errorf("unknown TarSum version name: %q", versionName)
}
hashConfig, ok := standardHashConfigs[hashName]
if !ok {
return nil, fmt.Errorf("unknown TarSum hash name: %q", hashName)
}
tHash := NewTHash(hashConfig.name, hashConfig.hash.New)
return NewTarSumHash(r, disableCompression, version, tHash)
}
// TarSum is the generic interface for calculating fixed time
// checksums of a tar archive.
type TarSum interface {
io.Reader
GetSums() FileInfoSums
Sum([]byte) string
Version() Version
Hash() THash
}
// tarSum struct is the structure for a Version0 checksum calculation.
type tarSum struct {
io.Reader
tarR *tar.Reader
tarW *tar.Writer
writer writeCloseFlusher
bufTar *bytes.Buffer
bufWriter *bytes.Buffer
bufData []byte
h hash.Hash
tHash THash
sums FileInfoSums
fileCounter int64
currentFile string
finished bool
first bool
DisableCompression bool // false by default. When false, the output gzip compressed.
tarSumVersion Version // this field is not exported so it can not be mutated during use
headerSelector tarHeaderSelector // handles selecting and ordering headers for files in the archive
}
func (ts tarSum) Hash() THash {
return ts.tHash
}
func (ts tarSum) Version() Version {
return ts.tarSumVersion
}
// THash provides a hash.Hash type generator and its name.
type THash interface {
Hash() hash.Hash
Name() string
}
// NewTHash is a convenience method for creating a THash.
func NewTHash(name string, h func() hash.Hash) THash {
return simpleTHash{n: name, h: h}
}
type tHashConfig struct {
name string
hash crypto.Hash
}
var (
// NOTE: DO NOT include MD5 or SHA1, which are considered insecure.
standardHashConfigs = map[string]tHashConfig{
"sha256": {name: "sha256", hash: crypto.SHA256},
"sha512": {name: "sha512", hash: crypto.SHA512},
}
)
// DefaultTHash is default TarSum hashing algorithm - "sha256".
var DefaultTHash = NewTHash("sha256", sha256.New)
type simpleTHash struct {
n string
h func() hash.Hash
}
func (sth simpleTHash) Name() string { return sth.n }
func (sth simpleTHash) Hash() hash.Hash { return sth.h() }
func (ts *tarSum) encodeHeader(h *tar.Header) error {
for _, elem := range ts.headerSelector.selectHeaders(h) {
if _, err := ts.h.Write([]byte(elem[0] + elem[1])); err != nil {
return err
}
}
return nil
}
func (ts *tarSum) initTarSum() error {
ts.bufTar = bytes.NewBuffer([]byte{})
ts.bufWriter = bytes.NewBuffer([]byte{})
ts.tarR = tar.NewReader(ts.Reader)
ts.tarW = tar.NewWriter(ts.bufTar)
if !ts.DisableCompression {
ts.writer = gzip.NewWriter(ts.bufWriter)
} else {
ts.writer = &nopCloseFlusher{Writer: ts.bufWriter}
}
if ts.tHash == nil {
ts.tHash = DefaultTHash
}
ts.h = ts.tHash.Hash()
ts.h.Reset()
ts.first = true
ts.sums = FileInfoSums{}
return nil
}
func (ts *tarSum) Read(buf []byte) (int, error) {
if ts.finished {
return ts.bufWriter.Read(buf)
}
if len(ts.bufData) < len(buf) {
switch {
case len(buf) <= buf8K:
ts.bufData = make([]byte, buf8K)
case len(buf) <= buf16K:
ts.bufData = make([]byte, buf16K)
case len(buf) <= buf32K:
ts.bufData = make([]byte, buf32K)
default:
ts.bufData = make([]byte, len(buf))
}
}
buf2 := ts.bufData[:len(buf)]
n, err := ts.tarR.Read(buf2)
if err != nil {
if err == io.EOF {
if _, err := ts.h.Write(buf2[:n]); err != nil {
return 0, err
}
if !ts.first {
ts.sums = append(ts.sums, fileInfoSum{name: ts.currentFile, sum: hex.EncodeToString(ts.h.Sum(nil)), pos: ts.fileCounter})
ts.fileCounter++
ts.h.Reset()
} else {
ts.first = false
}
currentHeader, err := ts.tarR.Next()
if err != nil {
if err == io.EOF {
if err := ts.tarW.Close(); err != nil {
return 0, err
}
if _, err := io.Copy(ts.writer, ts.bufTar); err != nil {
return 0, err
}
if err := ts.writer.Close(); err != nil {
return 0, err
}
ts.finished = true
return n, nil
}
return n, err
}
ts.currentFile = path.Clean(currentHeader.Name)
if err := ts.encodeHeader(currentHeader); err != nil {
return 0, err
}
if err := ts.tarW.WriteHeader(currentHeader); err != nil {
return 0, err
}
if _, err := ts.tarW.Write(buf2[:n]); err != nil {
return 0, err
}
ts.tarW.Flush()
if _, err := io.Copy(ts.writer, ts.bufTar); err != nil {
return 0, err
}
ts.writer.Flush()
return ts.bufWriter.Read(buf)
}
return n, err
}
// Filling the hash buffer
if _, err = ts.h.Write(buf2[:n]); err != nil {
return 0, err
}
// Filling the tar writer
if _, err = ts.tarW.Write(buf2[:n]); err != nil {
return 0, err
}
ts.tarW.Flush()
// Filling the output writer
if _, err = io.Copy(ts.writer, ts.bufTar); err != nil {
return 0, err
}
ts.writer.Flush()
return ts.bufWriter.Read(buf)
}
func (ts *tarSum) Sum(extra []byte) string {
ts.sums.SortBySums()
h := ts.tHash.Hash()
if extra != nil {
h.Write(extra)
}
for _, fis := range ts.sums {
h.Write([]byte(fis.Sum()))
}
checksum := ts.Version().String() + "+" + ts.tHash.Name() + ":" + hex.EncodeToString(h.Sum(nil))
return checksum
}
func (ts *tarSum) GetSums() FileInfoSums {
return ts.sums
}

View File

@ -1,158 +0,0 @@
package tarsum
import (
"archive/tar"
"errors"
"io"
"sort"
"strconv"
"strings"
)
// Version is used for versioning of the TarSum algorithm
// based on the prefix of the hash used
// i.e. "tarsum+sha256:e58fcf7418d4390dec8e8fb69d88c06ec07039d651fedd3aa72af9972e7d046b"
type Version int
// Prefix of "tarsum"
const (
Version0 Version = iota
Version1
// VersionDev this constant will be either the latest or an unsettled next-version of the TarSum calculation
VersionDev
)
// WriteV1Header writes a tar header to a writer in V1 tarsum format.
func WriteV1Header(h *tar.Header, w io.Writer) {
for _, elem := range v1TarHeaderSelect(h) {
w.Write([]byte(elem[0] + elem[1]))
}
}
// VersionLabelForChecksum returns the label for the given tarsum
// checksum, i.e., everything before the first `+` character in
// the string or an empty string if no label separator is found.
func VersionLabelForChecksum(checksum string) string {
// Checksums are in the form: {versionLabel}+{hashID}:{hex}
sepIndex := strings.Index(checksum, "+")
if sepIndex < 0 {
return ""
}
return checksum[:sepIndex]
}
// GetVersions gets a list of all known tarsum versions.
func GetVersions() []Version {
v := []Version{}
for k := range tarSumVersions {
v = append(v, k)
}
return v
}
var (
tarSumVersions = map[Version]string{
Version0: "tarsum",
Version1: "tarsum.v1",
VersionDev: "tarsum.dev",
}
tarSumVersionsByName = map[string]Version{
"tarsum": Version0,
"tarsum.v1": Version1,
"tarsum.dev": VersionDev,
}
)
func (tsv Version) String() string {
return tarSumVersions[tsv]
}
// GetVersionFromTarsum returns the Version from the provided string.
func GetVersionFromTarsum(tarsum string) (Version, error) {
tsv := tarsum
if strings.Contains(tarsum, "+") {
tsv = strings.SplitN(tarsum, "+", 2)[0]
}
for v, s := range tarSumVersions {
if s == tsv {
return v, nil
}
}
return -1, ErrNotVersion
}
// Errors that may be returned by functions in this package
var (
ErrNotVersion = errors.New("string does not include a TarSum Version")
ErrVersionNotImplemented = errors.New("TarSum Version is not yet implemented")
)
// tarHeaderSelector is the interface which different versions
// of tarsum should use for selecting and ordering tar headers
// for each item in the archive.
type tarHeaderSelector interface {
selectHeaders(h *tar.Header) (orderedHeaders [][2]string)
}
type tarHeaderSelectFunc func(h *tar.Header) (orderedHeaders [][2]string)
func (f tarHeaderSelectFunc) selectHeaders(h *tar.Header) (orderedHeaders [][2]string) {
return f(h)
}
func v0TarHeaderSelect(h *tar.Header) (orderedHeaders [][2]string) {
return [][2]string{
{"name", h.Name},
{"mode", strconv.FormatInt(h.Mode, 10)},
{"uid", strconv.Itoa(h.Uid)},
{"gid", strconv.Itoa(h.Gid)},
{"size", strconv.FormatInt(h.Size, 10)},
{"mtime", strconv.FormatInt(h.ModTime.UTC().Unix(), 10)},
{"typeflag", string([]byte{h.Typeflag})},
{"linkname", h.Linkname},
{"uname", h.Uname},
{"gname", h.Gname},
{"devmajor", strconv.FormatInt(h.Devmajor, 10)},
{"devminor", strconv.FormatInt(h.Devminor, 10)},
}
}
func v1TarHeaderSelect(h *tar.Header) (orderedHeaders [][2]string) {
// Get extended attributes.
xAttrKeys := make([]string, len(h.Xattrs))
for k := range h.Xattrs {
xAttrKeys = append(xAttrKeys, k)
}
sort.Strings(xAttrKeys)
// Make the slice with enough capacity to hold the 11 basic headers
// we want from the v0 selector plus however many xattrs we have.
orderedHeaders = make([][2]string, 0, 11+len(xAttrKeys))
// Copy all headers from v0 excluding the 'mtime' header (the 5th element).
v0headers := v0TarHeaderSelect(h)
orderedHeaders = append(orderedHeaders, v0headers[0:5]...)
orderedHeaders = append(orderedHeaders, v0headers[6:]...)
// Finally, append the sorted xattrs.
for _, k := range xAttrKeys {
orderedHeaders = append(orderedHeaders, [2]string{k, h.Xattrs[k]})
}
return
}
var registeredHeaderSelectors = map[Version]tarHeaderSelectFunc{
Version0: v0TarHeaderSelect,
Version1: v1TarHeaderSelect,
VersionDev: v1TarHeaderSelect,
}
func getTarHeaderSelector(v Version) (tarHeaderSelector, error) {
headerSelector, ok := registeredHeaderSelectors[v]
if !ok {
return nil, ErrVersionNotImplemented
}
return headerSelector, nil
}

View File

@ -1,22 +0,0 @@
package tarsum
import (
"io"
)
type writeCloseFlusher interface {
io.WriteCloser
Flush() error
}
type nopCloseFlusher struct {
io.Writer
}
func (n *nopCloseFlusher) Close() error {
return nil
}
func (n *nopCloseFlusher) Flush() error {
return nil
}

363
vendor/github.com/hashicorp/go-immutable-radix/LICENSE generated vendored Normal file
View File

@ -0,0 +1,363 @@
Mozilla Public License, version 2.0
1. Definitions
1.1. "Contributor"
means each individual or legal entity that creates, contributes to the
creation of, or owns Covered Software.
1.2. "Contributor Version"
means the combination of the Contributions of others (if any) used by a
Contributor and that particular Contributor's Contribution.
1.3. "Contribution"
means Covered Software of a particular Contributor.
1.4. "Covered Software"
means Source Code Form to which the initial Contributor has attached the
notice in Exhibit A, the Executable Form of such Source Code Form, and
Modifications of such Source Code Form, in each case including portions
thereof.
1.5. "Incompatible With Secondary Licenses"
means
a. that the initial Contributor has attached the notice described in
Exhibit B to the Covered Software; or
b. that the Covered Software was made available under the terms of
version 1.1 or earlier of the License, but not also under the terms of
a Secondary License.
1.6. "Executable Form"
means any form of the work other than Source Code Form.
1.7. "Larger Work"
means a work that combines Covered Software with other material, in a
separate file or files, that is not Covered Software.
1.8. "License"
means this document.
1.9. "Licensable"
means having the right to grant, to the maximum extent possible, whether
at the time of the initial grant or subsequently, any and all of the
rights conveyed by this License.
1.10. "Modifications"
means any of the following:
a. any file in Source Code Form that results from an addition to,
deletion from, or modification of the contents of Covered Software; or
b. any new file in Source Code Form that contains any Covered Software.
1.11. "Patent Claims" of a Contributor
means any patent claim(s), including without limitation, method,
process, and apparatus claims, in any patent Licensable by such
Contributor that would be infringed, but for the grant of the License,
by the making, using, selling, offering for sale, having made, import,
or transfer of either its Contributions or its Contributor Version.
1.12. "Secondary License"
means either the GNU General Public License, Version 2.0, the GNU Lesser
General Public License, Version 2.1, the GNU Affero General Public
License, Version 3.0, or any later versions of those licenses.
1.13. "Source Code Form"
means the form of the work preferred for making modifications.
1.14. "You" (or "Your")
means an individual or a legal entity exercising rights under this
License. For legal entities, "You" includes any entity that controls, is
controlled by, or is under common control with You. For purposes of this
definition, "control" means (a) the power, direct or indirect, to cause
the direction or management of such entity, whether by contract or
otherwise, or (b) ownership of more than fifty percent (50%) of the
outstanding shares or beneficial ownership of such entity.
2. License Grants and Conditions
2.1. Grants
Each Contributor hereby grants You a world-wide, royalty-free,
non-exclusive license:
a. under intellectual property rights (other than patent or trademark)
Licensable by such Contributor to use, reproduce, make available,
modify, display, perform, distribute, and otherwise exploit its
Contributions, either on an unmodified basis, with Modifications, or
as part of a Larger Work; and
b. under Patent Claims of such Contributor to make, use, sell, offer for
sale, have made, import, and otherwise transfer either its
Contributions or its Contributor Version.
2.2. Effective Date
The licenses granted in Section 2.1 with respect to any Contribution
become effective for each Contribution on the date the Contributor first
distributes such Contribution.
2.3. Limitations on Grant Scope
The licenses granted in this Section 2 are the only rights granted under
this License. No additional rights or licenses will be implied from the
distribution or licensing of Covered Software under this License.
Notwithstanding Section 2.1(b) above, no patent license is granted by a
Contributor:
a. for any code that a Contributor has removed from Covered Software; or
b. for infringements caused by: (i) Your and any other third party's
modifications of Covered Software, or (ii) the combination of its
Contributions with other software (except as part of its Contributor
Version); or
c. under Patent Claims infringed by Covered Software in the absence of
its Contributions.
This License does not grant any rights in the trademarks, service marks,
or logos of any Contributor (except as may be necessary to comply with
the notice requirements in Section 3.4).
2.4. Subsequent Licenses
No Contributor makes additional grants as a result of Your choice to
distribute the Covered Software under a subsequent version of this
License (see Section 10.2) or under the terms of a Secondary License (if
permitted under the terms of Section 3.3).
2.5. Representation
Each Contributor represents that the Contributor believes its
Contributions are its original creation(s) or it has sufficient rights to
grant the rights to its Contributions conveyed by this License.
2.6. Fair Use
This License is not intended to limit any rights You have under
applicable copyright doctrines of fair use, fair dealing, or other
equivalents.
2.7. Conditions
Sections 3.1, 3.2, 3.3, and 3.4 are conditions of the licenses granted in
Section 2.1.
3. Responsibilities
3.1. Distribution of Source Form
All distribution of Covered Software in Source Code Form, including any
Modifications that You create or to which You contribute, must be under
the terms of this License. You must inform recipients that the Source
Code Form of the Covered Software is governed by the terms of this
License, and how they can obtain a copy of this License. You may not
attempt to alter or restrict the recipients' rights in the Source Code
Form.
3.2. Distribution of Executable Form
If You distribute Covered Software in Executable Form then:
a. such Covered Software must also be made available in Source Code Form,
as described in Section 3.1, and You must inform recipients of the
Executable Form how they can obtain a copy of such Source Code Form by
reasonable means in a timely manner, at a charge no more than the cost
of distribution to the recipient; and
b. You may distribute such Executable Form under the terms of this
License, or sublicense it under different terms, provided that the
license for the Executable Form does not attempt to limit or alter the
recipients' rights in the Source Code Form under this License.
3.3. Distribution of a Larger Work
You may create and distribute a Larger Work under terms of Your choice,
provided that You also comply with the requirements of this License for
the Covered Software. If the Larger Work is a combination of Covered
Software with a work governed by one or more Secondary Licenses, and the
Covered Software is not Incompatible With Secondary Licenses, this
License permits You to additionally distribute such Covered Software
under the terms of such Secondary License(s), so that the recipient of
the Larger Work may, at their option, further distribute the Covered
Software under the terms of either this License or such Secondary
License(s).
3.4. Notices
You may not remove or alter the substance of any license notices
(including copyright notices, patent notices, disclaimers of warranty, or
limitations of liability) contained within the Source Code Form of the
Covered Software, except that You may alter any license notices to the
extent required to remedy known factual inaccuracies.
3.5. Application of Additional Terms
You may choose to offer, and to charge a fee for, warranty, support,
indemnity or liability obligations to one or more recipients of Covered
Software. However, You may do so only on Your own behalf, and not on
behalf of any Contributor. You must make it absolutely clear that any
such warranty, support, indemnity, or liability obligation is offered by
You alone, and You hereby agree to indemnify every Contributor for any
liability incurred by such Contributor as a result of warranty, support,
indemnity or liability terms You offer. You may include additional
disclaimers of warranty and limitations of liability specific to any
jurisdiction.
4. Inability to Comply Due to Statute or Regulation
If it is impossible for You to comply with any of the terms of this License
with respect to some or all of the Covered Software due to statute,
judicial order, or regulation then You must: (a) comply with the terms of
this License to the maximum extent possible; and (b) describe the
limitations and the code they affect. Such description must be placed in a
text file included with all distributions of the Covered Software under
this License. Except to the extent prohibited by statute or regulation,
such description must be sufficiently detailed for a recipient of ordinary
skill to be able to understand it.
5. Termination
5.1. The rights granted under this License will terminate automatically if You
fail to comply with any of its terms. However, if You become compliant,
then the rights granted under this License from a particular Contributor
are reinstated (a) provisionally, unless and until such Contributor
explicitly and finally terminates Your grants, and (b) on an ongoing
basis, if such Contributor fails to notify You of the non-compliance by
some reasonable means prior to 60 days after You have come back into
compliance. Moreover, Your grants from a particular Contributor are
reinstated on an ongoing basis if such Contributor notifies You of the
non-compliance by some reasonable means, this is the first time You have
received notice of non-compliance with this License from such
Contributor, and You become compliant prior to 30 days after Your receipt
of the notice.
5.2. If You initiate litigation against any entity by asserting a patent
infringement claim (excluding declaratory judgment actions,
counter-claims, and cross-claims) alleging that a Contributor Version
directly or indirectly infringes any patent, then the rights granted to
You by any and all Contributors for the Covered Software under Section
2.1 of this License shall terminate.
5.3. In the event of termination under Sections 5.1 or 5.2 above, all end user
license agreements (excluding distributors and resellers) which have been
validly granted by You or Your distributors under this License prior to
termination shall survive termination.
6. Disclaimer of Warranty
Covered Software is provided under this License on an "as is" basis,
without warranty of any kind, either expressed, implied, or statutory,
including, without limitation, warranties that the Covered Software is free
of defects, merchantable, fit for a particular purpose or non-infringing.
The entire risk as to the quality and performance of the Covered Software
is with You. Should any Covered Software prove defective in any respect,
You (not any Contributor) assume the cost of any necessary servicing,
repair, or correction. This disclaimer of warranty constitutes an essential
part of this License. No use of any Covered Software is authorized under
this License except under this disclaimer.
7. Limitation of Liability
Under no circumstances and under no legal theory, whether tort (including
negligence), contract, or otherwise, shall any Contributor, or anyone who
distributes Covered Software as permitted above, be liable to You for any
direct, indirect, special, incidental, or consequential damages of any
character including, without limitation, damages for lost profits, loss of
goodwill, work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses, even if such party shall have been
informed of the possibility of such damages. This limitation of liability
shall not apply to liability for death or personal injury resulting from
such party's negligence to the extent applicable law prohibits such
limitation. Some jurisdictions do not allow the exclusion or limitation of
incidental or consequential damages, so this exclusion and limitation may
not apply to You.
8. Litigation
Any litigation relating to this License may be brought only in the courts
of a jurisdiction where the defendant maintains its principal place of
business and such litigation shall be governed by laws of that
jurisdiction, without reference to its conflict-of-law provisions. Nothing
in this Section shall prevent a party's ability to bring cross-claims or
counter-claims.
9. Miscellaneous
This License represents the complete agreement concerning the subject
matter hereof. If any provision of this License is held to be
unenforceable, such provision shall be reformed only to the extent
necessary to make it enforceable. Any law or regulation which provides that
the language of a contract shall be construed against the drafter shall not
be used to construe this License against a Contributor.
10. Versions of the License
10.1. New Versions
Mozilla Foundation is the license steward. Except as provided in Section
10.3, no one other than the license steward has the right to modify or
publish new versions of this License. Each version will be given a
distinguishing version number.
10.2. Effect of New Versions
You may distribute the Covered Software under the terms of the version
of the License under which You originally received the Covered Software,
or under the terms of any subsequent version published by the license
steward.
10.3. Modified Versions
If you create software not governed by this License, and you want to
create a new license for such software, you may create and use a
modified version of this License if you rename the license and remove
any references to the name of the license steward (except to note that
such modified license differs from this License).
10.4. Distributing Source Code Form that is Incompatible With Secondary
Licenses If You choose to distribute Source Code Form that is
Incompatible With Secondary Licenses under the terms of this version of
the License, the notice described in Exhibit B of this License must be
attached.
Exhibit A - Source Code Form License Notice
This Source Code Form is subject to the
terms of the Mozilla Public License, v.
2.0. If a copy of the MPL was not
distributed with this file, You can
obtain one at
http://mozilla.org/MPL/2.0/.
If it is not possible or desirable to put the notice in a particular file,
then You may include the notice in a location (such as a LICENSE file in a
relevant directory) where a recipient would be likely to look for such a
notice.
You may add additional accurate notices of copyright ownership.
Exhibit B - "Incompatible With Secondary Licenses" Notice
This Source Code Form is "Incompatible
With Secondary Licenses", as defined by
the Mozilla Public License, v. 2.0.

View File

@ -0,0 +1,41 @@
go-immutable-radix [![Build Status](https://travis-ci.org/hashicorp/go-immutable-radix.png)](https://travis-ci.org/hashicorp/go-immutable-radix)
=========
Provides the `iradix` package that implements an immutable [radix tree](http://en.wikipedia.org/wiki/Radix_tree).
The package only provides a single `Tree` implementation, optimized for sparse nodes.
As a radix tree, it provides the following:
* O(k) operations. In many cases, this can be faster than a hash table since
the hash function is an O(k) operation, and hash tables have very poor cache locality.
* Minimum / Maximum value lookups
* Ordered iteration
A tree supports using a transaction to batch multiple updates (insert, delete)
in a more efficient manner than performing each operation one at a time.
For a mutable variant, see [go-radix](https://github.com/armon/go-radix).
Documentation
=============
The full documentation is available on [Godoc](http://godoc.org/github.com/hashicorp/go-immutable-radix).
Example
=======
Below is a simple example of usage
```go
// Create a tree
r := iradix.New()
r, _, _ = r.Insert([]byte("foo"), 1)
r, _, _ = r.Insert([]byte("bar"), 2)
r, _, _ = r.Insert([]byte("foobar"), 2)
// Find the longest prefix match
m, _, _ := r.Root().LongestPrefix([]byte("foozip"))
if string(m) != "foo" {
panic("should be foo")
}
```

View File

@ -0,0 +1,21 @@
package iradix
import "sort"
type edges []edge
func (e edges) Len() int {
return len(e)
}
func (e edges) Less(i, j int) bool {
return e[i].label < e[j].label
}
func (e edges) Swap(i, j int) {
e[i], e[j] = e[j], e[i]
}
func (e edges) Sort() {
sort.Sort(e)
}

View File

@ -0,0 +1,333 @@
package iradix
import (
"bytes"
"github.com/hashicorp/golang-lru/simplelru"
)
const (
// defaultModifiedCache is the default size of the modified node
// cache used per transaction. This is used to cache the updates
// to the nodes near the root, while the leaves do not need to be
// cached. This is important for very large transactions to prevent
// the modified cache from growing to be enormous.
defaultModifiedCache = 8192
)
// Tree implements an immutable radix tree. This can be treated as a
// Dictionary abstract data type. The main advantage over a standard
// hash map is prefix-based lookups and ordered iteration. The immutability
// means that it is safe to concurrently read from a Tree without any
// coordination.
type Tree struct {
root *Node
size int
}
// New returns an empty Tree
func New() *Tree {
t := &Tree{root: &Node{}}
return t
}
// Len is used to return the number of elements in the tree
func (t *Tree) Len() int {
return t.size
}
// Txn is a transaction on the tree. This transaction is applied
// atomically and returns a new tree when committed. A transaction
// is not thread safe, and should only be used by a single goroutine.
type Txn struct {
root *Node
size int
modified *simplelru.LRU
}
// Txn starts a new transaction that can be used to mutate the tree
func (t *Tree) Txn() *Txn {
txn := &Txn{
root: t.root,
size: t.size,
}
return txn
}
// writeNode returns a node to be modified, if the current
// node as already been modified during the course of
// the transaction, it is used in-place.
func (t *Txn) writeNode(n *Node) *Node {
// Ensure the modified set exists
if t.modified == nil {
lru, err := simplelru.NewLRU(defaultModifiedCache, nil)
if err != nil {
panic(err)
}
t.modified = lru
}
// If this node has already been modified, we can
// continue to use it during this transaction.
if _, ok := t.modified.Get(n); ok {
return n
}
// Copy the existing node
nc := new(Node)
if n.prefix != nil {
nc.prefix = make([]byte, len(n.prefix))
copy(nc.prefix, n.prefix)
}
if n.leaf != nil {
nc.leaf = new(leafNode)
*nc.leaf = *n.leaf
}
if len(n.edges) != 0 {
nc.edges = make([]edge, len(n.edges))
copy(nc.edges, n.edges)
}
// Mark this node as modified
t.modified.Add(n, nil)
return nc
}
// insert does a recursive insertion
func (t *Txn) insert(n *Node, k, search []byte, v interface{}) (*Node, interface{}, bool) {
// Handle key exhaution
if len(search) == 0 {
nc := t.writeNode(n)
if n.isLeaf() {
old := nc.leaf.val
nc.leaf.val = v
return nc, old, true
} else {
nc.leaf = &leafNode{
key: k,
val: v,
}
return nc, nil, false
}
}
// Look for the edge
idx, child := n.getEdge(search[0])
// No edge, create one
if child == nil {
e := edge{
label: search[0],
node: &Node{
leaf: &leafNode{
key: k,
val: v,
},
prefix: search,
},
}
nc := t.writeNode(n)
nc.addEdge(e)
return nc, nil, false
}
// Determine longest prefix of the search key on match
commonPrefix := longestPrefix(search, child.prefix)
if commonPrefix == len(child.prefix) {
search = search[commonPrefix:]
newChild, oldVal, didUpdate := t.insert(child, k, search, v)
if newChild != nil {
nc := t.writeNode(n)
nc.edges[idx].node = newChild
return nc, oldVal, didUpdate
}
return nil, oldVal, didUpdate
}
// Split the node
nc := t.writeNode(n)
splitNode := &Node{
prefix: search[:commonPrefix],
}
nc.replaceEdge(edge{
label: search[0],
node: splitNode,
})
// Restore the existing child node
modChild := t.writeNode(child)
splitNode.addEdge(edge{
label: modChild.prefix[commonPrefix],
node: modChild,
})
modChild.prefix = modChild.prefix[commonPrefix:]
// Create a new leaf node
leaf := &leafNode{
key: k,
val: v,
}
// If the new key is a subset, add to to this node
search = search[commonPrefix:]
if len(search) == 0 {
splitNode.leaf = leaf
return nc, nil, false
}
// Create a new edge for the node
splitNode.addEdge(edge{
label: search[0],
node: &Node{
leaf: leaf,
prefix: search,
},
})
return nc, nil, false
}
// delete does a recursive deletion
func (t *Txn) delete(parent, n *Node, search []byte) (*Node, *leafNode) {
// Check for key exhaution
if len(search) == 0 {
if !n.isLeaf() {
return nil, nil
}
// Remove the leaf node
nc := t.writeNode(n)
nc.leaf = nil
// Check if this node should be merged
if n != t.root && len(nc.edges) == 1 {
nc.mergeChild()
}
return nc, n.leaf
}
// Look for an edge
label := search[0]
idx, child := n.getEdge(label)
if child == nil || !bytes.HasPrefix(search, child.prefix) {
return nil, nil
}
// Consume the search prefix
search = search[len(child.prefix):]
newChild, leaf := t.delete(n, child, search)
if newChild == nil {
return nil, nil
}
// Copy this node
nc := t.writeNode(n)
// Delete the edge if the node has no edges
if newChild.leaf == nil && len(newChild.edges) == 0 {
nc.delEdge(label)
if n != t.root && len(nc.edges) == 1 && !nc.isLeaf() {
nc.mergeChild()
}
} else {
nc.edges[idx].node = newChild
}
return nc, leaf
}
// Insert is used to add or update a given key. The return provides
// the previous value and a bool indicating if any was set.
func (t *Txn) Insert(k []byte, v interface{}) (interface{}, bool) {
newRoot, oldVal, didUpdate := t.insert(t.root, k, k, v)
if newRoot != nil {
t.root = newRoot
}
if !didUpdate {
t.size++
}
return oldVal, didUpdate
}
// Delete is used to delete a given key. Returns the old value if any,
// and a bool indicating if the key was set.
func (t *Txn) Delete(k []byte) (interface{}, bool) {
newRoot, leaf := t.delete(nil, t.root, k)
if newRoot != nil {
t.root = newRoot
}
if leaf != nil {
t.size--
return leaf.val, true
}
return nil, false
}
// Root returns the current root of the radix tree within this
// transaction. The root is not safe across insert and delete operations,
// but can be used to read the current state during a transaction.
func (t *Txn) Root() *Node {
return t.root
}
// Get is used to lookup a specific key, returning
// the value and if it was found
func (t *Txn) Get(k []byte) (interface{}, bool) {
return t.root.Get(k)
}
// Commit is used to finalize the transaction and return a new tree
func (t *Txn) Commit() *Tree {
t.modified = nil
return &Tree{t.root, t.size}
}
// Insert is used to add or update a given key. The return provides
// the new tree, previous value and a bool indicating if any was set.
func (t *Tree) Insert(k []byte, v interface{}) (*Tree, interface{}, bool) {
txn := t.Txn()
old, ok := txn.Insert(k, v)
return txn.Commit(), old, ok
}
// Delete is used to delete a given key. Returns the new tree,
// old value if any, and a bool indicating if the key was set.
func (t *Tree) Delete(k []byte) (*Tree, interface{}, bool) {
txn := t.Txn()
old, ok := txn.Delete(k)
return txn.Commit(), old, ok
}
// Root returns the root node of the tree which can be used for richer
// query operations.
func (t *Tree) Root() *Node {
return t.root
}
// Get is used to lookup a specific key, returning
// the value and if it was found
func (t *Tree) Get(k []byte) (interface{}, bool) {
return t.root.Get(k)
}
// longestPrefix finds the length of the shared prefix
// of two strings
func longestPrefix(k1, k2 []byte) int {
max := len(k1)
if l := len(k2); l < max {
max = l
}
var i int
for i = 0; i < max; i++ {
if k1[i] != k2[i] {
break
}
}
return i
}
// concat two byte slices, returning a third new copy
func concat(a, b []byte) []byte {
c := make([]byte, len(a)+len(b))
copy(c, a)
copy(c[len(a):], b)
return c
}

81
vendor/github.com/hashicorp/go-immutable-radix/iter.go generated vendored Normal file
View File

@ -0,0 +1,81 @@
package iradix
import "bytes"
// Iterator is used to iterate over a set of nodes
// in pre-order
type Iterator struct {
node *Node
stack []edges
}
// SeekPrefix is used to seek the iterator to a given prefix
func (i *Iterator) SeekPrefix(prefix []byte) {
// Wipe the stack
i.stack = nil
n := i.node
search := prefix
for {
// Check for key exhaution
if len(search) == 0 {
i.node = n
return
}
// Look for an edge
_, n = n.getEdge(search[0])
if n == nil {
i.node = nil
return
}
// Consume the search prefix
if bytes.HasPrefix(search, n.prefix) {
search = search[len(n.prefix):]
} else if bytes.HasPrefix(n.prefix, search) {
i.node = n
return
} else {
i.node = nil
return
}
}
}
// Next returns the next node in order
func (i *Iterator) Next() ([]byte, interface{}, bool) {
// Initialize our stack if needed
if i.stack == nil && i.node != nil {
i.stack = []edges{
edges{
edge{node: i.node},
},
}
}
for len(i.stack) > 0 {
// Inspect the last element of the stack
n := len(i.stack)
last := i.stack[n-1]
elem := last[0].node
// Update the stack
if len(last) > 1 {
i.stack[n-1] = last[1:]
} else {
i.stack = i.stack[:n-1]
}
// Push the edges onto the frontier
if len(elem.edges) > 0 {
i.stack = append(i.stack, elem.edges)
}
// Return the leaf values if any
if elem.leaf != nil {
return elem.leaf.key, elem.leaf.val, true
}
}
return nil, nil, false
}

289
vendor/github.com/hashicorp/go-immutable-radix/node.go generated vendored Normal file
View File

@ -0,0 +1,289 @@
package iradix
import (
"bytes"
"sort"
)
// WalkFn is used when walking the tree. Takes a
// key and value, returning if iteration should
// be terminated.
type WalkFn func(k []byte, v interface{}) bool
// leafNode is used to represent a value
type leafNode struct {
key []byte
val interface{}
}
// edge is used to represent an edge node
type edge struct {
label byte
node *Node
}
// Node is an immutable node in the radix tree
type Node struct {
// leaf is used to store possible leaf
leaf *leafNode
// prefix is the common prefix we ignore
prefix []byte
// Edges should be stored in-order for iteration.
// We avoid a fully materialized slice to save memory,
// since in most cases we expect to be sparse
edges edges
}
func (n *Node) isLeaf() bool {
return n.leaf != nil
}
func (n *Node) addEdge(e edge) {
num := len(n.edges)
idx := sort.Search(num, func(i int) bool {
return n.edges[i].label >= e.label
})
n.edges = append(n.edges, e)
if idx != num {
copy(n.edges[idx+1:], n.edges[idx:num])
n.edges[idx] = e
}
}
func (n *Node) replaceEdge(e edge) {
num := len(n.edges)
idx := sort.Search(num, func(i int) bool {
return n.edges[i].label >= e.label
})
if idx < num && n.edges[idx].label == e.label {
n.edges[idx].node = e.node
return
}
panic("replacing missing edge")
}
func (n *Node) getEdge(label byte) (int, *Node) {
num := len(n.edges)
idx := sort.Search(num, func(i int) bool {
return n.edges[i].label >= label
})
if idx < num && n.edges[idx].label == label {
return idx, n.edges[idx].node
}
return -1, nil
}
func (n *Node) delEdge(label byte) {
num := len(n.edges)
idx := sort.Search(num, func(i int) bool {
return n.edges[i].label >= label
})
if idx < num && n.edges[idx].label == label {
copy(n.edges[idx:], n.edges[idx+1:])
n.edges[len(n.edges)-1] = edge{}
n.edges = n.edges[:len(n.edges)-1]
}
}
func (n *Node) mergeChild() {
e := n.edges[0]
child := e.node
n.prefix = concat(n.prefix, child.prefix)
if child.leaf != nil {
n.leaf = new(leafNode)
*n.leaf = *child.leaf
} else {
n.leaf = nil
}
if len(child.edges) != 0 {
n.edges = make([]edge, len(child.edges))
copy(n.edges, child.edges)
} else {
n.edges = nil
}
}
func (n *Node) Get(k []byte) (interface{}, bool) {
search := k
for {
// Check for key exhaution
if len(search) == 0 {
if n.isLeaf() {
return n.leaf.val, true
}
break
}
// Look for an edge
_, n = n.getEdge(search[0])
if n == nil {
break
}
// Consume the search prefix
if bytes.HasPrefix(search, n.prefix) {
search = search[len(n.prefix):]
} else {
break
}
}
return nil, false
}
// LongestPrefix is like Get, but instead of an
// exact match, it will return the longest prefix match.
func (n *Node) LongestPrefix(k []byte) ([]byte, interface{}, bool) {
var last *leafNode
search := k
for {
// Look for a leaf node
if n.isLeaf() {
last = n.leaf
}
// Check for key exhaution
if len(search) == 0 {
break
}
// Look for an edge
_, n = n.getEdge(search[0])
if n == nil {
break
}
// Consume the search prefix
if bytes.HasPrefix(search, n.prefix) {
search = search[len(n.prefix):]
} else {
break
}
}
if last != nil {
return last.key, last.val, true
}
return nil, nil, false
}
// Minimum is used to return the minimum value in the tree
func (n *Node) Minimum() ([]byte, interface{}, bool) {
for {
if n.isLeaf() {
return n.leaf.key, n.leaf.val, true
}
if len(n.edges) > 0 {
n = n.edges[0].node
} else {
break
}
}
return nil, nil, false
}
// Maximum is used to return the maximum value in the tree
func (n *Node) Maximum() ([]byte, interface{}, bool) {
for {
if num := len(n.edges); num > 0 {
n = n.edges[num-1].node
continue
}
if n.isLeaf() {
return n.leaf.key, n.leaf.val, true
} else {
break
}
}
return nil, nil, false
}
// Iterator is used to return an iterator at
// the given node to walk the tree
func (n *Node) Iterator() *Iterator {
return &Iterator{node: n}
}
// Walk is used to walk the tree
func (n *Node) Walk(fn WalkFn) {
recursiveWalk(n, fn)
}
// WalkPrefix is used to walk the tree under a prefix
func (n *Node) WalkPrefix(prefix []byte, fn WalkFn) {
search := prefix
for {
// Check for key exhaution
if len(search) == 0 {
recursiveWalk(n, fn)
return
}
// Look for an edge
_, n = n.getEdge(search[0])
if n == nil {
break
}
// Consume the search prefix
if bytes.HasPrefix(search, n.prefix) {
search = search[len(n.prefix):]
} else if bytes.HasPrefix(n.prefix, search) {
// Child may be under our search prefix
recursiveWalk(n, fn)
return
} else {
break
}
}
}
// WalkPath is used to walk the tree, but only visiting nodes
// from the root down to a given leaf. Where WalkPrefix walks
// all the entries *under* the given prefix, this walks the
// entries *above* the given prefix.
func (n *Node) WalkPath(path []byte, fn WalkFn) {
search := path
for {
// Visit the leaf values if any
if n.leaf != nil && fn(n.leaf.key, n.leaf.val) {
return
}
// Check for key exhaution
if len(search) == 0 {
return
}
// Look for an edge
_, n = n.getEdge(search[0])
if n == nil {
return
}
// Consume the search prefix
if bytes.HasPrefix(search, n.prefix) {
search = search[len(n.prefix):]
} else {
break
}
}
}
// recursiveWalk is used to do a pre-order walk of a node
// recursively. Returns true if the walk should be aborted
func recursiveWalk(n *Node, fn WalkFn) bool {
// Visit the leaf values if any
if n.leaf != nil && fn(n.leaf.key, n.leaf.val) {
return true
}
// Recurse on the children
for _, e := range n.edges {
if recursiveWalk(e.node, fn) {
return true
}
}
return false
}

362
vendor/github.com/hashicorp/golang-lru/LICENSE generated vendored Normal file
View File

@ -0,0 +1,362 @@
Mozilla Public License, version 2.0
1. Definitions
1.1. "Contributor"
means each individual or legal entity that creates, contributes to the
creation of, or owns Covered Software.
1.2. "Contributor Version"
means the combination of the Contributions of others (if any) used by a
Contributor and that particular Contributor's Contribution.
1.3. "Contribution"
means Covered Software of a particular Contributor.
1.4. "Covered Software"
means Source Code Form to which the initial Contributor has attached the
notice in Exhibit A, the Executable Form of such Source Code Form, and
Modifications of such Source Code Form, in each case including portions
thereof.
1.5. "Incompatible With Secondary Licenses"
means
a. that the initial Contributor has attached the notice described in
Exhibit B to the Covered Software; or
b. that the Covered Software was made available under the terms of
version 1.1 or earlier of the License, but not also under the terms of
a Secondary License.
1.6. "Executable Form"
means any form of the work other than Source Code Form.
1.7. "Larger Work"
means a work that combines Covered Software with other material, in a
separate file or files, that is not Covered Software.
1.8. "License"
means this document.
1.9. "Licensable"
means having the right to grant, to the maximum extent possible, whether
at the time of the initial grant or subsequently, any and all of the
rights conveyed by this License.
1.10. "Modifications"
means any of the following:
a. any file in Source Code Form that results from an addition to,
deletion from, or modification of the contents of Covered Software; or
b. any new file in Source Code Form that contains any Covered Software.
1.11. "Patent Claims" of a Contributor
means any patent claim(s), including without limitation, method,
process, and apparatus claims, in any patent Licensable by such
Contributor that would be infringed, but for the grant of the License,
by the making, using, selling, offering for sale, having made, import,
or transfer of either its Contributions or its Contributor Version.
1.12. "Secondary License"
means either the GNU General Public License, Version 2.0, the GNU Lesser
General Public License, Version 2.1, the GNU Affero General Public
License, Version 3.0, or any later versions of those licenses.
1.13. "Source Code Form"
means the form of the work preferred for making modifications.
1.14. "You" (or "Your")
means an individual or a legal entity exercising rights under this
License. For legal entities, "You" includes any entity that controls, is
controlled by, or is under common control with You. For purposes of this
definition, "control" means (a) the power, direct or indirect, to cause
the direction or management of such entity, whether by contract or
otherwise, or (b) ownership of more than fifty percent (50%) of the
outstanding shares or beneficial ownership of such entity.
2. License Grants and Conditions
2.1. Grants
Each Contributor hereby grants You a world-wide, royalty-free,
non-exclusive license:
a. under intellectual property rights (other than patent or trademark)
Licensable by such Contributor to use, reproduce, make available,
modify, display, perform, distribute, and otherwise exploit its
Contributions, either on an unmodified basis, with Modifications, or
as part of a Larger Work; and
b. under Patent Claims of such Contributor to make, use, sell, offer for
sale, have made, import, and otherwise transfer either its
Contributions or its Contributor Version.
2.2. Effective Date
The licenses granted in Section 2.1 with respect to any Contribution
become effective for each Contribution on the date the Contributor first
distributes such Contribution.
2.3. Limitations on Grant Scope
The licenses granted in this Section 2 are the only rights granted under
this License. No additional rights or licenses will be implied from the
distribution or licensing of Covered Software under this License.
Notwithstanding Section 2.1(b) above, no patent license is granted by a
Contributor:
a. for any code that a Contributor has removed from Covered Software; or
b. for infringements caused by: (i) Your and any other third party's
modifications of Covered Software, or (ii) the combination of its
Contributions with other software (except as part of its Contributor
Version); or
c. under Patent Claims infringed by Covered Software in the absence of
its Contributions.
This License does not grant any rights in the trademarks, service marks,
or logos of any Contributor (except as may be necessary to comply with
the notice requirements in Section 3.4).
2.4. Subsequent Licenses
No Contributor makes additional grants as a result of Your choice to
distribute the Covered Software under a subsequent version of this
License (see Section 10.2) or under the terms of a Secondary License (if
permitted under the terms of Section 3.3).
2.5. Representation
Each Contributor represents that the Contributor believes its
Contributions are its original creation(s) or it has sufficient rights to
grant the rights to its Contributions conveyed by this License.
2.6. Fair Use
This License is not intended to limit any rights You have under
applicable copyright doctrines of fair use, fair dealing, or other
equivalents.
2.7. Conditions
Sections 3.1, 3.2, 3.3, and 3.4 are conditions of the licenses granted in
Section 2.1.
3. Responsibilities
3.1. Distribution of Source Form
All distribution of Covered Software in Source Code Form, including any
Modifications that You create or to which You contribute, must be under
the terms of this License. You must inform recipients that the Source
Code Form of the Covered Software is governed by the terms of this
License, and how they can obtain a copy of this License. You may not
attempt to alter or restrict the recipients' rights in the Source Code
Form.
3.2. Distribution of Executable Form
If You distribute Covered Software in Executable Form then:
a. such Covered Software must also be made available in Source Code Form,
as described in Section 3.1, and You must inform recipients of the
Executable Form how they can obtain a copy of such Source Code Form by
reasonable means in a timely manner, at a charge no more than the cost
of distribution to the recipient; and
b. You may distribute such Executable Form under the terms of this
License, or sublicense it under different terms, provided that the
license for the Executable Form does not attempt to limit or alter the
recipients' rights in the Source Code Form under this License.
3.3. Distribution of a Larger Work
You may create and distribute a Larger Work under terms of Your choice,
provided that You also comply with the requirements of this License for
the Covered Software. If the Larger Work is a combination of Covered
Software with a work governed by one or more Secondary Licenses, and the
Covered Software is not Incompatible With Secondary Licenses, this
License permits You to additionally distribute such Covered Software
under the terms of such Secondary License(s), so that the recipient of
the Larger Work may, at their option, further distribute the Covered
Software under the terms of either this License or such Secondary
License(s).
3.4. Notices
You may not remove or alter the substance of any license notices
(including copyright notices, patent notices, disclaimers of warranty, or
limitations of liability) contained within the Source Code Form of the
Covered Software, except that You may alter any license notices to the
extent required to remedy known factual inaccuracies.
3.5. Application of Additional Terms
You may choose to offer, and to charge a fee for, warranty, support,
indemnity or liability obligations to one or more recipients of Covered
Software. However, You may do so only on Your own behalf, and not on
behalf of any Contributor. You must make it absolutely clear that any
such warranty, support, indemnity, or liability obligation is offered by
You alone, and You hereby agree to indemnify every Contributor for any
liability incurred by such Contributor as a result of warranty, support,
indemnity or liability terms You offer. You may include additional
disclaimers of warranty and limitations of liability specific to any
jurisdiction.
4. Inability to Comply Due to Statute or Regulation
If it is impossible for You to comply with any of the terms of this License
with respect to some or all of the Covered Software due to statute,
judicial order, or regulation then You must: (a) comply with the terms of
this License to the maximum extent possible; and (b) describe the
limitations and the code they affect. Such description must be placed in a
text file included with all distributions of the Covered Software under
this License. Except to the extent prohibited by statute or regulation,
such description must be sufficiently detailed for a recipient of ordinary
skill to be able to understand it.
5. Termination
5.1. The rights granted under this License will terminate automatically if You
fail to comply with any of its terms. However, if You become compliant,
then the rights granted under this License from a particular Contributor
are reinstated (a) provisionally, unless and until such Contributor
explicitly and finally terminates Your grants, and (b) on an ongoing
basis, if such Contributor fails to notify You of the non-compliance by
some reasonable means prior to 60 days after You have come back into
compliance. Moreover, Your grants from a particular Contributor are
reinstated on an ongoing basis if such Contributor notifies You of the
non-compliance by some reasonable means, this is the first time You have
received notice of non-compliance with this License from such
Contributor, and You become compliant prior to 30 days after Your receipt
of the notice.
5.2. If You initiate litigation against any entity by asserting a patent
infringement claim (excluding declaratory judgment actions,
counter-claims, and cross-claims) alleging that a Contributor Version
directly or indirectly infringes any patent, then the rights granted to
You by any and all Contributors for the Covered Software under Section
2.1 of this License shall terminate.
5.3. In the event of termination under Sections 5.1 or 5.2 above, all end user
license agreements (excluding distributors and resellers) which have been
validly granted by You or Your distributors under this License prior to
termination shall survive termination.
6. Disclaimer of Warranty
Covered Software is provided under this License on an "as is" basis,
without warranty of any kind, either expressed, implied, or statutory,
including, without limitation, warranties that the Covered Software is free
of defects, merchantable, fit for a particular purpose or non-infringing.
The entire risk as to the quality and performance of the Covered Software
is with You. Should any Covered Software prove defective in any respect,
You (not any Contributor) assume the cost of any necessary servicing,
repair, or correction. This disclaimer of warranty constitutes an essential
part of this License. No use of any Covered Software is authorized under
this License except under this disclaimer.
7. Limitation of Liability
Under no circumstances and under no legal theory, whether tort (including
negligence), contract, or otherwise, shall any Contributor, or anyone who
distributes Covered Software as permitted above, be liable to You for any
direct, indirect, special, incidental, or consequential damages of any
character including, without limitation, damages for lost profits, loss of
goodwill, work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses, even if such party shall have been
informed of the possibility of such damages. This limitation of liability
shall not apply to liability for death or personal injury resulting from
such party's negligence to the extent applicable law prohibits such
limitation. Some jurisdictions do not allow the exclusion or limitation of
incidental or consequential damages, so this exclusion and limitation may
not apply to You.
8. Litigation
Any litigation relating to this License may be brought only in the courts
of a jurisdiction where the defendant maintains its principal place of
business and such litigation shall be governed by laws of that
jurisdiction, without reference to its conflict-of-law provisions. Nothing
in this Section shall prevent a party's ability to bring cross-claims or
counter-claims.
9. Miscellaneous
This License represents the complete agreement concerning the subject
matter hereof. If any provision of this License is held to be
unenforceable, such provision shall be reformed only to the extent
necessary to make it enforceable. Any law or regulation which provides that
the language of a contract shall be construed against the drafter shall not
be used to construe this License against a Contributor.
10. Versions of the License
10.1. New Versions
Mozilla Foundation is the license steward. Except as provided in Section
10.3, no one other than the license steward has the right to modify or
publish new versions of this License. Each version will be given a
distinguishing version number.
10.2. Effect of New Versions
You may distribute the Covered Software under the terms of the version
of the License under which You originally received the Covered Software,
or under the terms of any subsequent version published by the license
steward.
10.3. Modified Versions
If you create software not governed by this License, and you want to
create a new license for such software, you may create and use a
modified version of this License if you rename the license and remove
any references to the name of the license steward (except to note that
such modified license differs from this License).
10.4. Distributing Source Code Form that is Incompatible With Secondary
Licenses If You choose to distribute Source Code Form that is
Incompatible With Secondary Licenses under the terms of this version of
the License, the notice described in Exhibit B of this License must be
attached.
Exhibit A - Source Code Form License Notice
This Source Code Form is subject to the
terms of the Mozilla Public License, v.
2.0. If a copy of the MPL was not
distributed with this file, You can
obtain one at
http://mozilla.org/MPL/2.0/.
If it is not possible or desirable to put the notice in a particular file,
then You may include the notice in a location (such as a LICENSE file in a
relevant directory) where a recipient would be likely to look for such a
notice.
You may add additional accurate notices of copyright ownership.
Exhibit B - "Incompatible With Secondary Licenses" Notice
This Source Code Form is "Incompatible
With Secondary Licenses", as defined by
the Mozilla Public License, v. 2.0.

25
vendor/github.com/hashicorp/golang-lru/README.md generated vendored Normal file
View File

@ -0,0 +1,25 @@
golang-lru
==========
This provides the `lru` package which implements a fixed-size
thread safe LRU cache. It is based on the cache in Groupcache.
Documentation
=============
Full docs are available on [Godoc](http://godoc.org/github.com/hashicorp/golang-lru)
Example
=======
Using the LRU is very simple:
```go
l, _ := New(128)
for i := 0; i < 256; i++ {
l.Add(i, nil)
}
if l.Len() != 128 {
panic(fmt.Sprintf("bad len: %v", l.Len()))
}
```

160
vendor/github.com/hashicorp/golang-lru/simplelru/lru.go generated vendored Normal file
View File

@ -0,0 +1,160 @@
package simplelru
import (
"container/list"
"errors"
)
// EvictCallback is used to get a callback when a cache entry is evicted
type EvictCallback func(key interface{}, value interface{})
// LRU implements a non-thread safe fixed size LRU cache
type LRU struct {
size int
evictList *list.List
items map[interface{}]*list.Element
onEvict EvictCallback
}
// entry is used to hold a value in the evictList
type entry struct {
key interface{}
value interface{}
}
// NewLRU constructs an LRU of the given size
func NewLRU(size int, onEvict EvictCallback) (*LRU, error) {
if size <= 0 {
return nil, errors.New("Must provide a positive size")
}
c := &LRU{
size: size,
evictList: list.New(),
items: make(map[interface{}]*list.Element),
onEvict: onEvict,
}
return c, nil
}
// Purge is used to completely clear the cache
func (c *LRU) Purge() {
for k, v := range c.items {
if c.onEvict != nil {
c.onEvict(k, v.Value.(*entry).value)
}
delete(c.items, k)
}
c.evictList.Init()
}
// Add adds a value to the cache. Returns true if an eviction occured.
func (c *LRU) Add(key, value interface{}) bool {
// Check for existing item
if ent, ok := c.items[key]; ok {
c.evictList.MoveToFront(ent)
ent.Value.(*entry).value = value
return false
}
// Add new item
ent := &entry{key, value}
entry := c.evictList.PushFront(ent)
c.items[key] = entry
evict := c.evictList.Len() > c.size
// Verify size not exceeded
if evict {
c.removeOldest()
}
return evict
}
// Get looks up a key's value from the cache.
func (c *LRU) Get(key interface{}) (value interface{}, ok bool) {
if ent, ok := c.items[key]; ok {
c.evictList.MoveToFront(ent)
return ent.Value.(*entry).value, true
}
return
}
// Check if a key is in the cache, without updating the recent-ness
// or deleting it for being stale.
func (c *LRU) Contains(key interface{}) (ok bool) {
_, ok = c.items[key]
return ok
}
// Returns the key value (or undefined if not found) without updating
// the "recently used"-ness of the key.
func (c *LRU) Peek(key interface{}) (value interface{}, ok bool) {
if ent, ok := c.items[key]; ok {
return ent.Value.(*entry).value, true
}
return nil, ok
}
// Remove removes the provided key from the cache, returning if the
// key was contained.
func (c *LRU) Remove(key interface{}) bool {
if ent, ok := c.items[key]; ok {
c.removeElement(ent)
return true
}
return false
}
// RemoveOldest removes the oldest item from the cache.
func (c *LRU) RemoveOldest() (interface{}, interface{}, bool) {
ent := c.evictList.Back()
if ent != nil {
c.removeElement(ent)
kv := ent.Value.(*entry)
return kv.key, kv.value, true
}
return nil, nil, false
}
// GetOldest returns the oldest entry
func (c *LRU) GetOldest() (interface{}, interface{}, bool) {
ent := c.evictList.Back()
if ent != nil {
kv := ent.Value.(*entry)
return kv.key, kv.value, true
}
return nil, nil, false
}
// Keys returns a slice of the keys in the cache, from oldest to newest.
func (c *LRU) Keys() []interface{} {
keys := make([]interface{}, len(c.items))
i := 0
for ent := c.evictList.Back(); ent != nil; ent = ent.Prev() {
keys[i] = ent.Value.(*entry).key
i++
}
return keys
}
// Len returns the number of items in the cache.
func (c *LRU) Len() int {
return c.evictList.Len()
}
// removeOldest removes the oldest item from the cache.
func (c *LRU) removeOldest() {
ent := c.evictList.Back()
if ent != nil {
c.removeElement(ent)
}
}
// removeElement is used to remove a given list element from the cache
func (c *LRU) removeElement(e *list.Element) {
c.evictList.Remove(e)
kv := e.Value.(*entry)
delete(c.items, kv.key)
if c.onEvict != nil {
c.onEvict(kv.key, kv.value)
}
}

View File

@ -1,111 +0,0 @@
package user
import (
"errors"
"golang.org/x/sys/unix"
)
var (
// The current operating system does not provide the required data for user lookups.
ErrUnsupported = errors.New("user lookup: operating system does not provide passwd-formatted data")
// No matching entries found in file.
ErrNoPasswdEntries = errors.New("no matching entries in passwd file")
ErrNoGroupEntries = errors.New("no matching entries in group file")
)
func lookupUser(filter func(u User) bool) (User, error) {
// Get operating system-specific passwd reader-closer.
passwd, err := GetPasswd()
if err != nil {
return User{}, err
}
defer passwd.Close()
// Get the users.
users, err := ParsePasswdFilter(passwd, filter)
if err != nil {
return User{}, err
}
// No user entries found.
if len(users) == 0 {
return User{}, ErrNoPasswdEntries
}
// Assume the first entry is the "correct" one.
return users[0], nil
}
// CurrentUser looks up the current user by their user id in /etc/passwd. If the
// user cannot be found (or there is no /etc/passwd file on the filesystem),
// then CurrentUser returns an error.
func CurrentUser() (User, error) {
return LookupUid(unix.Getuid())
}
// LookupUser looks up a user by their username in /etc/passwd. If the user
// cannot be found (or there is no /etc/passwd file on the filesystem), then
// LookupUser returns an error.
func LookupUser(username string) (User, error) {
return lookupUser(func(u User) bool {
return u.Name == username
})
}
// LookupUid looks up a user by their user id in /etc/passwd. If the user cannot
// be found (or there is no /etc/passwd file on the filesystem), then LookupId
// returns an error.
func LookupUid(uid int) (User, error) {
return lookupUser(func(u User) bool {
return u.Uid == uid
})
}
func lookupGroup(filter func(g Group) bool) (Group, error) {
// Get operating system-specific group reader-closer.
group, err := GetGroup()
if err != nil {
return Group{}, err
}
defer group.Close()
// Get the users.
groups, err := ParseGroupFilter(group, filter)
if err != nil {
return Group{}, err
}
// No user entries found.
if len(groups) == 0 {
return Group{}, ErrNoGroupEntries
}
// Assume the first entry is the "correct" one.
return groups[0], nil
}
// CurrentGroup looks up the current user's group by their primary group id's
// entry in /etc/passwd. If the group cannot be found (or there is no
// /etc/group file on the filesystem), then CurrentGroup returns an error.
func CurrentGroup() (Group, error) {
return LookupGid(unix.Getgid())
}
// LookupGroup looks up a group by its name in /etc/group. If the group cannot
// be found (or there is no /etc/group file on the filesystem), then LookupGroup
// returns an error.
func LookupGroup(groupname string) (Group, error) {
return lookupGroup(func(g Group) bool {
return g.Name == groupname
})
}
// LookupGid looks up a group by its group id in /etc/group. If the group cannot
// be found (or there is no /etc/group file on the filesystem), then LookupGid
// returns an error.
func LookupGid(gid int) (Group, error) {
return lookupGroup(func(g Group) bool {
return g.Gid == gid
})
}

View File

@ -1,30 +0,0 @@
// +build darwin dragonfly freebsd linux netbsd openbsd solaris
package user
import (
"io"
"os"
)
// Unix-specific path to the passwd and group formatted files.
const (
unixPasswdPath = "/etc/passwd"
unixGroupPath = "/etc/group"
)
func GetPasswdPath() (string, error) {
return unixPasswdPath, nil
}
func GetPasswd() (io.ReadCloser, error) {
return os.Open(unixPasswdPath)
}
func GetGroupPath() (string, error) {
return unixGroupPath, nil
}
func GetGroup() (io.ReadCloser, error) {
return os.Open(unixGroupPath)
}

View File

@ -1,21 +0,0 @@
// +build !darwin,!dragonfly,!freebsd,!linux,!netbsd,!openbsd,!solaris
package user
import "io"
func GetPasswdPath() (string, error) {
return "", ErrUnsupported
}
func GetPasswd() (io.ReadCloser, error) {
return nil, ErrUnsupported
}
func GetGroupPath() (string, error) {
return "", ErrUnsupported
}
func GetGroup() (io.ReadCloser, error) {
return nil, ErrUnsupported
}

View File

@ -1,441 +0,0 @@
package user
import (
"bufio"
"fmt"
"io"
"os"
"strconv"
"strings"
)
const (
minId = 0
maxId = 1<<31 - 1 //for 32-bit systems compatibility
)
var (
ErrRange = fmt.Errorf("uids and gids must be in range %d-%d", minId, maxId)
)
type User struct {
Name string
Pass string
Uid int
Gid int
Gecos string
Home string
Shell string
}
type Group struct {
Name string
Pass string
Gid int
List []string
}
func parseLine(line string, v ...interface{}) {
if line == "" {
return
}
parts := strings.Split(line, ":")
for i, p := range parts {
// Ignore cases where we don't have enough fields to populate the arguments.
// Some configuration files like to misbehave.
if len(v) <= i {
break
}
// Use the type of the argument to figure out how to parse it, scanf() style.
// This is legit.
switch e := v[i].(type) {
case *string:
*e = p
case *int:
// "numbers", with conversion errors ignored because of some misbehaving configuration files.
*e, _ = strconv.Atoi(p)
case *[]string:
// Comma-separated lists.
if p != "" {
*e = strings.Split(p, ",")
} else {
*e = []string{}
}
default:
// Someone goof'd when writing code using this function. Scream so they can hear us.
panic(fmt.Sprintf("parseLine only accepts {*string, *int, *[]string} as arguments! %#v is not a pointer!", e))
}
}
}
func ParsePasswdFile(path string) ([]User, error) {
passwd, err := os.Open(path)
if err != nil {
return nil, err
}
defer passwd.Close()
return ParsePasswd(passwd)
}
func ParsePasswd(passwd io.Reader) ([]User, error) {
return ParsePasswdFilter(passwd, nil)
}
func ParsePasswdFileFilter(path string, filter func(User) bool) ([]User, error) {
passwd, err := os.Open(path)
if err != nil {
return nil, err
}
defer passwd.Close()
return ParsePasswdFilter(passwd, filter)
}
func ParsePasswdFilter(r io.Reader, filter func(User) bool) ([]User, error) {
if r == nil {
return nil, fmt.Errorf("nil source for passwd-formatted data")
}
var (
s = bufio.NewScanner(r)
out = []User{}
)
for s.Scan() {
if err := s.Err(); err != nil {
return nil, err
}
line := strings.TrimSpace(s.Text())
if line == "" {
continue
}
// see: man 5 passwd
// name:password:UID:GID:GECOS:directory:shell
// Name:Pass:Uid:Gid:Gecos:Home:Shell
// root:x:0:0:root:/root:/bin/bash
// adm:x:3:4:adm:/var/adm:/bin/false
p := User{}
parseLine(line, &p.Name, &p.Pass, &p.Uid, &p.Gid, &p.Gecos, &p.Home, &p.Shell)
if filter == nil || filter(p) {
out = append(out, p)
}
}
return out, nil
}
func ParseGroupFile(path string) ([]Group, error) {
group, err := os.Open(path)
if err != nil {
return nil, err
}
defer group.Close()
return ParseGroup(group)
}
func ParseGroup(group io.Reader) ([]Group, error) {
return ParseGroupFilter(group, nil)
}
func ParseGroupFileFilter(path string, filter func(Group) bool) ([]Group, error) {
group, err := os.Open(path)
if err != nil {
return nil, err
}
defer group.Close()
return ParseGroupFilter(group, filter)
}
func ParseGroupFilter(r io.Reader, filter func(Group) bool) ([]Group, error) {
if r == nil {
return nil, fmt.Errorf("nil source for group-formatted data")
}
var (
s = bufio.NewScanner(r)
out = []Group{}
)
for s.Scan() {
if err := s.Err(); err != nil {
return nil, err
}
text := s.Text()
if text == "" {
continue
}
// see: man 5 group
// group_name:password:GID:user_list
// Name:Pass:Gid:List
// root:x:0:root
// adm:x:4:root,adm,daemon
p := Group{}
parseLine(text, &p.Name, &p.Pass, &p.Gid, &p.List)
if filter == nil || filter(p) {
out = append(out, p)
}
}
return out, nil
}
type ExecUser struct {
Uid int
Gid int
Sgids []int
Home string
}
// GetExecUserPath is a wrapper for GetExecUser. It reads data from each of the
// given file paths and uses that data as the arguments to GetExecUser. If the
// files cannot be opened for any reason, the error is ignored and a nil
// io.Reader is passed instead.
func GetExecUserPath(userSpec string, defaults *ExecUser, passwdPath, groupPath string) (*ExecUser, error) {
var passwd, group io.Reader
if passwdFile, err := os.Open(passwdPath); err == nil {
passwd = passwdFile
defer passwdFile.Close()
}
if groupFile, err := os.Open(groupPath); err == nil {
group = groupFile
defer groupFile.Close()
}
return GetExecUser(userSpec, defaults, passwd, group)
}
// GetExecUser parses a user specification string (using the passwd and group
// readers as sources for /etc/passwd and /etc/group data, respectively). In
// the case of blank fields or missing data from the sources, the values in
// defaults is used.
//
// GetExecUser will return an error if a user or group literal could not be
// found in any entry in passwd and group respectively.
//
// Examples of valid user specifications are:
// * ""
// * "user"
// * "uid"
// * "user:group"
// * "uid:gid
// * "user:gid"
// * "uid:group"
//
// It should be noted that if you specify a numeric user or group id, they will
// not be evaluated as usernames (only the metadata will be filled). So attempting
// to parse a user with user.Name = "1337" will produce the user with a UID of
// 1337.
func GetExecUser(userSpec string, defaults *ExecUser, passwd, group io.Reader) (*ExecUser, error) {
if defaults == nil {
defaults = new(ExecUser)
}
// Copy over defaults.
user := &ExecUser{
Uid: defaults.Uid,
Gid: defaults.Gid,
Sgids: defaults.Sgids,
Home: defaults.Home,
}
// Sgids slice *cannot* be nil.
if user.Sgids == nil {
user.Sgids = []int{}
}
// Allow for userArg to have either "user" syntax, or optionally "user:group" syntax
var userArg, groupArg string
parseLine(userSpec, &userArg, &groupArg)
// Convert userArg and groupArg to be numeric, so we don't have to execute
// Atoi *twice* for each iteration over lines.
uidArg, uidErr := strconv.Atoi(userArg)
gidArg, gidErr := strconv.Atoi(groupArg)
// Find the matching user.
users, err := ParsePasswdFilter(passwd, func(u User) bool {
if userArg == "" {
// Default to current state of the user.
return u.Uid == user.Uid
}
if uidErr == nil {
// If the userArg is numeric, always treat it as a UID.
return uidArg == u.Uid
}
return u.Name == userArg
})
// If we can't find the user, we have to bail.
if err != nil && passwd != nil {
if userArg == "" {
userArg = strconv.Itoa(user.Uid)
}
return nil, fmt.Errorf("unable to find user %s: %v", userArg, err)
}
var matchedUserName string
if len(users) > 0 {
// First match wins, even if there's more than one matching entry.
matchedUserName = users[0].Name
user.Uid = users[0].Uid
user.Gid = users[0].Gid
user.Home = users[0].Home
} else if userArg != "" {
// If we can't find a user with the given username, the only other valid
// option is if it's a numeric username with no associated entry in passwd.
if uidErr != nil {
// Not numeric.
return nil, fmt.Errorf("unable to find user %s: %v", userArg, ErrNoPasswdEntries)
}
user.Uid = uidArg
// Must be inside valid uid range.
if user.Uid < minId || user.Uid > maxId {
return nil, ErrRange
}
// Okay, so it's numeric. We can just roll with this.
}
// On to the groups. If we matched a username, we need to do this because of
// the supplementary group IDs.
if groupArg != "" || matchedUserName != "" {
groups, err := ParseGroupFilter(group, func(g Group) bool {
// If the group argument isn't explicit, we'll just search for it.
if groupArg == "" {
// Check if user is a member of this group.
for _, u := range g.List {
if u == matchedUserName {
return true
}
}
return false
}
if gidErr == nil {
// If the groupArg is numeric, always treat it as a GID.
return gidArg == g.Gid
}
return g.Name == groupArg
})
if err != nil && group != nil {
return nil, fmt.Errorf("unable to find groups for spec %v: %v", matchedUserName, err)
}
// Only start modifying user.Gid if it is in explicit form.
if groupArg != "" {
if len(groups) > 0 {
// First match wins, even if there's more than one matching entry.
user.Gid = groups[0].Gid
} else {
// If we can't find a group with the given name, the only other valid
// option is if it's a numeric group name with no associated entry in group.
if gidErr != nil {
// Not numeric.
return nil, fmt.Errorf("unable to find group %s: %v", groupArg, ErrNoGroupEntries)
}
user.Gid = gidArg
// Must be inside valid gid range.
if user.Gid < minId || user.Gid > maxId {
return nil, ErrRange
}
// Okay, so it's numeric. We can just roll with this.
}
} else if len(groups) > 0 {
// Supplementary group ids only make sense if in the implicit form.
user.Sgids = make([]int, len(groups))
for i, group := range groups {
user.Sgids[i] = group.Gid
}
}
}
return user, nil
}
// GetAdditionalGroups looks up a list of groups by name or group id
// against the given /etc/group formatted data. If a group name cannot
// be found, an error will be returned. If a group id cannot be found,
// or the given group data is nil, the id will be returned as-is
// provided it is in the legal range.
func GetAdditionalGroups(additionalGroups []string, group io.Reader) ([]int, error) {
var groups = []Group{}
if group != nil {
var err error
groups, err = ParseGroupFilter(group, func(g Group) bool {
for _, ag := range additionalGroups {
if g.Name == ag || strconv.Itoa(g.Gid) == ag {
return true
}
}
return false
})
if err != nil {
return nil, fmt.Errorf("Unable to find additional groups %v: %v", additionalGroups, err)
}
}
gidMap := make(map[int]struct{})
for _, ag := range additionalGroups {
var found bool
for _, g := range groups {
// if we found a matched group either by name or gid, take the
// first matched as correct
if g.Name == ag || strconv.Itoa(g.Gid) == ag {
if _, ok := gidMap[g.Gid]; !ok {
gidMap[g.Gid] = struct{}{}
found = true
break
}
}
}
// we asked for a group but didn't find it. let's check to see
// if we wanted a numeric group
if !found {
gid, err := strconv.Atoi(ag)
if err != nil {
return nil, fmt.Errorf("Unable to find group %s", ag)
}
// Ensure gid is inside gid range.
if gid < minId || gid > maxId {
return nil, ErrRange
}
gidMap[gid] = struct{}{}
}
}
gids := []int{}
for gid := range gidMap {
gids = append(gids, gid)
}
return gids, nil
}
// GetAdditionalGroupsPath is a wrapper around GetAdditionalGroups
// that opens the groupPath given and gives it as an argument to
// GetAdditionalGroups.
func GetAdditionalGroupsPath(additionalGroups []string, groupPath string) ([]int, error) {
var group io.Reader
if groupFile, err := os.Open(groupPath); err == nil {
group = groupFile
defer groupFile.Close()
}
return GetAdditionalGroups(additionalGroups, group)
}

View File

@ -1,6 +1,7 @@
package fsutil
import (
"hash"
"os"
"golang.org/x/net/context"
@ -14,6 +15,8 @@ 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)
func GetWalkerFn(root string) walkerFn {
return func(ctx context.Context, pathC chan<- *currentPath) error {
return Walk(ctx, root, nil, func(path string, f os.FileInfo, err error) error {

View File

@ -3,9 +3,6 @@
package fsutil
import (
"archive/tar"
"crypto/sha256"
"encoding/hex"
"hash"
"io"
"os"
@ -14,8 +11,7 @@ import (
"sync"
"time"
"github.com/docker/docker/pkg/archive"
"github.com/docker/docker/pkg/tarsum"
digest "github.com/opencontainers/go-digest"
"github.com/pkg/errors"
"golang.org/x/net/context"
"golang.org/x/sync/errgroup"
@ -27,6 +23,7 @@ type DiskWriterOpt struct {
AsyncDataCb WriteToFunc
SyncDataCb WriteToFunc
NotifyCb func(ChangeKind, string, os.FileInfo, error) error
ContentHasher ContentHasher
}
type DiskWriter struct {
@ -202,7 +199,7 @@ func (dw *DiskWriter) processChange(kind ChangeKind, p string, fi os.FileInfo, w
var hw *hashedWriter
if dw.opt.NotifyCb != nil {
var err error
if hw, err = newHashWriter(p, fi, w); err != nil {
if hw, err = newHashWriter(dw.opt.ContentHasher, fi, w); err != nil {
return err
}
w = hw
@ -231,11 +228,16 @@ type hashedWriter struct {
io.Writer
h hash.Hash
w io.WriteCloser
sum string
dgst digest.Digest
}
func newHashWriter(p string, fi os.FileInfo, w io.WriteCloser) (*hashedWriter, error) {
h, err := NewTarsumHash(p, fi)
func newHashWriter(ch ContentHasher, fi os.FileInfo, w io.WriteCloser) (*hashedWriter, error) {
stat, ok := fi.Sys().(*Stat)
if !ok {
return nil, errors.Errorf("invalid change without stat information")
}
h, err := ch(stat)
if err != nil {
return nil, err
}
@ -249,15 +251,15 @@ func newHashWriter(p string, fi os.FileInfo, w io.WriteCloser) (*hashedWriter, e
}
func (hw *hashedWriter) Close() error {
hw.sum = string(hex.EncodeToString(hw.h.Sum(nil)))
hw.dgst = digest.NewDigest(digest.SHA256, hw.h)
if hw.w != nil {
return hw.w.Close()
}
return nil
}
func (hw *hashedWriter) Hash() string {
return hw.sum
func (hw *hashedWriter) Digest() digest.Digest {
return hw.dgst
}
type lazyFileWriter struct {
@ -310,44 +312,3 @@ func nextSuffix() string {
randmu.Unlock()
return strconv.Itoa(int(1e9 + r%1e9))[1:]
}
func NewTarsumHash(p string, fi os.FileInfo) (hash.Hash, error) {
stat, ok := fi.Sys().(*Stat)
link := ""
if ok {
link = stat.Linkname
}
if fi.IsDir() {
p += string(os.PathSeparator)
}
h, err := archive.FileInfoHeader(p, fi, link)
if err != nil {
return nil, err
}
h.Name = p
if ok {
h.Uid = int(stat.Uid)
h.Gid = int(stat.Gid)
h.Linkname = stat.Linkname
if stat.Xattrs != nil {
h.Xattrs = make(map[string]string)
for k, v := range stat.Xattrs {
h.Xattrs[k] = string(v)
}
}
}
tsh := &tarsumHash{h: h, Hash: sha256.New()}
tsh.Reset()
return tsh, nil
}
// Reset resets the Hash to its initial state.
func (tsh *tarsumHash) Reset() {
tsh.Hash.Reset()
tarsum.WriteV1Header(tsh.h, tsh.Hash)
}
type tarsumHash struct {
hash.Hash
h *tar.Header
}

View File

@ -12,7 +12,7 @@ import (
"golang.org/x/sync/errgroup"
)
func Receive(ctx context.Context, conn Stream, dest string, notifyHashed ChangeFunc, progressCb func(int, bool)) error {
func Receive(ctx context.Context, conn Stream, dest string, notifyHashed ChangeFunc, contentHasher ContentHasher, progressCb func(int, bool)) error {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
@ -22,6 +22,7 @@ func Receive(ctx context.Context, conn Stream, dest string, notifyHashed ChangeF
files: make(map[string]uint32),
pipes: make(map[uint32]io.WriteCloser),
notifyHashed: notifyHashed,
contentHasher: contentHasher,
progressCb: progressCb,
}
return r.run(ctx)
@ -37,6 +38,7 @@ type receiver struct {
progressCb func(int, bool)
notifyHashed ChangeFunc
contentHasher ContentHasher
orderValidator Validator
hlValidator Hardlinks
}
@ -85,6 +87,7 @@ func (r *receiver) run(ctx context.Context) error {
dw, err := NewDiskWriter(ctx, r.dest, DiskWriterOpt{
AsyncDataCb: r.asyncDataFunc,
NotifyCb: r.notifyHashed,
ContentHasher: r.contentHasher,
})
if err != nil {
return err

View File

@ -9,6 +9,6 @@ import (
"golang.org/x/net/context"
)
func Receive(ctx context.Context, conn Stream, dest string, notifyHashed ChangeFunc, progressCb func(int, bool)) error {
func Receive(ctx context.Context, conn Stream, dest string, notifyHashed ChangeFunc, contentHasher ContentHasher, progressCb func(int, bool)) error {
return errors.Errorf("receive is unsupported in %s", runtime.GOOS)
}