From ef126149c5c104133a70a30b169cae9bc58b338c Mon Sep 17 00:00:00 2001 From: Alexander Larsson Date: Mon, 20 Apr 2020 13:47:16 +0200 Subject: [PATCH] Support layer deltas Deltas are a way to avoid downloading a full copy if a layer tar file if you have a previous version of the layer available locally. In testing these deltas have been shown to be around 10x to 100x smaller than the .tar.gz files for typical Linux base images. In the typical client-side case we have some previous version of the image stored in container-storage somewhere, which means that we have an uncompressed files available, but not the actual tarball (compressed or not). This means we can use github.com/containers/tar-diff which takes two tar files and produces a delta file which when applied on the untar:ed content of the first tarfile produces the (bitwise identical) content of the uncompressed second tarfile. It just happens that the uncompressed tarfile is exactly what we need to reproduce, because that is how the layers are refered to in the image config (the DiffIDs). How this works is that we use OCI artifacts to store, for each regular image a manifest with information about the available deltas for the image. This image looks like a regular manifest, except each layer contains a tar-diff (as a blob) an uses the existing annotations key to record which DiffIDs the layer applies to. For example, a manifest would look like this: ``` { "schemaVersion": 2, "config": { "mediaType": "application/vnd.oci.image.config.v1+json", "digest": "sha256:", "size": 3 }, "annotations": { "io.github.containers.delta.target": "sha256:", }, "layers": [ { "mediaType": "application/vnd.tar-diff", "digest": "sha256:", "size": 7059734, "annotations": { "io.github.containers.delta.from": "sha256:", "io.github.containers.delta.to": "sha256:" } } ] } ``` The config blob is just an json file containing "{}". Ideally it should not be of type application/vnd.oci.image.config.v1+json, because that is reserved for docker-style images. However, docker hub (and other registries) currently don't support any other type. For registries that support OCI artifacts we should instead use some other type so that tooling can know that this is not a regular image. The way we attach the delta manifest to the image is that we store it in the same repo and the we use a single tag named `_deltaindex` pointing to an index with all the delta manifest in the repository, with the digest of each target image in the `io.github.containers.delta.target` annotation key. The delta layers record which DiffID they apply to, which is what we want to use to look up the pre-existing layers to use as delta source material, and it is what the delta apply will generate. This means however that using the deltas only works if we're allowed to substitute blobs, but this doesn't seem to be an issue in the typical case. Signed-off-by: Alexander Larsson --- copy/copy.go | 144 +++++++++++++++++++++++++++++++++++-- copy/manifest_test.go | 10 +++ docker/docker_image_src.go | 38 ++++++++++ go.mod | 1 + go.sum | 2 + image/unparsed.go | 14 ++++ manifest/oci.go | 11 ++- openshift/openshift.go | 15 ++++ storage/storage_image.go | 49 +++++++++++++ types/types.go | 72 +++++++++++++++++++ 10 files changed, 350 insertions(+), 6 deletions(-) diff --git a/copy/copy.go b/copy/copy.go index 9fc0e5123b..3d5f9b896d 100644 --- a/copy/copy.go +++ b/copy/copy.go @@ -8,6 +8,7 @@ import ( "io/ioutil" "os" "reflect" + "sort" "strings" "sync" "time" @@ -23,6 +24,7 @@ import ( "github.com/containers/image/v5/types" "github.com/containers/ocicrypt" encconfig "github.com/containers/ocicrypt/config" + "github.com/containers/tar-diff/pkg/tar-patch" digest "github.com/opencontainers/go-digest" imgspecv1 "github.com/opencontainers/image-spec/specs-go/v1" "github.com/pkg/errors" @@ -790,6 +792,11 @@ func (ic *imageCopier) copyLayers(ctx context.Context) error { srcInfosUpdated = true } + deltaLayers, err := types.ImageDeltaLayers(ic.src, ctx) + if err != nil { + return err + } + type copyLayerData struct { destInfo types.BlobInfo diffID digest.Digest @@ -809,7 +816,7 @@ func (ic *imageCopier) copyLayers(ctx context.Context) error { } data := make([]copyLayerData, numLayers) - copyLayerHelper := func(index int, srcLayer types.BlobInfo, toEncrypt bool, pool *mpb.Progress) { + copyLayerHelper := func(index int, srcLayer types.BlobInfo, toEncrypt bool, pool *mpb.Progress, deltaLayers []types.BlobInfo) { defer copySemaphore.Release(1) defer copyGroup.Done() cld := copyLayerData{} @@ -824,7 +831,7 @@ func (ic *imageCopier) copyLayers(ctx context.Context) error { logrus.Debugf("Skipping foreign layer %q copy to %s", cld.destInfo.Digest, ic.c.dest.Reference().Transport().Name()) } } else { - cld.destInfo, cld.diffID, cld.err = ic.copyLayer(ctx, srcLayer, toEncrypt, pool) + cld.destInfo, cld.diffID, cld.err = ic.copyLayer(ctx, index, srcLayer, toEncrypt, pool, deltaLayers) } data[index] = cld } @@ -861,7 +868,7 @@ func (ic *imageCopier) copyLayers(ctx context.Context) error { return errors.Wrapf(err, "Can't acquire semaphore") } copyGroup.Add(1) - go copyLayerHelper(i, srcLayer, encLayerBitmap[i], progressPool) + go copyLayerHelper(i, srcLayer, encLayerBitmap[i], progressPool, deltaLayers) } // A call to copyGroup.Wait() is done at this point by the defer above. @@ -1043,9 +1050,83 @@ type diffIDResult struct { err error } +// Get all the deltas that apply to this layer +func (ic *imageCopier) getMatchingDeltaLayers(ctx context.Context, srcIndex int, deltaLayers []types.BlobInfo) (digest.Digest, []*types.BlobInfo) { + if deltaLayers == nil { + return "", nil + } + config, _ := ic.src.OCIConfig(ctx) + if config == nil || config.RootFS.DiffIDs == nil || len(config.RootFS.DiffIDs) <= srcIndex { + return "", nil + } + + layerDiffId := config.RootFS.DiffIDs[srcIndex] + + var matchingLayers []*types.BlobInfo + for i := range deltaLayers { + deltaLayer := &deltaLayers[i] + to := deltaLayer.Annotations["io.github.containers.delta.to"] + if to == layerDiffId.String() { + matchingLayers = append(matchingLayers, deltaLayer) + } + } + + return layerDiffId, matchingLayers +} + +// Looks at which of the matching delta froms have locally available data and picks the best one +func (ic *imageCopier) resolveDeltaLayer(ctx context.Context, matchingDeltas []*types.BlobInfo) (io.ReadCloser, tar_patch.DataSource, types.BlobInfo, error) { + // Sort smallest deltas so we favour the smallest useable one + sort.Slice(matchingDeltas, func(i, j int) bool { + return matchingDeltas[i].Size < matchingDeltas[j].Size + }) + + for i := range matchingDeltas { + matchingDelta := matchingDeltas[i] + from := matchingDelta.Annotations["io.github.containers.delta.from"] + fromDigest, err := digest.Parse(from) + if err != nil { + continue // Silently ignore if server specified a werid format + } + + dataSource, err := types.ImageDestinationGetLayerDeltaData(ic.c.dest, ctx, fromDigest) + if err != nil { + return nil, nil, types.BlobInfo{}, err // Internal error + } + if dataSource == nil { + continue // from layer doesn't exist + } + + logrus.Debugf("Using delta %v for DiffID %v", matchingDelta.Digest, fromDigest) + + deltaStream, _, err := ic.c.rawSource.GetBlob(ctx, *matchingDelta, ic.c.blobInfoCache) + if err != nil { + return nil, nil, types.BlobInfo{}, errors.Wrapf(err, "Error reading delta blob %s", matchingDelta.Digest) + } + return deltaStream, dataSource, *matchingDelta, nil + } + return nil, nil, types.BlobInfo{}, nil +} + +func (ic *imageCopier) canUseDeltas(srcInfo types.BlobInfo) (bool, string) { + // Deltas rewrite the manifest to refer to the uncompressed digest, so we must be able to substiture blobs + if !ic.canSubstituteBlobs { + return false, "" + } + + switch srcInfo.MediaType { + case manifest.DockerV2Schema2LayerMediaType, manifest.DockerV2SchemaLayerMediaTypeUncompressed: + return true, manifest.DockerV2SchemaLayerMediaTypeUncompressed + case imgspecv1.MediaTypeImageLayer, imgspecv1.MediaTypeImageLayerGzip, imgspecv1.MediaTypeImageLayerZstd: + return true, imgspecv1.MediaTypeImageLayer + } + + return false, "" +} + // copyLayer copies a layer with srcInfo (with known Digest and Annotations and possibly known Size) in src to dest, perhaps compressing it if canCompress, // and returns a complete blobInfo of the copied layer, and a value for LayerDiffIDs if diffIDIsNeeded -func (ic *imageCopier) copyLayer(ctx context.Context, srcInfo types.BlobInfo, toEncrypt bool, pool *mpb.Progress) (types.BlobInfo, digest.Digest, error) { +func (ic *imageCopier) copyLayer(ctx context.Context, srcIndex int, srcInfo types.BlobInfo, toEncrypt bool, pool *mpb.Progress, deltaLayers []types.BlobInfo) (types.BlobInfo, digest.Digest, error) { cachedDiffID := ic.c.blobInfoCache.UncompressedDigest(srcInfo.Digest) // May be "" // Diffs are needed if we are encrypting an image or trying to decrypt an image diffIDIsNeeded := ic.diffIDsAreNeeded && cachedDiffID == "" || toEncrypt || (isOciEncrypted(srcInfo.MediaType) && ic.ociDecryptConfig != nil) @@ -1064,6 +1145,52 @@ func (ic *imageCopier) copyLayer(ctx context.Context, srcInfo types.BlobInfo, to } } + // First look for a delta that matches this layer and substitute the result of that + if ok, deltaResultMediaType := ic.canUseDeltas(srcInfo); ok { + // Get deltas going TO this layer + deltaDiffID, matchingDeltas := ic.getMatchingDeltaLayers(ctx, srcIndex, deltaLayers) + // Get best possible FROM delta + deltaStream, deltaDataSource, matchingDelta, err := ic.resolveDeltaLayer(ctx, matchingDeltas) + if err != nil { + return types.BlobInfo{}, "", err + } + if deltaStream != nil { + bar := ic.c.createProgressBar(pool, matchingDelta, "delta", "done") + + wrappedDeltaStream := bar.ProxyReader(deltaStream) + + // Convert deltaStream to uncompressed tar layer stream + pr, pw := io.Pipe() + go func() { + if err := tar_patch.Apply(wrappedDeltaStream, deltaDataSource, pw); err != nil { + // We will notice this error when failing to verify the digest, so leave it be + logrus.Infof("Failed to apply layer delta: %v", err) + } + deltaDataSource.Close() + deltaStream.Close() + wrappedDeltaStream.Close() + pw.Close() + }() + defer pr.Close() + + // Copy uncompressed tar layer to destination, verifying the diffID + blobInfo, err := ic.c.copyBlobFromStream(ctx, pr, types.BlobInfo{Digest: deltaDiffID, Size: -1, MediaType: deltaResultMediaType, Annotations: srcInfo.Annotations}, nil, ic.canModifyManifest, false, toEncrypt, nil) + if err != nil { + return types.BlobInfo{}, "", err + } + + bar.SetTotal(matchingDelta.Size, true) + + // We verified this when streaming the applied delta above + diffID := deltaDiffID + + // Record the fact that this blob is uncompressed + ic.c.blobInfoCache.RecordDigestUncompressedPair(diffID, diffID) + + return blobInfo, diffID, err + } + } + // Fallback: copy the layer, computing the diffID if we need to do so srcStream, srcBlobSize, err := ic.c.rawSource.GetBlob(ctx, srcInfo, ic.c.blobInfoCache) if err != nil { @@ -1213,7 +1340,9 @@ func (c *copier) copyBlobFromStream(ctx context.Context, srcStream io.Reader, sr return types.BlobInfo{}, errors.Wrapf(err, "Error reading blob %s", srcInfo.Digest) } isCompressed := decompressor != nil - destStream = bar.ProxyReader(destStream) + if bar != nil { + destStream = bar.ProxyReader(destStream) + } // === Send a copy of the original, uncompressed, stream, to a separate path if necessary. var originalLayerReader io.Reader // DO NOT USE this other than to drain the input if no other consumer in the pipeline has done so. @@ -1232,6 +1361,11 @@ func (c *copier) copyBlobFromStream(ctx context.Context, srcStream io.Reader, sr logrus.Debugf("Using original blob without modification for encrypted blob") compressionOperation = types.PreserveOriginal inputInfo = srcInfo + } else if canModifyBlob && manifest.IsNoCompressType(srcInfo.MediaType) { + // This is a blob we should not repack, such as a delta + logrus.Debugf("Using original blob without modification for no-compress type") + compressionOperation = types.PreserveOriginal + inputInfo = srcInfo } else if canModifyBlob && c.dest.DesiredLayerCompression() == types.Compress && !isCompressed { logrus.Debugf("Compressing blob on the fly") compressionOperation = types.Compress diff --git a/copy/manifest_test.go b/copy/manifest_test.go index df10678f62..262d97d031 100644 --- a/copy/manifest_test.go +++ b/copy/manifest_test.go @@ -9,6 +9,7 @@ import ( "github.com/containers/image/v5/docker/reference" "github.com/containers/image/v5/manifest" "github.com/containers/image/v5/types" + digest "github.com/opencontainers/go-digest" "github.com/opencontainers/image-spec/specs-go/v1" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -77,6 +78,15 @@ func (f fakeImageSource) SupportsEncryption(ctx context.Context) bool { func (f fakeImageSource) Size() (int64, error) { panic("Unexpected call to a mock function") } +func (f fakeImageSource) GetDeltaManifest(ctx context.Context, instanceDigest *digest.Digest) ([]byte, string, error) { + panic("Unexpected call to a mock function") +} +func (f fakeImageSource) GetDeltaIndex(ctx context.Context) (types.ImageReference, error) { + panic("Unexpected call to a mock function") +} +func (f fakeImageSource) DeltaLayers(ctx context.Context) ([]types.BlobInfo, error) { + panic("Unexpected call to a mock function") +} func TestDetermineManifestConversion(t *testing.T) { supportS1S2OCI := []string{ diff --git a/docker/docker_image_src.go b/docker/docker_image_src.go index 55eb38824a..c47aff2584 100644 --- a/docker/docker_image_src.go +++ b/docker/docker_image_src.go @@ -203,6 +203,44 @@ func (s *dockerImageSource) fetchManifest(ctx context.Context, tagOrDigest strin return manblob, simplifyContentType(res.Header.Get("Content-Type")), nil } +func (s *dockerImageSource) GetDeltaManifest(ctx context.Context, instanceDigest *digest.Digest) ([]byte, string, error) { + // Get the real manifest digest + srcManifestDigest, err := s.manifestDigest(ctx, instanceDigest) + if err != nil { + return nil, "", err + } + + // Load the delta manifest index + ib, _, err := s.fetchManifest(ctx, "_deltaindex") + // Don't return error if the manifest doesn't exist, only for internal errors + // Deltas are an optional optimization anyway + if err == nil { + index, err := manifest.OCI1IndexFromManifest(ib) + if err != nil { + return nil, "", err + } + + // Look up the delta manifest in the index by the real manifest digest + for _, manifest := range index.Manifests { + if manifest.Annotations["io.github.containers.delta.target"] == srcManifestDigest.String() { + return s.fetchManifest(ctx, manifest.Digest.String()) + } + } + } + + // No delta + return nil, "", nil +} + +func (s *dockerImageSource) GetDeltaIndex(ctx context.Context) (types.ImageReference, error) { + deltaRef, err := reference.WithTag(s.logicalRef.ref, "_deltaindex") + if err != nil { + return nil, err + } + + return newReference(deltaRef) +} + // ensureManifestIsLoaded sets s.cachedManifest and s.cachedManifestMIMEType // // ImageSource implementations are not required or expected to do any caching, diff --git a/go.mod b/go.mod index 3bbfc9af8c..16b3eba454 100644 --- a/go.mod +++ b/go.mod @@ -9,6 +9,7 @@ require ( github.com/containers/libtrust v0.0.0-20190913040956-14b96171aa3b github.com/containers/ocicrypt v1.0.2 github.com/containers/storage v1.20.1 + github.com/containers/tar-diff v0.1.2 github.com/docker/distribution v2.7.1+incompatible github.com/docker/docker v1.4.2-0.20191219165747-a9416c67da9f github.com/docker/docker-credential-helpers v0.6.3 diff --git a/go.sum b/go.sum index a4e2067ded..272b2f26f2 100644 --- a/go.sum +++ b/go.sum @@ -53,6 +53,8 @@ github.com/containers/storage v1.19.2 h1:vhcUwEjDZiPJxaLPFsjvyavnEjFw6qQi9HAkVz1 github.com/containers/storage v1.19.2/go.mod h1:gYCp3jzgXkvubO0rI14QAjz5Mxm/qKJgLmHFyqayDnw= github.com/containers/storage v1.20.1 h1:2XE4eRIqSa6YjhAZjNwIkIKE6+Miy+5WV8l1KzY2ZKk= github.com/containers/storage v1.20.1/go.mod h1:RoKzO8KSDogCT6c06rEbanZTcKYxshorB33JikEGc3A= +github.com/containers/tar-diff v0.1.2 h1:6E04zGCdCsSJ8SoApFLiYkAxqkFllcKOaOATBUGydL4= +github.com/containers/tar-diff v0.1.2/go.mod h1:9/tnBUlqmoW1bz83CQAW9wC+EQCH+h1Wn8uq3VxLbMc= github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= diff --git a/image/unparsed.go b/image/unparsed.go index 4e3028d855..64b52a867b 100644 --- a/image/unparsed.go +++ b/image/unparsed.go @@ -66,6 +66,20 @@ func (i *UnparsedImage) Manifest(ctx context.Context) ([]byte, string, error) { return i.cachedManifest, i.cachedManifestMIMEType, nil } +func (i *UnparsedImage) DeltaLayers(ctx context.Context) ([]types.BlobInfo, error) { + // Note that GetDeltaManifest can return nil with a nil error. This is ok if no deltas exist + mb, mt, err := types.ImageSourceGetDeltaManifest(i.src, ctx, i.instanceDigest) + if mb == nil { + return nil, err + } + + m, err := manifestInstanceFromBlob(ctx, nil, i.src, mb, mt) + if err != nil { + return nil, err + } + return m.LayerInfos(), nil +} + // expectedManifestDigest returns a the expected value of the manifest digest, and an indicator whether it is known. // The bool return value seems redundant with digest != ""; it is used explicitly // to refuse (unexpected) situations when the digest exists but is "". diff --git a/manifest/oci.go b/manifest/oci.go index 2926145936..1f7ae3bb22 100644 --- a/manifest/oci.go +++ b/manifest/oci.go @@ -31,6 +31,11 @@ type OCI1 struct { imgspecv1.Manifest } +const ( + // MediaTypeDescriptor specifies the media type for a content descriptor. + MediaTypeTarDiff = "application/vnd.tar-diff" +) + // SupportedOCI1MediaType checks if the specified string is a supported OCI1 // media type. // @@ -41,7 +46,7 @@ type OCI1 struct { // useful for validation anyway. func SupportedOCI1MediaType(m string) error { switch m { - case imgspecv1.MediaTypeDescriptor, imgspecv1.MediaTypeImageConfig, imgspecv1.MediaTypeImageLayer, imgspecv1.MediaTypeImageLayerGzip, imgspecv1.MediaTypeImageLayerNonDistributable, imgspecv1.MediaTypeImageLayerNonDistributableGzip, imgspecv1.MediaTypeImageLayerNonDistributableZstd, imgspecv1.MediaTypeImageLayerZstd, imgspecv1.MediaTypeImageManifest, imgspecv1.MediaTypeLayoutHeader, ociencspec.MediaTypeLayerEnc, ociencspec.MediaTypeLayerGzipEnc: + case imgspecv1.MediaTypeDescriptor, imgspecv1.MediaTypeImageConfig, imgspecv1.MediaTypeImageLayer, imgspecv1.MediaTypeImageLayerGzip, imgspecv1.MediaTypeImageLayerNonDistributable, imgspecv1.MediaTypeImageLayerNonDistributableGzip, imgspecv1.MediaTypeImageLayerNonDistributableZstd, imgspecv1.MediaTypeImageLayerZstd, imgspecv1.MediaTypeImageManifest, imgspecv1.MediaTypeLayoutHeader, ociencspec.MediaTypeLayerEnc, ociencspec.MediaTypeLayerGzipEnc, MediaTypeTarDiff: return nil default: return fmt.Errorf("unsupported OCIv1 media type: %q", m) @@ -211,3 +216,7 @@ func getDecryptedMediaType(mediatype string) (string, error) { return strings.TrimSuffix(mediatype, "+encrypted"), nil } + +func IsNoCompressType(mediatype string) bool { + return mediatype == MediaTypeTarDiff +} diff --git a/openshift/openshift.go b/openshift/openshift.go index 28bfc456d5..45dceb161a 100644 --- a/openshift/openshift.go +++ b/openshift/openshift.go @@ -211,6 +211,17 @@ func (s *openshiftImageSource) GetManifest(ctx context.Context, instanceDigest * return s.docker.GetManifest(ctx, instanceDigest) } +func (s *openshiftImageSource) GetDeltaManifest(ctx context.Context, instanceDigest *digest.Digest) ([]byte, string, error) { + if err := s.ensureImageIsResolved(ctx); err != nil { + return nil, "", err + } + return types.ImageSourceGetDeltaManifest(s.docker, ctx, instanceDigest) +} + +func (s *openshiftImageSource) GetDeltaIndex(ctx context.Context) (types.ImageReference, error) { + return types.ImageSourceGetDeltaIndex(s.docker, ctx) +} + // HasThreadSafeGetBlob indicates whether GetBlob can be executed concurrently. func (s *openshiftImageSource) HasThreadSafeGetBlob() bool { return false @@ -511,6 +522,10 @@ func (d *openshiftImageDestination) Commit(ctx context.Context, unparsedToplevel return d.docker.Commit(ctx, unparsedToplevel) } +func (d *openshiftImageDestination) GetLayerDeltaData(ctx context.Context, diffID digest.Digest) (types.DeltaDataSource, error) { + return types.ImageDestinationGetLayerDeltaData(d.docker, ctx, diffID) +} + // These structs are subsets of github.com/openshift/origin/pkg/image/api/v1 and its dependencies. type imageStream struct { Status imageStreamStatus `json:"status,omitempty"` diff --git a/storage/storage_image.go b/storage/storage_image.go index df4b67c7a7..be52501fc0 100644 --- a/storage/storage_image.go +++ b/storage/storage_image.go @@ -24,6 +24,7 @@ import ( "github.com/containers/storage" "github.com/containers/storage/pkg/archive" "github.com/containers/storage/pkg/ioutils" + "github.com/containers/tar-diff/pkg/tar-patch" digest "github.com/opencontainers/go-digest" imgspecv1 "github.com/opencontainers/image-spec/specs-go/v1" "github.com/pkg/errors" @@ -900,6 +901,54 @@ func (s *storageImageDestination) Commit(ctx context.Context, unparsedToplevel t return nil } +type LayerDeltaDataSource struct { + fs *tar_patch.FilesystemDataSource + store storage.Store + id string +} + +func (s *LayerDeltaDataSource) Close() error { + err := s.fs.Close() + if _, err := s.store.Unmount(s.id, false); err != nil { + logrus.Infof("Failed to unmount layer %v: %v", s.id, err) + } + return err +} + +func (s *LayerDeltaDataSource) Read(data []byte) (n int, err error) { + return s.fs.Read(data) +} + +func (s *LayerDeltaDataSource) SetCurrentFile(file string) error { + return s.fs.SetCurrentFile(file) +} + +func (s *LayerDeltaDataSource) Seek(offset int64, whence int) (int64, error) { + return s.fs.Seek(offset, whence) +} + +func (s *storageImageDestination) GetLayerDeltaData(ctx context.Context, diffID digest.Digest) (types.DeltaDataSource, error) { + layers, err := s.imageRef.transport.store.LayersByUncompressedDigest(diffID) + if err != nil && err != storage.ErrLayerUnknown { + return nil, err // Internal error + } + if len(layers) == 0 { + return nil, nil // Unknown layer + } + + layerId := layers[len(layers)-1].ID + mountPoint, err := s.imageRef.transport.store.Mount(layerId, "") + if err != nil { + return nil, err + } + + return &LayerDeltaDataSource{ + fs: tar_patch.NewFilesystemDataSource(mountPoint), + store: s.imageRef.transport.store, + id: layerId, + }, nil +} + var manifestMIMETypes = []string{ imgspecv1.MediaTypeImageManifest, manifest.DockerV2Schema2MediaType, diff --git a/types/types.go b/types/types.go index d469e03b53..dc96c2a362 100644 --- a/types/types.go +++ b/types/types.go @@ -258,6 +258,36 @@ type ImageSource interface { LayerInfosForCopy(ctx context.Context, instanceDigest *digest.Digest) ([]BlobInfo, error) } +// This is an optional extension of ImageSource +type DeltaImageSource interface { + ImageSource + // GetDeltaManifest returns the delta manifest for the current image, as well as its type, if it exist. + // No error is returned if no delta manifest exists, just a nil slice + // It may use a remote (= slow) service. + // If instanceDigest is not nil, it contains a digest of the specific manifest instance to retrieve deltas for (when the primary manifest is a manifest list); + // this never happens if the primary manifest is not a manifest list (e.g. if the source never returns manifest lists). + GetDeltaManifest(ctx context.Context, instanceDigest *digest.Digest) ([]byte, string, error) + // GetDeltaIndex returns an ImageReference that can be used to update the delta index for deltas for Image. + // If deltas are not supported it will return nil + GetDeltaIndex(ctx context.Context) (ImageReference, error) +} + +// These are wrappers for ImageSource extensions that do the dynamic type cast required to work if the type is supported +// and the default implementation otherwise +func ImageSourceGetDeltaManifest(src ImageSource, ctx context.Context, instanceDigest *digest.Digest) ([]byte, string, error) { + if d, ok := src.(DeltaImageSource); ok { + return d.GetDeltaManifest(ctx, instanceDigest) + } + return nil, "", nil +} + +func ImageSourceGetDeltaIndex(src ImageSource, ctx context.Context) (ImageReference, error) { + if d, ok := src.(DeltaImageSource); ok { + return d.GetDeltaIndex(ctx) + } + return nil, nil +} + // ImageDestination is a service, possibly remote (= slow), to store components of a single image. // // There is a specific required order for some of the calls: @@ -331,6 +361,23 @@ type ImageDestination interface { Commit(ctx context.Context, unparsedToplevel UnparsedImage) error } +// This is an optional extension of ImageSource +type DeltaImageDestination interface { + ImageDestination + // Tries to get access to the uncompressed data of a given DiffID that is locally available + // This data is used to apply a delta from this layer + // If deltas are not supported or the layer is not available, nil is returned (and no error) + GetLayerDeltaData(ctx context.Context, diffID digest.Digest) (DeltaDataSource, error) +} + +func ImageDestinationGetLayerDeltaData(dst ImageDestination, ctx context.Context, diffID digest.Digest) (DeltaDataSource, error) { + if d, ok := dst.(DeltaImageDestination); ok { + return d.GetLayerDeltaData(ctx, diffID) + } + return nil, nil + +} + // ManifestTypeRejectedError is returned by ImageDestination.PutManifest if the destination is in principle available, // refuses specifically this manifest type, but may accept a different manifest type. type ManifestTypeRejectedError struct { // We only use a struct to allow a type assertion, without limiting the contents of the error otherwise. @@ -409,6 +456,21 @@ type Image interface { Size() (int64, error) } +// This is an optional extension of Image +type DeltaImage interface { + Image + // Downloads and parses the delta manifest for the image, returning the available delta layers + // If no deltas available, returns nil without an error + DeltaLayers(ctx context.Context) ([]BlobInfo, error) +} + +func ImageDeltaLayers(img Image, ctx context.Context) ([]BlobInfo, error) { + if d, ok := img.(DeltaImage); ok { + return d.DeltaLayers(ctx) + } + return nil, nil +} + // ImageCloser is an Image with a Close() method which must be called by the user. // This is returned by ImageReference.NewImage, which transparently instantiates a types.ImageSource, // to ensure that the ImageSource is closed. @@ -623,3 +685,13 @@ type ProgressProperties struct { // interval. Will be reset after each ProgressEventRead event. OffsetUpdate uint64 } + +// This is an interface that allows you to access existing local data to +// use as existing content when applying a delta file +// It is identical to the DataSource interface in tar_diff but re-exported +// here to avoid unncecceary dependencies +type DeltaDataSource interface { + io.ReadSeeker + io.Closer + SetCurrentFile(file string) error +}