Browse Source

LCOW: Pass platform through into layer store

Signed-off-by: John Howard <jhoward@microsoft.com>
John Howard 8 years ago
parent
commit
42c5c1a9ec

+ 6 - 1
daemon/commit.go

@@ -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
 	}

+ 8 - 1
daemon/images.go

@@ -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")
 	}

+ 6 - 1
daemon/import.go

@@ -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
 	}

+ 11 - 7
distribution/config.go

@@ -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 {

+ 1 - 1
distribution/pull_v1.go

@@ -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
 	}

+ 14 - 13
distribution/pull_v2.go

@@ -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.
 	}

+ 1 - 1
distribution/push_v2.go

@@ -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)
 	}

+ 10 - 10
distribution/xfer/download.go

@@ -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)

+ 11 - 6
distribution/xfer/download_test.go

@@ -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")
 	}

+ 22 - 5
image/tarexport/load.go

@@ -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
 	}

+ 2 - 2
layer/layer.go

@@ -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

+ 4 - 5
layer/layer_store.go

@@ -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
 	}

+ 2 - 2
layer/layer_store_windows.go

@@ -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)
 }

+ 7 - 7
layer/layer_test.go

@@ -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)
 	}

+ 4 - 4
layer/migration_test.go

@@ -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)
 	}

+ 2 - 2
plugin/backend_linux.go

@@ -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)
 }
 

+ 3 - 2
plugin/blobstore.go

@@ -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)
 }

+ 11 - 4
plugin/manager.go

@@ -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 {