1
0
Fork 0
mirror of https://github.com/moby/moby.git synced 2022-11-09 12:21:53 -05:00

LCOW: Pass platform through into layer store

Signed-off-by: John Howard <jhoward@microsoft.com>
This commit is contained in:
John Howard 2017-04-25 16:45:42 -07:00
parent fc21bf280b
commit 42c5c1a9ec
18 changed files with 125 additions and 74 deletions

View file

@ -160,6 +160,7 @@ func (daemon *Daemon) Commit(name string, c *backend.ContainerCommitConfig) (str
}()
var parent *image.Image
os := runtime.GOOS
if container.ImageID == "" {
parent = new(image.Image)
parent.RootFS = image.NewRootFS()
@ -168,9 +169,13 @@ func (daemon *Daemon) Commit(name string, c *backend.ContainerCommitConfig) (str
if err != nil {
return "", err
}
// To support LCOW, Windows needs to pass the platform in when registering the layer in the store
if runtime.GOOS == "windows" {
os = parent.OS
}
}
l, err := daemon.layerStore.Register(rwTar, parent.RootFS.ChainID())
l, err := daemon.layerStore.Register(rwTar, parent.RootFS.ChainID(), layer.Platform(os))
if err != nil {
return "", err
}

View file

@ -3,6 +3,7 @@ package daemon
import (
"encoding/json"
"fmt"
"runtime"
"sort"
"time"
@ -272,7 +273,13 @@ func (daemon *Daemon) SquashImage(id, parent string) (string, error) {
}
defer ts.Close()
newL, err := daemon.layerStore.Register(ts, parentChainID)
// To support LCOW, Windows needs to pass the platform into the store when registering the layer.
platform := layer.Platform("")
if runtime.GOOS == "windows" {
platform = l.Platform()
}
newL, err := daemon.layerStore.Register(ts, parentChainID, platform)
if err != nil {
return "", errors.Wrap(err, "error registering layer")
}

View file

@ -86,7 +86,12 @@ func (daemon *Daemon) ImportImage(src string, repository, tag string, msg string
return err
}
// TODO: support windows baselayer?
l, err := daemon.layerStore.Register(inflatedLayerData, "")
// TODO: LCOW support @jhowardmsft. For now, pass in a null platform when
// registering the layer. Windows doesn't currently support import,
// but for Linux images, there's no reason it couldn't. However it
// would need another CLI flag as there's no meta-data indicating
// the OS of the thing being imported.
l, err := daemon.layerStore.Register(inflatedLayerData, "", "")
if err != nil {
return err
}

View file

@ -83,7 +83,7 @@ type ImagePushConfig struct {
type ImageConfigStore interface {
Put([]byte) (digest.Digest, error)
Get(digest.Digest) ([]byte, error)
RootFSFromConfig([]byte) (*image.RootFS, error)
RootFSAndPlatformFromConfig([]byte) (*image.RootFS, layer.Platform, error)
}
// PushLayerProvider provides layers to be pushed by ChainID.
@ -109,7 +109,7 @@ type RootFSDownloadManager interface {
// returns the final rootfs.
// Given progress output to track download progress
// Returns function to release download resources
Download(ctx context.Context, initialRootFS image.RootFS, layers []xfer.DownloadDescriptor, progressOutput progress.Output) (image.RootFS, func(), error)
Download(ctx context.Context, initialRootFS image.RootFS, platform layer.Platform, layers []xfer.DownloadDescriptor, progressOutput progress.Output) (image.RootFS, func(), error)
}
type imageConfigStore struct {
@ -137,21 +137,25 @@ func (s *imageConfigStore) Get(d digest.Digest) ([]byte, error) {
return img.RawJSON(), nil
}
func (s *imageConfigStore) RootFSFromConfig(c []byte) (*image.RootFS, error) {
func (s *imageConfigStore) RootFSAndPlatformFromConfig(c []byte) (*image.RootFS, layer.Platform, error) {
var unmarshalledConfig image.Image
if err := json.Unmarshal(c, &unmarshalledConfig); err != nil {
return nil, err
return nil, "", err
}
// fail immediately on Windows when downloading a non-Windows image
// and vice versa. Exception on Windows if Linux Containers are enabled.
if runtime.GOOS == "windows" && unmarshalledConfig.OS == "linux" && !system.LCOWSupported() {
return nil, fmt.Errorf("image operating system %q cannot be used on this platform", unmarshalledConfig.OS)
return nil, "", fmt.Errorf("image operating system %q cannot be used on this platform", unmarshalledConfig.OS)
} else if runtime.GOOS != "windows" && unmarshalledConfig.OS == "windows" {
return nil, fmt.Errorf("image operating system %q cannot be used on this platform", unmarshalledConfig.OS)
return nil, "", fmt.Errorf("image operating system %q cannot be used on this platform", unmarshalledConfig.OS)
}
return unmarshalledConfig.RootFS, nil
platform := ""
if runtime.GOOS == "windows" {
platform = unmarshalledConfig.OS
}
return unmarshalledConfig.RootFS, layer.Platform(platform), nil
}
type storeLayerProvider struct {

View file

@ -232,7 +232,7 @@ func (p *v1Puller) pullImage(ctx context.Context, v1ID, endpoint string, localNa
}
rootFS := image.NewRootFS()
resultRootFS, release, err := p.config.DownloadManager.Download(ctx, *rootFS, descriptors, p.config.ProgressOutput)
resultRootFS, release, err := p.config.DownloadManager.Download(ctx, *rootFS, "", descriptors, p.config.ProgressOutput)
if err != nil {
return err
}

View file

@ -486,7 +486,7 @@ func (p *v2Puller) pullSchema1(ctx context.Context, ref reference.Named, unverif
descriptors = append(descriptors, layerDescriptor)
}
resultRootFS, release, err := p.config.DownloadManager.Download(ctx, *rootFS, descriptors, p.config.ProgressOutput)
resultRootFS, release, err := p.config.DownloadManager.Download(ctx, *rootFS, "", descriptors, p.config.ProgressOutput)
if err != nil {
return "", "", err
}
@ -556,10 +556,11 @@ func (p *v2Puller) pullSchema2(ctx context.Context, ref reference.Named, mfst *s
}()
var (
configJSON []byte // raw serialized image config
downloadedRootFS *image.RootFS // rootFS from registered layers
configRootFS *image.RootFS // rootFS from configuration
release func() // release resources from rootFS download
configJSON []byte // raw serialized image config
downloadedRootFS *image.RootFS // rootFS from registered layers
configRootFS *image.RootFS // rootFS from configuration
release func() // release resources from rootFS download
platform layer.Platform // for LCOW when registering downloaded layers
)
// https://github.com/docker/docker/issues/24766 - Err on the side of caution,
@ -571,7 +572,7 @@ func (p *v2Puller) pullSchema2(ctx context.Context, ref reference.Named, mfst *s
// check to block Windows images being pulled on Linux is implemented, it
// may be necessary to perform the same type of serialisation.
if runtime.GOOS == "windows" {
configJSON, configRootFS, err = receiveConfig(p.config.ImageStore, configChan, configErrChan)
configJSON, configRootFS, platform, err = receiveConfig(p.config.ImageStore, configChan, configErrChan)
if err != nil {
return "", "", err
}
@ -598,7 +599,7 @@ func (p *v2Puller) pullSchema2(ctx context.Context, ref reference.Named, mfst *s
rootFS image.RootFS
)
downloadRootFS := *image.NewRootFS()
rootFS, release, err = p.config.DownloadManager.Download(ctx, downloadRootFS, descriptors, p.config.ProgressOutput)
rootFS, release, err = p.config.DownloadManager.Download(ctx, downloadRootFS, platform, descriptors, p.config.ProgressOutput)
if err != nil {
// Intentionally do not cancel the config download here
// as the error from config download (if there is one)
@ -616,7 +617,7 @@ func (p *v2Puller) pullSchema2(ctx context.Context, ref reference.Named, mfst *s
}
if configJSON == nil {
configJSON, configRootFS, err = receiveConfig(p.config.ImageStore, configChan, configErrChan)
configJSON, configRootFS, _, err = receiveConfig(p.config.ImageStore, configChan, configErrChan)
if err == nil && configRootFS == nil {
err = errRootFSInvalid
}
@ -663,16 +664,16 @@ func (p *v2Puller) pullSchema2(ctx context.Context, ref reference.Named, mfst *s
return imageID, manifestDigest, nil
}
func receiveConfig(s ImageConfigStore, configChan <-chan []byte, errChan <-chan error) ([]byte, *image.RootFS, error) {
func receiveConfig(s ImageConfigStore, configChan <-chan []byte, errChan <-chan error) ([]byte, *image.RootFS, layer.Platform, error) {
select {
case configJSON := <-configChan:
rootfs, err := s.RootFSFromConfig(configJSON)
rootfs, platform, err := s.RootFSAndPlatformFromConfig(configJSON)
if err != nil {
return nil, nil, err
return nil, nil, "", err
}
return configJSON, rootfs, nil
return configJSON, rootfs, platform, nil
case err := <-errChan:
return nil, nil, err
return nil, nil, "", err
// Don't need a case for ctx.Done in the select because cancellation
// will trigger an error in p.pullSchema2ImageConfig.
}

View file

@ -118,7 +118,7 @@ func (p *v2Pusher) pushV2Tag(ctx context.Context, ref reference.NamedTagged, id
return fmt.Errorf("could not find image from tag %s: %v", reference.FamiliarString(ref), err)
}
rootfs, err := p.config.ImageStore.RootFSFromConfig(imgConfig)
rootfs, _, err := p.config.ImageStore.RootFSAndPlatformFromConfig(imgConfig)
if err != nil {
return fmt.Errorf("unable to get rootfs for image %s: %s", reference.FamiliarString(ref), err)
}

View file

@ -94,7 +94,7 @@ type DownloadDescriptorWithRegistered interface {
// Download method is called to get the layer tar data. Layers are then
// registered in the appropriate order. The caller must call the returned
// release function once it is done with the returned RootFS object.
func (ldm *LayerDownloadManager) Download(ctx context.Context, initialRootFS image.RootFS, layers []DownloadDescriptor, progressOutput progress.Output) (image.RootFS, func(), error) {
func (ldm *LayerDownloadManager) Download(ctx context.Context, initialRootFS image.RootFS, platform layer.Platform, layers []DownloadDescriptor, progressOutput progress.Output) (image.RootFS, func(), error) {
var (
topLayer layer.Layer
topDownload *downloadTransfer
@ -140,7 +140,7 @@ func (ldm *LayerDownloadManager) Download(ctx context.Context, initialRootFS ima
// the stack? If so, avoid downloading it more than once.
var topDownloadUncasted Transfer
if existingDownload, ok := downloadsByKey[key]; ok {
xferFunc := ldm.makeDownloadFuncFromDownload(descriptor, existingDownload, topDownload)
xferFunc := ldm.makeDownloadFuncFromDownload(descriptor, existingDownload, topDownload, platform)
defer topDownload.Transfer.Release(watcher)
topDownloadUncasted, watcher = ldm.tm.Transfer(transferKey, xferFunc, progressOutput)
topDownload = topDownloadUncasted.(*downloadTransfer)
@ -152,10 +152,10 @@ func (ldm *LayerDownloadManager) Download(ctx context.Context, initialRootFS ima
var xferFunc DoFunc
if topDownload != nil {
xferFunc = ldm.makeDownloadFunc(descriptor, "", topDownload)
xferFunc = ldm.makeDownloadFunc(descriptor, "", topDownload, platform)
defer topDownload.Transfer.Release(watcher)
} else {
xferFunc = ldm.makeDownloadFunc(descriptor, rootFS.ChainID(), nil)
xferFunc = ldm.makeDownloadFunc(descriptor, rootFS.ChainID(), nil, platform)
}
topDownloadUncasted, watcher = ldm.tm.Transfer(transferKey, xferFunc, progressOutput)
topDownload = topDownloadUncasted.(*downloadTransfer)
@ -212,7 +212,7 @@ func (ldm *LayerDownloadManager) Download(ctx context.Context, initialRootFS ima
// complete before the registration step, and registers the downloaded data
// on top of parentDownload's resulting layer. Otherwise, it registers the
// layer on top of the ChainID given by parentLayer.
func (ldm *LayerDownloadManager) makeDownloadFunc(descriptor DownloadDescriptor, parentLayer layer.ChainID, parentDownload *downloadTransfer) DoFunc {
func (ldm *LayerDownloadManager) makeDownloadFunc(descriptor DownloadDescriptor, parentLayer layer.ChainID, parentDownload *downloadTransfer, platform layer.Platform) DoFunc {
return func(progressChan chan<- progress.Progress, start <-chan struct{}, inactive chan<- struct{}) Transfer {
d := &downloadTransfer{
Transfer: NewTransfer(),
@ -335,9 +335,9 @@ func (ldm *LayerDownloadManager) makeDownloadFunc(descriptor DownloadDescriptor,
src = fs.Descriptor()
}
if ds, ok := d.layerStore.(layer.DescribableStore); ok {
d.layer, err = ds.RegisterWithDescriptor(inflatedLayerData, parentLayer, src)
d.layer, err = ds.RegisterWithDescriptor(inflatedLayerData, parentLayer, platform, src)
} else {
d.layer, err = d.layerStore.Register(inflatedLayerData, parentLayer)
d.layer, err = d.layerStore.Register(inflatedLayerData, parentLayer, platform)
}
if err != nil {
select {
@ -376,7 +376,7 @@ func (ldm *LayerDownloadManager) makeDownloadFunc(descriptor DownloadDescriptor,
// parentDownload. This function does not log progress output because it would
// interfere with the progress reporting for sourceDownload, which has the same
// Key.
func (ldm *LayerDownloadManager) makeDownloadFuncFromDownload(descriptor DownloadDescriptor, sourceDownload *downloadTransfer, parentDownload *downloadTransfer) DoFunc {
func (ldm *LayerDownloadManager) makeDownloadFuncFromDownload(descriptor DownloadDescriptor, sourceDownload *downloadTransfer, parentDownload *downloadTransfer, platform layer.Platform) DoFunc {
return func(progressChan chan<- progress.Progress, start <-chan struct{}, inactive chan<- struct{}) Transfer {
d := &downloadTransfer{
Transfer: NewTransfer(),
@ -434,9 +434,9 @@ func (ldm *LayerDownloadManager) makeDownloadFuncFromDownload(descriptor Downloa
src = fs.Descriptor()
}
if ds, ok := d.layerStore.(layer.DescribableStore); ok {
d.layer, err = ds.RegisterWithDescriptor(layerReader, parentLayer, src)
d.layer, err = ds.RegisterWithDescriptor(layerReader, parentLayer, platform, src)
} else {
d.layer, err = d.layerStore.Register(layerReader, parentLayer)
d.layer, err = d.layerStore.Register(layerReader, parentLayer, platform)
}
if err != nil {
d.err = fmt.Errorf("failed to register layer: %v", err)

View file

@ -91,7 +91,7 @@ func (ls *mockLayerStore) Map() map[layer.ChainID]layer.Layer {
return layers
}
func (ls *mockLayerStore) Register(reader io.Reader, parentID layer.ChainID) (layer.Layer, error) {
func (ls *mockLayerStore) Register(reader io.Reader, parentID layer.ChainID, platform layer.Platform) (layer.Layer, error) {
return ls.RegisterWithDescriptor(reader, parentID, distribution.Descriptor{})
}
@ -272,7 +272,9 @@ func TestSuccessfulDownload(t *testing.T) {
}
layerStore := &mockLayerStore{make(map[layer.ChainID]*mockLayer)}
ldm := NewLayerDownloadManager(layerStore, maxDownloadConcurrency, func(m *LayerDownloadManager) { m.waitDuration = time.Millisecond })
lsMap := make(map[string]layer.Store)
lsMap[runtime.GOOS] = layerStore
ldm := NewLayerDownloadManager(lsMap, maxDownloadConcurrency, func(m *LayerDownloadManager) { m.waitDuration = time.Millisecond })
progressChan := make(chan progress.Progress)
progressDone := make(chan struct{})
@ -291,13 +293,13 @@ func TestSuccessfulDownload(t *testing.T) {
firstDescriptor := descriptors[0].(*mockDownloadDescriptor)
// Pre-register the first layer to simulate an already-existing layer
l, err := layerStore.Register(firstDescriptor.mockTarStream(), "")
l, err := layerStore.Register(firstDescriptor.mockTarStream(), "", layer.Platform(runtime.GOOS))
if err != nil {
t.Fatal(err)
}
firstDescriptor.diffID = l.DiffID()
rootFS, releaseFunc, err := ldm.Download(context.Background(), *image.NewRootFS(), descriptors, progress.ChanOutput(progressChan))
rootFS, releaseFunc, err := ldm.Download(context.Background(), *image.NewRootFS(), layer.Platform(runtime.GOOS), descriptors, progress.ChanOutput(progressChan))
if err != nil {
t.Fatalf("download error: %v", err)
}
@ -333,7 +335,10 @@ func TestSuccessfulDownload(t *testing.T) {
}
func TestCancelledDownload(t *testing.T) {
ldm := NewLayerDownloadManager(&mockLayerStore{make(map[layer.ChainID]*mockLayer)}, maxDownloadConcurrency, func(m *LayerDownloadManager) { m.waitDuration = time.Millisecond })
layerStore := &mockLayerStore{make(map[layer.ChainID]*mockLayer)}
lsMap := make(map[string]layer.Store)
lsMap[runtime.GOOS] = layerStore
ldm := NewLayerDownloadManager(lsMap, maxDownloadConcurrency, func(m *LayerDownloadManager) { m.waitDuration = time.Millisecond })
progressChan := make(chan progress.Progress)
progressDone := make(chan struct{})
@ -352,7 +357,7 @@ func TestCancelledDownload(t *testing.T) {
}()
descriptors := downloadDescriptors(nil)
_, _, err := ldm.Download(ctx, *image.NewRootFS(), descriptors, progress.ChanOutput(progressChan))
_, _, err := ldm.Download(ctx, *image.NewRootFS(), layer.Platform(runtime.GOOS), descriptors, progress.ChanOutput(progressChan))
if err != context.Canceled {
t.Fatal("expected download to be cancelled")
}

View file

@ -2,12 +2,14 @@ package tarexport
import (
"encoding/json"
"errors"
"fmt"
"io"
"io/ioutil"
"os"
"path/filepath"
"reflect"
"runtime"
"github.com/Sirupsen/logrus"
"github.com/docker/distribution"
@ -85,6 +87,17 @@ func (l *tarexporter) Load(inTar io.ReadCloser, outStream io.Writer, quiet bool)
return fmt.Errorf("invalid manifest, layers length mismatch: expected %d, got %d", expected, actual)
}
// On Windows, validate the platform, defaulting to windows if not present.
platform := layer.Platform(img.OS)
if runtime.GOOS == "windows" {
if platform == "" {
platform = "windows"
}
if (platform != "windows") && (platform != "linux") {
return fmt.Errorf("configuration for this image has an unsupported platform: %s", platform)
}
}
for i, diffID := range img.RootFS.DiffIDs {
layerPath, err := safePath(tmpDir, m.Layers[i])
if err != nil {
@ -94,7 +107,7 @@ func (l *tarexporter) Load(inTar io.ReadCloser, outStream io.Writer, quiet bool)
r.Append(diffID)
newLayer, err := l.ls.Get(r.ChainID())
if err != nil {
newLayer, err = l.loadLayer(layerPath, rootFS, diffID.String(), m.LayerSources[diffID], progressOutput)
newLayer, err = l.loadLayer(layerPath, rootFS, diffID.String(), platform, m.LayerSources[diffID], progressOutput)
if err != nil {
return err
}
@ -161,7 +174,7 @@ func (l *tarexporter) setParentID(id, parentID image.ID) error {
return l.is.SetParent(id, parentID)
}
func (l *tarexporter) loadLayer(filename string, rootFS image.RootFS, id string, foreignSrc distribution.Descriptor, progressOutput progress.Output) (layer.Layer, error) {
func (l *tarexporter) loadLayer(filename string, rootFS image.RootFS, id string, platform layer.Platform, foreignSrc distribution.Descriptor, progressOutput progress.Output) (layer.Layer, error) {
// We use system.OpenSequential to use sequential file access on Windows, avoiding
// depleting the standby list. On Linux, this equates to a regular os.Open.
rawTar, err := system.OpenSequential(filename)
@ -191,9 +204,9 @@ func (l *tarexporter) loadLayer(filename string, rootFS image.RootFS, id string,
defer inflatedLayerData.Close()
if ds, ok := l.ls.(layer.DescribableStore); ok {
return ds.RegisterWithDescriptor(inflatedLayerData, rootFS.ChainID(), foreignSrc)
return ds.RegisterWithDescriptor(inflatedLayerData, rootFS.ChainID(), platform, foreignSrc)
}
return l.ls.Register(inflatedLayerData, rootFS.ChainID())
return l.ls.Register(inflatedLayerData, rootFS.ChainID(), platform)
}
func (l *tarexporter) setLoadedTag(ref reference.NamedTagged, imgID digest.Digest, outStream io.Writer) error {
@ -208,6 +221,10 @@ func (l *tarexporter) setLoadedTag(ref reference.NamedTagged, imgID digest.Diges
}
func (l *tarexporter) legacyLoad(tmpDir string, outStream io.Writer, progressOutput progress.Output) error {
if runtime.GOOS == "windows" {
return errors.New("Windows does not support legacy loading of images")
}
legacyLoadedMap := make(map[string]image.ID)
dirs, err := ioutil.ReadDir(tmpDir)
@ -312,7 +329,7 @@ func (l *tarexporter) legacyLoadImage(oldID, sourceDir string, loadedMap map[str
if err != nil {
return err
}
newLayer, err := l.loadLayer(layerPath, *rootFS, oldID, distribution.Descriptor{}, progressOutput)
newLayer, err := l.loadLayer(layerPath, *rootFS, oldID, "", distribution.Descriptor{}, progressOutput)
if err != nil {
return err
}

View file

@ -190,7 +190,7 @@ type CreateRWLayerOpts struct {
// Store represents a backend for managing both
// read-only and read-write layers.
type Store interface {
Register(io.Reader, ChainID) (Layer, error)
Register(io.Reader, ChainID, Platform) (Layer, error)
Get(ChainID) (Layer, error)
Map() map[ChainID]Layer
Release(Layer) ([]Metadata, error)
@ -208,7 +208,7 @@ type Store interface {
// DescribableStore represents a layer store capable of storing
// descriptors for layers.
type DescribableStore interface {
RegisterWithDescriptor(io.Reader, ChainID, distribution.Descriptor) (Layer, error)
RegisterWithDescriptor(io.Reader, ChainID, Platform, distribution.Descriptor) (Layer, error)
}
// MetadataTransaction represents functions for setting layer metadata

View file

@ -253,11 +253,11 @@ func (ls *layerStore) applyTar(tx MetadataTransaction, ts io.Reader, parent stri
return nil
}
func (ls *layerStore) Register(ts io.Reader, parent ChainID) (Layer, error) {
return ls.registerWithDescriptor(ts, parent, distribution.Descriptor{})
func (ls *layerStore) Register(ts io.Reader, parent ChainID, platform Platform) (Layer, error) {
return ls.registerWithDescriptor(ts, parent, platform, distribution.Descriptor{})
}
func (ls *layerStore) registerWithDescriptor(ts io.Reader, parent ChainID, descriptor distribution.Descriptor) (Layer, error) {
func (ls *layerStore) registerWithDescriptor(ts io.Reader, parent ChainID, platform Platform, descriptor distribution.Descriptor) (Layer, error) {
// err is used to hold the error which will always trigger
// cleanup of creates sources but may not be an error returned
// to the caller (already exists).
@ -292,6 +292,7 @@ func (ls *layerStore) registerWithDescriptor(ts io.Reader, parent ChainID, descr
layerStore: ls,
references: map[Layer]struct{}{},
descriptor: descriptor,
platform: platform,
}
if err = ls.driver.Create(layer.cacheID, pid, nil); err != nil {
@ -394,7 +395,6 @@ func (ls *layerStore) deleteLayer(layer *roLayer, metadata *Metadata) error {
if err != nil {
return err
}
err = ls.store.Remove(layer.chainID)
if err != nil {
return err
@ -524,7 +524,6 @@ func (ls *layerStore) CreateRWLayer(name string, parent ChainID, opts *CreateRWL
if err = ls.driver.CreateReadWrite(m.mountID, pid, createOpts); err != nil {
return nil, err
}
if err = ls.saveMount(m); err != nil {
return nil, err
}

View file

@ -6,6 +6,6 @@ import (
"github.com/docker/distribution"
)
func (ls *layerStore) RegisterWithDescriptor(ts io.Reader, parent ChainID, descriptor distribution.Descriptor) (Layer, error) {
return ls.registerWithDescriptor(ts, parent, descriptor)
func (ls *layerStore) RegisterWithDescriptor(ts io.Reader, parent ChainID, platform Platform, descriptor distribution.Descriptor) (Layer, error) {
return ls.registerWithDescriptor(ts, parent, platform, descriptor)
}

View file

@ -106,7 +106,7 @@ func createLayer(ls Store, parent ChainID, layerFunc layerInit) (Layer, error) {
}
defer ts.Close()
layer, err := ls.Register(ts, parent)
layer, err := ls.Register(ts, parent, Platform(runtime.GOOS))
if err != nil {
return nil, err
}
@ -499,7 +499,7 @@ func TestTarStreamStability(t *testing.T) {
t.Fatal(err)
}
layer1, err := ls.Register(bytes.NewReader(tar1), "")
layer1, err := ls.Register(bytes.NewReader(tar1), "", Platform(runtime.GOOS))
if err != nil {
t.Fatal(err)
}
@ -518,7 +518,7 @@ func TestTarStreamStability(t *testing.T) {
t.Fatal(err)
}
layer2, err := ls.Register(bytes.NewReader(tar2), layer1.ChainID())
layer2, err := ls.Register(bytes.NewReader(tar2), layer1.ChainID(), Platform(runtime.GOOS))
if err != nil {
t.Fatal(err)
}
@ -686,12 +686,12 @@ func TestRegisterExistingLayer(t *testing.T) {
t.Fatal(err)
}
layer2a, err := ls.Register(bytes.NewReader(tar1), layer1.ChainID())
layer2a, err := ls.Register(bytes.NewReader(tar1), layer1.ChainID(), Platform(runtime.GOOS))
if err != nil {
t.Fatal(err)
}
layer2b, err := ls.Register(bytes.NewReader(tar1), layer1.ChainID())
layer2b, err := ls.Register(bytes.NewReader(tar1), layer1.ChainID(), Platform(runtime.GOOS))
if err != nil {
t.Fatal(err)
}
@ -726,12 +726,12 @@ func TestTarStreamVerification(t *testing.T) {
t.Fatal(err)
}
layer1, err := ls.Register(bytes.NewReader(tar1), "")
layer1, err := ls.Register(bytes.NewReader(tar1), "", Platform(runtime.GOOS))
if err != nil {
t.Fatal(err)
}
layer2, err := ls.Register(bytes.NewReader(tar2), "")
layer2, err := ls.Register(bytes.NewReader(tar2), "", Platform(runtime.GOOS))
if err != nil {
t.Fatal(err)
}

View file

@ -110,14 +110,14 @@ func TestLayerMigration(t *testing.T) {
t.Fatal(err)
}
layer1b, err := ls.Register(bytes.NewReader(tar1), "")
layer1b, err := ls.Register(bytes.NewReader(tar1), "", Platform(runtime.GOOS))
if err != nil {
t.Fatal(err)
}
assertReferences(t, layer1a, layer1b)
// Attempt register, should be same
layer2a, err := ls.Register(bytes.NewReader(tar2), layer1a.ChainID())
layer2a, err := ls.Register(bytes.NewReader(tar2), layer1a.ChainID(), Platform(runtime.GOOS))
if err != nil {
t.Fatal(err)
}
@ -238,7 +238,7 @@ func TestLayerMigrationNoTarsplit(t *testing.T) {
t.Fatal(err)
}
layer1b, err := ls.Register(bytes.NewReader(tar1), "")
layer1b, err := ls.Register(bytes.NewReader(tar1), "", Platform(runtime.GOOS))
if err != nil {
t.Fatal(err)
}
@ -246,7 +246,7 @@ func TestLayerMigrationNoTarsplit(t *testing.T) {
assertReferences(t, layer1a, layer1b)
// Attempt register, should be same
layer2a, err := ls.Register(bytes.NewReader(tar2), layer1a.ChainID())
layer2a, err := ls.Register(bytes.NewReader(tar2), layer1a.ChainID(), Platform(runtime.GOOS))
if err != nil {
t.Fatal(err)
}

View file

@ -145,7 +145,7 @@ func (s *tempConfigStore) Get(d digest.Digest) ([]byte, error) {
return s.config, nil
}
func (s *tempConfigStore) RootFSFromConfig(c []byte) (*image.RootFS, error) {
func (s *tempConfigStore) RootFSAndPlatformFromConfig(c []byte) (*image.RootFS, layer.Platform, error) {
return configToRootFS(c)
}
@ -525,7 +525,7 @@ func (s *pluginConfigStore) Get(d digest.Digest) ([]byte, error) {
return ioutil.ReadAll(rwc)
}
func (s *pluginConfigStore) RootFSFromConfig(c []byte) (*image.RootFS, error) {
func (s *pluginConfigStore) RootFSAndPlatformFromConfig(c []byte) (*image.RootFS, layer.Platform, error) {
return configToRootFS(c)
}

View file

@ -126,7 +126,8 @@ type downloadManager struct {
configDigest digest.Digest
}
func (dm *downloadManager) Download(ctx context.Context, initialRootFS image.RootFS, layers []xfer.DownloadDescriptor, progressOutput progress.Output) (image.RootFS, func(), error) {
func (dm *downloadManager) Download(ctx context.Context, initialRootFS image.RootFS, platform layer.Platform, layers []xfer.DownloadDescriptor, progressOutput progress.Output) (image.RootFS, func(), error) {
// TODO @jhowardmsft LCOW: May need revisiting.
for _, l := range layers {
b, err := dm.blobStore.New()
if err != nil {
@ -178,6 +179,6 @@ func (dm *downloadManager) Put(dt []byte) (digest.Digest, error) {
func (dm *downloadManager) Get(d digest.Digest) ([]byte, error) {
return nil, fmt.Errorf("digest not found")
}
func (dm *downloadManager) RootFSFromConfig(c []byte) (*image.RootFS, error) {
func (dm *downloadManager) RootFSAndPlatformFromConfig(c []byte) (*image.RootFS, layer.Platform, error) {
return configToRootFS(c)
}

View file

@ -8,6 +8,7 @@ import (
"path/filepath"
"reflect"
"regexp"
"runtime"
"sort"
"strings"
"sync"
@ -21,6 +22,7 @@ import (
"github.com/docker/docker/pkg/authorization"
"github.com/docker/docker/pkg/ioutils"
"github.com/docker/docker/pkg/mount"
"github.com/docker/docker/pkg/system"
"github.com/docker/docker/plugin/v2"
"github.com/docker/docker/registry"
"github.com/opencontainers/go-digest"
@ -348,17 +350,22 @@ func isEqualPrivilege(a, b types.PluginPrivilege) bool {
return reflect.DeepEqual(a.Value, b.Value)
}
func configToRootFS(c []byte) (*image.RootFS, error) {
func configToRootFS(c []byte) (*image.RootFS, layer.Platform, error) {
// TODO @jhowardmsft LCOW - Will need to revisit this. For now, calculate the platform.
platform := layer.Platform(runtime.GOOS)
if platform == "windows" && system.LCOWSupported() {
platform = "linux"
}
var pluginConfig types.PluginConfig
if err := json.Unmarshal(c, &pluginConfig); err != nil {
return nil, err
return nil, "", err
}
// validation for empty rootfs is in distribution code
if pluginConfig.Rootfs == nil {
return nil, nil
return nil, platform, nil
}
return rootFSFromPlugin(pluginConfig.Rootfs), nil
return rootFSFromPlugin(pluginConfig.Rootfs), platform, nil
}
func rootFSFromPlugin(pluginfs *types.PluginConfigRootfs) *image.RootFS {