Bump stargz-snapshotter and partial registry logic integration

Current stargz snapshotter pkg doesn't contain registry configuration and the
client of that pkg needs to pass it to that pkg. So this commit includes changes
of propagating buildkitd's registry configuration to stargz snapshotter.

But this is a partial integration of registry logic between buildkitd and stargz
snapshotter because buildkitd's session-based authentication logic is still not
applied to stargz snapshotter. This means private images that require
`~/.docker/config.json` creds can't be lazily pulled yet.

Signed-off-by: ktock <ktokunaga.mail@gmail.com>
v0.8
ktock 2020-10-28 23:10:14 +09:00
parent 5c201fa7b1
commit e3f6e0d249
71 changed files with 1675 additions and 3619 deletions

View File

@ -11,7 +11,7 @@ ARG ROOTLESSKIT_VERSION=v0.9.5
ARG CNI_VERSION=v0.8.6
ARG SHADOW_VERSION=4.8.1
ARG FUSEOVERLAYFS_VERSION=v1.1.2
ARG STARGZ_SNAPSHOTTER_VERSION=2ee75e91f8f98f3d324290a2503269812e019fc3
ARG STARGZ_SNAPSHOTTER_VERSION=3a04e4c2c116c85b4b66d01945cf7ebcb7a2eb5a
# git stage is used for checking out remote repository sources
FROM --platform=$BUILDPLATFORM alpine AS git

View File

