add Image content converter

Go example:
```go
opts := []converter.Opt{
  // convert Docker media types to OCI ones
  converter.WithDocker2OCI(true),
  // convert tar.gz layers to uncompressed tar layers
  converter.WithLayerConvertFunc(uncompress.LayerConvertFunc),
}
srcRef := "example.com/foo:orig"
dstRef := "example.com/foo:converted"
dstImg, err = converter.Convert(ctx, client, dstRef, srcRef, opts...)
fmt.Println(dstImg.Target)
```

ctr example: `ctr images convert --oci --uncompress example.com/foo:orig example.com/foo:converted`

Go test: `go test -exec sudo -test.root -test.run TestConvert`

The implementation is from https://github.com/containerd/stargz-snapshotter/pull/224,
but eStargz-specific functions are not included in this PR.

eStargz converter can be specified by importing `estargz` package and using `WithLayerConvertFunc(estargz.LayerConvertFunc)` option.

This converter interface will be potentially useful for converting zstd and ocicrypt layers as well.

Signed-off-by: Akihiro Suda <akihiro.suda.cz@hco.ntt.co.jp>
This commit is contained in:
Akihiro Suda 2020-12-24 17:21:31 +09:00
parent 9b9de47eb9
commit 5ca3ac65c4
No known key found for this signature in database
GPG Key ID: 49524C6F9F638F1A
9 changed files with 1024 additions and 1 deletions

View File

