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:
parent
9b9de47eb9
commit
5ca3ac65c4
108
cmd/ctr/commands/images/convert.go
Normal file
108
cmd/ctr/commands/images/convert.go
Normal 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
|
||||
},
|
||||
}
|
@ -50,6 +50,7 @@ var Command = cli.Command{
|
||||
removeCommand,
|
||||
tagCommand,
|
||||
setLabelsCommand,
|
||||
convertCommand,
|
||||
},
|
||||
}
|
||||
|
||||
|
88
convert_test.go
Normal file
88
convert_test.go
Normal 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)
|
||||
}
|
||||
}
|
||||
}
|
126
images/converter/converter.go
Normal file
126
images/converter/converter.go
Normal 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
441
images/converter/default.go
Normal 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)
|
||||
}
|
||||
}
|
||||
}
|
120
images/converter/uncompress/uncompress.go
Normal file
120
images/converter/uncompress/uncompress.go
Normal 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
80
images/diffid.go
Normal 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
|
||||
}
|
@ -124,7 +124,45 @@ func IsLayerType(mt string) bool {
|
||||
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 {
|
||||
switch mt {
|
||||
case MediaTypeDockerSchema2Config, ocispec.MediaTypeImageConfig,
|
||||
|
21
labels/labels.go
Normal file
21
labels/labels.go
Normal 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"
|
Loading…
Reference in New Issue
Block a user