@ -3,6 +3,7 @@
package main
import (
"context"
"os"
"os/exec"
"path/filepath"
@ -13,6 +14,7 @@ import (
snapshotsapi "github.com/containerd/containerd/api/services/snapshots/v1"
"github.com/containerd/containerd/defaults"
"github.com/containerd/containerd/pkg/dialer"
"github.com/containerd/containerd/remotes/docker"
ctdsnapshot "github.com/containerd/containerd/snapshots"
"github.com/containerd/containerd/snapshots/native"
"github.com/containerd/containerd/snapshots/overlay"
@ -20,6 +22,7 @@ import (
"github.com/containerd/containerd/sys"
remotesn "github.com/containerd/stargz-snapshotter/snapshot"
"github.com/containerd/stargz-snapshotter/stargz"
sgzsource "github.com/containerd/stargz-snapshotter/stargz/source"
"github.com/moby/buildkit/cmd/buildkitd/config"
"github.com/moby/buildkit/executor/oci"
"github.com/moby/buildkit/util/network/cniprovider"
@ -230,7 +233,8 @@ func ociWorkerInitializer(c *cli.Context, common workerInitializerOpt) ([]worker
return nil, err
}
snFactory, err := snapshotterFactory(common.config.Root, cfg)
hosts := resolverFunc(common.config)
snFactory, err := snapshotterFactory(common.config.Root, cfg, hosts)
if err != nil {
return nil, err
}
@ -267,7 +271,7 @@ func ociWorkerInitializer(c *cli.Context, common workerInitializerOpt) ([]worker
return nil, err
}
opt.GCPolicy = getGCPolicy(cfg.GCConfig, common.config.Root)
opt.RegistryHosts = resolverFunc(common.config)
opt.RegistryHosts = hosts
if platformsStr := cfg.Platforms; len(platformsStr) != 0 {
platforms, err := parsePlatforms(platformsStr)
@ -283,7 +287,7 @@ func ociWorkerInitializer(c *cli.Context, common workerInitializerOpt) ([]worker
return []worker.Worker{w}, nil
}
func snapshotterFactory(commonRoot string, cfg config.OCIConfig) (runc.SnapshotterFactory, error) {
func snapshotterFactory(commonRoot string, cfg config.OCIConfig, hosts docker.RegistryHosts) (runc.SnapshotterFactory, error) {
var (
name = cfg.Snapshotter
address = cfg.ProxySnapshotterPath
@ -348,13 +352,36 @@ func snapshotterFactory(commonRoot string, cfg config.OCIConfig) (runc.Snapshott
return fuseoverlayfs.NewSnapshotter(root)
}
case "stargz":
snFactory.New = func(root string) (ctdsnapshot.Snapshotter, error) {
fs, err := stargz.NewFilesystem(filepath.Join(root, "stargz"),
cfg.StargzSnapshotterConfig)
// Pass the registry configuration to stargz snapshotter
sgzhosts := func(host string) ([]docker.RegistryHost, error) {
base, err := hosts(host)
if err != nil {
return nil, err
}
return remotesn.NewSnapshotter(filepath.Join(root, "snapshotter"),
for i := range base {
if base[i].Authorizer == nil {
// Default authorizer that don't fetch creds via session
// TODO(ktock): use session-based authorizer
base[i].Authorizer = docker.NewDockerAuthorizer(
docker.WithAuthClient(base[i].Client))
}
}
return base, nil
}
snFactory.New = func(root string) (ctdsnapshot.Snapshotter, error) {
fs, err := stargz.NewFilesystem(filepath.Join(root, "stargz"),
cfg.StargzSnapshotterConfig,
stargz.WithGetSources(
// provides source info based on the registry config and
// default labels.
sgzsource.FromDefaultLabels(sgzhosts),
),
)
if err != nil {
return nil, err
}
return remotesn.NewSnapshotter(context.Background(),
filepath.Join(root, "snapshotter"),
fs, remotesn.AsynchronousRemove)
}
default:

12
go.mod
View File

@ -13,7 +13,7 @@ require (
github.com/containerd/continuity v0.0.0-20200710164510-efbc4488d8fe
github.com/containerd/go-cni v1.0.1
github.com/containerd/go-runc v0.0.0-20201020171139-16b287bc67d0
github.com/containerd/stargz-snapshotter v0.0.0-20200903042824-2ee75e91f8f9
github.com/containerd/stargz-snapshotter v0.0.0-20201027054423-3a04e4c2c116
github.com/containerd/typeurl v1.0.1
github.com/coreos/go-systemd/v22 v22.1.0
github.com/docker/cli v0.0.0-20200227165822-2298e6a3fe24
@ -26,7 +26,7 @@ require (
github.com/gogo/protobuf v1.3.1
// protobuf: the actual version is replaced in replace()
github.com/golang/protobuf v1.4.2
github.com/google/go-cmp v0.4.0
github.com/google/go-cmp v0.4.1
github.com/google/shlex v0.0.0-20150127133951-6f45313302b9
github.com/gorilla/mux v1.7.4 // indirect
github.com/grpc-ecosystem/go-grpc-middleware v1.2.0
@ -57,12 +57,12 @@ require (
go.etcd.io/bbolt v1.3.5
golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9
golang.org/x/net v0.0.0-20200707034311-ab3426394381
golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e
golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208
golang.org/x/sys v0.0.0-20200916030750-2334cc1a136f
golang.org/x/time v0.0.0-20191024005414-555d28b269f0
golang.org/x/time v0.0.0-20200416051211-89c76fbcd5d1
// genproto: the actual version is replaced in replace()
google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013
google.golang.org/grpc v1.28.1
google.golang.org/genproto v0.0.0-20200527145253-8367513e4ece
google.golang.org/grpc v1.29.1
)
replace (

464
go.sum

File diff suppressed because it is too large Load Diff

View File

@ -63,7 +63,7 @@ const (
// directory.
type FileSystem interface {
Mount(ctx context.Context, mountpoint string, labels map[string]string) error
Check(ctx context.Context, mountpoint string) error
Check(ctx context.Context, mountpoint string, labels map[string]string) error
Unmount(ctx context.Context, mountpoint string) error
}
@ -97,7 +97,7 @@ type snapshotter struct {
// as snapshots. This is implemented based on the overlayfs snapshotter, so
// diffs are stored under the provided root and a metadata file is stored under
// the root as same as overlayfs snapshotter.
func NewSnapshotter(root string, targetFs FileSystem, opts ...Opt) (snapshots.Snapshotter, error) {
func NewSnapshotter(ctx context.Context, root string, targetFs FileSystem, opts ...Opt) (snapshots.Snapshotter, error) {
if targetFs == nil {
return nil, fmt.Errorf("Specify filesystem to use")
}
@ -128,12 +128,18 @@ func NewSnapshotter(root string, targetFs FileSystem, opts ...Opt) (snapshots.Sn
return nil, err
}
return &snapshotter{
o := &snapshotter{
root: root,
ms: ms,
asyncRemove: config.asyncRemove,
fs: targetFs,
}, nil
}
if err := o.restoreRemoteSnapshot(ctx); err != nil {
return nil, errors.Wrap(err, "failed to restore remote snapshot")
}
return o, nil
}
// Stat returns the info for an active or committed snapshot by name or
@ -661,7 +667,7 @@ func (o *snapshotter) checkAvailability(ctx context.Context, key string) bool {
if _, ok := info.Labels[remoteLabel]; ok {
eg.Go(func() error {
log.G(lCtx).Debug("checking mount point")
if err := o.fs.Check(egCtx, mp); err != nil {
if err := o.fs.Check(egCtx, mp, info.Labels); err != nil {
log.G(lCtx).WithError(err).Warn("layer is unavailable")
return err
}
@ -677,3 +683,34 @@ func (o *snapshotter) checkAvailability(ctx context.Context, key string) bool {
}
return true
}
func (o *snapshotter) restoreRemoteSnapshot(ctx context.Context) error {
mounts, err := mount.Self()
if err != nil {
return err
}
for _, m := range mounts {
if strings.HasPrefix(m.Mountpoint, filepath.Join(o.root, "snapshots")) {
if err := syscall.Unmount(m.Mountpoint, syscall.MNT_FORCE); err != nil {
return errors.Wrapf(err, "failed to unmount %s", m.Mountpoint)
}
}
}
var task []snapshots.Info
if err := o.Walk(ctx, func(ctx context.Context, info snapshots.Info) error {
if _, ok := info.Labels[remoteLabel]; ok {
task = append(task, info)
}
return nil
}); err != nil && !errdefs.IsNotFound(err) {
return err
}
for _, info := range task {
if err := o.prepareRemoteSnapshot(ctx, info.Name, info.Labels); err != nil {
return errors.Wrapf(err, "failed to prepare remote snapshot: %s", info.Name)
}
}
return nil
}

View File

@ -22,6 +22,17 @@
package config
const (
// TargetSkipVerifyLabel is a snapshot label key that indicates to skip content
// verification for the layer.
TargetSkipVerifyLabel = "containerd.io/snapshot/remote/stargz.skipverify"
// TargetPrefetchSizeLabel is a snapshot label key that indicates size to prefetch
// the layer. If the layer is eStargz and contains prefetch landmarks, these config
// will be respeced.
TargetPrefetchSizeLabel = "containerd.io/snapshot/remote/stargz.prefetch"
)
type Config struct {
HTTPCacheType string `toml:"http_cache_type"`
FSCacheType string `toml:"filesystem_cache_type"`
@ -29,11 +40,10 @@ type Config struct {
PrefetchSize int64 `toml:"prefetch_size"`
PrefetchTimeoutSec int64 `toml:"prefetch_timeout_sec"`
NoPrefetch bool `toml:"noprefetch"`
NoBackgroundFetch bool `toml:"no_background_fetch"`
Debug bool `toml:"debug"`
AllowNoVerification bool `toml:"allow_no_verification"`
// ResolverConfig is config for resolving registries.
ResolverConfig `toml:"resolver"`
DisableVerification bool `toml:"disable_verification"`
// BlobConfig is config for layer blob management.
BlobConfig `toml:"blob"`
@ -42,20 +52,6 @@ type Config struct {
DirectoryCacheConfig `toml:"directory_cache"`
}
type ResolverConfig struct {
Host map[string]HostConfig `toml:"host"`
ConnectionPoolEntry int `toml:"connection_pool_entry"`
}
type HostConfig struct {
Mirrors []MirrorConfig `toml:"mirrors"`
}
type MirrorConfig struct {
Host string `toml:"host"`
Insecure bool `toml:"insecure"`
}
type BlobConfig struct {
ValidInterval int64 `toml:"valid_interval"`
CheckAlways bool `toml:"check_always"`

View File

@ -43,7 +43,6 @@ import (
"encoding/json"
"fmt"
"io"
"net/http"
"os"
"path/filepath"
"strconv"
@ -54,21 +53,24 @@ import (
"unsafe"
"github.com/containerd/containerd/log"
"github.com/containerd/containerd/reference"
"github.com/containerd/containerd/remotes/docker"
"github.com/containerd/stargz-snapshotter/cache"
snbase "github.com/containerd/stargz-snapshotter/snapshot"
"github.com/containerd/stargz-snapshotter/stargz/config"
"github.com/containerd/stargz-snapshotter/stargz/handler"
"github.com/containerd/stargz-snapshotter/stargz/reader"
"github.com/containerd/stargz-snapshotter/stargz/remote"
"github.com/containerd/stargz-snapshotter/stargz/source"
"github.com/containerd/stargz-snapshotter/stargz/verify"
"github.com/containerd/stargz-snapshotter/task"
"github.com/golang/groupcache/lru"
"github.com/google/crfs/stargz"
"github.com/google/go-containerregistry/pkg/authn"
fusefs "github.com/hanwen/go-fuse/v2/fs"
"github.com/hanwen/go-fuse/v2/fuse"
digest "github.com/opencontainers/go-digest"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/pkg/errors"
"golang.org/x/sync/singleflight"
"golang.org/x/sys/unix"
)
@ -85,14 +87,6 @@ const (
statFileMode = syscall.S_IFREG | 0400 // -r--------
stateDirMode = syscall.S_IFDIR | 0500 // dr-x------
// targetRefLabelCRI is a label which contains image reference passed from CRI plugin
targetRefLabelCRI = "containerd.io/snapshot/cri.image-ref"
// targetDigestLabelCRI is a label which contains layer digest passed from CRI plugin
targetDigestLabelCRI = "containerd.io/snapshot/cri.layer-digest"
// targetImageLayersLabel is a label which contains layer digests contained in
// the target image and is passed from CRI plugin.
targetImageLayersLabel = "containerd.io/snapshot/cri.image-layers"
// PrefetchLandmark is a file entry which indicates the end position of
// prefetch in the stargz file.
PrefetchLandmark = ".prefetch.landmark"
@ -100,20 +94,17 @@ const (
// NoPrefetchLandmark is a file entry which indicates that no prefetch should
// occur in the stargz file.
NoPrefetchLandmark = ".no.prefetch.landmark"
// TargetSkipVerifyLabel indicates to skip content verification for the layer.
TargetSkipVerifyLabel = "containerd.io/snapshot/remote/stargz.skipverify"
)
type Option func(*options)
type options struct {
keychain []authn.Keychain
getSources source.GetSources
}
func WithKeychain(keychain []authn.Keychain) Option {
func WithGetSources(s source.GetSources) Option {
return func(opts *options) {
opts.keychain = keychain
opts.getSources = s
}
}
@ -154,9 +145,6 @@ func NewFilesystem(root string, cfg config.Config, opts ...Option) (_ snbase.Fil
return nil, errors.Wrap(err, "failed to prepare filesystem cache")
}
}
keychain := authn.NewMultiKeychain(append(
[]authn.Keychain{authn.DefaultKeychain},
fsOpts.keychain...)...)
resolveResultEntry := cfg.ResolveResultEntry
if resolveResultEntry == 0 {
resolveResultEntry = defaultResolveResultEntry
@ -165,37 +153,48 @@ func NewFilesystem(root string, cfg config.Config, opts ...Option) (_ snbase.Fil
if prefetchTimeout == 0 {
prefetchTimeout = defaultPrefetchTimeoutSec * time.Second
}
getSources := fsOpts.getSources
if getSources == nil {
getSources = source.FromDefaultLabels(
docker.ConfigureDefaultRegistries(docker.WithPlainHTTP(docker.MatchLocalhost)))
}
return &filesystem{
resolver: remote.NewResolver(keychain, cfg.ResolverConfig),
blobConfig: cfg.BlobConfig,
httpCache: httpCache,
resolver: remote.NewResolver(httpCache, cfg.BlobConfig),
getSources: getSources,
fsCache: fsCache,
prefetchSize: cfg.PrefetchSize,
prefetchTimeout: prefetchTimeout,
noprefetch: cfg.NoPrefetch,
noBackgroundFetch: cfg.NoBackgroundFetch,
debug: cfg.Debug,
layer: make(map[string]*layer),
resolveResult: lru.New(resolveResultEntry),
blobResult: lru.New(resolveResultEntry),
backgroundTaskManager: task.NewBackgroundTaskManager(2, 5*time.Second),
allowNoVerification: cfg.AllowNoVerification,
disableVerification: cfg.DisableVerification,
}, nil
}
type filesystem struct {
resolver *remote.Resolver
blobConfig config.BlobConfig
httpCache cache.BlobCache
fsCache cache.BlobCache
prefetchSize int64
prefetchTimeout time.Duration
noprefetch bool
noBackgroundFetch bool
debug bool
layer map[string]*layer
layerMu sync.Mutex
resolveResult *lru.Cache
resolveResultMu sync.Mutex
blobResult *lru.Cache
blobResultMu sync.Mutex
backgroundTaskManager *task.BackgroundTaskManager
allowNoVerification bool
disableVerification bool
getSources source.GetSources
resolveG singleflight.Group
}
func (fs *filesystem) Mount(ctx context.Context, mountpoint string, labels map[string]string) error {
@ -206,58 +205,60 @@ func (fs *filesystem) Mount(ctx context.Context, mountpoint string, labels map[s
defer fs.backgroundTaskManager.DonePrioritizedTask()
ctx = log.WithLogger(ctx, log.G(ctx).WithField("mountpoint", mountpoint))
// Get basic information of this layer.
ref, ldgst, layers, prefetchSize, err := fs.parseLabels(labels)
// Get source information of this layer.
src, err := fs.getSources(labels)
if err != nil {
log.G(ctx).WithError(err).Debug("failed to get necessary information from labels")
return err
} else if len(src) == 0 {
return fmt.Errorf("source must be passed")
}
// Resolve the target layer and the all chained layers
// Resolve the target layer
var (
resolved *resolveResult
target = append([]string{ldgst}, layers...)
resultChan = make(chan *layer)
errChan = make(chan error)
)
for _, dgst := range target {
var (
rr *resolveResult
key = fmt.Sprintf("%s/%s", ref, dgst)
)
fs.resolveResultMu.Lock()
if c, ok := fs.resolveResult.Get(key); ok {
if c.(*resolveResult).isInProgress() {
rr = c.(*resolveResult) // resolving in progress
} else if _, err := c.(*resolveResult).get(); err == nil {
rr = c.(*resolveResult) // hit successfully resolved cache
go func() {
rErr := fmt.Errorf("failed to resolve target")
for _, s := range src {
l, err := fs.resolveLayer(ctx, s.Hosts, s.Name, s.Target)
if err == nil {
resultChan <- l
return
}
rErr = errors.Wrapf(rErr, "failed to resolve layer %q from %q: %v",
s.Target.Digest, s.Name, err)
}
if rr == nil { // missed cache
rr = fs.resolve(ctx, ref, dgst)
fs.resolveResult.Add(key, rr)
errChan <- rErr
}()
// Also resolve other layers in parallel
preResolve := src[0] // TODO: should we pre-resolve blobs in other sources as well?
for _, desc := range preResolve.Manifest.Layers {
if desc.Digest.String() != preResolve.Target.Digest.String() {
go fs.resolveLayer(ctx, preResolve.Hosts, preResolve.Name, desc)
}
if dgst == ldgst {
resolved = rr
}
fs.resolveResultMu.Unlock()
}
// Get the resolved layer
ctx = log.WithLogger(ctx, log.G(ctx).WithField("ref", ref).WithField("digest", ldgst))
if resolved == nil {
log.G(ctx).Debug("resolve result isn't registered")
return fmt.Errorf("resolve result(%q,%q) isn't registered", ref, ldgst)
}
l, err := resolved.waitAndGet(30 * time.Second) // get layer with timeout
if err != nil {
// Wait for resolving completion
var l *layer
select {
case l = <-resultChan:
case err := <-errChan:
log.G(ctx).WithError(err).Debug("failed to resolve layer")
return errors.Wrapf(err, "failed to resolve layer(%q,%q)", ref, ldgst)
}
if err := fs.check(ctx, l); err != nil { // check the connectivity
return err
return errors.Wrapf(err, "failed to resolve layer")
case <-time.After(30 * time.Second):
log.G(ctx).Debug("failed to resolve layer (timeout)")
return fmt.Errorf("failed to resolve layer (timeout)")
}
// Verify this layer using the TOC JSON digest passed through label.
if tocDigest, ok := labels[verify.TOCJSONDigestAnnotation]; ok {
// Verify layer's content
if fs.disableVerification {
// Skip if verification is disabled completely
l.skipVerify()
log.G(ctx).Debugf("Verification forcefully skipped")
} else if tocDigest, ok := labels[verify.TOCJSONDigestAnnotation]; ok {
// Verify this layer using the TOC JSON digest passed through label.
dgst, err := digest.Parse(tocDigest)
if err != nil {
log.G(ctx).WithError(err).Debugf("failed to parse passed TOC digest %q", dgst)
@ -268,7 +269,7 @@ func (fs *filesystem) Mount(ctx context.Context, mountpoint string, labels map[s
return errors.Wrapf(err, "invalid stargz layer")
}
log.G(ctx).Debugf("verified")
} else if _, ok := labels[TargetSkipVerifyLabel]; ok && fs.allowNoVerification {
} else if _, ok := labels[config.TargetSkipVerifyLabel]; ok && fs.allowNoVerification {
// If unverified layer is allowed, use it with warning.
// This mode is for legacy stargz archives which don't contain digests
// necessary for layer verification.
@ -289,28 +290,19 @@ func (fs *filesystem) Mount(ctx context.Context, mountpoint string, labels map[s
fs.layer[mountpoint] = l
fs.layerMu.Unlock()
// RoundTripper only used for pre-/background-fetch.
// We use a separated transport because we don't want these fetching
// functionalities to disturb other HTTP-related operations
fetchTr := lazyTransport(func() (http.RoundTripper, error) {
return l.blob.Authn(http.DefaultTransport.(*http.Transport).Clone())
})
// Prefetch this layer. We prefetch several layers in parallel. The first
// Check() for this layer waits for the prefetch completion. We recreate
// RoundTripper to avoid disturbing other NW-related operations.
// Check() for this layer waits for the prefetch completion.
if !fs.noprefetch {
l.doPrefetch()
prefetchSize := fs.prefetchSize
if psStr, ok := labels[config.TargetPrefetchSizeLabel]; ok {
if ps, err := strconv.ParseInt(psStr, 10, 64); err == nil {
prefetchSize = ps
}
}
go func() {
defer l.donePrefetch()
fs.backgroundTaskManager.DoPrioritizedTask()
defer fs.backgroundTaskManager.DonePrioritizedTask()
tr, err := fetchTr()
if err != nil {
log.G(ctx).WithError(err).Debug("failed to prepare transport for prefetch")
return
}
if err := l.prefetch(prefetchSize, remote.WithRoundTripper(tr)); err != nil {
if err := l.prefetch(prefetchSize); err != nil {
log.G(ctx).WithError(err).Debug("failed to prefetched layer")
return
}
@ -321,36 +313,30 @@ func (fs *filesystem) Mount(ctx context.Context, mountpoint string, labels map[s
// Fetch whole layer aggressively in background. We use background
// reader for this so prioritized tasks(Mount, Check, etc...) can
// interrupt the reading. This can avoid disturbing prioritized tasks
// about NW traffic. We read layer with a buffer to reduce num of
// requests to the registry.
go func() {
br := io.NewSectionReader(readerAtFunc(func(p []byte, offset int64) (retN int, retErr error) {
fs.backgroundTaskManager.InvokeBackgroundTask(func(ctx context.Context) {
tr, err := fetchTr()
if err != nil {
log.G(ctx).WithError(err).Debug("failed to prepare transport for background fetch")
retN, retErr = 0, err
return
}
retN, retErr = l.blob.ReadAt(
p,
offset,
remote.WithContext(ctx), // Make cancellable
remote.WithRoundTripper(tr), // Use dedicated Transport
remote.WithCacheOpts(cache.Direct()), // Do not pollute mem cache
)
}, 120*time.Second)
return
}), 0, l.blob.Size())
if err := layerReader.Cache(
reader.WithReader(br), // Read contents in background
reader.WithCacheOpts(cache.Direct()), // Do not pollute mem cache
); err != nil {
log.G(ctx).WithError(err).Debug("failed to fetch whole layer")
return
}
log.G(ctx).Debug("completed to fetch all layer data in background")
}()
// about NW traffic.
if !fs.noBackgroundFetch {
go func() {
br := io.NewSectionReader(readerAtFunc(func(p []byte, offset int64) (retN int, retErr error) {
fs.backgroundTaskManager.InvokeBackgroundTask(func(ctx context.Context) {
retN, retErr = l.blob.ReadAt(
p,
offset,
remote.WithContext(ctx), // Make cancellable
remote.WithCacheOpts(cache.Direct()), // Do not pollute mem cache
)
}, 120*time.Second)
return
}), 0, l.blob.Size())
if err := layerReader.Cache(
reader.WithReader(br), // Read contents in background
reader.WithCacheOpts(cache.Direct()), // Do not pollute mem cache
); err != nil {
log.G(ctx).WithError(err).Debug("failed to fetch whole layer")
return
}
log.G(ctx).Debug("completed to fetch all layer data in background")
}()
}
// Mounting stargz
// TODO: bind mount the state directory as a read-only fs on snapshotter's side
@ -359,7 +345,7 @@ func (fs *filesystem) Mount(ctx context.Context, mountpoint string, labels map[s
fs: fs,
layer: layerReader,
e: l.root,
s: newState(ldgst, l.blob),
s: newState(l.desc.Digest.String(), l.blob),
root: mountpoint,
}, &fusefs.Options{
AttrTimeout: &timeSec,
@ -381,16 +367,39 @@ func (fs *filesystem) Mount(ctx context.Context, mountpoint string, labels map[s
return server.WaitMount()
}
func (fs *filesystem) resolve(ctx context.Context, ref, digest string) *resolveResult {
ctx = log.WithLogger(ctx, log.G(ctx).WithField("ref", ref).WithField("digest", digest))
return newResolveResult(func() (*layer, error) {
log.G(ctx).Debugf("resolving")
defer log.G(ctx).Debugf("resolved")
func (fs *filesystem) resolveLayer(ctx context.Context, hosts docker.RegistryHosts, refspec reference.Spec, desc ocispec.Descriptor) (*layer, error) {
name := refspec.String() + "/" + desc.Digest.String()
ctx, cancel := context.WithCancel(log.WithLogger(ctx, log.G(ctx).WithField("src", name)))
defer cancel()
// Resolve the reference and digest
blob, err := fs.resolver.Resolve(ref, digest, fs.httpCache, fs.blobConfig)
if err != nil {
return nil, errors.Wrap(err, "failed to resolve the reference")
fs.resolveResultMu.Lock()
c, ok := fs.resolveResult.Get(name)
fs.resolveResultMu.Unlock()
if ok && c.(*layer).blob.Check() == nil {
return c.(*layer), nil
}
resultChan := fs.resolveG.DoChan(name, func() (interface{}, error) {
log.G(ctx).Debugf("resolving")
// Resolve the blob. The result will be cached for future use. This is effective
// in some failure cases including resolving is succeeded but the blob is non-stargz.
var blob remote.Blob
fs.blobResultMu.Lock()
c, ok := fs.blobResult.Get(name)
fs.blobResultMu.Unlock()
if ok && c.(remote.Blob).Check() == nil {
blob = c.(remote.Blob)
} else {
var err error
blob, err = fs.resolver.Resolve(ctx, hosts, refspec, desc)
if err != nil {
log.G(ctx).WithError(err).Debugf("failed to resolve source")
return nil, errors.Wrap(err, "failed to resolve the source")
}
fs.blobResultMu.Lock()
fs.blobResult.Add(name, blob)
fs.blobResultMu.Unlock()
}
// Get a reader for stargz archive.
@ -404,14 +413,34 @@ func (fs *filesystem) resolve(ctx context.Context, ref, digest string) *resolveR
}), 0, blob.Size())
vr, root, err := reader.NewReader(sr, fs.fsCache)
if err != nil {
log.G(ctx).WithError(err).Debugf("failed to resolve: layer cannot be read")
return nil, errors.Wrap(err, "failed to read layer")
}
return newLayer(blob, vr, root, fs.prefetchTimeout), nil
// Combine layer information together
l := newLayer(desc, blob, vr, root, fs.prefetchTimeout)
fs.resolveResultMu.Lock()
fs.resolveResult.Add(name, l)
fs.resolveResultMu.Unlock()
log.G(ctx).Debugf("resolved")
return l, nil
})
var res singleflight.Result
select {
case res = <-resultChan:
case <-time.After(30 * time.Second):
fs.resolveG.Forget(name)
return nil, fmt.Errorf("failed to resolve layer (timeout)")
}
if res.Err != nil || res.Val == nil {
return nil, fmt.Errorf("failed to resolve layer: %v", res.Err)
}
return res.Val.(*layer), nil
}
func (fs *filesystem) Check(ctx context.Context, mountpoint string) error {
func (fs *filesystem) Check(ctx context.Context, mountpoint string, labels map[string]string) error {
// This is a prioritized task and all background tasks will be stopped
// execution so this can avoid being disturbed for NW traffic by background
// tasks.
@ -428,41 +457,54 @@ func (fs *filesystem) Check(ctx context.Context, mountpoint string) error {
return fmt.Errorf("layer not registered")
}
// Check the blob connectivity and refresh the connection if possible
if err := fs.check(ctx, l); err != nil {
// Check the blob connectivity and try to refresh the connection on failure
if err := fs.check(ctx, l, labels); err != nil {
log.G(ctx).WithError(err).Warn("check failed")
return err
}
// Wait for prefetch compeletion
if err := l.waitForPrefetchCompletion(); err != nil {
log.G(ctx).WithError(err).Warn("failed to sync with prefetch completion")
if !fs.noprefetch {
if err := l.waitForPrefetchCompletion(); err != nil {
log.G(ctx).WithError(err).Warn("failed to sync with prefetch completion")
}
}
return nil
}
func (fs *filesystem) check(ctx context.Context, l *layer) error {
if err := l.blob.Check(); err != nil {
// Check failed. Try to refresh the connection
log.G(ctx).WithError(err).Warn("failed to connect to blob; refreshing...")
for retry := 0; retry < 3; retry++ {
if iErr := fs.resolver.Refresh(l.blob); iErr != nil {
log.G(ctx).WithError(iErr).Warnf("failed to refresh connection(%d)",
retry)
err = errors.Wrapf(err, "error(%d): %v", retry, iErr)
continue // retry
func (fs *filesystem) check(ctx context.Context, l *layer, labels map[string]string) error {
err := l.blob.Check()
if err == nil {
return nil
}
log.G(ctx).WithError(err).Warn("failed to connect to blob")
// Check failed. Try to refresh the connection with fresh source information
src, err := fs.getSources(labels)
if err != nil {
return err
}
var (
retrynum = 1
rErr = fmt.Errorf("failed to refresh connection")
)
for retry := 0; retry < retrynum; retry++ {
log.G(ctx).Warnf("refreshing(%d)...", retry)
for _, s := range src {
err := l.blob.Refresh(ctx, s.Hosts, s.Name, s.Target)
if err == nil {
log.G(ctx).Debug("Successfully refreshed connection")
return nil
}
log.G(ctx).Debug("Successfully refreshed connection")
err = nil
break
}
if err != nil {
return err
log.G(ctx).WithError(err).Warnf("failed to refresh the layer %q from %q",
s.Target.Digest, s.Name)
rErr = errors.Wrapf(rErr, "failed(layer:%q, ref:%q): %v",
s.Target.Digest, s.Name, err)
}
}
return nil
return rErr
}
func (fs *filesystem) Unmount(ctx context.Context, mountpoint string) error {
@ -481,107 +523,9 @@ func (fs *filesystem) Unmount(ctx context.Context, mountpoint string) error {
return syscall.Unmount(mountpoint, syscall.MNT_FORCE)
}
func (fs *filesystem) parseLabels(labels map[string]string) (rRef, rDigest string, rLayers []string, rPrefetchSize int64, _ error) {
// mandatory labels
if ref, ok := labels[targetRefLabelCRI]; ok {
rRef = ref
} else if ref, ok := labels[handler.TargetRefLabel]; ok {
rRef = ref
} else {
return "", "", nil, 0, fmt.Errorf("reference hasn't been passed")
}
if digest, ok := labels[targetDigestLabelCRI]; ok {
rDigest = digest
} else if digest, ok := labels[handler.TargetDigestLabel]; ok {
rDigest = digest
} else {
return "", "", nil, 0, fmt.Errorf("digest hasn't been passed")
}
if l, ok := labels[targetImageLayersLabel]; ok {
rLayers = strings.Split(l, ",")
} else if l, ok := labels[handler.TargetImageLayersLabel]; ok {
rLayers = strings.Split(l, ",")
} else {
return "", "", nil, 0, fmt.Errorf("image layers hasn't been passed")
}
// optional label
rPrefetchSize = fs.prefetchSize
if psStr, ok := labels[handler.TargetPrefetchSizeLabel]; ok {
if ps, err := strconv.ParseInt(psStr, 10, 64); err == nil {
rPrefetchSize = ps
}
}
return
}
func lazyTransport(trFunc func() (http.RoundTripper, error)) func() (http.RoundTripper, error) {
var (
tr http.RoundTripper
trMu sync.Mutex
)
return func() (http.RoundTripper, error) {
trMu.Lock()
defer trMu.Unlock()
if tr != nil {
return tr, nil
}
gotTr, err := trFunc()
if err != nil {
return nil, err
}
tr = gotTr
return tr, nil
}
}
func newResolveResult(init func() (*layer, error)) *resolveResult {
rr := &resolveResult{
progress: newWaiter(),
}
rr.progress.start()
go func() {
rr.resultMu.Lock()
rr.layer, rr.err = init()
rr.resultMu.Unlock()
rr.progress.done()
}()
return rr
}
type resolveResult struct {
layer *layer
err error
resultMu sync.Mutex
progress *waiter
}
func (rr *resolveResult) waitAndGet(timeout time.Duration) (*layer, error) {
if err := rr.progress.wait(timeout); err != nil {
return nil, err
}
return rr.get()
}
func (rr *resolveResult) get() (*layer, error) {
rr.resultMu.Lock()
defer rr.resultMu.Unlock()
if rr.layer == nil && rr.err == nil {
return nil, fmt.Errorf("failed to get result")
}
return rr.layer, rr.err
}
func (rr *resolveResult) isInProgress() bool {
return rr.progress.isInProgress()
}
func newLayer(blob remote.Blob, vr reader.VerifiableReader, root *stargz.TOCEntry, prefetchTimeout time.Duration) *layer {
func newLayer(desc ocispec.Descriptor, blob remote.Blob, vr reader.VerifiableReader, root *stargz.TOCEntry, prefetchTimeout time.Duration) *layer {
return &layer{
desc: desc,
blob: blob,
verifiableReader: vr,
root: root,
@ -591,6 +535,7 @@ func newLayer(blob remote.Blob, vr reader.VerifiableReader, root *stargz.TOCEntr
}
type layer struct {
desc ocispec.Descriptor
blob remote.Blob
verifiableReader reader.VerifiableReader
root *stargz.TOCEntry
@ -623,15 +568,9 @@ func (l *layer) verify(tocDigest digest.Digest) error {
return nil
}
func (l *layer) doPrefetch() {
l.prefetchWaiter.start()
}
func (l *layer) prefetch(prefetchSize int64) error {
defer l.prefetchWaiter.done() // Notify the completion
func (l *layer) donePrefetch() {
l.prefetchWaiter.done()
}
func (l *layer) prefetch(prefetchSize int64, opts ...remote.Option) error {
lr, err := l.reader()
if err != nil {
return err
@ -648,7 +587,7 @@ func (l *layer) prefetch(prefetchSize int64, opts ...remote.Option) error {
}
// Fetch the target range
if err := l.blob.Cache(0, prefetchSize, opts...); err != nil {
if err := l.blob.Cache(0, prefetchSize); err != nil {
return errors.Wrap(err, "failed to prefetch layer")
}
@ -689,40 +628,28 @@ func newWaiter() *waiter {
}
type waiter struct {
inProgress bool
inProgressMu sync.Mutex
isDone bool
isDoneMu sync.Mutex
completionCond *sync.Cond
}
func (w *waiter) start() {
w.inProgressMu.Lock()
w.inProgress = true
w.inProgressMu.Unlock()
}
func (w *waiter) done() {
w.inProgressMu.Lock()
w.inProgress = false
w.inProgressMu.Unlock()
w.isDoneMu.Lock()
w.isDone = true
w.isDoneMu.Unlock()
w.completionCond.Broadcast()
}
func (w *waiter) isInProgress() bool {
w.inProgressMu.Lock()
defer w.inProgressMu.Unlock()
return w.inProgress
}
func (w *waiter) wait(timeout time.Duration) error {
wait := func() <-chan struct{} {
ch := make(chan struct{})
go func() {
w.inProgressMu.Lock()
inProgress := w.inProgress
w.inProgressMu.Unlock()
w.isDoneMu.Lock()
isDone := w.isDone
w.isDoneMu.Unlock()
w.completionCond.L.Lock()
if inProgress {
if !isDone {
w.completionCond.Wait()
}
w.completionCond.L.Unlock()
@ -732,9 +659,9 @@ func (w *waiter) wait(timeout time.Duration) error {
}
select {
case <-time.After(timeout):
w.inProgressMu.Lock()
w.inProgress = false
w.inProgressMu.Unlock()
w.isDoneMu.Lock()
w.isDone = true
w.isDoneMu.Unlock()
w.completionCond.Broadcast()
return fmt.Errorf("timeout(%v)", timeout)
case <-wait():
@ -1140,8 +1067,8 @@ func entryToAttr(e *stargz.TOCEntry, out *fuse.Attr) fusefs.StableAttr {
if out.Size%uint64(out.Blksize) > 0 {
out.Blocks++
}
out.Mtime = uint64(e.ModTime().Unix())
out.Mtimensec = uint32(e.ModTime().UnixNano())
mtime := e.ModTime()
out.SetTimes(nil, &mtime, nil)
out.Mode = modeOfEntry(e)
out.Owner = fuse.Owner{Uid: uint32(e.Uid), Gid: uint32(e.Gid)}
out.Rdev = uint32(unix.Mkdev(uint32(e.DevMajor), uint32(e.DevMinor)))
@ -1167,8 +1094,8 @@ func entryToWhAttr(e *stargz.TOCEntry, out *fuse.Attr) fusefs.StableAttr {
out.Size = 0
out.Blksize = blockSize
out.Blocks = 0
out.Mtime = uint64(fi.ModTime().Unix())
out.Mtimensec = uint32(fi.ModTime().UnixNano())
mtime := fi.ModTime()
out.SetTimes(nil, &mtime, nil)
out.Mode = syscall.S_IFCHR
out.Owner = fuse.Owner{Uid: 0, Gid: 0}
out.Rdev = uint32(unix.Mkdev(0, 0))

View File

@ -1,72 +0,0 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package handler
import (
"context"
"fmt"
"github.com/containerd/containerd/images"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
)
const (
TargetRefLabel = "containerd.io/snapshot/remote/stargz.reference"
TargetDigestLabel = "containerd.io/snapshot/remote/stargz.digest"
TargetImageLayersLabel = "containerd.io/snapshot/remote/stargz.layers"
TargetPrefetchSizeLabel = "containerd.io/snapshot/remote/stargz.prefetch" // optional
)
// AppendInfoHandlerWrapper makes a handler which appends image's basic
// information to each layer descriptor as annotations during unpack. These
// annotations will be passed to this remote snapshotter as labels and used by
// this filesystem for searching/preparing layers.
func AppendInfoHandlerWrapper(ref string, prefetchSize int64) func(f images.Handler) images.Handler {
return func(f images.Handler) images.Handler {
return images.HandlerFunc(func(ctx context.Context, desc ocispec.Descriptor) ([]ocispec.Descriptor, error) {
children, err := f.Handle(ctx, desc)
if err != nil {
return nil, err
}
switch desc.MediaType {
case ocispec.MediaTypeImageManifest, images.MediaTypeDockerSchema2Manifest:
var layers string
for _, c := range children {
if images.IsLayerType(c.MediaType) {
layers += fmt.Sprintf("%s,", c.Digest.String())
}
}
if len(layers) >= 1 {
layers = layers[:len(layers)-1]
}
for i := range children {
c := &children[i]
if images.IsLayerType(c.MediaType) {
if c.Annotations == nil {
c.Annotations = make(map[string]string)
}
c.Annotations[TargetRefLabel] = ref
c.Annotations[TargetDigestLabel] = c.Digest.String()
c.Annotations[TargetImageLayersLabel] = layers
c.Annotations[TargetPrefetchSizeLabel] = fmt.Sprintf("%d", prefetchSize)
}
}
}
return children, nil
})
}
}

View File

@ -29,6 +29,7 @@ import (
"fmt"
"io"
"path/filepath"
"runtime"
"sync"
"github.com/containerd/stargz-snapshotter/cache"
@ -36,6 +37,7 @@ import (
"github.com/google/crfs/stargz"
"github.com/pkg/errors"
"golang.org/x/sync/errgroup"
"golang.org/x/sync/semaphore"
)
type Reader interface {
@ -133,22 +135,35 @@ func (gr *reader) Cache(opts ...CacheOption) (err error) {
filter = cacheOpts.filter
}
return gr.cacheWithReader(root, r, filter, cacheOpts.cacheOpts...)
eg, egCtx := errgroup.WithContext(context.Background())
eg.Go(func() error {
return gr.cacheWithReader(egCtx, eg, semaphore.NewWeighted(int64(runtime.GOMAXPROCS(0))),
root, r, filter, cacheOpts.cacheOpts...)
})
return eg.Wait()
}
func (gr *reader) cacheWithReader(dir *stargz.TOCEntry, r *stargz.Reader, filter func(*stargz.TOCEntry) bool, opts ...cache.Option) (rErr error) {
eg, _ := errgroup.WithContext(context.Background())
func (gr *reader) cacheWithReader(ctx context.Context, eg *errgroup.Group, sem *semaphore.Weighted, dir *stargz.TOCEntry, r *stargz.Reader, filter func(*stargz.TOCEntry) bool, opts ...cache.Option) (rErr error) {
dir.ForeachChild(func(_ string, e *stargz.TOCEntry) bool {
if e.Type == "dir" {
// Walk through all files on this stargz file.
eg.Go(func() error {
// Make sure the entry is the immediate child for avoiding loop.
if filepath.Dir(filepath.Clean(e.Name)) != filepath.Clean(dir.Name) {
return fmt.Errorf("invalid child path %q; must be child of %q",
e.Name, dir.Name)
}
return gr.cacheWithReader(e, r, filter, opts...)
})
// Ignore a TOCEntry of "./" (formated as "" by stargz lib) on root directory
// because this points to the root directory itself.
if e.Name == "" && dir.Name == "" {
return true
}
// Make sure the entry is the immediate child for avoiding loop.
if filepath.Dir(filepath.Clean(e.Name)) != filepath.Clean(dir.Name) {
rErr = fmt.Errorf("invalid child path %q; must be child of %q",
e.Name, dir.Name)
return false
}
if err := gr.cacheWithReader(ctx, eg, sem, e, r, filter, opts...); err != nil {
rErr = err
return false
}
return true
} else if e.Type != "reg" {
// Only cache regular files
@ -175,7 +190,14 @@ func (gr *reader) cacheWithReader(dir *stargz.TOCEntry, r *stargz.Reader, filter
}
nr += ce.ChunkSize
if err := sem.Acquire(ctx, 1); err != nil {
rErr = err
return false
}
eg.Go(func() error {
defer sem.Release(1)
// Check if the target chunks exists in the cache
id := genID(e.Digest, ce.ChunkOffset, ce.ChunkSize)
if _, err := gr.cache.FetchAt(id, 0, nil, opts...); err == nil {
@ -215,13 +237,6 @@ func (gr *reader) cacheWithReader(dir *stargz.TOCEntry, r *stargz.Reader, filter
return true
})
if err := eg.Wait(); err != nil {
if rErr != nil {
return errors.Wrapf(rErr, "failed to pre-cache some files: %v", err)
}
return err
}
return
}

View File

@ -28,25 +28,26 @@ import (
"fmt"
"io"
"io/ioutil"
"net/http"
"regexp"
"sync"
"time"
"github.com/containerd/containerd/reference"
"github.com/containerd/containerd/remotes/docker"
"github.com/containerd/stargz-snapshotter/cache"
"github.com/google/go-containerregistry/pkg/authn"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/pkg/errors"
)
var contentRangeRegexp = regexp.MustCompile(`bytes ([0-9]+)-([0-9]+)/([0-9]+|\\*)`)
type Blob interface {
Authn(tr http.RoundTripper) (http.RoundTripper, error)
Check() error
Size() int64
FetchedSize() int64
ReadAt(p []byte, offset int64, opts ...Option) (int, error)
Cache(offset int64, size int64, opts ...Option) error
Refresh(ctx context.Context, host docker.RegistryHosts, refspec reference.Spec, desc ocispec.Descriptor) error
}
type blob struct {
@ -54,10 +55,10 @@ type blob struct {
fetcherMu sync.Mutex
size int64
keychain authn.Keychain
chunkSize int64
cache cache.BlobCache
lastCheck time.Time
lastCheckMu sync.Mutex
checkInterval time.Duration
fetchedRegionSet regionSet
@ -66,24 +67,48 @@ type blob struct {
resolver *Resolver
}
func (b *blob) Authn(tr http.RoundTripper) (http.RoundTripper, error) {
func (b *blob) Refresh(ctx context.Context, hosts docker.RegistryHosts, refspec reference.Spec, desc ocispec.Descriptor) error {
// refresh the fetcher
new, newSize, err := newFetcher(ctx, hosts, refspec, desc)
if err != nil {
return err
} else if newSize != b.size {
return fmt.Errorf("Invalid size of new blob %d; want %d", newSize, b.size)
}
// update the blob's fetcher with new one
b.fetcherMu.Lock()
fr := b.fetcher
b.fetcher = new
b.fetcherMu.Unlock()
return fr.authn(tr, b.keychain)
b.lastCheckMu.Lock()
b.lastCheck = time.Now()
b.lastCheckMu.Unlock()
return nil
}
func (b *blob) Check() error {
now := time.Now()
if now.Sub(b.lastCheck) < b.checkInterval {
b.lastCheckMu.Lock()
lastCheck := b.lastCheck
b.lastCheckMu.Unlock()
if now.Sub(lastCheck) < b.checkInterval {
// do nothing if not expired
return nil
}
b.lastCheck = now
b.fetcherMu.Lock()
fr := b.fetcher
b.fetcherMu.Unlock()
return fr.check()
err := fr.check()
if err == nil {
// update lastCheck only if check succeeded.
// on failure, we should check this layer next time again.
b.lastCheckMu.Lock()
b.lastCheck = now
b.lastCheckMu.Unlock()
}
return err
}
func (b *blob) Size() int64 {
@ -244,14 +269,16 @@ func (b *blob) fetchRange(allData map[region]io.Writer, opts *options) error {
}
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
defer cancel()
mr, err := fr.fetch(ctx, req, opts)
mr, err := fr.fetch(ctx, req, true, opts)
if err != nil {
return err
}
defer mr.Close()
// Update the check timer because we succeeded to access the blob
b.lastCheckMu.Lock()
b.lastCheck = time.Now()
b.lastCheckMu.Unlock()
// chunk and cache responsed data. Regions must be aligned by chunk size.
// TODO: Reorganize remoteData to make it be aligned by chunk size

View File

@ -32,221 +32,189 @@ import (
"mime"
"mime/multipart"
"net/http"
"net/url"
"path"
"strconv"
"strings"
"sync"
"time"
"github.com/containerd/containerd/reference/docker"
"github.com/containerd/containerd/errdefs"
"github.com/containerd/containerd/reference"
"github.com/containerd/containerd/remotes/docker"
"github.com/containerd/stargz-snapshotter/cache"
"github.com/containerd/stargz-snapshotter/stargz/config"
"github.com/golang/groupcache/lru"
"github.com/google/go-containerregistry/pkg/authn"
"github.com/google/go-containerregistry/pkg/name"
"github.com/google/go-containerregistry/pkg/v1/remote/transport"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/pkg/errors"
)
const (
defaultChunkSize = 50000
defaultValidInterval = 60 * time.Second
defaultPoolEntry = 3000
defaultChunkSize = 50000
defaultValidIntervalSec = 60
)
func NewResolver(keychain authn.Keychain, cfg config.ResolverConfig) *Resolver {
if cfg.Host == nil {
cfg.Host = make(map[string]config.HostConfig)
func NewResolver(cache cache.BlobCache, cfg config.BlobConfig) *Resolver {
if cfg.ChunkSize == 0 { // zero means "use default chunk size"
cfg.ChunkSize = defaultChunkSize
}
poolEntry := cfg.ConnectionPoolEntry
if poolEntry == 0 {
poolEntry = defaultPoolEntry
if cfg.ValidInterval == 0 { // zero means "use default interval"
cfg.ValidInterval = defaultValidIntervalSec
}
if cfg.CheckAlways {
cfg.ValidInterval = 0
}
return &Resolver{
transport: http.DefaultTransport,
trPool: lru.New(poolEntry),
keychain: keychain,
config: cfg,
bufPool: sync.Pool{
New: func() interface{} {
return new(bytes.Buffer)
},
},
blobCache: cache,
blobConfig: cfg,
}
}
type Resolver struct {
transport http.RoundTripper
trPool *lru.Cache
trPoolMu sync.Mutex
keychain authn.Keychain
config config.ResolverConfig
bufPool sync.Pool
blobCache cache.BlobCache
blobConfig config.BlobConfig
bufPool sync.Pool
}
func (r *Resolver) Resolve(ref, digest string, cache cache.BlobCache, cfg config.BlobConfig) (Blob, error) {
fetcher, size, err := r.resolve(ref, digest)
func (r *Resolver) Resolve(ctx context.Context, hosts docker.RegistryHosts, refspec reference.Spec, desc ocispec.Descriptor) (Blob, error) {
fetcher, size, err := newFetcher(ctx, hosts, refspec, desc)
if err != nil {
return nil, err
}
var (
chunkSize int64
checkInterval time.Duration
)
chunkSize = cfg.ChunkSize
if chunkSize == 0 { // zero means "use default chunk size"
chunkSize = defaultChunkSize
}
if cfg.ValidInterval == 0 { // zero means "use default interval"
checkInterval = defaultValidInterval
} else {
checkInterval = time.Duration(cfg.ValidInterval) * time.Second
}
if cfg.CheckAlways {
checkInterval = 0
}
return &blob{
fetcher: fetcher,
size: size,
keychain: r.keychain,
chunkSize: chunkSize,
cache: cache,
chunkSize: r.blobConfig.ChunkSize,
cache: r.blobCache,
lastCheck: time.Now(),
checkInterval: checkInterval,
checkInterval: time.Duration(r.blobConfig.ValidInterval) * time.Second,
resolver: r,
}, nil
}
func (r *Resolver) Refresh(target Blob) error {
b, ok := target.(*blob)
if !ok {
return fmt.Errorf("invalid type of blob. must be *blob")
}
// refresh the fetcher
b.fetcherMu.Lock()
defer b.fetcherMu.Unlock()
new, newSize, err := b.fetcher.refresh()
if err != nil {
return err
} else if newSize != b.size {
return fmt.Errorf("Invalid size of new blob %d; want %d", newSize, b.size)
}
// update the blob's fetcher with new one
b.fetcher = new
b.lastCheck = time.Now()
return nil
}
func (r *Resolver) resolve(ref, digest string) (*fetcher, int64, error) {
var (
nref name.Reference
url string
tr http.RoundTripper
size int64
)
named, err := docker.ParseDockerRef(ref)
func newFetcher(ctx context.Context, hosts docker.RegistryHosts, refspec reference.Spec, desc ocispec.Descriptor) (*fetcher, int64, error) {
reghosts, err := hosts(refspec.Hostname())
if err != nil {
return nil, 0, err
}
hosts := append(r.config.Host[docker.Domain(named)].Mirrors, config.MirrorConfig{
Host: docker.Domain(named),
})
if desc.Digest.String() == "" {
return nil, 0, fmt.Errorf("Digest is mandatory in layer descriptor")
}
digest := desc.Digest
u, err := url.Parse("dummy://" + refspec.Locator)
if err != nil {
return nil, 0, err
}
// Try to create fetcher until succeeded
rErr := fmt.Errorf("failed to resolve")
for _, h := range hosts {
// Parse reference
if h.Host == "" || strings.Contains(h.Host, "/") {
rErr = errors.Wrapf(rErr, "host %q: mirror must be a domain name", h.Host)
continue // try another host
for _, host := range reghosts {
if host.Host == "" || strings.Contains(host.Host, "/") {
rErr = errors.Wrapf(rErr, "invalid destination (host %q, ref:%q, digest:%q)",
host.Host, refspec, digest)
continue // Try another
}
var opts []name.Option
if h.Insecure {
opts = append(opts, name.Insecure)
}
sref := fmt.Sprintf("%s/%s", h.Host, docker.Path(named))
nref, err = name.ParseReference(sref, opts...)
if err != nil {
rErr = errors.Wrapf(rErr, "host %q: failed to parse ref %q (%q): %v",
h.Host, sref, digest, err)
continue // try another host
// Prepare transport with authorization functionality
tr := host.Client.Transport
if host.Authorizer != nil {
tr = &transport{
inner: tr,
auth: host.Authorizer,
// Specify pull scope
// TODO: The scope generator function in containerd (github.com/containerd/containerd/remotes/docker/scope.go) should be exported and used here.
scope: "repository:" + strings.TrimPrefix(u.Path, "/") + ":pull",
}
}
// Resolve redirection and get blob URL
url, tr, err = r.resolveReference(nref, digest)
blobURL := fmt.Sprintf("%s://%s/%s/blobs/%s",
host.Scheme,
path.Join(host.Host, host.Path),
strings.TrimPrefix(refspec.Locator, refspec.Hostname()+"/"),
digest)
url, err := redirect(ctx, blobURL, tr)
if err != nil {
rErr = errors.Wrapf(rErr, "host %q: failed to resolve ref %q (%q): %v",
h.Host, nref.String(), digest, err)
continue // try another host
rErr = errors.Wrapf(rErr, "failed to redirect (host %q, ref:%q, digest:%q): %v",
host.Host, refspec, digest, err)
continue // Try another
}
// Get size information
size, err = getSize(url, tr)
// TODO: we should try to use the Size field in the descriptor here.
size, err := getSize(ctx, url, tr)
if err != nil {
rErr = errors.Wrapf(rErr, "host %q: failed to get size: %v", h.Host, err)
continue // try another host
rErr = errors.Wrapf(rErr, "failed to get size (host %q, ref:%q, digest:%q): %v",
host.Host, refspec, digest, err)
continue // Try another
}
rErr = nil // Hit one accessible mirror
break
}
if rErr != nil {
return nil, 0, errors.Wrapf(rErr, "cannot resolve ref %q (%q)", ref, digest)
// Hit one destination
return &fetcher{
url: url,
tr: tr,
blobURL: blobURL,
}, size, nil
}
return &fetcher{
resolver: r,
ref: ref,
digest: digest,
nref: nref,
url: url,
tr: tr,
}, size, nil
return nil, 0, errors.Wrapf(rErr, "cannot resolve layer")
}
func (r *Resolver) resolveReference(ref name.Reference, digest string) (string, http.RoundTripper, error) {
r.trPoolMu.Lock()
defer r.trPoolMu.Unlock()
type transport struct {
inner http.RoundTripper
auth docker.Authorizer
scope string
}
// Construct endpoint URL from given ref
endpointURL := fmt.Sprintf("%s://%s/v2/%s/blobs/%s",
ref.Context().Registry.Scheme(),
ref.Context().RegistryStr(),
ref.Context().RepositoryStr(),
digest)
// Try to use cached transport (cahced per reference name)
if tr, ok := r.trPool.Get(ref.Name()); ok {
if url, err := redirect(endpointURL, tr.(http.RoundTripper)); err == nil {
return url, tr.(http.RoundTripper), nil
func (tr *transport) RoundTrip(req *http.Request) (*http.Response, error) {
ctx := docker.WithScope(req.Context(), tr.scope)
roundTrip := func(req *http.Request) (*http.Response, error) {
// authorize the request using docker.Authorizer
if err := tr.auth.Authorize(ctx, req); err != nil {
return nil, err
}
// send the request
return tr.inner.RoundTrip(req)
}
// Remove the stale transport from cache
r.trPool.Remove(ref.Name())
// transport is unavailable/expired so refresh the transport and try again
tr, err := authnTransport(ref, r.transport, r.keychain)
resp, err := roundTrip(req)
if err != nil {
return "", nil, err
}
url, err := redirect(endpointURL, tr)
if err != nil {
return "", nil, err
return nil, err
}
// Update transports cache
r.trPool.Add(ref.Name(), tr)
// TODO: support more status codes and retries
if resp.StatusCode == http.StatusUnauthorized {
return url, tr, nil
// prepare authorization for the target host using docker.Authorizer
if err := tr.auth.AddResponses(ctx, []*http.Response{resp}); err != nil {
if errdefs.IsNotImplemented(err) {
return resp, nil
}
return nil, err
}
// re-authorize and send the request
return roundTrip(req.Clone(ctx))
}
return resp, nil
}
func redirect(endpointURL string, tr http.RoundTripper) (url string, err error) {
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
func redirect(ctx context.Context, blobURL string, tr http.RoundTripper) (url string, err error) {
ctx, cancel := context.WithTimeout(ctx, 10*time.Second)
defer cancel()
// We use GET request for GCR.
req, err := http.NewRequestWithContext(ctx, "GET", endpointURL, nil)
// We use GET request for redirect.
// gcr.io returns 200 on HEAD without Location header (2020).
// ghcr.io returns 200 on HEAD without Location header (2020).
req, err := http.NewRequestWithContext(ctx, "GET", blobURL, nil)
if err != nil {
return "", errors.Wrapf(err, "failed to make request to the registry")
}
@ -262,7 +230,7 @@ func redirect(endpointURL string, tr http.RoundTripper) (url string, err error)
}()
if res.StatusCode/100 == 2 {
url = endpointURL
url = blobURL
} else if redir := res.Header.Get("Location"); redir != "" && res.StatusCode/100 == 3 {
// TODO: Support nested redirection
url = redir
@ -273,8 +241,8 @@ func redirect(endpointURL string, tr http.RoundTripper) (url string, err error)
return
}
func getSize(url string, tr http.RoundTripper) (int64, error) {
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
func getSize(ctx context.Context, url string, tr http.RoundTripper) (int64, error) {
ctx, cancel := context.WithTimeout(ctx, 10*time.Second)
defer cancel()
req, err := http.NewRequestWithContext(ctx, "HEAD", url, nil)
if err != nil {
@ -286,60 +254,55 @@ func getSize(url string, tr http.RoundTripper) (int64, error) {
return 0, err
}
defer res.Body.Close()
if res.StatusCode != http.StatusOK {
return 0, fmt.Errorf("failed HEAD request with code %v", res.StatusCode)
if res.StatusCode == http.StatusOK {
return strconv.ParseInt(res.Header.Get("Content-Length"), 10, 64)
}
return strconv.ParseInt(res.Header.Get("Content-Length"), 10, 64)
}
headStatusCode := res.StatusCode
func authnTransport(ref name.Reference, tr http.RoundTripper, keychain authn.Keychain) (http.RoundTripper, error) {
if keychain == nil {
keychain = authn.DefaultKeychain
}
authn, err := keychain.Resolve(ref.Context())
// Failed to do HEAD request. Fall back to GET.
// ghcr.io (https://github-production-container-registry.s3.amazonaws.com) doesn't allow
// HEAD request (2020).
req, err = http.NewRequestWithContext(ctx, "GET", url, nil)
if err != nil {
return nil, errors.Wrapf(err, "failed to resolve the reference %q", ref)
return 0, errors.Wrapf(err, "failed to make request to the registry")
}
errCh := make(chan error)
var rTr http.RoundTripper
go func() {
rTr, err = transport.New(
ref.Context().Registry,
authn,
tr,
[]string{ref.Scope(transport.PullScope)},
)
errCh <- err
req.Close = false
req.Header.Set("Range", "bytes=0-1")
res, err = tr.RoundTrip(req)
if err != nil {
return 0, errors.Wrapf(err, "failed to request")
}
defer func() {
io.Copy(ioutil.Discard, res.Body)
res.Body.Close()
}()
select {
case err = <-errCh:
case <-time.After(10 * time.Second):
return nil, fmt.Errorf("authentication timeout")
if res.StatusCode == http.StatusOK {
return strconv.ParseInt(res.Header.Get("Content-Length"), 10, 64)
} else if res.StatusCode == http.StatusPartialContent {
_, size, err := parseRange(res.Header.Get("Content-Range"))
return size, err
}
return rTr, err
return 0, fmt.Errorf("failed to get size with code (HEAD=%v, GET=%v)",
headStatusCode, res.StatusCode)
}
type fetcher struct {
resolver *Resolver
ref string
digest string
nref name.Reference
url string
urlMu sync.Mutex
tr http.RoundTripper
blobURL string
singleRange bool
singleRangeMu sync.Mutex
}
func (f *fetcher) refresh() (*fetcher, int64, error) {
return f.resolver.resolve(f.ref, f.digest)
}
type multipartReadCloser interface {
Next() (region, io.Reader, error)
Close() error
}
func (f *fetcher) fetch(ctx context.Context, rs []region, opts *options) (multipartReadCloser, error) {
func (f *fetcher) fetch(ctx context.Context, rs []region, retry bool, opts *options) (multipartReadCloser, error) {
if len(rs) == 0 {
return nil, fmt.Errorf("no request queried")
}
@ -371,7 +334,10 @@ func (f *fetcher) fetch(ctx context.Context, rs []region, opts *options) (multip
}
// Request to the registry
req, err := http.NewRequestWithContext(ctx, "GET", f.url, nil)
f.urlMu.Lock()
url := f.url
f.urlMu.Unlock()
req, err := http.NewRequestWithContext(ctx, "GET", url, nil)
if err != nil {
return nil, err
}
@ -404,14 +370,21 @@ func (f *fetcher) fetch(ctx context.Context, rs []region, opts *options) (multip
}
// We are getting single range
reg, err := parseRange(res.Header.Get("Content-Range"))
reg, _, err := parseRange(res.Header.Get("Content-Range"))
if err != nil {
return nil, errors.Wrapf(err, "failed to parse Content-Range")
}
return singlePartReader(reg, res.Body), nil
} else if !singleRangeMode {
f.singleRangeMode() // fallbacks to singe range request mode
return f.fetch(ctx, rs, opts) // retries with the single range mode
} else if retry && res.StatusCode == http.StatusForbidden {
// re-redirect and retry this once.
if err := f.refreshURL(ctx); err != nil {
return nil, errors.Wrapf(err, "failed to refresh URL on %v", res.Status)
}
return f.fetch(ctx, rs, false, opts)
} else if retry && res.StatusCode == http.StatusBadRequest && !singleRangeMode {
// gcr.io (https://storage.googleapis.com) returns 400 on multi-range request (2020 #81)
f.singleRangeMode() // fallbacks to singe range request mode
return f.fetch(ctx, rs, false, opts) // retries with the single range mode
}
return nil, fmt.Errorf("unexpected status code: %v", res.Status)
@ -420,7 +393,10 @@ func (f *fetcher) fetch(ctx context.Context, rs []region, opts *options) (multip
func (f *fetcher) check() error {
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
defer cancel()
req, err := http.NewRequestWithContext(ctx, "GET", f.url, nil)
f.urlMu.Lock()
url := f.url
f.urlMu.Unlock()
req, err := http.NewRequestWithContext(ctx, "GET", url, nil)
if err != nil {
return errors.Wrapf(err, "check failed: failed to make request")
}
@ -434,19 +410,34 @@ func (f *fetcher) check() error {
io.Copy(ioutil.Discard, res.Body)
res.Body.Close()
}()
if res.StatusCode != http.StatusOK && res.StatusCode != http.StatusPartialContent {
return fmt.Errorf("unexpected status code %v", res.StatusCode)
if res.StatusCode == http.StatusOK || res.StatusCode == http.StatusPartialContent {
return nil
} else if res.StatusCode == http.StatusForbidden {
// Try to re-redirect this blob
rCtx, rCancel := context.WithTimeout(context.Background(), 10*time.Second)
defer rCancel()
if err := f.refreshURL(rCtx); err == nil {
return nil
}
return fmt.Errorf("failed to refresh URL on status %v", res.Status)
}
return fmt.Errorf("unexpected status code %v", res.StatusCode)
}
func (f *fetcher) refreshURL(ctx context.Context) error {
newURL, err := redirect(ctx, f.blobURL, f.tr)
if err != nil {
return err
}
f.urlMu.Lock()
f.url = newURL
f.urlMu.Unlock()
return nil
}
func (f *fetcher) authn(tr http.RoundTripper, keychain authn.Keychain) (http.RoundTripper, error) {
return authnTransport(f.nref, tr, keychain)
}
func (f *fetcher) genID(reg region) string {
sum := sha256.Sum256([]byte(fmt.Sprintf("%s-%d-%d", f.url, reg.b, reg.e)))
sum := sha256.Sum256([]byte(fmt.Sprintf("%s-%d-%d", f.blobURL, reg.b, reg.e)))
return fmt.Sprintf("%x", sum)
}
@ -503,28 +494,32 @@ func (sr *multipartReader) Next() (region, io.Reader, error) {
if err != nil {
return region{}, nil, err
}
reg, err := parseRange(p.Header.Get("Content-Range"))
reg, _, err := parseRange(p.Header.Get("Content-Range"))
if err != nil {
return region{}, nil, errors.Wrapf(err, "failed to parse Content-Range")
}
return reg, p, nil
}
func parseRange(header string) (region, error) {
func parseRange(header string) (region, int64, error) {
submatches := contentRangeRegexp.FindStringSubmatch(header)
if len(submatches) < 4 {
return region{}, fmt.Errorf("Content-Range %q doesn't have enough information", header)
return region{}, 0, fmt.Errorf("Content-Range %q doesn't have enough information", header)
}
begin, err := strconv.ParseInt(submatches[1], 10, 64)
if err != nil {
return region{}, errors.Wrapf(err, "failed to parse beginning offset %q", submatches[1])
return region{}, 0, errors.Wrapf(err, "failed to parse beginning offset %q", submatches[1])
}
end, err := strconv.ParseInt(submatches[2], 10, 64)
if err != nil {
return region{}, errors.Wrapf(err, "failed to parse end offset %q", submatches[2])
return region{}, 0, errors.Wrapf(err, "failed to parse end offset %q", submatches[2])
}
blobSize, err := strconv.ParseInt(submatches[3], 10, 64)
if err != nil {
return region{}, 0, errors.Wrapf(err, "failed to parse blob size %q", submatches[3])
}
return region{begin, end}, nil
return region{begin, end}, blobSize, nil
}
type Option func(*options)

View File

@ -0,0 +1,165 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package source
import (
"context"
"fmt"
"strings"
"github.com/containerd/containerd/images"
"github.com/containerd/containerd/labels"
"github.com/containerd/containerd/reference"
"github.com/containerd/containerd/remotes/docker"
"github.com/containerd/stargz-snapshotter/stargz/config"
digest "github.com/opencontainers/go-digest"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
)
// GetSource is a function for converting snapshot labels into typed blob sources
// information. This package defines a default converter which provides source
// information based on some labels but implementations aren't required to use labels.
// Implementations are allowed to return several sources (registry config + image refs)
// about the blob.
type GetSources func(labels map[string]string) (source []Source, err error)
// Source is a typed blob source information. This contains information about
// a blob stored in registries and some contexts of the blob.
type Source struct {
// Hosts is a registry configuration where this blob is stored.
Hosts docker.RegistryHosts
// Name is an image reference which contains this blob.
Name reference.Spec
// Target is a descriptor of this blob.
Target ocispec.Descriptor
// Manifest is an image manifest which contains the blob. This will
// be used by the filesystem to pre-resolve some layers contained in
// the manifest.
// Currently, only layer digests (Manifest.Layers.Digest) will be used.
Manifest ocispec.Manifest
}
const (
// targetRefLabel is a label which contains image reference.
targetRefLabel = "containerd.io/snapshot/remote/stargz.reference"
// targetDigestLabel is a label which contains layer digest.
targetDigestLabel = "containerd.io/snapshot/remote/stargz.digest"
// targetImageLayersLabel is a label which contains layer digests contained in
// the target image.
targetImageLayersLabel = "containerd.io/snapshot/remote/stargz.layers"
)
// FromDefaultLabels returns a function for converting snapshot labels to
// source information based on labels.
func FromDefaultLabels(hosts docker.RegistryHosts) GetSources {
return func(labels map[string]string) ([]Source, error) {
refStr, ok := labels[targetRefLabel]
if !ok {
return nil, fmt.Errorf("reference hasn't been passed")
}
refspec, err := reference.Parse(refStr)
if err != nil {
return nil, err
}
digestStr, ok := labels[targetDigestLabel]
if !ok {
return nil, fmt.Errorf("digest hasn't been passed")
}
target, err := digest.Parse(digestStr)
if err != nil {
return nil, err
}
var layersDgst []digest.Digest
if l, ok := labels[targetImageLayersLabel]; ok {
layersStr := strings.Split(l, ",")
for _, l := range layersStr {
d, err := digest.Parse(l)
if err != nil {
return nil, err
}
if d.String() != target.String() {
layersDgst = append(layersDgst, d)
}
}
}
var layers []ocispec.Descriptor
for _, dgst := range append([]digest.Digest{target}, layersDgst...) {
layers = append(layers, ocispec.Descriptor{Digest: dgst})
}
return []Source{
{
Hosts: hosts,
Name: refspec,
Target: ocispec.Descriptor{Digest: target},
Manifest: ocispec.Manifest{Layers: layers},
},
}, nil
}
}
// AppendDefaultLabelsHandlerWrapper makes a handler which appends image's basic
// information to each layer descriptor as annotations during unpack. These
// annotations will be passed to this remote snapshotter as labels and used to
// construct source information.
func AppendDefaultLabelsHandlerWrapper(ref string, prefetchSize int64) func(f images.Handler) images.Handler {
return func(f images.Handler) images.Handler {
return images.HandlerFunc(func(ctx context.Context, desc ocispec.Descriptor) ([]ocispec.Descriptor, error) {
children, err := f.Handle(ctx, desc)
if err != nil {
return nil, err
}
switch desc.MediaType {
case ocispec.MediaTypeImageManifest, images.MediaTypeDockerSchema2Manifest:
for i := range children {
c := &children[i]
if images.IsLayerType(c.MediaType) {
if c.Annotations == nil {
c.Annotations = make(map[string]string)
}
c.Annotations[targetRefLabel] = ref
c.Annotations[targetDigestLabel] = c.Digest.String()
var layers string
for _, l := range children[i:] {
if images.IsLayerType(l.MediaType) {
ls := fmt.Sprintf("%s,", l.Digest.String())
// This avoids the label hits the size limitation.
// Skipping layers is allowed here and only affects performance.
if err := labels.Validate(targetImageLayersLabel, layers+ls); err != nil {
break
}
layers += ls
}
}
c.Annotations[targetImageLayersLabel] = strings.TrimSuffix(layers, ",")
c.Annotations[config.TargetPrefetchSizeLabel] = fmt.Sprintf("%d", prefetchSize)
}
}
}
return children, nil
})
}
}

View File

@ -6,6 +6,10 @@
//
// This package is intended to be a more powerful and safer alternative to
// reflect.DeepEqual for comparing whether two values are semantically equal.
// It is intended to only be used in tests, as performance is not a goal and
// it may panic if it cannot compare the values. Its propensity towards
// panicking means that its unsuitable for production environments where a
// spurious panic may be fatal.
//
// The primary features of cmp are:
//

View File

@ -81,14 +81,19 @@ func (opts formatOptions) FormatDiff(v *valueNode) textNode {
return opts.FormatDiffSlice(v)
}
var withinSlice bool
if v.parent != nil && (v.parent.Type.Kind() == reflect.Slice || v.parent.Type.Kind() == reflect.Array) {
withinSlice = true
}
// For leaf nodes, format the value based on the reflect.Values alone.
if v.MaxDepth == 0 {
switch opts.DiffMode {
case diffUnknown, diffIdentical:
// Format Equal.
if v.NumDiff == 0 {
outx := opts.FormatValue(v.ValueX, visitedPointers{})
outy := opts.FormatValue(v.ValueY, visitedPointers{})
outx := opts.FormatValue(v.ValueX, withinSlice, visitedPointers{})
outy := opts.FormatValue(v.ValueY, withinSlice, visitedPointers{})
if v.NumIgnored > 0 && v.NumSame == 0 {
return textEllipsis
} else if outx.Len() < outy.Len() {
@ -101,8 +106,8 @@ func (opts formatOptions) FormatDiff(v *valueNode) textNode {
// Format unequal.
assert(opts.DiffMode == diffUnknown)
var list textList
outx := opts.WithTypeMode(elideType).FormatValue(v.ValueX, visitedPointers{})
outy := opts.WithTypeMode(elideType).FormatValue(v.ValueY, visitedPointers{})
outx := opts.WithTypeMode(elideType).FormatValue(v.ValueX, withinSlice, visitedPointers{})
outy := opts.WithTypeMode(elideType).FormatValue(v.ValueY, withinSlice, visitedPointers{})
if outx != nil {
list = append(list, textRecord{Diff: '-', Value: outx})
}
@ -111,9 +116,9 @@ func (opts formatOptions) FormatDiff(v *valueNode) textNode {
}
return opts.WithTypeMode(emitType).FormatType(v.Type, list)
case diffRemoved:
return opts.FormatValue(v.ValueX, visitedPointers{})
return opts.FormatValue(v.ValueX, withinSlice, visitedPointers{})
case diffInserted:
return opts.FormatValue(v.ValueY, visitedPointers{})
return opts.FormatValue(v.ValueY, withinSlice, visitedPointers{})
default:
panic("invalid diff mode")
}

View File

@ -74,7 +74,7 @@ func (opts formatOptions) FormatType(t reflect.Type, s textNode) textNode {
// FormatValue prints the reflect.Value, taking extra care to avoid descending
// into pointers already in m. As pointers are visited, m is also updated.
func (opts formatOptions) FormatValue(v reflect.Value, m visitedPointers) (out textNode) {
func (opts formatOptions) FormatValue(v reflect.Value, withinSlice bool, m visitedPointers) (out textNode) {
if !v.IsValid() {
return nil
}
@ -108,12 +108,15 @@ func (opts formatOptions) FormatValue(v reflect.Value, m visitedPointers) (out t
return textLine(fmt.Sprint(v.Bool()))
case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64:
return textLine(fmt.Sprint(v.Int()))
case reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, reflect.Uintptr:
// Unnamed uints are usually bytes or words, so use hexadecimal.
if t.PkgPath() == "" || t.Kind() == reflect.Uintptr {
case reflect.Uint, reflect.Uint16, reflect.Uint32, reflect.Uint64:
return textLine(fmt.Sprint(v.Uint()))
case reflect.Uint8:
if withinSlice {
return textLine(formatHex(v.Uint()))
}
return textLine(fmt.Sprint(v.Uint()))
case reflect.Uintptr:
return textLine(formatHex(v.Uint()))
case reflect.Float32, reflect.Float64:
return textLine(fmt.Sprint(v.Float()))
case reflect.Complex64, reflect.Complex128:
@ -129,7 +132,7 @@ func (opts formatOptions) FormatValue(v reflect.Value, m visitedPointers) (out t
if value.IsZero(vv) {
continue // Elide fields with zero values
}
s := opts.WithTypeMode(autoType).FormatValue(vv, m)
s := opts.WithTypeMode(autoType).FormatValue(vv, false, m)
list = append(list, textRecord{Key: t.Field(i).Name, Value: s})
}
return textWrap{"{", list, "}"}
@ -156,7 +159,7 @@ func (opts formatOptions) FormatValue(v reflect.Value, m visitedPointers) (out t
continue
}
}
s := opts.WithTypeMode(elideType).FormatValue(vi, m)
s := opts.WithTypeMode(elideType).FormatValue(vi, true, m)
list = append(list, textRecord{Value: s})
}
return textWrap{ptr + "{", list, "}"}
@ -171,7 +174,7 @@ func (opts formatOptions) FormatValue(v reflect.Value, m visitedPointers) (out t
var list textList
for _, k := range value.SortKeys(v.MapKeys()) {
sk := formatMapKey(k)
sv := opts.WithTypeMode(elideType).FormatValue(v.MapIndex(k), m)
sv := opts.WithTypeMode(elideType).FormatValue(v.MapIndex(k), false, m)
list = append(list, textRecord{Key: sk, Value: sv})
}
if opts.PrintAddresses {
@ -189,7 +192,7 @@ func (opts formatOptions) FormatValue(v reflect.Value, m visitedPointers) (out t
ptr = formatPointer(v)
}
skipType = true // Let the underlying value print the type instead
return textWrap{"&" + ptr, opts.FormatValue(v.Elem(), m), ""}
return textWrap{"&" + ptr, opts.FormatValue(v.Elem(), false, m), ""}
case reflect.Interface:
if v.IsNil() {
return textNil
@ -197,7 +200,7 @@ func (opts formatOptions) FormatValue(v reflect.Value, m visitedPointers) (out t
// Interfaces accept different concrete types,
// so configure the underlying value to explicitly print the type.
skipType = true // Print the concrete type instead
return opts.WithTypeMode(emitType).FormatValue(v.Elem(), m)
return opts.WithTypeMode(emitType).FormatValue(v.Elem(), false, m)
default:
panic(fmt.Sprintf("%v kind not handled", v.Kind()))
}
@ -209,7 +212,7 @@ func formatMapKey(v reflect.Value) string {
var opts formatOptions
opts.TypeMode = elideType
opts.ShallowPointers = true
s := opts.FormatValue(v, visitedPointers{}).String()
s := opts.FormatValue(v, false, visitedPointers{}).String()
return strings.TrimSpace(s)
}

View File

@ -172,7 +172,9 @@ func (opts formatOptions) FormatDiffSlice(v *valueNode) textNode {
switch t.Elem().Kind() {
case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64:
ss = append(ss, fmt.Sprint(v.Index(i).Int()))
case reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, reflect.Uintptr:
case reflect.Uint, reflect.Uint16, reflect.Uint32, reflect.Uint64:
ss = append(ss, fmt.Sprint(v.Index(i).Uint()))
case reflect.Uint8, reflect.Uintptr:
ss = append(ss, formatHex(v.Index(i).Uint()))
case reflect.Bool, reflect.Float32, reflect.Float64, reflect.Complex64, reflect.Complex128:
ss = append(ss, fmt.Sprint(v.Index(i).Interface()))

View File

@ -1,202 +0,0 @@
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction,
and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by
the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity
exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications,
including but not limited to software source code, documentation
source, and configuration files.
"Object" form shall mean any form resulting from mechanical
transformation or translation of a Source form, including but
not limited to compiled object code, generated documentation,
and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or
Object form, made available under the License, as indicated by a
copyright notice that is included in or attached to the work
(an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object
form, that is based on (or derived from) the Work and for which the
editorial revisions, annotations, elaborations, or other modifications
represent, as a whole, an original work of authorship. For the purposes
of this License, Derivative Works shall not include works that remain
separable from, or merely link (or bind by name) to the interfaces of,
the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including
the original version of the Work and any modifications or additions
to that Work or Derivative Works thereof, that is intentionally
submitted to Licensor for inclusion in the Work by the copyright owner
or by an individual or Legal Entity authorized to submit on behalf of
the copyright owner. For the purposes of this definition, "submitted"
means any form of electronic, verbal, or written communication sent
to the Licensor or its representatives, including but not limited to
communication on electronic mailing lists, source code control systems,
and issue tracking systems that are managed by, or on behalf of, the
Licensor for the purpose of discussing and improving the Work, but
excluding communication that is conspicuously marked or otherwise
designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity
on behalf of whom a Contribution has been received by Licensor and
subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
copyright license to reproduce, prepare Derivative Works of,
publicly display, publicly perform, sublicense, and distribute the
Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
(except as stated in this section) patent license to make, have made,
use, offer to sell, sell, import, and otherwise transfer the Work,
where such license applies only to those patent claims licensable
by such Contributor that are necessarily infringed by their
Contribution(s) alone or by combination of their Contribution(s)
with the Work to which such Contribution(s) was submitted. If You
institute patent litigation against any entity (including a
cross-claim or counterclaim in a lawsuit) alleging that the Work
or a Contribution incorporated within the Work constitutes direct
or contributory patent infringement, then any patent licenses
granted to You under this License for that Work shall terminate
as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the
Work or Derivative Works thereof in any medium, with or without
modifications, and in Source or Object form, provided that You
meet the following conditions:
(a) You must give any other recipients of the Work or
Derivative Works a copy of this License; and
(b) You must cause any modified files to carry prominent notices
stating that You changed the files; and
(c) You must retain, in the Source form of any Derivative Works
that You distribute, all copyright, patent, trademark, and
attribution notices from the Source form of the Work,
excluding those notices that do not pertain to any part of
the Derivative Works; and
(d) If the Work includes a "NOTICE" text file as part of its
distribution, then any Derivative Works that You distribute must
include a readable copy of the attribution notices contained
within such NOTICE file, excluding those notices that do not
pertain to any part of the Derivative Works, in at least one
of the following places: within a NOTICE text file distributed
as part of the Derivative Works; within the Source form or
documentation, if provided along with the Derivative Works; or,
within a display generated by the Derivative Works, if and
wherever such third-party notices normally appear. The contents
of the NOTICE file are for informational purposes only and
do not modify the License. You may add Your own attribution
notices within Derivative Works that You distribute, alongside
or as an addendum to the NOTICE text from the Work, provided
that such additional attribution notices cannot be construed
as modifying the License.
You may add Your own copyright statement to Your modifications and
may provide additional or different license terms and conditions
for use, reproduction, or distribution of Your modifications, or
for any such Derivative Works as a whole, provided Your use,
reproduction, and distribution of the Work otherwise complies with
the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise,
any Contribution intentionally submitted for inclusion in the Work
by You to the Licensor shall be under the terms and conditions of
this License, without any additional terms or conditions.
Notwithstanding the above, nothing herein shall supersede or modify
the terms of any separate license agreement you may have executed
with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade
names, trademarks, service marks, or product names of the Licensor,
except as required for reasonable and customary use in describing the
origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or
agreed to in writing, Licensor provides the Work (and each
Contributor provides its Contributions) on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
implied, including, without limitation, any warranties or conditions
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
PARTICULAR PURPOSE. You are solely responsible for determining the
appropriateness of using or redistributing the Work and assume any
risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory,
whether in tort (including negligence), contract, or otherwise,
unless required by applicable law (such as deliberate and grossly
negligent acts) or agreed to in writing, shall any Contributor be
liable to You for damages, including any direct, indirect, special,
incidental, or consequential damages of any character arising as a
result of this License or out of the use or inability to use the
Work (including but not limited to damages for loss of goodwill,
work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses), even if such Contributor
has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing
the Work or Derivative Works thereof, You may choose to offer,
and charge a fee for, acceptance of support, warranty, indemnity,
or other liability obligations and/or rights consistent with this
License. However, in accepting such obligations, You may act only
on Your own behalf and on Your sole responsibility, not on behalf
of any other Contributor, and only if You agree to indemnify,
defend, and hold each Contributor harmless for any liability
incurred by, or claims asserted against, such Contributor by reason
of your accepting any such warranty or additional liability.
END OF TERMS AND CONDITIONS
APPENDIX: How to apply the Apache License to your work.
To apply the Apache License to your work, attach the following
boilerplate notice, with the fields enclosed by brackets "[]"
replaced with your own identifying information. (Don't include
the brackets!) The text should be enclosed in the appropriate
comment syntax for the file format. We also recommend that a
file or class name and description of purpose be included on the
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright [yyyy] [name of copyright owner]
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.

View File

@ -1,242 +0,0 @@
# `authn`
[![GoDoc](https://godoc.org/github.com/google/go-containerregistry/pkg/authn?status.svg)](https://godoc.org/github.com/google/go-containerregistry/pkg/authn)
This README outlines how we acquire and use credentials when interacting with a registry.
As much as possible, we attempt to emulate docker's authentication behavior and configuration so that this library "just works" if you've already configured credentials that work with docker; however, when things don't work, a basic understanding of what's going on can help with debugging.
The official documentation for how docker authentication works is (reasonably) scattered across several different sites and GitHub repositories, so we've tried to summarize the relevant bits here.
## tl;dr for consumers of this package
By default, [`pkg/v1/remote`](https://godoc.org/github.com/google/go-containerregistry/pkg/v1/remote) uses [`Anonymous`](https://godoc.org/github.com/google/go-containerregistry/pkg/authn#Anonymous) credentials (i.e. _none_), which for most registries will only allow read access to public images.
To use the credentials found in your docker config file, you can use the [`DefaultKeychain`](https://godoc.org/github.com/google/go-containerregistry/pkg/authn#DefaultKeychain), e.g.:
```go
package main
import (
"fmt"
"github.com/google/go-containerregistry/pkg/authn"
"github.com/google/go-containerregistry/pkg/name"
"github.com/google/go-containerregistry/pkg/v1/remote"
)
func main() {
ref, err := name.ParseReference("registry.example.com/private/repo")
if err != nil {
panic(err)
}
// Fetch the manifest using default credentials.
img, err := remote.Get(ref, remote.WithAuthFromKeychain(authn.DefaultKeychain))
if err != nil {
panic(err)
}
// Prints the digest of registry.example.com/private/repo
fmt.Println(img.Digest)
}
```
(If you're only using [gcr.io](https://gcr.io), see the [`pkg/v1/google.Keychain`](https://godoc.org/github.com/google/go-containerregistry/pkg/v1/google#Keychain), which emulates [`docker-credential-gcr`](https://github.com/GoogleCloudPlatform/docker-credential-gcr).)
## The Config File
This file contains various configuration options for docker and is (by default) located at:
* `$HOME/.docker/config.json` (on linux and darwin), or
* `%USERPROFILE%\.docker\config.json` (on windows).
You can override this location with the `DOCKER_CONFIG` environment variable.
### Plaintext
The config file is where your credentials are stored when you invoke `docker login`, e.g. the contents may look something like this:
```json
{
"auths": {
"registry.example.com": {
"auth": "QXp1cmVEaWFtb25kOmh1bnRlcjI="
}
}
}
```
The `auths` map has an entry per registry, and the `auth` field contains your username and password encoded as [HTTP 'Basic' Auth](https://tools.ietf.org/html/rfc7617).
**NOTE**: This means that your credentials are stored _in plaintext_:
```bash
$ echo "QXp1cmVEaWFtb25kOmh1bnRlcjI=" | base64 -d
AzureDiamond:hunter2
```
For what it's worth, this config file is equivalent to:
```json
{
"auths": {
"registry.example.com": {
"username": "AzureDiamond",
"password": "hunter2"
}
}
}
```
... which is useful to know if e.g. your CI system provides you a registry username and password via environment variables and you want to populate this file manually without invoking `docker login`.
### Helpers
If you log in like this, docker will warn you that you should use a [credential helper](https://docs.docker.com/engine/reference/commandline/login/#credentials-store), and you should!
To configure a global credential helper:
```json
{
"credsStore": "osxkeychain"
}
```
To configure a per-registry credential helper:
```json
{
"credHelpers": {
"gcr.io": "gcr"
}
}
```
We use [`github.com/docker/cli/cli/config.Load`](https://godoc.org/github.com/docker/cli/cli/config#Load) to parse the config file and invoke any necessary credential helpers. This handles the logic of taking a [`ConfigFile`](https://github.com/docker/cli/blob/ba63a92655c0bea4857b8d6cc4991498858b3c60/cli/config/configfile/file.go#L25-L54) + registry domain and producing an [`AuthConfig`](https://github.com/docker/cli/blob/ba63a92655c0bea4857b8d6cc4991498858b3c60/cli/config/types/authconfig.go#L3-L22), which determines how we authenticate to the registry.
## Credential Helpers
The [credential helper protocol](https://github.com/docker/docker-credential-helpers) allows you to configure a binary that supplies credentials for the registry, rather than hard-coding them in the config file.
The protocol has several verbs, but the one we most care about is `get`.
For example, using the following config file:
```json
{
"credHelpers": {
"gcr.io": "gcr",
"eu.gcr.io": "gcr"
}
}
```
To acquire credentials for `gcr.io`, we look in the `credHelpers` map to find
the credential helper for `gcr.io` is `gcr`. By appending that value to
`docker-credential-`, we can get the name of the binary we need to use.
For this example, that's `docker-credential-gcr`, which must be on our `$PATH`.
We'll then invoke that binary to get credentials:
```bash
$ echo "gcr.io" | docker-credential-gcr get
{"Username":"_token","Secret":"<long access token>"}
```
You can configure the same credential helper for multiple registries, which is
why we need to pass the domain in via STDIN, e.g. if we were trying to access
`eu.gcr.io`, we'd do this instead:
```bash
$ echo "eu.gcr.io" | docker-credential-gcr get
{"Username":"_token","Secret":"<long access token>"}
```
### Debugging credential helpers
If a credential helper is configured but doesn't seem to be working, it can be
challenging to debug. Implementing a fake credential helper lets you poke around
to make it easier to see where the failure is happening.
This "implements" a credential helper with hard-coded values:
```
#!/usr/bin/env bash
echo '{"Username":"<token>","Secret":"hunter2"}'
```
This implements a credential helper that prints the output of
`docker-credential-gcr` to both stderr and whatever called it, which allows you
to snoop on another credential helper:
```
#!/usr/bin/env bash
docker-credential-gcr $@ | tee >(cat 1>&2)
```
Put those files somewhere on your path, naming them e.g.
`docker-credential-hardcoded` and `docker-credential-tee`, then modify the
config file to use them:
```json
{
"credHelpers": {
"gcr.io": "tee",
"eu.gcr.io": "hardcoded"
}
}
```
The `docker-credential-tee` trick works with both `crane` and `docker`:
```bash
$ crane manifest gcr.io/google-containers/pause > /dev/null
{"ServerURL":"","Username":"_dcgcr_1_5_0_token","Secret":"<redacted>"}
$ docker pull gcr.io/google-containers/pause
Using default tag: latest
{"ServerURL":"","Username":"_dcgcr_1_5_0_token","Secret":"<redacted>"}
latest: Pulling from google-containers/pause
a3ed95caeb02: Pull complete
4964c72cd024: Pull complete
Digest: sha256:a78c2d6208eff9b672de43f880093100050983047b7b0afe0217d3656e1b0d5f
Status: Downloaded newer image for gcr.io/google-containers/pause:latest
gcr.io/google-containers/pause:latest
```
## The Registry
There are two methods for authenticating against a registry:
[token](https://docs.docker.com/registry/spec/auth/token/) and
[oauth2](https://docs.docker.com/registry/spec/auth/oauth/).
Both methods are used to acquire an opaque `Bearer` token (or
[RegistryToken](https://github.com/docker/cli/blob/ba63a92655c0bea4857b8d6cc4991498858b3c60/cli/config/types/authconfig.go#L21))
to use in the `Authorization` header. The registry will return a `401
Unauthorized` during the [version
check](https://github.com/opencontainers/distribution-spec/blob/2c3975d1f03b67c9a0203199038adea0413f0573/spec.md#api-version-check)
(or during normal operations) with
[Www-Authenticate](https://tools.ietf.org/html/rfc7235#section-4.1) challenge
indicating how to proceed.
### Token
If we get back an `AuthConfig` containing a [`Username/Password`](https://github.com/docker/cli/blob/ba63a92655c0bea4857b8d6cc4991498858b3c60/cli/config/types/authconfig.go#L5-L6)
or
[`Auth`](https://github.com/docker/cli/blob/ba63a92655c0bea4857b8d6cc4991498858b3c60/cli/config/types/authconfig.go#L7),
we'll use the token method for authentication:
![basic](../../images/credhelper-basic.svg)
### OAuth 2
If we get back an `AuthConfig` containing an [`IdentityToken`](https://github.com/docker/cli/blob/ba63a92655c0bea4857b8d6cc4991498858b3c60/cli/config/types/authconfig.go#L18)
we'll use the oauth2 method for authentication:
![oauth](../../images/credhelper-oauth.svg)
This happens when a credential helper returns a response with the
[`Username`](https://github.com/docker/docker-credential-helpers/blob/f78081d1f7fef6ad74ad6b79368de6348386e591/credentials/credentials.go#L16)
set to `<token>` (no, that's not a placeholder, the literal string `"<token>"`).
It is unclear why: [moby/moby#36926](https://github.com/moby/moby/issues/36926).
We only support the oauth2 `grant_type` for `refresh_token` ([#629](https://github.com/google/go-containerregistry/issues/629)),
since it's impossible to determine from the registry response whether we should
use oauth, and the token method for authentication is widely implemented by
registries.

View File

@ -1,26 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package authn
// anonymous implements Authenticator for anonymous authentication.
type anonymous struct{}
// Authorization implements Authenticator.
func (a *anonymous) Authorization() (*AuthConfig, error) {
return &AuthConfig{}, nil
}
// Anonymous is a singleton Authenticator for providing anonymous auth.
var Anonymous Authenticator = &anonymous{}

View File

@ -1,30 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package authn
// auth is an Authenticator that simply returns the wrapped AuthConfig.
type auth struct {
config AuthConfig
}
// FromConfig returns an Authenticator that just returns the given AuthConfig.
func FromConfig(cfg AuthConfig) Authenticator {
return &auth{cfg}
}
// Authorization implements Authenticator.
func (a *auth) Authorization() (*AuthConfig, error) {
return &a.config, nil
}

View File

@ -1,36 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package authn
// AuthConfig contains authorization information for connecting to a Registry
// Inlined what we use from github.com/cli/cli/config/types
type AuthConfig struct {
Username string `json:"username,omitempty"`
Password string `json:"password,omitempty"`
Auth string `json:"auth,omitempty"`
// IdentityToken is used to authenticate the user and get
// an access token for the registry.
IdentityToken string `json:"identitytoken,omitempty"`
// RegistryToken is a bearer token to be sent to a registry
RegistryToken string `json:"registrytoken,omitempty"`
}
// Authenticator is used to authenticate Docker transports.
type Authenticator interface {
// Authorization returns the value to use in an http transport's Authorization header.
Authorization() (*AuthConfig, error)
}

View File

@ -1,29 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package authn
// Basic implements Authenticator for basic authentication.
type Basic struct {
Username string
Password string
}
// Authorization implements Authenticator.
func (b *Basic) Authorization() (*AuthConfig, error) {
return &AuthConfig{
Username: b.Username,
Password: b.Password,
}, nil
}

View File

@ -1,27 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package authn
// Bearer implements Authenticator for bearer authentication.
type Bearer struct {
Token string `json:"token"`
}
// Authorization implements Authenticator.
func (b *Bearer) Authorization() (*AuthConfig, error) {
return &AuthConfig{
RegistryToken: b.Token,
}, nil
}

View File

@ -1,17 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// Package authn defines different methods of authentication for
// talking to a container registry.
package authn

View File

@ -1,97 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package authn
import (
"encoding/json"
"os"
"github.com/docker/cli/cli/config"
"github.com/docker/cli/cli/config/types"
"github.com/google/go-containerregistry/pkg/logs"
"github.com/google/go-containerregistry/pkg/name"
)
// Resource represents a registry or repository that can be authenticated against.
type Resource interface {
// String returns the full string representation of the target, e.g.
// gcr.io/my-project or just gcr.io.
String() string
// RegistryStr returns just the registry portion of the target, e.g. for
// gcr.io/my-project, this should just return gcr.io. This is needed to
// pull out an appropriate hostname.
RegistryStr() string
}
// Keychain is an interface for resolving an image reference to a credential.
type Keychain interface {
// Resolve looks up the most appropriate credential for the specified target.
Resolve(Resource) (Authenticator, error)
}
// defaultKeychain implements Keychain with the semantics of the standard Docker
// credential keychain.
type defaultKeychain struct{}
var (
// DefaultKeychain implements Keychain by interpreting the docker config file.
DefaultKeychain Keychain = &defaultKeychain{}
)
const (
// DefaultAuthKey is the key used for dockerhub in config files, which
// is hardcoded for historical reasons.
DefaultAuthKey = "https://" + name.DefaultRegistry + "/v1/"
)
// Resolve implements Keychain.
func (dk *defaultKeychain) Resolve(target Resource) (Authenticator, error) {
cf, err := config.Load(os.Getenv("DOCKER_CONFIG"))
if err != nil {
return nil, err
}
// See:
// https://github.com/google/ko/issues/90
// https://github.com/moby/moby/blob/fc01c2b481097a6057bec3cd1ab2d7b4488c50c4/registry/config.go#L397-L404
key := target.RegistryStr()
if key == name.DefaultRegistry {
key = DefaultAuthKey
}
cfg, err := cf.GetAuthConfig(key)
if err != nil {
return nil, err
}
if logs.Enabled(logs.Debug) {
b, err := json.Marshal(cfg)
if err == nil {
logs.Debug.Printf("defaultKeychain.Resolve(%q) = %s", key, string(b))
}
}
empty := types.AuthConfig{}
if cfg == empty {
return Anonymous, nil
}
return FromConfig(AuthConfig{
Username: cfg.Username,
Password: cfg.Password,
Auth: cfg.Auth,
IdentityToken: cfg.IdentityToken,
RegistryToken: cfg.RegistryToken,
}), nil
}

View File

@ -1,41 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package authn
type multiKeychain struct {
keychains []Keychain
}
// Assert that our multi-keychain implements Keychain.
var _ (Keychain) = (*multiKeychain)(nil)
// NewMultiKeychain composes a list of keychains into one new keychain.
func NewMultiKeychain(kcs ...Keychain) Keychain {
return &multiKeychain{keychains: kcs}
}
// Resolve implements Keychain.
func (mk *multiKeychain) Resolve(target Resource) (Authenticator, error) {
for _, kc := range mk.keychains {
auth, err := kc.Resolve(target)
if err != nil {
return nil, err
}
if auth != Anonymous {
return auth, nil
}
}
return Anonymous, nil
}

View File

@ -1,77 +0,0 @@
// Copyright 2019 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// Package retry provides methods for retrying operations. It is a thin wrapper
// around k8s.io/apimachinery/pkg/util/wait to make certain operations easier.
package retry
import (
"context"
"fmt"
"github.com/google/go-containerregistry/pkg/internal/retry/wait"
)
// Backoff is an alias of our own wait.Backoff to avoid name conflicts with
// the kubernetes wait package. Typing retry.Backoff is aesier than fixing
// the wrong import every time you use wait.Backoff.
type Backoff = wait.Backoff
// This is implemented by several errors in the net package as well as our
// transport.Error.
type temporary interface {
Temporary() bool
}
// IsTemporary returns true if err implements Temporary() and it returns true.
func IsTemporary(err error) bool {
if err == context.DeadlineExceeded {
return false
}
if te, ok := err.(temporary); ok && te.Temporary() {
return true
}
return false
}
// IsNotNil returns true if err is not nil.
func IsNotNil(err error) bool {
return err != nil
}
// Predicate determines whether an error should be retried.
type Predicate func(error) (retry bool)
// Retry retries a given function, f, until a predicate is satisfied, using
// exponential backoff. If the predicate is never satisfied, it will return the
// last error returned by f.
func Retry(f func() error, p Predicate, backoff wait.Backoff) (err error) {
if f == nil {
return fmt.Errorf("nil f passed to retry")
}
if p == nil {
return fmt.Errorf("nil p passed to retry")
}
condition := func() (bool, error) {
err = f()
if p(err) {
return false, nil
}
return true, err
}
wait.ExponentialBackoff(backoff, condition)
return
}

View File

@ -1,123 +0,0 @@
/*
Copyright 2014 The Kubernetes Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
// Package wait is a subset of k8s.io/apimachinery to avoid conflicts
// in dependencies (specifically, logging).
package wait
import (
"errors"
"math/rand"
"time"
)
// Jitter returns a time.Duration between duration and duration + maxFactor *
// duration.
//
// This allows clients to avoid converging on periodic behavior. If maxFactor
// is 0.0, a suggested default value will be chosen.
func Jitter(duration time.Duration, maxFactor float64) time.Duration {
if maxFactor <= 0.0 {
maxFactor = 1.0
}
wait := duration + time.Duration(rand.Float64()*maxFactor*float64(duration))
return wait
}
// ErrWaitTimeout is returned when the condition exited without success.
var ErrWaitTimeout = errors.New("timed out waiting for the condition")
// ConditionFunc returns true if the condition is satisfied, or an error
// if the loop should be aborted.
type ConditionFunc func() (done bool, err error)
// Backoff holds parameters applied to a Backoff function.
type Backoff struct {
// The initial duration.
Duration time.Duration
// Duration is multiplied by factor each iteration, if factor is not zero
// and the limits imposed by Steps and Cap have not been reached.
// Should not be negative.
// The jitter does not contribute to the updates to the duration parameter.
Factor float64
// The sleep at each iteration is the duration plus an additional
// amount chosen uniformly at random from the interval between
// zero and `jitter*duration`.
Jitter float64
// The remaining number of iterations in which the duration
// parameter may change (but progress can be stopped earlier by
// hitting the cap). If not positive, the duration is not
// changed. Used for exponential backoff in combination with
// Factor and Cap.
Steps int
// A limit on revised values of the duration parameter. If a
// multiplication by the factor parameter would make the duration
// exceed the cap then the duration is set to the cap and the
// steps parameter is set to zero.
Cap time.Duration
}
// Step (1) returns an amount of time to sleep determined by the
// original Duration and Jitter and (2) mutates the provided Backoff
// to update its Steps and Duration.
func (b *Backoff) Step() time.Duration {
if b.Steps < 1 {
if b.Jitter > 0 {
return Jitter(b.Duration, b.Jitter)
}
return b.Duration
}
b.Steps--
duration := b.Duration
// calculate the next step
if b.Factor != 0 {
b.Duration = time.Duration(float64(b.Duration) * b.Factor)
if b.Cap > 0 && b.Duration > b.Cap {
b.Duration = b.Cap
b.Steps = 0
}
}
if b.Jitter > 0 {
duration = Jitter(duration, b.Jitter)
}
return duration
}
// ExponentialBackoff repeats a condition check with exponential backoff.
//
// It repeatedly checks the condition and then sleeps, using `backoff.Step()`
// to determine the length of the sleep and adjust Duration and Steps.
// Stops and returns as soon as:
// 1. the condition check returns true or an error,
// 2. `backoff.Steps` checks of the condition have been done, or
// 3. a sleep truncated by the cap on duration has been completed.
// In case (1) the returned error is what the condition function returned.
// In all other cases, ErrWaitTimeout is returned.
func ExponentialBackoff(backoff Backoff, condition ConditionFunc) error {
for backoff.Steps > 0 {
if ok, err := condition(); err != nil || ok {
return err
}
if backoff.Steps == 1 {
break
}
time.Sleep(backoff.Step())
}
return ErrWaitTimeout
}

View File

@ -1,39 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// Package logs exposes the loggers used by this library.
package logs
import (
"io/ioutil"
"log"
)
var (
// Warn is used to log non-fatal errors.
Warn = log.New(ioutil.Discard, "", log.LstdFlags)
// Progress is used to log notable, successful events.
Progress = log.New(ioutil.Discard, "", log.LstdFlags)
// Debug is used to log information that is useful for debugging.
Debug = log.New(ioutil.Discard, "", log.LstdFlags)
)
// Enabled checks to see if the logger's writer is set to something other
// than ioutil.Discard. This allows callers to avoid expensive operations
// that will end up in /dev/null anyway.
func Enabled(l *log.Logger) bool {
return l.Writer() != ioutil.Discard
}

View File

@ -1,3 +0,0 @@
# `name`
[![GoDoc](https://godoc.org/github.com/google/go-containerregistry/pkg/name?status.svg)](https://godoc.org/github.com/google/go-containerregistry/pkg/name)

View File

@ -1,43 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package name
import (
"strings"
"unicode/utf8"
)
// stripRunesFn returns a function which returns -1 (i.e. a value which
// signals deletion in strings.Map) for runes in 'runes', and the rune otherwise.
func stripRunesFn(runes string) func(rune) rune {
return func(r rune) rune {
if strings.ContainsRune(runes, r) {
return -1
}
return r
}
}
// checkElement checks a given named element matches character and length restrictions.
// Returns true if the given element adheres to the given restrictions, false otherwise.
func checkElement(name, element, allowedRunes string, minRunes, maxRunes int) error {
numRunes := utf8.RuneCountInString(element)
if (numRunes < minRunes) || (maxRunes < numRunes) {
return NewErrBadName("%s must be between %d and %d runes in length: %s", name, minRunes, maxRunes, element)
} else if len(strings.Map(stripRunesFn(allowedRunes), element)) != 0 {
return NewErrBadName("%s can only contain the runes `%s`: %s", name, allowedRunes, element)
}
return nil
}

View File

@ -1,96 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package name
import (
"strings"
)
const (
// These have the form: sha256:<hex string>
// TODO(dekkagaijin): replace with opencontainers/go-digest or docker/distribution's validation.
digestChars = "sh:0123456789abcdef"
digestDelim = "@"
)
// Digest stores a digest name in a structured form.
type Digest struct {
Repository
digest string
original string
}
// Ensure Digest implements Reference
var _ Reference = (*Digest)(nil)
// Context implements Reference.
func (d Digest) Context() Repository {
return d.Repository
}
// Identifier implements Reference.
func (d Digest) Identifier() string {
return d.DigestStr()
}
// DigestStr returns the digest component of the Digest.
func (d Digest) DigestStr() string {
return d.digest
}
// Name returns the name from which the Digest was derived.
func (d Digest) Name() string {
return d.Repository.Name() + digestDelim + d.DigestStr()
}
// String returns the original input string.
func (d Digest) String() string {
return d.original
}
func checkDigest(name string) error {
return checkElement("digest", name, digestChars, 7+64, 7+64)
}
// NewDigest returns a new Digest representing the given name.
func NewDigest(name string, opts ...Option) (Digest, error) {
// Split on "@"
parts := strings.Split(name, digestDelim)
if len(parts) != 2 {
return Digest{}, NewErrBadName("a digest must contain exactly one '@' separator (e.g. registry/repository@digest) saw: %s", name)
}
base := parts[0]
digest := parts[1]
// Always check that the digest is valid.
if err := checkDigest(digest); err != nil {
return Digest{}, err
}
tag, err := NewTag(base, opts...)
if err == nil {
base = tag.Repository.Name()
}
repo, err := NewRepository(base, opts...)
if err != nil {
return Digest{}, err
}
return Digest{
Repository: repo,
digest: digest,
original: name,
}, nil
}

View File

@ -1,42 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// Package name defines structured types for representing image references.
//
// What's in a name? For image references, not nearly enough!
//
// Image references look a lot like URLs, but they differ in that they don't
// contain the scheme (http or https), they can end with a :tag or a @digest
// (the latter being validated), and they perform defaulting for missing
// components.
//
// Since image references don't contain the scheme, we do our best to infer
// if we use http or https from the given hostname. We allow http fallback for
// any host that looks like localhost (localhost, 127.0.0.1, ::1), ends in
// ".local", or is in the "private" address space per RFC 1918. For everything
// else, we assume https only. To override this heuristic, use the Insecure
// option.
//
// Image references with a digest signal to us that we should verify the content
// of the image matches the digest. E.g. when pulling a Digest reference, we'll
// calculate the sha256 of the manifest returned by the registry and error out
// if it doesn't match what we asked for.
//
// For defaulting, we interpret "ubuntu" as
// "index.docker.io/library/ubuntu:latest" because we add the missing repo
// "library", the missing registry "index.docker.io", and the missing tag
// "latest". To disable this defaulting, use the StrictValidation option. This
// is useful e.g. to only allow image references that explicitly set a tag or
// digest, so that you don't accidentally pull "latest".
package name

View File

@ -1,37 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package name
import "fmt"
// ErrBadName is an error for when a bad docker name is supplied.
type ErrBadName struct {
info string
}
func (e *ErrBadName) Error() string {
return e.info
}
// NewErrBadName returns a ErrBadName which returns the given formatted string from Error().
func NewErrBadName(fmtStr string, args ...interface{}) *ErrBadName {
return &ErrBadName{fmt.Sprintf(fmtStr, args...)}
}
// IsErrBadName returns true if the given error is an ErrBadName.
func IsErrBadName(err error) bool {
_, ok := err.(*ErrBadName)
return ok
}

View File

@ -1,49 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package name
type options struct {
strict bool // weak by default
insecure bool // secure by default
}
func makeOptions(opts ...Option) options {
opt := options{}
for _, o := range opts {
o(&opt)
}
return opt
}
// Option is a functional option for name parsing.
type Option func(*options)
// StrictValidation is an Option that requires image references to be fully
// specified; i.e. no defaulting for registry (dockerhub), repo (library),
// or tag (latest).
func StrictValidation(opts *options) {
opts.strict = true
}
// WeakValidation is an Option that sets defaults when parsing names, see
// StrictValidation.
func WeakValidation(opts *options) {
opts.strict = false
}
// Insecure is an Option that allows image references to be fetched without TLS.
func Insecure(opts *options) {
opts.insecure = true
}

View File

@ -1,49 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package name
import (
"fmt"
)
// Reference defines the interface that consumers use when they can
// take either a tag or a digest.
type Reference interface {
fmt.Stringer
// Context accesses the Repository context of the reference.
Context() Repository
// Identifier accesses the type-specific portion of the reference.
Identifier() string
// Name is the fully-qualified reference name.
Name() string
// Scope is the scope needed to access this reference.
Scope(string) string
}
// ParseReference parses the string as a reference, either by tag or digest.
func ParseReference(s string, opts ...Option) (Reference, error) {
if t, err := NewTag(s, opts...); err == nil {
return t, nil
}
if d, err := NewDigest(s, opts...); err == nil {
return d, nil
}
return nil, NewErrBadName("could not parse reference: " + s)
}

View File

@ -1,142 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package name
import (
"net"
"net/url"
"regexp"
"strings"
)
const (
// DefaultRegistry is Docker Hub, assumed when a hostname is omitted.
DefaultRegistry = "index.docker.io"
defaultRegistryAlias = "docker.io"
)
// Detect more complex forms of local references.
var reLocal = regexp.MustCompile(`.*\.local(?:host)?(?::\d{1,5})?$`)
// Detect the loopback IP (127.0.0.1)
var reLoopback = regexp.MustCompile(regexp.QuoteMeta("127.0.0.1"))
// Detect the loopback IPV6 (::1)
var reipv6Loopback = regexp.MustCompile(regexp.QuoteMeta("::1"))
// Registry stores a docker registry name in a structured form.
type Registry struct {
insecure bool
registry string
}
// RegistryStr returns the registry component of the Registry.
func (r Registry) RegistryStr() string {
if r.registry != "" {
return r.registry
}
return DefaultRegistry
}
// Name returns the name from which the Registry was derived.
func (r Registry) Name() string {
return r.RegistryStr()
}
func (r Registry) String() string {
return r.Name()
}
// Scope returns the scope required to access the registry.
func (r Registry) Scope(string) string {
// The only resource under 'registry' is 'catalog'. http://goo.gl/N9cN9Z
return "registry:catalog:*"
}
func (r Registry) isRFC1918() bool {
ipStr := strings.Split(r.Name(), ":")[0]
ip := net.ParseIP(ipStr)
if ip == nil {
return false
}
for _, cidr := range []string{"10.0.0.0/8", "172.16.0.0/12", "192.168.0.0/16"} {
_, block, _ := net.ParseCIDR(cidr)
if block.Contains(ip) {
return true
}
}
return false
}
// Scheme returns https scheme for all the endpoints except localhost or when explicitly defined.
func (r Registry) Scheme() string {
if r.insecure {
return "http"
}
if r.isRFC1918() {
return "http"
}
if strings.HasPrefix(r.Name(), "localhost:") {
return "http"
}
if reLocal.MatchString(r.Name()) {
return "http"
}
if reLoopback.MatchString(r.Name()) {
return "http"
}
if reipv6Loopback.MatchString(r.Name()) {
return "http"
}
return "https"
}
func checkRegistry(name string) error {
// Per RFC 3986, registries (authorities) are required to be prefixed with "//"
// url.Host == hostname[:port] == authority
if url, err := url.Parse("//" + name); err != nil || url.Host != name {
return NewErrBadName("registries must be valid RFC 3986 URI authorities: %s", name)
}
return nil
}
// NewRegistry returns a Registry based on the given name.
// Strict validation requires explicit, valid RFC 3986 URI authorities to be given.
func NewRegistry(name string, opts ...Option) (Registry, error) {
opt := makeOptions(opts...)
if opt.strict && len(name) == 0 {
return Registry{}, NewErrBadName("strict validation requires the registry to be explicitly defined")
}
if err := checkRegistry(name); err != nil {
return Registry{}, err
}
// Rewrite "docker.io" to "index.docker.io".
// See: https://github.com/google/go-containerregistry/issues/68
if name == defaultRegistryAlias {
name = DefaultRegistry
}
return Registry{registry: name, insecure: opt.insecure}, nil
}
// NewInsecureRegistry returns an Insecure Registry based on the given name.
//
// Deprecated: Use the Insecure Option with NewRegistry instead.
func NewInsecureRegistry(name string, opts ...Option) (Registry, error) {
opts = append(opts, Insecure)
return NewRegistry(name, opts...)
}

View File

@ -1,121 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package name
import (
"fmt"
"strings"
)
const (
defaultNamespace = "library"
repositoryChars = "abcdefghijklmnopqrstuvwxyz0123456789_-./"
regRepoDelimiter = "/"
)
// Repository stores a docker repository name in a structured form.
type Repository struct {
Registry
repository string
}
// See https://docs.docker.com/docker-hub/official_repos
func hasImplicitNamespace(repo string, reg Registry) bool {
return !strings.ContainsRune(repo, '/') && reg.RegistryStr() == DefaultRegistry
}
// RepositoryStr returns the repository component of the Repository.
func (r Repository) RepositoryStr() string {
if hasImplicitNamespace(r.repository, r.Registry) {
return fmt.Sprintf("%s/%s", defaultNamespace, r.repository)
}
return r.repository
}
// Name returns the name from which the Repository was derived.
func (r Repository) Name() string {
regName := r.Registry.Name()
if regName != "" {
return regName + regRepoDelimiter + r.RepositoryStr()
}
// TODO: As far as I can tell, this is unreachable.
return r.RepositoryStr()
}
func (r Repository) String() string {
return r.Name()
}
// Scope returns the scope required to perform the given action on the registry.
// TODO(jonjohnsonjr): consider moving scopes to a separate package.
func (r Repository) Scope(action string) string {
return fmt.Sprintf("repository:%s:%s", r.RepositoryStr(), action)
}
func checkRepository(repository string) error {
return checkElement("repository", repository, repositoryChars, 2, 255)
}
// NewRepository returns a new Repository representing the given name, according to the given strictness.
func NewRepository(name string, opts ...Option) (Repository, error) {
opt := makeOptions(opts...)
if len(name) == 0 {
return Repository{}, NewErrBadName("a repository name must be specified")
}
var registry string
repo := name
parts := strings.SplitN(name, regRepoDelimiter, 2)
if len(parts) == 2 && (strings.ContainsRune(parts[0], '.') || strings.ContainsRune(parts[0], ':')) {
// The first part of the repository is treated as the registry domain
// iff it contains a '.' or ':' character, otherwise it is all repository
// and the domain defaults to Docker Hub.
registry = parts[0]
repo = parts[1]
}
if err := checkRepository(repo); err != nil {
return Repository{}, err
}
reg, err := NewRegistry(registry, opts...)
if err != nil {
return Repository{}, err
}
if hasImplicitNamespace(repo, reg) && opt.strict {
return Repository{}, NewErrBadName("strict validation requires the full repository path (missing 'library')")
}
return Repository{reg, repo}, nil
}
// Tag returns a Tag in this Repository.
func (r Repository) Tag(identifier string) Tag {
t := Tag{
tag: identifier,
Repository: r,
}
t.original = t.Name()
return t
}
// Digest returns a Digest in this Repository.
func (r Repository) Digest(identifier string) Digest {
d := Digest{
digest: identifier,
Repository: r,
}
d.original = d.Name()
return d
}

View File

@ -1,108 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package name
import (
"strings"
)
const (
defaultTag = "latest"
// TODO(dekkagaijin): use the docker/distribution regexes for validation.
tagChars = "abcdefghijklmnopqrstuvwxyz0123456789_-.ABCDEFGHIJKLMNOPQRSTUVWXYZ"
tagDelim = ":"
)
// Tag stores a docker tag name in a structured form.
type Tag struct {
Repository
tag string
original string
}
// Ensure Tag implements Reference
var _ Reference = (*Tag)(nil)
// Context implements Reference.
func (t Tag) Context() Repository {
return t.Repository
}
// Identifier implements Reference.
func (t Tag) Identifier() string {
return t.TagStr()
}
// TagStr returns the tag component of the Tag.
func (t Tag) TagStr() string {
if t.tag != "" {
return t.tag
}
return defaultTag
}
// Name returns the name from which the Tag was derived.
func (t Tag) Name() string {
return t.Repository.Name() + tagDelim + t.TagStr()
}
// String returns the original input string.
func (t Tag) String() string {
return t.original
}
// Scope returns the scope required to perform the given action on the tag.
func (t Tag) Scope(action string) string {
return t.Repository.Scope(action)
}
func checkTag(name string) error {
return checkElement("tag", name, tagChars, 1, 127)
}
// NewTag returns a new Tag representing the given name, according to the given strictness.
func NewTag(name string, opts ...Option) (Tag, error) {
opt := makeOptions(opts...)
base := name
tag := ""
// Split on ":"
parts := strings.Split(name, tagDelim)
// Verify that we aren't confusing a tag for a hostname w/ port for the purposes of weak validation.
if len(parts) > 1 && !strings.Contains(parts[len(parts)-1], regRepoDelimiter) {
base = strings.Join(parts[:len(parts)-1], tagDelim)
tag = parts[len(parts)-1]
}
// We don't require a tag, but if we get one check it's valid,
// even when not being strict.
// If we are being strict, we want to validate the tag regardless in case
// it's empty.
if tag != "" || opt.strict {
if err := checkTag(tag); err != nil {
return Tag{}, err
}
}
repo, err := NewRepository(base, opts...)
if err != nil {
return Tag{}, err
}
return Tag{
Repository: repo,
tag: tag,
original: name,
}, nil
}

View File

@ -1,129 +0,0 @@
# `transport`
[![GoDoc](https://godoc.org/github.com/google/go-containerregistry/pkg/v1/transport?status.svg)](https://godoc.org/github.com/google/go-containerregistry/pkg/v1/transport)
The [distribution protocol](https://github.com/opencontainers/distribution-spec) is fairly simple, but correctly [implementing authentication](../../../authn/README.md) is **hard**.
This package [implements](https://godoc.org/github.com/google/go-containerregistry/pkg/v1/remote/transport#New) an [`http.RoundTripper`](https://godoc.org/net/http#RoundTripper)
that transparently performs:
* [Token
Authentication](https://docs.docker.com/registry/spec/auth/token/) and
* [OAuth2
Authentication](https://docs.docker.com/registry/spec/auth/oauth/)
for registry clients.
## Raison d'être
> Why not just use the [`docker/distribution`](https://godoc.org/github.com/docker/distribution/registry/client/auth) client?
Great question! Mostly, because I don't want to depend on [`prometheus/client_golang`](https://github.com/prometheus/client_golang).
As a performance optimization, that client uses [a cache](https://github.com/docker/distribution/blob/a8371794149d1d95f1e846744b05c87f2f825e5a/registry/client/repository.go#L173) to keep track of a mapping between blob digests and their [descriptors](https://github.com/docker/distribution/blob/a8371794149d1d95f1e846744b05c87f2f825e5a/blobs.go#L57-L86). Unfortunately, the cache [uses prometheus](https://github.com/docker/distribution/blob/a8371794149d1d95f1e846744b05c87f2f825e5a/registry/storage/cache/cachedblobdescriptorstore.go#L44) to track hits and misses, so if you want to use that client you have to pull in all of prometheus, which is pretty large.
![docker/distribution](../../../../images/docker.dot.svg)
> Why does it matter if you depend on prometheus? Who cares?
It's generally polite to your downstream to reduce the number of dependencies your package requires:
* Downloading your package is faster, which helps our Australian friends and people on airplanes.
* There is less code to compile, which speeds up builds and saves the planet from global warming.
* You reduce the likelihood of inflicting dependency hell upon your consumers.
* [Tim Hockin](https://twitter.com/thockin/status/958606077456654336) prefers it based on his experience working on Kubernetes, and he's a pretty smart guy.
> Okay, what about [`containerd/containerd`](https://godoc.org/github.com/containerd/containerd/remotes/docker)?
Similar reasons! That ends up pulling in grpc, protobuf, and logrus.
![containerd/containerd](../../../../images/containerd.dot.svg)
> Well... what about [`containers/image`](https://godoc.org/github.com/containers/image/docker)?
That just uses the the `docker/distribution` client... and more!
![containerd/containerd](../../../../images/containers.dot.svg)
> Wow, what about this package?
Of course, this package isn't perfect either. `transport` depends on `authn`,
which in turn depends on docker's config file parsing and handling package,
which you don't strictly need but almost certainly want if you're going to be
interacting with a registry.
![google/go-containerregistry](../../../../images/ggcr.dot.svg)
*These graphs were generated by
[`kisielk/godepgraph`](https://github.com/kisielk/godepgraph).*
## Usage
This is heavily used by the
[`remote`](https://godoc.org/github.com/google/go-containerregistry/pkg/v1/remote)
package, which implements higher level image-centric functionality, but this
package is useful if you want to interact directly with the registry to do
something that `remote` doesn't support, e.g. [to handle with schema 1
images](https://github.com/google/go-containerregistry/pull/509).
This package also includes some [error
handling](https://github.com/opencontainers/distribution-spec/blob/60be706c34ee7805bdd1d3d11affec53b0dfb8fb/spec.md#errors)
facilities in the form of
[`CheckError`](https://godoc.org/github.com/google/go-containerregistry/pkg/v1/remote/transport#CheckError),
which will parse the response body into a structured error for unexpected http
status codes.
Here's a "simple" program that writes the result of
[listing tags](https://github.com/opencontainers/distribution-spec/blob/60be706c34ee7805bdd1d3d11affec53b0dfb8fb/spec.md#tags)
for [`gcr.io/google-containers/pause`](https://gcr.io/google-containers/pause)
to stdout.
```go
package main
import (
"io"
"net/http"
"os"
"github.com/google/go-containerregistry/pkg/authn"
"github.com/google/go-containerregistry/pkg/name"
"github.com/google/go-containerregistry/pkg/v1/remote/transport"
)
func main() {
repo, err := name.NewRepository("gcr.io/google-containers/pause")
if err != nil {
panic(err)
}
// Fetch credentials based on your docker config file, which is $HOME/.docker/config.json or $DOCKER_CONFIG.
auth, err := authn.DefaultKeychain.Resolve(repo.Registry)
if err != nil {
panic(err)
}
// Construct an http.Client that is authorized to pull from gcr.io/google-containers/pause.
scopes := []string{repo.Scope(transport.PullScope)}
t, err := transport.New(repo.Registry, auth, http.DefaultTransport, scopes)
if err != nil {
panic(err)
}
client := &http.Client{Transport: t}
// Make the actual request.
resp, err := client.Get("https://gcr.io/v2/google-containers/pause/tags/list")
if err != nil {
panic(err)
}
// Assert that we get a 200, otherwise attempt to parse body as a structured error.
if err := transport.CheckError(resp, http.StatusOK); err != nil {
panic(err)
}
// Write the response to stdout.
if _, err := io.Copy(os.Stdout, resp.Body); err != nil {
panic(err)
}
}
```

View File

@ -1,63 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package transport
import (
"encoding/base64"
"fmt"
"net/http"
"github.com/google/go-containerregistry/pkg/authn"
)
type basicTransport struct {
inner http.RoundTripper
auth authn.Authenticator
target string
}
var _ http.RoundTripper = (*basicTransport)(nil)
// RoundTrip implements http.RoundTripper
func (bt *basicTransport) RoundTrip(in *http.Request) (*http.Response, error) {
if bt.auth != authn.Anonymous {
auth, err := bt.auth.Authorization()
if err != nil {
return nil, err
}
// http.Client handles redirects at a layer above the http.RoundTripper
// abstraction, so to avoid forwarding Authorization headers to places
// we are redirected, only set it when the authorization header matches
// the host with which we are interacting.
// In case of redirect http.Client can use an empty Host, check URL too.
if in.Host == bt.target || in.URL.Host == bt.target {
if bearer := auth.RegistryToken; bearer != "" {
hdr := fmt.Sprintf("Bearer %s", bearer)
in.Header.Set("Authorization", hdr)
} else if user, pass := auth.Username, auth.Password; user != "" && pass != "" {
delimited := fmt.Sprintf("%s:%s", user, pass)
encoded := base64.StdEncoding.EncodeToString([]byte(delimited))
hdr := fmt.Sprintf("Basic %s", encoded)
in.Header.Set("Authorization", hdr)
} else if token := auth.Auth; token != "" {
hdr := fmt.Sprintf("Basic %s", token)
in.Header.Set("Authorization", hdr)
}
}
}
in.Header.Set("User-Agent", transportName)
return bt.inner.RoundTrip(in)
}

View File

@ -1,257 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package transport
import (
"encoding/json"
"fmt"
"io/ioutil"
"net"
"net/http"
"net/url"
"strings"
"github.com/google/go-containerregistry/pkg/authn"
"github.com/google/go-containerregistry/pkg/name"
)
type bearerTransport struct {
// Wrapped by bearerTransport.
inner http.RoundTripper
// Basic credentials that we exchange for bearer tokens.
basic authn.Authenticator
// Holds the bearer response from the token service.
bearer authn.AuthConfig
// Registry to which we send bearer tokens.
registry name.Registry
// See https://tools.ietf.org/html/rfc6750#section-3
realm string
// See https://docs.docker.com/registry/spec/auth/token/
service string
scopes []string
// Scheme we should use, determined by ping response.
scheme string
}
var _ http.RoundTripper = (*bearerTransport)(nil)
var portMap = map[string]string{
"http": "80",
"https": "443",
}
// RoundTrip implements http.RoundTripper
func (bt *bearerTransport) RoundTrip(in *http.Request) (*http.Response, error) {
sendRequest := func() (*http.Response, error) {
// http.Client handles redirects at a layer above the http.RoundTripper
// abstraction, so to avoid forwarding Authorization headers to places
// we are redirected, only set it when the authorization header matches
// the registry with which we are interacting.
// In case of redirect http.Client can use an empty Host, check URL too.
if matchesHost(bt.registry, in, bt.scheme) {
hdr := fmt.Sprintf("Bearer %s", bt.bearer.RegistryToken)
in.Header.Set("Authorization", hdr)
}
return bt.inner.RoundTrip(in)
}
res, err := sendRequest()
if err != nil {
return nil, err
}
// Perform a token refresh() and retry the request in case the token has expired
if res.StatusCode == http.StatusUnauthorized {
if err = bt.refresh(); err != nil {
return nil, err
}
return sendRequest()
}
return res, err
}
// It's unclear which authentication flow to use based purely on the protocol,
// so we rely on heuristics and fallbacks to support as many registries as possible.
// The basic token exchange is attempted first, falling back to the oauth flow.
// If the IdentityToken is set, this indicates that we should start with the oauth flow.
func (bt *bearerTransport) refresh() error {
auth, err := bt.basic.Authorization()
if err != nil {
return err
}
if auth.RegistryToken != "" {
bt.bearer.RegistryToken = auth.RegistryToken
return nil
}
var content []byte
if auth.IdentityToken != "" {
// If the secret being stored is an identity token,
// the Username should be set to <token>, which indicates
// we are using an oauth flow.
content, err = bt.refreshOauth()
if terr, ok := err.(*Error); ok && terr.StatusCode == http.StatusNotFound {
// Note: Not all token servers implement oauth2.
// If the request to the endpoint returns 404 using the HTTP POST method,
// refer to Token Documentation for using the HTTP GET method supported by all token servers.
content, err = bt.refreshBasic()
}
} else {
content, err = bt.refreshBasic()
}
if err != nil {
return err
}
// Some registries don't have "token" in the response. See #54.
type tokenResponse struct {
Token string `json:"token"`
AccessToken string `json:"access_token"`
RefreshToken string `json:"refresh_token"`
// TODO: handle expiry?
}
var response tokenResponse
if err := json.Unmarshal(content, &response); err != nil {
return err
}
// Some registries set access_token instead of token.
if response.AccessToken != "" {
response.Token = response.AccessToken
}
// Find a token to turn into a Bearer authenticator
if response.Token != "" {
bt.bearer.RegistryToken = response.Token
} else {
return fmt.Errorf("no token in bearer response:\n%s", content)
}
// If we obtained a refresh token from the oauth flow, use that for refresh() now.
if response.RefreshToken != "" {
bt.basic = authn.FromConfig(authn.AuthConfig{
IdentityToken: response.RefreshToken,
})
}
return nil
}
func matchesHost(reg name.Registry, in *http.Request, scheme string) bool {
canonicalHeaderHost := canonicalAddress(in.Host, scheme)
canonicalURLHost := canonicalAddress(in.URL.Host, scheme)
canonicalRegistryHost := canonicalAddress(reg.RegistryStr(), scheme)
return canonicalHeaderHost == canonicalRegistryHost || canonicalURLHost == canonicalRegistryHost
}
func canonicalAddress(host, scheme string) (address string) {
// The host may be any one of:
// - hostname
// - hostname:port
// - ipv4
// - ipv4:port
// - ipv6
// - [ipv6]:port
// As net.SplitHostPort returns an error if the host does not contain a port, we should only attempt
// to call it when we know that the address contains a port
if strings.Count(host, ":") == 1 || (strings.Count(host, ":") >= 2 && strings.Contains(host, "]:")) {
hostname, port, err := net.SplitHostPort(host)
if err != nil {
return host
}
if port == "" {
port = portMap[scheme]
}
return net.JoinHostPort(hostname, port)
}
return net.JoinHostPort(host, portMap[scheme])
}
// https://docs.docker.com/registry/spec/auth/oauth/
func (bt *bearerTransport) refreshOauth() ([]byte, error) {
auth, err := bt.basic.Authorization()
if err != nil {
return nil, err
}
u, err := url.Parse(bt.realm)
if err != nil {
return nil, err
}
v := url.Values{}
v.Set("scope", strings.Join(bt.scopes, " "))
v.Set("service", bt.service)
v.Set("client_id", transportName)
if auth.IdentityToken != "" {
v.Set("grant_type", "refresh_token")
v.Set("refresh_token", auth.IdentityToken)
} else if auth.Username != "" && auth.Password != "" {
// TODO(#629): This is unreachable.
v.Set("grant_type", "password")
v.Set("username", auth.Username)
v.Set("password", auth.Password)
v.Set("access_type", "offline")
}
client := http.Client{Transport: bt.inner}
resp, err := client.PostForm(u.String(), v)
if err != nil {
return nil, err
}
defer resp.Body.Close()
if err := CheckError(resp, http.StatusOK); err != nil {
return nil, err
}
return ioutil.ReadAll(resp.Body)
}
// https://docs.docker.com/registry/spec/auth/token/
func (bt *bearerTransport) refreshBasic() ([]byte, error) {
u, err := url.Parse(bt.realm)
if err != nil {
return nil, err
}
b := &basicTransport{
inner: bt.inner,
auth: bt.basic,
target: u.Host,
}
client := http.Client{Transport: b}
u.RawQuery = url.Values{
"scope": bt.scopes,
"service": []string{bt.service},
}.Encode()
resp, err := client.Get(u.String())
if err != nil {
return nil, err
}
defer resp.Body.Close()
if err := CheckError(resp, http.StatusOK); err != nil {
return nil, err
}
return ioutil.ReadAll(resp.Body)
}

View File

@ -1,18 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// Package transport provides facilities for setting up an authenticated
// http.RoundTripper given an Authenticator and base RoundTripper. See
// transport.New for more information.
package transport

View File

@ -1,175 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package transport
import (
"encoding/json"
"fmt"
"io/ioutil"
"net/http"
"net/url"
"strings"
)
// The set of query string keys that we expect to send as part of the registry
// protocol. Anything else is potentially dangerous to leak, as it's probably
// from a redirect. These redirects often included tokens or signed URLs.
var paramWhitelist = map[string]struct{}{
// Token exchange
"scope": struct{}{},
"service": struct{}{},
// Cross-repo mounting
"mount": struct{}{},
"from": struct{}{},
// Layer PUT
"digest": struct{}{},
// Listing tags and catalog
"n": struct{}{},
"last": struct{}{},
}
// Error implements error to support the following error specification:
// https://github.com/docker/distribution/blob/master/docs/spec/api.md#errors
type Error struct {
Errors []Diagnostic `json:"errors,omitempty"`
// The http status code returned.
StatusCode int
// The raw body if we couldn't understand it.
rawBody string
// The request that failed.
request *http.Request
}
// Check that Error implements error
var _ error = (*Error)(nil)
// Error implements error
func (e *Error) Error() string {
prefix := ""
if e.request != nil {
prefix = fmt.Sprintf("%s %s: ", e.request.Method, redact(e.request.URL))
}
return prefix + e.responseErr()
}
func (e *Error) responseErr() string {
switch len(e.Errors) {
case 0:
if len(e.rawBody) == 0 {
return fmt.Sprintf("unsupported status code %d", e.StatusCode)
}
return fmt.Sprintf("unsupported status code %d; body: %s", e.StatusCode, e.rawBody)
case 1:
return e.Errors[0].String()
default:
var errors []string
for _, d := range e.Errors {
errors = append(errors, d.String())
}
return fmt.Sprintf("multiple errors returned: %s",
strings.Join(errors, "; "))
}
}
// Temporary returns whether the request that preceded the error is temporary.
func (e *Error) Temporary() bool {
if len(e.Errors) == 0 {
return false
}
for _, d := range e.Errors {
// TODO: Include other error types.
if d.Code != BlobUploadInvalidErrorCode {
return false
}
}
return true
}
func redact(original *url.URL) *url.URL {
qs := original.Query()
for k, v := range qs {
for i := range v {
if _, ok := paramWhitelist[k]; !ok {
// key is not in the whitelist
v[i] = "REDACTED"
}
}
}
redacted := *original
redacted.RawQuery = qs.Encode()
return &redacted
}
// Diagnostic represents a single error returned by a Docker registry interaction.
type Diagnostic struct {
Code ErrorCode `json:"code"`
Message string `json:"message,omitempty"`
Detail interface{} `json:"detail,omitempty"`
}
// String stringifies the Diagnostic in the form: $Code: $Message[; $Detail]
func (d Diagnostic) String() string {
msg := fmt.Sprintf("%s: %s", d.Code, d.Message)
if d.Detail != nil {
msg = fmt.Sprintf("%s; %v", msg, d.Detail)
}
return msg
}
// ErrorCode is an enumeration of supported error codes.
type ErrorCode string
// The set of error conditions a registry may return:
// https://github.com/docker/distribution/blob/master/docs/spec/api.md#errors-2
const (
BlobUnknownErrorCode ErrorCode = "BLOB_UNKNOWN"
BlobUploadInvalidErrorCode ErrorCode = "BLOB_UPLOAD_INVALID"
BlobUploadUnknownErrorCode ErrorCode = "BLOB_UPLOAD_UNKNOWN"
DigestInvalidErrorCode ErrorCode = "DIGEST_INVALID"
ManifestBlobUnknownErrorCode ErrorCode = "MANIFEST_BLOB_UNKNOWN"
ManifestInvalidErrorCode ErrorCode = "MANIFEST_INVALID"
ManifestUnknownErrorCode ErrorCode = "MANIFEST_UNKNOWN"
ManifestUnverifiedErrorCode ErrorCode = "MANIFEST_UNVERIFIED"
NameInvalidErrorCode ErrorCode = "NAME_INVALID"
NameUnknownErrorCode ErrorCode = "NAME_UNKNOWN"
SizeInvalidErrorCode ErrorCode = "SIZE_INVALID"
TagInvalidErrorCode ErrorCode = "TAG_INVALID"
UnauthorizedErrorCode ErrorCode = "UNAUTHORIZED"
DeniedErrorCode ErrorCode = "DENIED"
UnsupportedErrorCode ErrorCode = "UNSUPPORTED"
)
// CheckError returns a structured error if the response status is not in codes.
func CheckError(resp *http.Response, codes ...int) error {
for _, code := range codes {
if resp.StatusCode == code {
// This is one of the supported status codes.
return nil
}
}
b, err := ioutil.ReadAll(resp.Body)
if err != nil {
return err
}
// https://github.com/docker/distribution/blob/master/docs/spec/api.md#errors
structuredError := &Error{}
if err := json.Unmarshal(b, structuredError); err != nil {
structuredError.rawBody = string(b)
}
structuredError.StatusCode = resp.StatusCode
structuredError.request = resp.Request
return structuredError
}

View File

@ -1,44 +0,0 @@
package transport
import (
"fmt"
"net/http"
"net/http/httputil"
"github.com/google/go-containerregistry/pkg/logs"
)
type logTransport struct {
inner http.RoundTripper
}
// NewLogger returns a transport that logs requests and responses to
// github.com/google/go-containerregistry/pkg/logs.Debug.
func NewLogger(inner http.RoundTripper) http.RoundTripper {
return &logTransport{inner}
}
func (t *logTransport) RoundTrip(in *http.Request) (out *http.Response, err error) {
// Inspired by: github.com/motemen/go-loghttp
logs.Debug.Printf("--> %s %s", in.Method, in.URL)
b, err := httputil.DumpRequestOut(in, true)
if err == nil {
logs.Debug.Println(string(b))
}
out, err = t.inner.RoundTrip(in)
if err != nil {
logs.Debug.Printf("<-- %v %s", err, in.URL)
}
if out != nil {
msg := fmt.Sprintf("<-- %d", out.StatusCode)
if out.Request != nil {
msg = fmt.Sprintf("%s %s", msg, out.Request.URL)
}
logs.Debug.Print(msg)
b, err := httputil.DumpResponse(out, true)
if err == nil {
logs.Debug.Println(string(b))
}
}
return
}

View File

@ -1,122 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package transport
import (
"fmt"
"io"
"io/ioutil"
"net/http"
"strings"
"github.com/google/go-containerregistry/pkg/name"
)
type challenge string
const (
anonymous challenge = "anonymous"
basic challenge = "basic"
bearer challenge = "bearer"
)
type pingResp struct {
challenge challenge
// Following the challenge there are often key/value pairs
// e.g. Bearer service="gcr.io",realm="https://auth.gcr.io/v36/tokenz"
parameters map[string]string
// The registry's scheme to use. Communicates whether we fell back to http.
scheme string
}
func (c challenge) Canonical() challenge {
return challenge(strings.ToLower(string(c)))
}
func parseChallenge(suffix string) map[string]string {
kv := make(map[string]string)
for _, token := range strings.Split(suffix, ",") {
// Trim any whitespace around each token.
token = strings.Trim(token, " ")
// Break the token into a key/value pair
if parts := strings.SplitN(token, "=", 2); len(parts) == 2 {
// Unquote the value, if it is quoted.
kv[parts[0]] = strings.Trim(parts[1], `"`)
} else {
// If there was only one part, treat is as a key with an empty value
kv[token] = ""
}
}
return kv
}
func ping(reg name.Registry, t http.RoundTripper) (*pingResp, error) {
client := http.Client{Transport: t}
// This first attempts to use "https" for every request, falling back to http
// if the registry matches our localhost heuristic or if it is intentionally
// set to insecure via name.NewInsecureRegistry.
schemes := []string{"https"}
if reg.Scheme() == "http" {
schemes = append(schemes, "http")
}
var connErr error
for _, scheme := range schemes {
url := fmt.Sprintf("%s://%s/v2/", scheme, reg.Name())
resp, err := client.Get(url)
if err != nil {
connErr = err
// Potentially retry with http.
continue
}
defer func() {
// By draining the body, make sure to reuse the connection made by
// the ping for the following access to the registry
io.Copy(ioutil.Discard, resp.Body)
resp.Body.Close()
}()
switch resp.StatusCode {
case http.StatusOK:
// If we get a 200, then no authentication is needed.
return &pingResp{
challenge: anonymous,
scheme: scheme,
}, nil
case http.StatusUnauthorized:
wac := resp.Header.Get("WWW-Authenticate")
if parts := strings.SplitN(wac, " ", 2); len(parts) == 2 {
// If there are two parts, then parse the challenge parameters.
return &pingResp{
challenge: challenge(parts[0]).Canonical(),
parameters: parseChallenge(parts[1]),
scheme: scheme,
}, nil
}
// Otherwise, just return the challenge without parameters.
return &pingResp{
challenge: challenge(wac).Canonical(),
scheme: scheme,
}, nil
default:
return nil, CheckError(resp, http.StatusOK, http.StatusUnauthorized)
}
}
return nil, connErr
}

View File

@ -1,88 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package transport
import (
"net/http"
"time"
"github.com/google/go-containerregistry/pkg/internal/retry"
)
// Sleep for 0.1, 0.3, 0.9, 2.7 seconds. This should cover networking blips.
var defaultBackoff = retry.Backoff{
Duration: 100 * time.Millisecond,
Factor: 3.0,
Jitter: 0.1,
Steps: 5,
}
var _ http.RoundTripper = (*retryTransport)(nil)
// retryTransport wraps a RoundTripper and retries temporary network errors.
type retryTransport struct {
inner http.RoundTripper
backoff retry.Backoff
predicate retry.Predicate
}
// Option is a functional option for retryTransport.
type Option func(*options)
type options struct {
backoff retry.Backoff
predicate retry.Predicate
}
// WithRetryBackoff sets the backoff for retry operations.
func WithRetryBackoff(backoff retry.Backoff) Option {
return func(o *options) {
o.backoff = backoff
}
}
// WithRetryPredicate sets the predicate for retry operations.
func WithRetryPredicate(predicate func(error) bool) Option {
return func(o *options) {
o.predicate = predicate
}
}
// NewRetry returns a transport that retries errors.
func NewRetry(inner http.RoundTripper, opts ...Option) http.RoundTripper {
o := &options{
backoff: defaultBackoff,
predicate: retry.IsTemporary,
}
for _, opt := range opts {
opt(o)
}
return &retryTransport{
inner: inner,
backoff: o.backoff,
predicate: o.predicate,
}
}
func (t *retryTransport) RoundTrip(in *http.Request) (out *http.Response, err error) {
roundtrip := func() error {
out, err = t.inner.RoundTrip(in)
return err
}
retry.Retry(roundtrip, t.predicate, t.backoff)
return
}

View File

@ -1,44 +0,0 @@
// Copyright 2019 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package transport
import (
"net/http"
"github.com/google/go-containerregistry/pkg/name"
)
type schemeTransport struct {
// Scheme we should use, determined by ping response.
scheme string
// Registry we're talking to.
registry name.Registry
// Wrapped by schemeTransport.
inner http.RoundTripper
}
// RoundTrip implements http.RoundTripper
func (st *schemeTransport) RoundTrip(in *http.Request) (*http.Response, error) {
// When we ping() the registry, we determine whether to use http or https
// based on which scheme was successful. That is only valid for the
// registry server and not e.g. a separate token server or blob storage,
// so we should only override the scheme if the host is the registry.
if matchesHost(st.registry, in, st.scheme) {
in.URL.Scheme = st.scheme
}
return st.inner.RoundTrip(in)
}

View File

@ -1,24 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package transport
// Scopes suitable to qualify each Repository
const (
PullScope string = "pull"
PushScope string = "push,pull"
// For now DELETE is PUSH, which is the read/write ACL.
DeleteScope string = PushScope
CatalogScope string = "catalog"
)

View File

@ -1,91 +0,0 @@
// Copyright 2018 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package transport
import (
"fmt"
"net/http"
"github.com/google/go-containerregistry/pkg/authn"
"github.com/google/go-containerregistry/pkg/name"
)
// New returns a new RoundTripper based on the provided RoundTripper that has been
// setup to authenticate with the remote registry "reg", in the capacity
// laid out by the specified scopes.
func New(reg name.Registry, auth authn.Authenticator, t http.RoundTripper, scopes []string) (http.RoundTripper, error) {
// The handshake:
// 1. Use "t" to ping() the registry for the authentication challenge.
//
// 2a. If we get back a 200, then simply use "t".
//
// 2b. If we get back a 401 with a Basic challenge, then use a transport
// that just attachs auth each roundtrip.
//
// 2c. If we get back a 401 with a Bearer challenge, then use a transport
// that attaches a bearer token to each request, and refreshes is on 401s.
// Perform an initial refresh to seed the bearer token.
// First we ping the registry to determine the parameters of the authentication handshake
// (if one is even necessary).
pr, err := ping(reg, t)
if err != nil {
return nil, err
}
// Wrap the given transport in transports that use an appropriate scheme,
// (based on the ping response) and set the user agent.
t = &useragentTransport{
inner: &schemeTransport{
scheme: pr.scheme,
registry: reg,
inner: t,
},
}
switch pr.challenge.Canonical() {
case anonymous:
return t, nil
case basic:
return &basicTransport{inner: t, auth: auth, target: reg.RegistryStr()}, nil
case bearer:
// We require the realm, which tells us where to send our Basic auth to turn it into Bearer auth.
realm, ok := pr.parameters["realm"]
if !ok {
return nil, fmt.Errorf("malformed www-authenticate, missing realm: %v", pr.parameters)
}
service, ok := pr.parameters["service"]
if !ok {
// If the service parameter is not specified, then default it to the registry
// with which we are talking.
service = reg.String()
}
bt := &bearerTransport{
inner: t,
basic: auth,
realm: realm,
registry: reg,
service: service,
scopes: scopes,
scheme: pr.scheme,
}
if err := bt.refresh(); err != nil {
return nil, err
}
return bt, nil
default:
return nil, fmt.Errorf("unrecognized challenge: %s", pr.challenge)
}
}

View File

@ -1,32 +0,0 @@
// Copyright 2019 Google LLC All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package transport
import "net/http"
const (
transportName = "go-containerregistry"
)
type useragentTransport struct {
// Wrapped by useragentTransport.
inner http.RoundTripper
}
// RoundTrip implements http.RoundTripper
func (ut *useragentTransport) RoundTrip(in *http.Request) (*http.Response, error) {
in.Header.Set("User-Agent", transportName)
return ut.inner.RoundTrip(in)
}

1
vendor/go.opencensus.io/go.sum generated vendored
View File

@ -67,6 +67,7 @@ google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZi
google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
google.golang.org/grpc v1.20.1 h1:Hz2g2wirWK7H0qIIhGIqRGTuMwTE8HEKFnDZZ7lm9NU=
google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38=
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM=
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw=
gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=

View File

@ -67,7 +67,12 @@ func (s *Weighted) Acquire(ctx context.Context, n int64) error {
// fix up the queue, just pretend we didn't notice the cancelation.
err = nil
default:
isFront := s.waiters.Front() == elem
s.waiters.Remove(elem)
// If we're at the front and there're extra tokens left, notify other waiters.
if isFront && s.size > s.cur {
s.notifyWaiters()
}
}
s.mu.Unlock()
return err
@ -97,6 +102,11 @@ func (s *Weighted) Release(n int64) {
s.mu.Unlock()
panic("semaphore: released more than held")
}
s.notifyWaiters()
s.mu.Unlock()
}
func (s *Weighted) notifyWaiters() {
for {
next := s.waiters.Front()
if next == nil {
@ -123,5 +133,4 @@ func (s *Weighted) Release(n int64) {
s.waiters.Remove(next)
close(w.ready)
}
s.mu.Unlock()
}

120
vendor/golang.org/x/sync/singleflight/singleflight.go generated vendored Normal file
View File

@ -0,0 +1,120 @@
// Copyright 2013 The Go Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
// Package singleflight provides a duplicate function call suppression
// mechanism.
package singleflight // import "golang.org/x/sync/singleflight"
import "sync"
// call is an in-flight or completed singleflight.Do call
type call struct {
wg sync.WaitGroup
// These fields are written once before the WaitGroup is done
// and are only read after the WaitGroup is done.
val interface{}
err error
// forgotten indicates whether Forget was called with this call's key
// while the call was still in flight.
forgotten bool
// These fields are read and written with the singleflight
// mutex held before the WaitGroup is done, and are read but
// not written after the WaitGroup is done.
dups int
chans []chan<- Result
}
// Group represents a class of work and forms a namespace in
// which units of work can be executed with duplicate suppression.
type Group struct {
mu sync.Mutex // protects m
m map[string]*call // lazily initialized
}
// Result holds the results of Do, so they can be passed
// on a channel.
type Result struct {
Val interface{}
Err error
Shared bool
}
// Do executes and returns the results of the given function, making
// sure that only one execution is in-flight for a given key at a
// time. If a duplicate comes in, the duplicate caller waits for the
// original to complete and receives the same results.
// The return value shared indicates whether v was given to multiple callers.
func (g *Group) Do(key string, fn func() (interface{}, error)) (v interface{}, err error, shared bool) {
g.mu.Lock()
if g.m == nil {
g.m = make(map[string]*call)
}
if c, ok := g.m[key]; ok {
c.dups++
g.mu.Unlock()
c.wg.Wait()
return c.val, c.err, true
}
c := new(call)
c.wg.Add(1)
g.m[key] = c
g.mu.Unlock()
g.doCall(c, key, fn)
return c.val, c.err, c.dups > 0
}
// DoChan is like Do but returns a channel that will receive the
// results when they are ready.
func (g *Group) DoChan(key string, fn func() (interface{}, error)) <-chan Result {
ch := make(chan Result, 1)
g.mu.Lock()
if g.m == nil {
g.m = make(map[string]*call)
}
if c, ok := g.m[key]; ok {
c.dups++
c.chans = append(c.chans, ch)
g.mu.Unlock()
return ch
}
c := &call{chans: []chan<- Result{ch}}
c.wg.Add(1)
g.m[key] = c
g.mu.Unlock()
go g.doCall(c, key, fn)
return ch
}
// doCall handles the single call for a key.
func (g *Group) doCall(c *call, key string, fn func() (interface{}, error)) {
c.val, c.err = fn()
c.wg.Done()
g.mu.Lock()
if !c.forgotten {
delete(g.m, key)
}
for _, ch := range c.chans {
ch <- Result{c.val, c.err, c.dups > 0}
}
g.mu.Unlock()
}
// Forget tells the singleflight to forget about a key. Future calls
// to Do for this key will call the function rather than waiting for
// an earlier call to complete.
func (g *Group) Forget(key string) {
g.mu.Lock()
if c, ok := g.m[key]; ok {
c.forgotten = true
}
delete(g.m, key)
g.mu.Unlock()
}

View File

@ -196,7 +196,7 @@ func (lim *Limiter) Reserve() *Reservation {
// ReserveN returns a Reservation that indicates how long the caller must wait before n events happen.
// The Limiter takes this Reservation into account when allowing future events.
// ReserveN returns false if n exceeds the Limiter's burst size.
// The returned Reservations OK() method returns false if n exceeds the Limiter's burst size.
// Usage example:
// r := lim.ReserveN(time.Now(), 1)
// if !r.OK() {

View File

@ -130,6 +130,10 @@ possible reasons, including:
1. mis-configured transport credentials, connection failed on handshaking
1. bytes disrupted, possibly by a proxy in between
1. server shutdown
1. Keepalive parameters caused connection shutdown, for example if you have configured
your server to terminate connections regularly to [trigger DNS lookups](https://github.com/grpc/grpc-go/issues/3170#issuecomment-552517779).
If this is the case, you may want to increase your [MaxConnectionAgeGrace](https://pkg.go.dev/google.golang.org/grpc/keepalive?tab=doc#ServerParameters),
to allow longer RPC calls to finish.
It can be tricky to debug this because the error happens on the client side but
the root cause of the connection being closed is on the server side. Turn on

View File

@ -194,12 +194,13 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn *
cc.mkp = cc.dopts.copts.KeepaliveParams
if cc.dopts.copts.Dialer == nil {
cc.dopts.copts.Dialer = newProxyDialer(
func(ctx context.Context, addr string) (net.Conn, error) {
network, addr := parseDialTarget(addr)
return (&net.Dialer{}).DialContext(ctx, network, addr)
},
)
cc.dopts.copts.Dialer = func(ctx context.Context, addr string) (net.Conn, error) {
network, addr := parseDialTarget(addr)
return (&net.Dialer{}).DialContext(ctx, network, addr)
}
if cc.dopts.withProxy {
cc.dopts.copts.Dialer = newProxyDialer(cc.dopts.copts.Dialer)
}
}
if cc.dopts.copts.UserAgent != "" {
@ -1525,9 +1526,9 @@ var ErrClientConnTimeout = errors.New("grpc: timed out when dialing")
func (cc *ClientConn) getResolver(scheme string) resolver.Builder {
for _, rb := range cc.dopts.resolvers {
if cc.parsedTarget.Scheme == rb.Scheme() {
if scheme == rb.Scheme() {
return rb
}
}
return resolver.Get(cc.parsedTarget.Scheme)
return resolver.Get(scheme)
}

View File

@ -135,16 +135,26 @@ func NewTLS(c *tls.Config) TransportCredentials {
return tc
}
// NewClientTLSFromCert constructs TLS credentials from the input certificate for client.
// NewClientTLSFromCert constructs TLS credentials from the provided root
// certificate authority certificate(s) to validate server connections. If
// certificates to establish the identity of the client need to be included in
// the credentials (eg: for mTLS), use NewTLS instead, where a complete
// tls.Config can be specified.
// serverNameOverride is for testing only. If set to a non empty string,
// it will override the virtual host name of authority (e.g. :authority header field) in requests.
// it will override the virtual host name of authority (e.g. :authority header
// field) in requests.
func NewClientTLSFromCert(cp *x509.CertPool, serverNameOverride string) TransportCredentials {
return NewTLS(&tls.Config{ServerName: serverNameOverride, RootCAs: cp})
}
// NewClientTLSFromFile constructs TLS credentials from the input certificate file for client.
// NewClientTLSFromFile constructs TLS credentials from the provided root
// certificate authority certificate file(s) to validate server connections. If
// certificates to establish the identity of the client need to be included in
// the credentials (eg: for mTLS), use NewTLS instead, where a complete
// tls.Config can be specified.
// serverNameOverride is for testing only. If set to a non empty string,
// it will override the virtual host name of authority (e.g. :authority header field) in requests.
// it will override the virtual host name of authority (e.g. :authority header
// field) in requests.
func NewClientTLSFromFile(certFile, serverNameOverride string) (TransportCredentials, error) {
b, err := ioutil.ReadFile(certFile)
if err != nil {

View File

@ -72,6 +72,7 @@ type dialOptions struct {
// we need to be able to configure this in tests.
resolveNowBackoff func(int) time.Duration
resolvers []resolver.Builder
withProxy bool
}
// DialOption configures how we set up the connection.
@ -307,6 +308,16 @@ func WithInsecure() DialOption {
})
}
// WithNoProxy returns a DialOption which disables the use of proxies for this
// ClientConn. This is ignored if WithDialer or WithContextDialer are used.
//
// This API is EXPERIMENTAL.
func WithNoProxy() DialOption {
return newFuncDialOption(func(o *dialOptions) {
o.withProxy = false
})
}
// WithTransportCredentials returns a DialOption which configures a connection
// level security credentials (e.g., TLS/SSL). This should not be used together
// with WithCredentialsBundle.
@ -557,6 +568,7 @@ func defaultDialOptions() dialOptions {
ReadBufferSize: defaultReadBufSize,
},
resolveNowBackoff: internalbackoff.DefaultExponential.Backoff,
withProxy: true,
}
}

View File

@ -3,8 +3,8 @@ module google.golang.org/grpc
go 1.11
require (
github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f
github.com/envoyproxy/go-control-plane v0.9.4
github.com/envoyproxy/protoc-gen-validate v0.1.0
github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b
github.com/golang/mock v1.1.1
github.com/golang/protobuf v1.3.3

View File

@ -37,11 +37,6 @@ var (
// KeepaliveMinPingTime is the minimum ping interval. This must be 10s by
// default, but tests may wish to set it lower for convenience.
KeepaliveMinPingTime = 10 * time.Second
// StatusRawProto is exported by status/status.go. This func returns a
// pointer to the wrapped Status proto for a given status.Status without a
// call to proto.Clone(). The returned Status proto should not be mutated by
// the caller.
StatusRawProto interface{} // func (*status.Status) *spb.Status
// NewRequestInfoContext creates a new context based on the argument context attaching
// the passed in RequestInfo to the new context.
NewRequestInfoContext interface{} // func(context.Context, credentials.RequestInfo) context.Context

161
vendor/google.golang.org/grpc/internal/status/status.go generated vendored Normal file
View File

@ -0,0 +1,161 @@
/*
*
* Copyright 2020 gRPC authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*
*/
// Package status implements errors returned by gRPC. These errors are
// serialized and transmitted on the wire between server and client, and allow
// for additional data to be transmitted via the Details field in the status
// proto. gRPC service handlers should return an error created by this
// package, and gRPC clients should expect a corresponding error to be
// returned from the RPC call.
//
// This package upholds the invariants that a non-nil error may not
// contain an OK code, and an OK code must result in a nil error.
package status
import (
"errors"
"fmt"
"github.com/golang/protobuf/proto"
"github.com/golang/protobuf/ptypes"
spb "google.golang.org/genproto/googleapis/rpc/status"
"google.golang.org/grpc/codes"
)
// Status represents an RPC status code, message, and details. It is immutable
// and should be created with New, Newf, or FromProto.
type Status struct {
s *spb.Status
}
// New returns a Status representing c and msg.
func New(c codes.Code, msg string) *Status {
return &Status{s: &spb.Status{Code: int32(c), Message: msg}}
}
// Newf returns New(c, fmt.Sprintf(format, a...)).
func Newf(c codes.Code, format string, a ...interface{}) *Status {
return New(c, fmt.Sprintf(format, a...))
}
// FromProto returns a Status representing s.
func FromProto(s *spb.Status) *Status {
return &Status{s: proto.Clone(s).(*spb.Status)}
}
// Err returns an error representing c and msg. If c is OK, returns nil.
func Err(c codes.Code, msg string) error {
return New(c, msg).Err()
}
// Errorf returns Error(c, fmt.Sprintf(format, a...)).
func Errorf(c codes.Code, format string, a ...interface{}) error {
return Err(c, fmt.Sprintf(format, a...))
}
// Code returns the status code contained in s.
func (s *Status) Code() codes.Code {
if s == nil || s.s == nil {
return codes.OK
}
return codes.Code(s.s.Code)
}
// Message returns the message contained in s.
func (s *Status) Message() string {
if s == nil || s.s == nil {
return ""
}
return s.s.Message
}
// Proto returns s's status as an spb.Status proto message.
func (s *Status) Proto() *spb.Status {
if s == nil {
return nil
}
return proto.Clone(s.s).(*spb.Status)
}
// Err returns an immutable error representing s; returns nil if s.Code() is OK.
func (s *Status) Err() error {
if s.Code() == codes.OK {
return nil
}
return (*Error)(s.Proto())
}
// WithDetails returns a new status with the provided details messages appended to the status.
// If any errors are encountered, it returns nil and the first error encountered.
func (s *Status) WithDetails(details ...proto.Message) (*Status, error) {
if s.Code() == codes.OK {
return nil, errors.New("no error details for status with code OK")
}
// s.Code() != OK implies that s.Proto() != nil.
p := s.Proto()
for _, detail := range details {
any, err := ptypes.MarshalAny(detail)
if err != nil {
return nil, err
}
p.Details = append(p.Details, any)
}
return &Status{s: p}, nil
}
// Details returns a slice of details messages attached to the status.
// If a detail cannot be decoded, the error is returned in place of the detail.
func (s *Status) Details() []interface{} {
if s == nil || s.s == nil {
return nil
}
details := make([]interface{}, 0, len(s.s.Details))
for _, any := range s.s.Details {
detail := &ptypes.DynamicAny{}
if err := ptypes.UnmarshalAny(any, detail); err != nil {
details = append(details, err)
continue
}
details = append(details, detail.Message)
}
return details
}
// Error is an alias of a status proto. It implements error and Status,
// and a nil Error should never be returned by this package.
type Error spb.Status
func (se *Error) Error() string {
p := (*spb.Status)(se)
return fmt.Sprintf("rpc error: code = %s desc = %s", codes.Code(p.GetCode()), p.GetMessage())
}
// GRPCStatus returns the Status represented by se.
func (se *Error) GRPCStatus() *Status {
return FromProto((*spb.Status)(se))
}
// Is implements future error.Is functionality.
// A Error is equivalent if the code and message are identical.
func (se *Error) Is(target error) bool {
tse, ok := target.(*Error)
if !ok {
return false
}
return proto.Equal((*spb.Status)(se), (*spb.Status)(tse))
}

View File

@ -112,11 +112,10 @@ func NewServerHandlerTransport(w http.ResponseWriter, r *http.Request, stats sta
// at this point to be speaking over HTTP/2, so it's able to speak valid
// gRPC.
type serverHandlerTransport struct {
rw http.ResponseWriter
req *http.Request
timeoutSet bool
timeout time.Duration
didCommonHeaders bool
rw http.ResponseWriter
req *http.Request
timeoutSet bool
timeout time.Duration
headerMD metadata.MD
@ -186,8 +185,11 @@ func (ht *serverHandlerTransport) WriteStatus(s *Stream, st *status.Status) erro
ht.writeStatusMu.Lock()
defer ht.writeStatusMu.Unlock()
headersWritten := s.updateHeaderSent()
err := ht.do(func() {
ht.writeCommonHeaders(s)
if !headersWritten {
ht.writePendingHeaders(s)
}
// And flush, in case no header or body has been sent yet.
// This forces a separation of headers and trailers if this is the
@ -227,6 +229,8 @@ func (ht *serverHandlerTransport) WriteStatus(s *Stream, st *status.Status) erro
if err == nil { // transport has not been closed
if ht.stats != nil {
// Note: The trailer fields are compressed with hpack after this call returns.
// No WireLength field is set here.
ht.stats.HandleRPC(s.Context(), &stats.OutTrailer{
Trailer: s.trailer.Copy(),
})
@ -236,14 +240,16 @@ func (ht *serverHandlerTransport) WriteStatus(s *Stream, st *status.Status) erro
return err
}
// writePendingHeaders sets common and custom headers on the first
// write call (Write, WriteHeader, or WriteStatus)
func (ht *serverHandlerTransport) writePendingHeaders(s *Stream) {
ht.writeCommonHeaders(s)
ht.writeCustomHeaders(s)
}
// writeCommonHeaders sets common headers on the first write
// call (Write, WriteHeader, or WriteStatus).
func (ht *serverHandlerTransport) writeCommonHeaders(s *Stream) {
if ht.didCommonHeaders {
return
}
ht.didCommonHeaders = true
h := ht.rw.Header()
h["Date"] = nil // suppress Date to make tests happy; TODO: restore
h.Set("Content-Type", ht.contentType)
@ -262,9 +268,30 @@ func (ht *serverHandlerTransport) writeCommonHeaders(s *Stream) {
}
}
// writeCustomHeaders sets custom headers set on the stream via SetHeader
// on the first write call (Write, WriteHeader, or WriteStatus).
func (ht *serverHandlerTransport) writeCustomHeaders(s *Stream) {
h := ht.rw.Header()
s.hdrMu.Lock()
for k, vv := range s.header {
if isReservedHeader(k) {
continue
}
for _, v := range vv {
h.Add(k, encodeMetadataHeader(k, v))
}
}
s.hdrMu.Unlock()
}
func (ht *serverHandlerTransport) Write(s *Stream, hdr []byte, data []byte, opts *Options) error {
headersWritten := s.updateHeaderSent()
return ht.do(func() {
ht.writeCommonHeaders(s)
if !headersWritten {
ht.writePendingHeaders(s)
}
ht.rw.Write(hdr)
ht.rw.Write(data)
ht.rw.(http.Flusher).Flush()
@ -272,27 +299,27 @@ func (ht *serverHandlerTransport) Write(s *Stream, hdr []byte, data []byte, opts
}
func (ht *serverHandlerTransport) WriteHeader(s *Stream, md metadata.MD) error {
if err := s.SetHeader(md); err != nil {
return err
}
headersWritten := s.updateHeaderSent()
err := ht.do(func() {
ht.writeCommonHeaders(s)
h := ht.rw.Header()
for k, vv := range md {
// Clients don't tolerate reading restricted headers after some non restricted ones were sent.
if isReservedHeader(k) {
continue
}
for _, v := range vv {
v = encodeMetadataHeader(k, v)
h.Add(k, v)
}
if !headersWritten {
ht.writePendingHeaders(s)
}
ht.rw.WriteHeader(200)
ht.rw.(http.Flusher).Flush()
})
if err == nil {
if ht.stats != nil {
// Note: The header fields are compressed with hpack after this call returns.
// No WireLength field is set here.
ht.stats.HandleRPC(s.Context(), &stats.OutHeader{
Header: md.Copy(),
Header: md.Copy(),
Compression: s.sendCompress,
})
}
}

View File

@ -686,6 +686,8 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
} else {
header = metadata.Pairs("user-agent", t.userAgent)
}
// Note: The header fields are compressed with hpack after this call returns.
// No WireLength field is set here.
outHeader := &stats.OutHeader{
Client: true,
FullMethod: callHdr.Method,
@ -1193,9 +1195,10 @@ func (t *http2Client) operateHeaders(frame *http2.MetaHeadersFrame) {
if t.statsHandler != nil {
if isHeader {
inHeader := &stats.InHeader{
Client: true,
WireLength: int(frame.Header().Length),
Header: s.header.Copy(),
Client: true,
WireLength: int(frame.Header().Length),
Header: s.header.Copy(),
Compression: s.recvCompress,
}
t.statsHandler.HandleRPC(s.ctx, inHeader)
} else {

View File

@ -35,11 +35,9 @@ import (
"golang.org/x/net/http2"
"golang.org/x/net/http2/hpack"
spb "google.golang.org/genproto/googleapis/rpc/status"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/grpclog"
"google.golang.org/grpc/internal"
"google.golang.org/grpc/internal/channelz"
"google.golang.org/grpc/internal/grpcrand"
"google.golang.org/grpc/keepalive"
@ -57,9 +55,6 @@ var (
// ErrHeaderListSizeLimitViolation indicates that the header list size is larger
// than the limit set by peer.
ErrHeaderListSizeLimitViolation = errors.New("transport: trying to send header list size larger than the limit set by peer")
// statusRawProto is a function to get to the raw status proto wrapped in a
// status.Status without a proto.Clone().
statusRawProto = internal.StatusRawProto.(func(*status.Status) *spb.Status)
)
// serverConnectionCounter counts the number of connections a server has seen
@ -813,10 +808,11 @@ func (t *http2Server) writeHeaderLocked(s *Stream) error {
return ErrHeaderListSizeLimitViolation
}
if t.stats != nil {
// Note: WireLength is not set in outHeader.
// TODO(mmukhi): Revisit this later, if needed.
// Note: Headers are compressed with hpack after this call returns.
// No WireLength field is set here.
outHeader := &stats.OutHeader{
Header: s.header.Copy(),
Header: s.header.Copy(),
Compression: s.sendCompress,
}
t.stats.HandleRPC(s.Context(), outHeader)
}
@ -849,7 +845,7 @@ func (t *http2Server) WriteStatus(s *Stream, st *status.Status) error {
headerFields = append(headerFields, hpack.HeaderField{Name: "grpc-status", Value: strconv.Itoa(int(st.Code()))})
headerFields = append(headerFields, hpack.HeaderField{Name: "grpc-message", Value: encodeGrpcMessage(st.Message())})
if p := statusRawProto(st); p != nil && len(p.Details) > 0 {
if p := st.Proto(); p != nil && len(p.Details) > 0 {
stBytes, err := proto.Marshal(p)
if err != nil {
// TODO: return error instead, when callers are able to handle it.
@ -880,6 +876,8 @@ func (t *http2Server) WriteStatus(s *Stream, st *status.Status) error {
rst := s.getState() == streamActive
t.finishStream(s, rst, http2.ErrCodeNo, trailingHeader, true)
if t.stats != nil {
// Note: The trailer fields are compressed with hpack after this call returns.
// No WireLength field is set here.
t.stats.HandleRPC(s.Context(), &stats.OutTrailer{
Trailer: s.trailer.Copy(),
})

View File

@ -81,6 +81,10 @@ type InHeader struct {
Client bool
// WireLength is the wire length of header.
WireLength int
// Compression is the compression algorithm used for the RPC.
Compression string
// Header contains the header metadata received.
Header metadata.MD
// The following fields are valid only if Client is false.
// FullMethod is the full RPC method string, i.e., /package.service/method.
@ -89,10 +93,6 @@ type InHeader struct {
RemoteAddr net.Addr
// LocalAddr is the local address of the corresponding connection.
LocalAddr net.Addr
// Compression is the compression algorithm used for the RPC.
Compression string
// Header contains the header metadata received.
Header metadata.MD
}
// IsClient indicates if the stats information is from client side.
@ -141,6 +141,10 @@ func (s *OutPayload) isRPCStats() {}
type OutHeader struct {
// Client is true if this OutHeader is from client side.
Client bool
// Compression is the compression algorithm used for the RPC.
Compression string
// Header contains the header metadata sent.
Header metadata.MD
// The following fields are valid only if Client is true.
// FullMethod is the full RPC method string, i.e., /package.service/method.
@ -149,10 +153,6 @@ type OutHeader struct {
RemoteAddr net.Addr
// LocalAddr is the local address of the corresponding connection.
LocalAddr net.Addr
// Compression is the compression algorithm used for the RPC.
Compression string
// Header contains the header metadata sent.
Header metadata.MD
}
// IsClient indicates if this stats information is from client side.
@ -165,6 +165,9 @@ type OutTrailer struct {
// Client is true if this OutTrailer is from client side.
Client bool
// WireLength is the wire length of trailer.
//
// Deprecated: This field is never set. The length is not known when this message is
// emitted because the trailer fields are compressed with hpack after that.
WireLength int
// Trailer contains the trailer metadata sent to the client. This
// field is only valid if this OutTrailer is from the server side.

View File

@ -29,88 +29,23 @@ package status
import (
"context"
"errors"
"fmt"
"github.com/golang/protobuf/proto"
"github.com/golang/protobuf/ptypes"
spb "google.golang.org/genproto/googleapis/rpc/status"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/internal"
"google.golang.org/grpc/internal/status"
)
func init() {
internal.StatusRawProto = statusRawProto
}
func statusRawProto(s *Status) *spb.Status { return s.s }
// statusError is an alias of a status proto. It implements error and Status,
// and a nil statusError should never be returned by this package.
type statusError spb.Status
func (se *statusError) Error() string {
p := (*spb.Status)(se)
return fmt.Sprintf("rpc error: code = %s desc = %s", codes.Code(p.GetCode()), p.GetMessage())
}
func (se *statusError) GRPCStatus() *Status {
return &Status{s: (*spb.Status)(se)}
}
// Is implements future error.Is functionality.
// A statusError is equivalent if the code and message are identical.
func (se *statusError) Is(target error) bool {
tse, ok := target.(*statusError)
if !ok {
return false
}
return proto.Equal((*spb.Status)(se), (*spb.Status)(tse))
}
// Status represents an RPC status code, message, and details. It is immutable
// and should be created with New, Newf, or FromProto.
type Status struct {
s *spb.Status
}
// Code returns the status code contained in s.
func (s *Status) Code() codes.Code {
if s == nil || s.s == nil {
return codes.OK
}
return codes.Code(s.s.Code)
}
// Message returns the message contained in s.
func (s *Status) Message() string {
if s == nil || s.s == nil {
return ""
}
return s.s.Message
}
// Proto returns s's status as an spb.Status proto message.
func (s *Status) Proto() *spb.Status {
if s == nil {
return nil
}
return proto.Clone(s.s).(*spb.Status)
}
// Err returns an immutable error representing s; returns nil if s.Code() is
// OK.
func (s *Status) Err() error {
if s.Code() == codes.OK {
return nil
}
return (*statusError)(s.s)
}
// Status references google.golang.org/grpc/internal/status. It represents an
// RPC status code, message, and details. It is immutable and should be
// created with New, Newf, or FromProto.
// https://godoc.org/google.golang.org/grpc/internal/status
type Status = status.Status
// New returns a Status representing c and msg.
func New(c codes.Code, msg string) *Status {
return &Status{s: &spb.Status{Code: int32(c), Message: msg}}
return status.New(c, msg)
}
// Newf returns New(c, fmt.Sprintf(format, a...)).
@ -135,7 +70,7 @@ func ErrorProto(s *spb.Status) error {
// FromProto returns a Status representing s.
func FromProto(s *spb.Status) *Status {
return &Status{s: proto.Clone(s).(*spb.Status)}
return status.FromProto(s)
}
// FromError returns a Status representing err if it was produced from this
@ -160,42 +95,6 @@ func Convert(err error) *Status {
return s
}
// WithDetails returns a new status with the provided details messages appended to the status.
// If any errors are encountered, it returns nil and the first error encountered.
func (s *Status) WithDetails(details ...proto.Message) (*Status, error) {
if s.Code() == codes.OK {
return nil, errors.New("no error details for status with code OK")
}
// s.Code() != OK implies that s.Proto() != nil.
p := s.Proto()
for _, detail := range details {
any, err := ptypes.MarshalAny(detail)
if err != nil {
return nil, err
}
p.Details = append(p.Details, any)
}
return &Status{s: p}, nil
}
// Details returns a slice of details messages attached to the status.
// If a detail cannot be decoded, the error is returned in place of the detail.
func (s *Status) Details() []interface{} {
if s == nil || s.s == nil {
return nil
}
details := make([]interface{}, 0, len(s.s.Details))
for _, any := range s.s.Details {
detail := &ptypes.DynamicAny{}
if err := ptypes.UnmarshalAny(any, detail); err != nil {
details = append(details, err)
continue
}
details = append(details, detail.Message)
}
return details
}
// Code returns the Code of the error if it is a Status error, codes.OK if err
// is nil, or codes.Unknown otherwise.
func Code(err error) codes.Code {

View File

@ -19,4 +19,4 @@
package grpc
// Version is the current grpc version.
const Version = "1.28.1"
const Version = "1.29.1"

39
vendor/google.golang.org/grpc/vet.sh generated vendored
View File

@ -1,20 +1,22 @@
#!/bin/bash
if [[ `uname -a` = *"Darwin"* ]]; then
echo "It seems you are running on Mac. This script does not work on Mac. See https://github.com/grpc/grpc-go/issues/2047"
exit 1
fi
set -ex # Exit on error; debugging enabled.
set -o pipefail # Fail a pipe if any sub-command fails.
# not makes sure the command passed to it does not exit with a return code of 0.
not() {
# This is required instead of the earlier (! $COMMAND) because subshells and
# pipefail don't work the same on Darwin as in Linux.
! "$@"
}
die() {
echo "$@" >&2
exit 1
}
fail_on_output() {
tee /dev/stderr | (! read)
tee /dev/stderr | not read
}
# Check to make sure it's safe to modify the user's git repo.
@ -60,7 +62,7 @@ if [[ "$1" = "-install" ]]; then
unzip ${PROTOC_FILENAME}
bin/protoc --version
popd
elif ! which protoc > /dev/null; then
elif not which protoc > /dev/null; then
die "Please install protoc into your path"
fi
fi
@ -70,21 +72,21 @@ elif [[ "$#" -ne 0 ]]; then
fi
# - Ensure all source files contain a copyright message.
(! git grep -L "\(Copyright [0-9]\{4,\} gRPC authors\)\|DO NOT EDIT" -- '*.go')
not git grep -L "\(Copyright [0-9]\{4,\} gRPC authors\)\|DO NOT EDIT" -- '*.go'
# - Make sure all tests in grpc and grpc/test use leakcheck via Teardown.
(! grep 'func Test[^(]' *_test.go)
(! grep 'func Test[^(]' test/*.go)
not grep 'func Test[^(]' *_test.go
not grep 'func Test[^(]' test/*.go
# - Do not import x/net/context.
(! git grep -l 'x/net/context' -- "*.go")
not git grep -l 'x/net/context' -- "*.go"
# - Do not import math/rand for real library code. Use internal/grpcrand for
# thread safety.
git grep -l '"math/rand"' -- "*.go" 2>&1 | (! grep -v '^examples\|^stress\|grpcrand\|^benchmark\|wrr_test')
git grep -l '"math/rand"' -- "*.go" 2>&1 | not grep -v '^examples\|^stress\|grpcrand\|^benchmark\|wrr_test'
# - Ensure all ptypes proto packages are renamed when importing.
(! git grep "\(import \|^\s*\)\"github.com/golang/protobuf/ptypes/" -- "*.go")
not git grep "\(import \|^\s*\)\"github.com/golang/protobuf/ptypes/" -- "*.go"
# - Check imports that are illegal in appengine (until Go 1.11).
# TODO: Remove when we drop Go 1.10 support
@ -92,8 +94,8 @@ go list -f {{.Dir}} ./... | xargs go run test/go_vet/vet.go
# - gofmt, goimports, golint (with exceptions for generated code), go vet.
gofmt -s -d -l . 2>&1 | fail_on_output
goimports -l . 2>&1 | (! grep -vE "(_mock|\.pb)\.go")
golint ./... 2>&1 | (! grep -vE "(_mock|\.pb)\.go:")
goimports -l . 2>&1 | not grep -vE "(_mock|\.pb)\.go"
golint ./... 2>&1 | not grep -vE "(_mock|\.pb)\.go:"
go vet -all ./...
misspell -error .
@ -119,9 +121,9 @@ fi
SC_OUT="$(mktemp)"
staticcheck -go 1.9 -checks 'inherit,-ST1015' ./... > "${SC_OUT}" || true
# Error if anything other than deprecation warnings are printed.
(! grep -v "is deprecated:.*SA1019" "${SC_OUT}")
not grep -v "is deprecated:.*SA1019" "${SC_OUT}"
# Only ignore the following deprecated types/fields/functions.
(! grep -Fv '.HandleResolvedAddrs
not grep -Fv '.HandleResolvedAddrs
.HandleSubConnStateChange
.HeaderMap
.NewAddress
@ -157,4 +159,5 @@ naming.Update
naming.Watcher
resolver.Backend
resolver.GRPCLB' "${SC_OUT}"
)
echo SUCCESS

1
vendor/gopkg.in/yaml.v2/apic.go generated vendored
View File

@ -86,6 +86,7 @@ func yaml_emitter_initialize(emitter *yaml_emitter_t) {
raw_buffer: make([]byte, 0, output_raw_buffer_size),
states: make([]yaml_emitter_state_t, 0, initial_stack_size),
events: make([]yaml_event_t, 0, initial_queue_size),
best_width: -1,
}
}

29
vendor/modules.txt vendored
View File

@ -119,14 +119,14 @@ github.com/containerd/fifo
github.com/containerd/go-cni
# github.com/containerd/go-runc v0.0.0-20201020171139-16b287bc67d0
github.com/containerd/go-runc
# github.com/containerd/stargz-snapshotter v0.0.0-20200903042824-2ee75e91f8f9
# github.com/containerd/stargz-snapshotter v0.0.0-20201027054423-3a04e4c2c116
github.com/containerd/stargz-snapshotter/cache
github.com/containerd/stargz-snapshotter/snapshot
github.com/containerd/stargz-snapshotter/stargz
github.com/containerd/stargz-snapshotter/stargz/config
github.com/containerd/stargz-snapshotter/stargz/handler
github.com/containerd/stargz-snapshotter/stargz/reader
github.com/containerd/stargz-snapshotter/stargz/remote
github.com/containerd/stargz-snapshotter/stargz/source
github.com/containerd/stargz-snapshotter/stargz/verify
github.com/containerd/stargz-snapshotter/task
# github.com/containerd/ttrpc v1.0.1
@ -235,7 +235,7 @@ github.com/gogo/protobuf/proto
github.com/gogo/protobuf/protoc-gen-gogo/descriptor
github.com/gogo/protobuf/sortkeys
github.com/gogo/protobuf/types
# github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7
# github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e
github.com/golang/groupcache/lru
# github.com/golang/protobuf v1.4.2 => github.com/golang/protobuf v1.3.5
github.com/golang/protobuf/proto
@ -245,19 +245,12 @@ github.com/golang/protobuf/ptypes/duration
github.com/golang/protobuf/ptypes/timestamp
# github.com/google/crfs v0.0.0-20191108021818-71d77da419c9
github.com/google/crfs/stargz
# github.com/google/go-cmp v0.4.0
# github.com/google/go-cmp v0.4.1
github.com/google/go-cmp/cmp
github.com/google/go-cmp/cmp/internal/diff
github.com/google/go-cmp/cmp/internal/flags
github.com/google/go-cmp/cmp/internal/function
github.com/google/go-cmp/cmp/internal/value
# github.com/google/go-containerregistry v0.0.0-20200425101607-48f605c3b60a
github.com/google/go-containerregistry/pkg/authn
github.com/google/go-containerregistry/pkg/internal/retry
github.com/google/go-containerregistry/pkg/internal/retry/wait
github.com/google/go-containerregistry/pkg/logs
github.com/google/go-containerregistry/pkg/name
github.com/google/go-containerregistry/pkg/v1/remote/transport
# github.com/google/shlex v0.0.0-20150127133951-6f45313302b9
github.com/google/shlex
# github.com/google/uuid v1.1.1
@ -353,7 +346,7 @@ github.com/urfave/cli
github.com/willf/bitset
# go.etcd.io/bbolt v1.3.5
go.etcd.io/bbolt
# go.opencensus.io v0.22.2
# go.opencensus.io v0.22.3
go.opencensus.io
go.opencensus.io/internal
go.opencensus.io/trace
@ -384,9 +377,10 @@ golang.org/x/net/internal/socks
golang.org/x/net/internal/timeseries
golang.org/x/net/proxy
golang.org/x/net/trace
# golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e
# golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208
golang.org/x/sync/errgroup
golang.org/x/sync/semaphore
golang.org/x/sync/singleflight
# golang.org/x/sys v0.0.0-20200916030750-2334cc1a136f
golang.org/x/sys/cpu
golang.org/x/sys/internal/unsafeheader
@ -398,11 +392,11 @@ golang.org/x/text/secure/bidirule
golang.org/x/text/transform
golang.org/x/text/unicode/bidi
golang.org/x/text/unicode/norm
# golang.org/x/time v0.0.0-20191024005414-555d28b269f0
# golang.org/x/time v0.0.0-20200416051211-89c76fbcd5d1
golang.org/x/time/rate
# google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013 => google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63
# google.golang.org/genproto v0.0.0-20200527145253-8367513e4ece => google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63
google.golang.org/genproto/googleapis/rpc/status
# google.golang.org/grpc v1.28.1
# google.golang.org/grpc v1.29.1
google.golang.org/grpc
google.golang.org/grpc/attributes
google.golang.org/grpc/backoff
@ -432,6 +426,7 @@ google.golang.org/grpc/internal/grpcsync
google.golang.org/grpc/internal/grpcutil
google.golang.org/grpc/internal/resolver/dns
google.golang.org/grpc/internal/resolver/passthrough
google.golang.org/grpc/internal/status
google.golang.org/grpc/internal/syscall
google.golang.org/grpc/internal/transport
google.golang.org/grpc/keepalive
@ -443,7 +438,7 @@ google.golang.org/grpc/serviceconfig
google.golang.org/grpc/stats
google.golang.org/grpc/status
google.golang.org/grpc/tap
# gopkg.in/yaml.v2 v2.2.8
# gopkg.in/yaml.v2 v2.3.0
gopkg.in/yaml.v2
# gotest.tools/v3 v3.0.2
gotest.tools/v3/assert