@ -0,0 +1,108 @@
/*
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 images
import (
"fmt"
"github.com/containerd/containerd/cmd/ctr/commands"
"github.com/containerd/containerd/images/converter"
"github.com/containerd/containerd/images/converter/uncompress"
"github.com/containerd/containerd/platforms"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/pkg/errors"
"github.com/urfave/cli"
)
var convertCommand = cli.Command{
Name: "convert",
Usage: "convert an image",
ArgsUsage: "[flags] <source_ref> <target_ref>",
Description: `Convert an image format.
e.g., 'ctr convert --uncompress --oci example.com/foo:orig example.com/foo:converted'
Use '--platform' to define the output platform.
When '--all-platforms' is given all images in a manifest list must be available.
`,
Flags: []cli.Flag{
// generic flags
cli.BoolFlag{
Name: "uncompress",
Usage: "convert tar.gz layers to uncompressed tar layers",
},
cli.BoolFlag{
Name: "oci",
Usage: "convert Docker media types to OCI media types",
},
// platform flags
cli.StringSliceFlag{
Name: "platform",
Usage: "Pull content from a specific platform",
Value: &cli.StringSlice{},
},
cli.BoolFlag{
Name: "all-platforms",
Usage: "exports content from all platforms",
},
},
Action: func(context *cli.Context) error {
var convertOpts []converter.Opt
srcRef := context.Args().Get(0)
targetRef := context.Args().Get(1)
if srcRef == "" || targetRef == "" {
return errors.New("src and target image need to be specified")
}
if !context.Bool("all-platforms") {
if pss := context.StringSlice("platform"); len(pss) > 0 {
var all []ocispec.Platform
for _, ps := range pss {
p, err := platforms.Parse(ps)
if err != nil {
return errors.Wrapf(err, "invalid platform %q", ps)
}
all = append(all, p)
}
convertOpts = append(convertOpts, converter.WithPlatform(platforms.Ordered(all...)))
} else {
convertOpts = append(convertOpts, converter.WithPlatform(platforms.DefaultStrict()))
}
}
if context.Bool("uncompress") {
convertOpts = append(convertOpts, converter.WithLayerConvertFunc(uncompress.LayerConvertFunc))
}
if context.Bool("oci") {
convertOpts = append(convertOpts, converter.WithDockerToOCI(true))
}
client, ctx, cancel, err := commands.NewClient(context)
if err != nil {
return err
}
defer cancel()
newImg, err := converter.Convert(ctx, client, targetRef, srcRef, convertOpts...)
if err != nil {
return err
}
fmt.Fprintln(context.App.Writer, newImg.Target.Digest.String())
return nil
},
}

View File

@ -50,6 +50,7 @@ var Command = cli.Command{
removeCommand, removeCommand,
tagCommand, tagCommand,
setLabelsCommand, setLabelsCommand,
convertCommand,
}, },
} }

88
convert_test.go Normal file
View File

@ -0,0 +1,88 @@
/*
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 containerd
import (
"testing"
"github.com/containerd/containerd/images"
"github.com/containerd/containerd/images/converter"
"github.com/containerd/containerd/images/converter/uncompress"
"github.com/containerd/containerd/platforms"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"gotest.tools/v3/assert"
)
// TestConvert creates an image from testImage, with the following conversion:
// - Media type: Docker -> OCI
// - Layer type: tar.gz -> tar
// - Arch: Multi -> Single
func TestConvert(t *testing.T) {
if testing.Short() {
t.Skip()
}
ctx, cancel := testContext(t)
defer cancel()
client, err := New(address)
if err != nil {
t.Fatal(err)
}
defer client.Close()
_, err = client.Fetch(ctx, testImage)
if err != nil {
t.Fatal(err)
}
dstRef := testImage + "-testconvert"
defPlat := platforms.DefaultStrict()
opts := []converter.Opt{
converter.WithDockerToOCI(true),
converter.WithLayerConvertFunc(uncompress.LayerConvertFunc),
converter.WithPlatform(defPlat),
}
dstImg, err := converter.Convert(ctx, client, dstRef, testImage, opts...)
if err != nil {
t.Fatal(err)
}
defer func() {
if deleteErr := client.ImageService().Delete(ctx, dstRef); deleteErr != nil {
t.Fatal(deleteErr)
}
}()
cs := client.ContentStore()
plats, err := images.Platforms(ctx, cs, dstImg.Target)
if err != nil {
t.Fatal(err)
}
// Assert that the image does not have any extra arch.
assert.Equal(t, 1, len(plats))
assert.Check(t, defPlat.Match(plats[0]))
// Assert that the media type is converted to OCI and also uncompressed
mani, err := images.Manifest(ctx, cs, dstImg.Target, defPlat)
if err != nil {
t.Fatal(err)
}
for _, l := range mani.Layers {
if plats[0].OS == "windows" {
assert.Equal(t, ocispec.MediaTypeImageLayerNonDistributable, l.MediaType)
} else {
assert.Equal(t, ocispec.MediaTypeImageLayer, l.MediaType)
}
}
}

View File

@ -0,0 +1,126 @@
/*
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 converter provides image converter
package converter
import (
"context"
"github.com/containerd/containerd/content"
"github.com/containerd/containerd/images"
"github.com/containerd/containerd/leases"
"github.com/containerd/containerd/platforms"
)
type convertOpts struct {
layerConvertFunc ConvertFunc
docker2oci bool
indexConvertFunc ConvertFunc
platformMC platforms.MatchComparer
}
// Opt is an option for Convert()
type Opt func(*convertOpts) error
// WithLayerConvertFunc specifies the function that converts layers.
func WithLayerConvertFunc(fn ConvertFunc) Opt {
return func(copts *convertOpts) error {
copts.layerConvertFunc = fn
return nil
}
}
// WithDockerToOCI converts Docker media types into OCI ones.
func WithDockerToOCI(v bool) Opt {
return func(copts *convertOpts) error {
copts.docker2oci = true
return nil
}
}
// WithPlatform specifies the platform.
// Defaults to all platforms.
func WithPlatform(p platforms.MatchComparer) Opt {
return func(copts *convertOpts) error {
copts.platformMC = p
return nil
}
}
// WithIndexConvertFunc specifies the function that converts manifests and index (manifest lists).
// Defaults to DefaultIndexConvertFunc.
func WithIndexConvertFunc(fn ConvertFunc) Opt {
return func(copts *convertOpts) error {
copts.indexConvertFunc = fn
return nil
}
}
// Client is implemented by *containerd.Client .
type Client interface {
WithLease(ctx context.Context, opts ...leases.Opt) (context.Context, func(context.Context) error, error)
ContentStore() content.Store
ImageService() images.Store
}
// Convert converts an image.
func Convert(ctx context.Context, client Client, dstRef, srcRef string, opts ...Opt) (*images.Image, error) {
var copts convertOpts
for _, o := range opts {
if err := o(&copts); err != nil {
return nil, err
}
}
if copts.platformMC == nil {
copts.platformMC = platforms.All
}
if copts.indexConvertFunc == nil {
copts.indexConvertFunc = DefaultIndexConvertFunc(copts.layerConvertFunc, copts.docker2oci, copts.platformMC)
}
ctx, done, err := client.WithLease(ctx)
if err != nil {
return nil, err
}
defer done(ctx)
cs := client.ContentStore()
is := client.ImageService()
srcImg, err := is.Get(ctx, srcRef)
if err != nil {
return nil, err
}
dstDesc, err := copts.indexConvertFunc(ctx, cs, srcImg.Target)
if err != nil {
return nil, err
}
dstImg := srcImg
dstImg.Name = dstRef
if dstDesc != nil {
dstImg.Target = *dstDesc
}
var res images.Image
if dstRef != srcRef {
_ = is.Delete(ctx, dstRef)
res, err = is.Create(ctx, dstImg)
} else {
res, err = is.Update(ctx, dstImg)
}
return &res, err
}

441
images/converter/default.go Normal file
View File

@ -0,0 +1,441 @@
/*
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 converter
import (
"bytes"
"context"
"encoding/json"
"fmt"
"strings"
"sync"
"github.com/containerd/containerd/content"
"github.com/containerd/containerd/images"
"github.com/containerd/containerd/platforms"
"github.com/opencontainers/go-digest"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/sirupsen/logrus"
"golang.org/x/sync/errgroup"
)
// ConvertFunc returns a converted content descriptor.
// When the content was not converted, ConvertFunc returns nil.
type ConvertFunc func(ctx context.Context, cs content.Store, desc ocispec.Descriptor) (*ocispec.Descriptor, error)
// DefaultIndexConvertFunc is the default convert func used by Convert.
func DefaultIndexConvertFunc(layerConvertFunc ConvertFunc, docker2oci bool, platformMC platforms.MatchComparer) ConvertFunc {
c := &defaultConverter{
layerConvertFunc: layerConvertFunc,
docker2oci: docker2oci,
platformMC: platformMC,
diffIDMap: make(map[digest.Digest]digest.Digest),
}
return c.convert
}
type defaultConverter struct {
layerConvertFunc ConvertFunc
docker2oci bool
platformMC platforms.MatchComparer
diffIDMap map[digest.Digest]digest.Digest // key: old diffID, value: new diffID
diffIDMapMu sync.RWMutex
}
// convert dispatches desc.MediaType and calls c.convert{Layer,Manifest,Index,Config}.
//
// Also converts media type if c.docker2oci is set.
func (c *defaultConverter) convert(ctx context.Context, cs content.Store, desc ocispec.Descriptor) (*ocispec.Descriptor, error) {
var (
newDesc *ocispec.Descriptor
err error
)
if images.IsLayerType(desc.MediaType) {
newDesc, err = c.convertLayer(ctx, cs, desc)
} else if images.IsManifestType(desc.MediaType) {
newDesc, err = c.convertManifest(ctx, cs, desc)
} else if images.IsIndexType(desc.MediaType) {
newDesc, err = c.convertIndex(ctx, cs, desc)
} else if images.IsConfigType(desc.MediaType) {
newDesc, err = c.convertConfig(ctx, cs, desc)
}
if err != nil {
return nil, err
}
if images.IsDockerType(desc.MediaType) {
if c.docker2oci {
if newDesc == nil {
newDesc = copyDesc(desc)
}
newDesc.MediaType = ConvertDockerMediaTypeToOCI(newDesc.MediaType)
} else if (newDesc == nil && len(desc.Annotations) != 0) || (newDesc != nil && len(newDesc.Annotations) != 0) {
// Annotations is supported only on OCI manifest.
// We need to remove annotations for Docker media types.
if newDesc == nil {
newDesc = copyDesc(desc)
}
newDesc.Annotations = nil
}
}
logrus.WithField("old", desc).WithField("new", newDesc).Debugf("converted")
return newDesc, nil
}
func copyDesc(desc ocispec.Descriptor) *ocispec.Descriptor {
descCopy := desc
return &descCopy
}
// convertLayer converts image image layers if c.layerConvertFunc is set.
//
// c.layerConvertFunc can be nil, e.g., for converting Docker media types to OCI ones.
func (c *defaultConverter) convertLayer(ctx context.Context, cs content.Store, desc ocispec.Descriptor) (*ocispec.Descriptor, error) {
if c.layerConvertFunc != nil {
return c.layerConvertFunc(ctx, cs, desc)
}
return nil, nil
}
// convertManifest converts image manifests.
//
// - clears `.mediaType` if the target format is OCI
//
// - records diff ID changes in c.diffIDMap
func (c *defaultConverter) convertManifest(ctx context.Context, cs content.Store, desc ocispec.Descriptor) (*ocispec.Descriptor, error) {
var (
manifest DualManifest
modified bool
)
labels, err := readJSON(ctx, cs, &manifest, desc)
if err != nil {
return nil, err
}
if labels == nil {
labels = make(map[string]string)
}
if images.IsDockerType(manifest.MediaType) && c.docker2oci {
manifest.MediaType = ""
modified = true
}
var mu sync.Mutex
eg, ctx2 := errgroup.WithContext(ctx)
for i, l := range manifest.Layers {
i := i
l := l
oldDiffID, err := images.GetDiffID(ctx, cs, l)
if err != nil {
return nil, err
}
eg.Go(func() error {
newL, err := c.convert(ctx2, cs, l)
if err != nil {
return err
}
if newL != nil {
mu.Lock()
// update GC labels
ClearGCLabels(labels, l.Digest)
labelKey := fmt.Sprintf("containerd.io/gc.ref.content.l.%d", i)
labels[labelKey] = newL.Digest.String()
manifest.Layers[i] = *newL
modified = true
mu.Unlock()
// diffID changes if the tar entries were modified.
// diffID stays same if only the compression type was changed.
// When diffID changed, add a map entry so that we can update image config.
newDiffID, err := images.GetDiffID(ctx, cs, *newL)
if err != nil {
return err
}
if newDiffID != oldDiffID {
c.diffIDMapMu.Lock()
c.diffIDMap[oldDiffID] = newDiffID
c.diffIDMapMu.Unlock()
}
}
return nil
})
}
if err := eg.Wait(); err != nil {
return nil, err
}
newConfig, err := c.convert(ctx, cs, manifest.Config)
if err != nil {
return nil, err
}
if newConfig != nil {
ClearGCLabels(labels, manifest.Config.Digest)
labels["containerd.io/gc.ref.content.config"] = newConfig.Digest.String()
manifest.Config = *newConfig
modified = true
}
if modified {
return writeJSON(ctx, cs, &manifest, desc, labels)
}
return nil, nil
}
// convertIndex converts image index.
//
// - clears `.mediaType` if the target format is OCI
//
// - clears manifest entries that do not match c.platformMC
func (c *defaultConverter) convertIndex(ctx context.Context, cs content.Store, desc ocispec.Descriptor) (*ocispec.Descriptor, error) {
var (
index DualIndex
modified bool
)
labels, err := readJSON(ctx, cs, &index, desc)
if err != nil {
return nil, err
}
if labels == nil {
labels = make(map[string]string)
}
if images.IsDockerType(index.MediaType) && c.docker2oci {
index.MediaType = ""
modified = true
}
newManifests := make([]ocispec.Descriptor, len(index.Manifests))
newManifestsToBeRemoved := make(map[int]struct{}) // slice index
var mu sync.Mutex
eg, ctx2 := errgroup.WithContext(ctx)
for i, mani := range index.Manifests {
i := i
mani := mani
labelKey := fmt.Sprintf("containerd.io/gc.ref.content.m.%d", i)
eg.Go(func() error {
if mani.Platform != nil && !c.platformMC.Match(*mani.Platform) {
mu.Lock()
ClearGCLabels(labels, mani.Digest)
newManifestsToBeRemoved[i] = struct{}{}
modified = true
mu.Unlock()
return nil
}
newMani, err := c.convert(ctx2, cs, mani)
if err != nil {
return err
}
mu.Lock()
if newMani != nil {
ClearGCLabels(labels, mani.Digest)
labels[labelKey] = newMani.Digest.String()
// NOTE: for keeping manifest order, we specify `i` index explicitly
newManifests[i] = *newMani
modified = true
} else {
newManifests[i] = mani
}
mu.Unlock()
return nil
})
}
if err := eg.Wait(); err != nil {
return nil, err
}
if modified {
var newManifestsClean []ocispec.Descriptor
for i, m := range newManifests {
if _, ok := newManifestsToBeRemoved[i]; !ok {
newManifestsClean = append(newManifestsClean, m)
}
}
index.Manifests = newManifestsClean
return writeJSON(ctx, cs, &index, desc, labels)
}
return nil, nil
}
// convertConfig converts image config contents.
//
// - updates `.rootfs.diff_ids` using c.diffIDMap .
//
// - clears legacy `.config.Image` and `.container_config.Image` fields if `.rootfs.diff_ids` was updated.
func (c *defaultConverter) convertConfig(ctx context.Context, cs content.Store, desc ocispec.Descriptor) (*ocispec.Descriptor, error) {
var (
cfg DualConfig
cfgAsOCI ocispec.Image // read only, used for parsing cfg
modified bool
)
labels, err := readJSON(ctx, cs, &cfg, desc)
if err != nil {
return nil, err
}
if labels == nil {
labels = make(map[string]string)
}
if _, err := readJSON(ctx, cs, &cfgAsOCI, desc); err != nil {
return nil, err
}
if rootfs := cfgAsOCI.RootFS; rootfs.Type == "layers" {
rootfsModified := false
c.diffIDMapMu.RLock()
for i, oldDiffID := range rootfs.DiffIDs {
if newDiffID, ok := c.diffIDMap[oldDiffID]; ok && newDiffID != oldDiffID {
rootfs.DiffIDs[i] = newDiffID
rootfsModified = true
}
}
c.diffIDMapMu.RUnlock()
if rootfsModified {
rootfsB, err := json.Marshal(rootfs)
if err != nil {
return nil, err
}
cfg["rootfs"] = (*json.RawMessage)(&rootfsB)
modified = true
}
}
if modified {
// cfg may have dummy value for legacy `.config.Image` and `.container_config.Image`
// We should clear the ID if we changed the diff IDs.
if _, err := clearDockerV1DummyID(cfg); err != nil {
return nil, err
}
return writeJSON(ctx, cs, &cfg, desc, labels)
}
return nil, nil
}
// clearDockerV1DummyID clears the dummy values for legacy `.config.Image` and `.container_config.Image`.
// Returns true if the cfg was modified.
func clearDockerV1DummyID(cfg DualConfig) (bool, error) {
var modified bool
f := func(k string) error {
if configX, ok := cfg[k]; ok && configX != nil {
var configField map[string]*json.RawMessage
if err := json.Unmarshal(*configX, &configField); err != nil {
return err
}
delete(configField, "Image")
b, err := json.Marshal(configField)
if err != nil {
return err
}
cfg[k] = (*json.RawMessage)(&b)
modified = true
}
return nil
}
if err := f("config"); err != nil {
return modified, err
}
if err := f("container_config"); err != nil {
return modified, err
}
return modified, nil
}
type ObjectWithMediaType struct {
// MediaType appears on Docker manifests and manifest lists.
// MediaType does not appear on OCI manifests and index
MediaType string `json:"mediaType,omitempty"`
}
// DualManifest covers Docker manifest and OCI manifest
type DualManifest struct {
ocispec.Manifest
ObjectWithMediaType
}
// DualIndex covers Docker manifest list and OCI index
type DualIndex struct {
ocispec.Index
ObjectWithMediaType
}
// DualConfig covers Docker config (v1.0, v1.1, v1.2) and OCI config.
// Unmarshalled as map[string]*json.RawMessage to retain unknown fields on remarshalling.
type DualConfig map[string]*json.RawMessage
func readJSON(ctx context.Context, cs content.Store, x interface{}, desc ocispec.Descriptor) (map[string]string, error) {
info, err := cs.Info(ctx, desc.Digest)
if err != nil {
return nil, err
}
labels := info.Labels
b, err := content.ReadBlob(ctx, cs, desc)
if err != nil {
return nil, err
}
if err := json.Unmarshal(b, x); err != nil {
return nil, err
}
return labels, nil
}
func writeJSON(ctx context.Context, cs content.Store, x interface{}, oldDesc ocispec.Descriptor, labels map[string]string) (*ocispec.Descriptor, error) {
b, err := json.Marshal(x)
if err != nil {
return nil, err
}
dgst := digest.SHA256.FromBytes(b)
ref := fmt.Sprintf("converter-write-json-%s", dgst.String())
w, err := content.OpenWriter(ctx, cs, content.WithRef(ref))
if err != nil {
return nil, err
}
if err := content.Copy(ctx, w, bytes.NewReader(b), int64(len(b)), dgst, content.WithLabels(labels)); err != nil {
return nil, err
}
if err := w.Close(); err != nil {
return nil, err
}
newDesc := oldDesc
newDesc.Size = int64(len(b))
newDesc.Digest = dgst
return &newDesc, nil
}
// ConvertDockerMediaTypeToOCI converts a media type string
func ConvertDockerMediaTypeToOCI(mt string) string {
switch mt {
case images.MediaTypeDockerSchema2ManifestList:
return ocispec.MediaTypeImageIndex
case images.MediaTypeDockerSchema2Manifest:
return ocispec.MediaTypeImageManifest
case images.MediaTypeDockerSchema2LayerGzip:
return ocispec.MediaTypeImageLayerGzip
case images.MediaTypeDockerSchema2LayerForeignGzip:
return ocispec.MediaTypeImageLayerNonDistributableGzip
case images.MediaTypeDockerSchema2Layer:
return ocispec.MediaTypeImageLayer
case images.MediaTypeDockerSchema2LayerForeign:
return ocispec.MediaTypeImageLayerNonDistributable
case images.MediaTypeDockerSchema2Config:
return ocispec.MediaTypeImageConfig
default:
return mt
}
}
// ClearGCLabels clears GC labels for the given digest.
func ClearGCLabels(labels map[string]string, dgst digest.Digest) {
for k, v := range labels {
if v == dgst.String() && strings.HasPrefix(k, "containerd.io/gc.ref.content") {
delete(labels, k)
}
}
}

View File

@ -0,0 +1,120 @@
/*
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 uncompress
import (
"compress/gzip"
"context"
"fmt"
"io"
"github.com/containerd/containerd/content"
"github.com/containerd/containerd/errdefs"
"github.com/containerd/containerd/images"
"github.com/containerd/containerd/images/converter"
"github.com/containerd/containerd/labels"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
)
var _ converter.ConvertFunc = LayerConvertFunc
// LayerConvertFunc converts tar.gz layers into uncompressed tar layers.
// Media type is changed, e.g., "application/vnd.oci.image.layer.v1.tar+gzip" -> "application/vnd.oci.image.layer.v1.tar"
func LayerConvertFunc(ctx context.Context, cs content.Store, desc ocispec.Descriptor) (*ocispec.Descriptor, error) {
if !images.IsLayerType(desc.MediaType) || IsUncompressedType(desc.MediaType) {
// No conversion. No need to return an error here.
return nil, nil
}
info, err := cs.Info(ctx, desc.Digest)
if err != nil {
return nil, err
}
readerAt, err := cs.ReaderAt(ctx, desc)
if err != nil {
return nil, err
}
defer readerAt.Close()
sr := io.NewSectionReader(readerAt, 0, desc.Size)
newR, err := gzip.NewReader(sr)
if err != nil {
return nil, err
}
defer newR.Close()
ref := fmt.Sprintf("convert-uncompress-from-%s", desc.Digest)
w, err := cs.Writer(ctx, content.WithRef(ref))
if err != nil {
return nil, err
}
defer w.Close()
// Reset the writing position
// Old writer possibly remains without aborted
// (e.g. conversion interrupted by a signal)
if err := w.Truncate(0); err != nil {
return nil, err
}
n, err := io.Copy(w, newR)
if err != nil {
return nil, err
}
if err := newR.Close(); err != nil {
return nil, err
}
// no need to retain "containerd.io/uncompressed" label, but retain other labels ("containerd.io/distribution.source.*")
labelsMap := info.Labels
delete(labelsMap, labels.LabelUncompressed)
if err = w.Commit(ctx, 0, "", content.WithLabels(labelsMap)); err != nil && !errdefs.IsAlreadyExists(err) {
return nil, err
}
if err := w.Close(); err != nil {
return nil, err
}
newDesc := desc
newDesc.Digest = w.Digest()
newDesc.Size = n
newDesc.MediaType = convertMediaType(newDesc.MediaType)
return &newDesc, nil
}
func IsUncompressedType(mt string) bool {
switch mt {
case
images.MediaTypeDockerSchema2Layer,
images.MediaTypeDockerSchema2LayerForeign,
ocispec.MediaTypeImageLayer,
ocispec.MediaTypeImageLayerNonDistributable:
return true
default:
return false
}
}
func convertMediaType(mt string) string {
switch mt {
case images.MediaTypeDockerSchema2LayerGzip:
return images.MediaTypeDockerSchema2Layer
case images.MediaTypeDockerSchema2LayerForeignGzip:
return images.MediaTypeDockerSchema2LayerForeign
case ocispec.MediaTypeImageLayerGzip:
return ocispec.MediaTypeImageLayer
case ocispec.MediaTypeImageLayerNonDistributableGzip:
return ocispec.MediaTypeImageLayerNonDistributable
default:
return mt
}
}

80
images/diffid.go Normal file
View File

@ -0,0 +1,80 @@
/*
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 images
import (
"compress/gzip"
"context"
"io"
"github.com/containerd/containerd/content"
"github.com/containerd/containerd/labels"
"github.com/opencontainers/go-digest"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/sirupsen/logrus"
)
// GetDiffID gets the diff ID of the layer blob descriptor.
func GetDiffID(ctx context.Context, cs content.Store, desc ocispec.Descriptor) (digest.Digest, error) {
switch desc.MediaType {
case
// If the layer is already uncompressed, we can just return its digest
MediaTypeDockerSchema2Layer,
ocispec.MediaTypeImageLayer,
MediaTypeDockerSchema2LayerForeign,
ocispec.MediaTypeImageLayerNonDistributable:
return desc.Digest, nil
}
info, err := cs.Info(ctx, desc.Digest)
if err != nil {
return "", err
}
v, ok := info.Labels[labels.LabelUncompressed]
if ok {
// Fast path: if the image is already unpacked, we can use the label value
return digest.Parse(v)
}
// if the image is not unpacked, we may not have the label
ra, err := cs.ReaderAt(ctx, desc)
if err != nil {
return "", err
}
defer ra.Close()
r := content.NewReader(ra)
gzR, err := gzip.NewReader(r)
if err != nil {
return "", err
}
digester := digest.Canonical.Digester()
hashW := digester.Hash()
if _, err := io.Copy(hashW, gzR); err != nil {
return "", err
}
if err := ra.Close(); err != nil {
return "", err
}
digest := digester.Digest()
// memorize the computed value
if info.Labels == nil {
info.Labels = make(map[string]string)
}
info.Labels[labels.LabelUncompressed] = digest.String()
if _, err := cs.Update(ctx, info, "labels"); err != nil {
logrus.WithError(err).Warnf("failed to set %s label for %s", labels.LabelUncompressed, desc.Digest)
}
return digest, nil
}

View File

@ -124,7 +124,45 @@ func IsLayerType(mt string) bool {
return false return false
} }
// IsKnownConfig returns true if the media type is a known config type // IsDockerType returns true if the media type has "application/vnd.docker." prefix
func IsDockerType(mt string) bool {
return strings.HasPrefix(mt, "application/vnd.docker.")
}
// IsManifestType returns true if the media type is an OCI-compatible manifest.
// No support for schema1 manifest.
func IsManifestType(mt string) bool {
switch mt {
case MediaTypeDockerSchema2Manifest, ocispec.MediaTypeImageManifest:
return true
default:
return false
}
}
// IsIndexType returns true if the media type is an OCI-compatible index.
func IsIndexType(mt string) bool {
switch mt {
case ocispec.MediaTypeImageIndex, MediaTypeDockerSchema2ManifestList:
return true
default:
return false
}
}
// IsConfigType returns true if the media type is an OCI-compatible image config.
// No support for containerd checkpoint configs.
func IsConfigType(mt string) bool {
switch mt {
case MediaTypeDockerSchema2Config, ocispec.MediaTypeImageConfig:
return true
default:
return false
}
}
// IsKnownConfig returns true if the media type is a known config type,
// including containerd checkpoint configs
func IsKnownConfig(mt string) bool { func IsKnownConfig(mt string) bool {
switch mt { switch mt {
case MediaTypeDockerSchema2Config, ocispec.MediaTypeImageConfig, case MediaTypeDockerSchema2Config, ocispec.MediaTypeImageConfig,

21
labels/labels.go Normal file
View File

@ -0,0 +1,21 @@
/*
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 labels
// LabelUncompressed is added to compressed layer contents.
// The value is digest of the uncompressed content.
const LabelUncompressed = "containerd.io/uncompressed"