0
Fork 0
mirror of https://github.com/project-zot/zot.git synced 2024-12-16 21:56:37 -05:00

feat(storage): rebuild storage(s3/local) dedupe index when switching dedupe status (#1062)

Signed-off-by: Petu Eusebiu <peusebiu@cisco.com>
This commit is contained in:
peusebiu 2023-04-07 19:49:24 +03:00 committed by GitHub
parent 96232bb11c
commit f35ff53146
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
23 changed files with 2520 additions and 203 deletions

57
.github/workflows/nightly.yaml vendored Normal file
View file

@ -0,0 +1,57 @@
name: 'Nightly jobs'
on:
schedule:
- cron: '30 1 * * *'
workflow_dispatch:
permissions: read-all
# Here we are running two tests:
# 1. run zot with local storage and dedupe disabled, push images, restart zot with dedupe enabled
# task scheduler will start a dedupe all blobs process at zot startup and it shouldn't interfere with clients.
# 2. run zot with s3 storage and dynamodb and dedupe enabled, push images, restart zot with dedupe false and no cache
# task scheduler will start a restore all blobs process at zot startup, after it finishes all blobs should be restored to their original state (have content)
jobs:
client-tools:
name: Dedupe/restore blobs
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v3
- uses: actions/setup-go@v3
with:
go-version: 1.19.x
- name: Install dependencies
run: |
cd $GITHUB_WORKSPACE
go install github.com/swaggo/swag/cmd/swag
go mod download
sudo apt-get update
sudo apt-get install libgpgme-dev libassuan-dev libbtrfs-dev libdevmapper-dev pkg-config rpm uidmap
# install skopeo
git clone -b v1.9.0 https://github.com/containers/skopeo.git
cd skopeo
make bin/skopeo
sudo cp bin/skopeo /usr/bin
skopeo -v
- name: Install localstack
run: |
pip install --upgrade pyopenssl
pip install localstack awscli-local[ver1] # install LocalStack cli and awslocal
docker pull localstack/localstack # Make sure to pull the latest version of the image
localstack start -d # Start LocalStack in the background
echo "Waiting for LocalStack startup..." # Wait 30 seconds for the LocalStack container
localstack wait -t 30 # to become ready before timing out
echo "Startup complete"
- name: Run restore s3 blobs after cache is deleted
run: |
make test-restore-s3-blobs
env:
AWS_ACCESS_KEY_ID: fake
AWS_SECRET_ACCESS_KEY: fake
- name: Run dedupe filesystem blobs after switching dedupe to enable.
run: |
make test-push-pull-running-dedupe
env:
AWS_ACCESS_KEY_ID: fake
AWS_SECRET_ACCESS_KEY: fake

View file

@ -307,6 +307,22 @@ test-push-pull: binary check-skopeo $(BATS) $(REGCLIENT) $(ORAS) $(HELM)
test-push-pull-verbose: binary check-skopeo $(BATS)
$(BATS) --trace --verbose-run --print-output-on-failure --show-output-of-passing-tests test/blackbox/pushpull.bats
.PHONY: test-push-pull-running-dedupe
test-push-pull-running-dedupe: binary check-skopeo $(BATS) $(REGCLIENT) $(ORAS) $(HELM)
$(BATS) --trace --print-output-on-failure test/blackbox/pushpull_running_dedupe.bats
.PHONY: test-push-pull-running-dedupe-verbose
test-push-pull-running-dedupe-verbose: binary check-skopeo $(BATS) $(REGCLIENT) $(ORAS) $(HELM)
$(BATS) --trace --verbose-run --print-output-on-failure --show-output-of-passing-tests test/blackbox/pushpull_running_dedupe.bats
.PHONY: test-restore-s3-blobs
test-restore-s3-blobs: binary check-skopeo $(BATS) $(REGCLIENT) $(ORAS) $(HELM)
$(BATS) --trace --print-output-on-failure test/blackbox/restore_s3_blobs.bats
.PHONY: test-restore-s3-blobs-verbose
test-restore-s3-blobs-verbose: binary check-skopeo $(BATS) $(REGCLIENT) $(ORAS) $(HELM)
$(BATS) --trace --verbose-run --print-output-on-failure --show-output-of-passing-tests test/blackbox/restore_s3_blobs.bats
.PHONY: test-bats-referrers
test-bats-referrers: EXTENSIONS=search
test-bats-referrers: binary check-skopeo $(BATS) $(ORAS)

View file

@ -81,4 +81,5 @@ var (
ErrMediaTypeNotSupported = errors.New("repodb: media type is not supported")
ErrTimeout = errors.New("operation timeout")
ErrNotImplemented = errors.New("not implemented")
ErrDedupeRebuild = errors.New("dedupe: couldn't rebuild dedupe index")
)

View file

@ -451,11 +451,16 @@ func getUseRelPaths(storageConfig *config.StorageConfig) bool {
}
func CreateCacheDatabaseDriver(storageConfig config.StorageConfig, log log.Logger) cache.Cache {
if storageConfig.Dedupe {
if storageConfig.Dedupe || storageConfig.StorageDriver != nil {
if !storageConfig.RemoteCache {
params := cache.BoltDBDriverParameters{}
params.RootDir = storageConfig.RootDirectory
params.Name = constants.BoltdbName
if storageConfig.StorageDriver != nil {
params.Name = s3.CacheDBName
}
params.UseRelPaths = getUseRelPaths(&storageConfig)
driver, _ := storage.Create("boltdb", params, log)
@ -548,6 +553,9 @@ func (c *Controller) StartBackgroundTasks(reloadCtx context.Context) {
c.StoreController.DefaultStore.RunGCPeriodically(c.Config.Storage.GCInterval, taskScheduler)
}
// Enable running dedupe blobs both ways (dedupe or restore deduped blobs)
c.StoreController.DefaultStore.RunDedupeBlobs(time.Duration(0), taskScheduler)
// Enable extensions if extension config is provided for DefaultStore
if c.Config != nil && c.Config.Extensions != nil {
ext.EnableMetricsExtension(c.Config, c.Log, c.Config.Storage.RootDirectory)
@ -565,6 +573,12 @@ func (c *Controller) StartBackgroundTasks(reloadCtx context.Context) {
if c.Config != nil && c.Config.Extensions != nil {
ext.EnableMetricsExtension(c.Config, c.Log, storageConfig.RootDirectory)
}
// Enable running dedupe blobs both ways (dedupe or restore deduped blobs) for subpaths
substore := c.StoreController.SubStore[route]
if substore != nil {
substore.RunDedupeBlobs(time.Duration(0), taskScheduler)
}
}
}

View file

@ -3345,10 +3345,10 @@ func TestCrossRepoMount(t *testing.T) {
dir := t.TempDir()
ctlr := makeController(conf, dir, "../../test/data")
ctlr.Config.Storage.RemoteCache = false
ctlr.Config.Storage.Dedupe = false
cm := test.NewControllerManager(ctlr)
cm := test.NewControllerManager(ctlr) //nolint: varnamelen
cm.StartAndWait(port)
defer cm.StopServer()
params := make(map[string]string)
@ -3449,6 +3449,14 @@ func TestCrossRepoMount(t *testing.T) {
// in cache, now try mount blob request status and it should be 201 because now blob is present in cache
// and it should do hard link.
// restart server with dedupe enabled
cm.StopServer()
ctlr.Config.Storage.Dedupe = true
cm.StartAndWait(port)
// wait for dedupe task to run
time.Sleep(15 * time.Second)
params["mount"] = string(manifestDigest)
postResponse, err = client.R().
SetBasicAuth(username, passphrase).SetQueryParams(params).
@ -6479,7 +6487,7 @@ func TestPeriodicGC(t *testing.T) {
ctlr := api.NewController(conf)
dir := t.TempDir()
ctlr.Config.Storage.RootDirectory = dir
ctlr.Config.Storage.Dedupe = false
ctlr.Config.Storage.GC = true
ctlr.Config.Storage.GCInterval = 1 * time.Hour
ctlr.Config.Storage.GCDelay = 1 * time.Second
@ -6518,8 +6526,8 @@ func TestPeriodicGC(t *testing.T) {
subPaths := make(map[string]config.StorageConfig)
subPaths["/a"] = config.StorageConfig{RootDirectory: subDir, GC: true, GCDelay: 1 * time.Second, GCInterval: 24 * time.Hour, RemoteCache: false} //nolint:lll // gofumpt conflicts with lll
subPaths["/a"] = config.StorageConfig{RootDirectory: subDir, GC: true, GCDelay: 1 * time.Second, GCInterval: 24 * time.Hour, RemoteCache: false, Dedupe: false} //nolint:lll // gofumpt conflicts with lll
ctlr.Config.Storage.Dedupe = false
ctlr.Config.Storage.SubPaths = subPaths
cm := test.NewControllerManager(ctlr)
@ -6552,8 +6560,8 @@ func TestPeriodicGC(t *testing.T) {
ctlr := api.NewController(conf)
dir := t.TempDir()
ctlr.Config.Storage.RootDirectory = dir
ctlr.Config.Storage.Dedupe = false
ctlr.Config.Storage.RootDirectory = dir
ctlr.Config.Storage.GC = true
ctlr.Config.Storage.GCInterval = 1 * time.Hour
ctlr.Config.Storage.GCDelay = 1 * time.Second

View file

@ -16,6 +16,8 @@ import (
"time"
"unicode/utf8"
"github.com/opencontainers/go-digest"
"zotregistry.io/zot/pkg/log"
)
@ -233,3 +235,13 @@ func MarshalThroughStruct(obj interface{}, throughStruct interface{}) ([]byte, e
return toJSON, nil
}
func DContains(slice []digest.Digest, item digest.Digest) bool {
for _, v := range slice {
if item == v {
return true
}
}
return false
}

View file

@ -91,7 +91,10 @@ func TestMetricsExtension(t *testing.T) {
ctlrManager := test.NewControllerManager(ctlr)
subPaths := make(map[string]config.StorageConfig)
subPaths["/a"] = config.StorageConfig{}
subPaths["/a"] = config.StorageConfig{
Dedupe: false,
RootDirectory: t.TempDir(),
}
ctlr.Config.Storage.RootDirectory = globalDir
ctlr.Config.Storage.SubPaths = subPaths

View file

@ -45,6 +45,8 @@ func TestScrubExtension(t *testing.T) {
subdir := t.TempDir()
conf.Storage.RootDirectory = dir
conf.Storage.Dedupe = false
substore := config.StorageConfig{RootDirectory: subdir}
conf.Storage.SubPaths = map[string]config.StorageConfig{"/a": substore}
conf.Log.Output = logFile.Name()
@ -86,6 +88,8 @@ func TestScrubExtension(t *testing.T) {
dir := t.TempDir()
conf.Storage.RootDirectory = dir
conf.Storage.Dedupe = false
conf.Log.Output = logFile.Name()
trueValue := true
scrubConfig := &extconf.ScrubConfig{
@ -132,6 +136,8 @@ func TestScrubExtension(t *testing.T) {
dir := t.TempDir()
conf.Storage.RootDirectory = dir
conf.Storage.Dedupe = false
conf.Log.Output = logFile.Name()
trueValue := true
scrubConfig := &extconf.ScrubConfig{

View file

@ -52,7 +52,11 @@ func (pq *generatorsPriorityQueue) Pop() any {
return item
}
const rateLimiterScheduler = 400
const (
rateLimiterScheduler = 400
rateLimit = 5 * time.Second
numWorkers = 3
)
type Scheduler struct {
tasksQLow chan Task
@ -63,6 +67,7 @@ type Scheduler struct {
generatorsLock *sync.Mutex
log log.Logger
stopCh chan struct{}
RateLimit time.Duration
}
func NewScheduler(logC log.Logger) *Scheduler {
@ -82,14 +87,11 @@ func NewScheduler(logC log.Logger) *Scheduler {
generatorsLock: new(sync.Mutex),
log: log.Logger{Logger: sublogger},
stopCh: make(chan struct{}),
// default value
RateLimit: rateLimit,
}
}
const (
rateLimit = 5 * time.Second
numWorkers = 3
)
func (scheduler *Scheduler) poolWorker(numWorkers int, tasks chan Task) {
for i := 0; i < numWorkers; i++ {
go func(workerID int) {
@ -120,6 +122,8 @@ func (scheduler *Scheduler) RunScheduler(ctx context.Context) {
close(tasksWorker)
close(scheduler.stopCh)
scheduler.log.Debug().Msg("scheduler: received stop signal, exiting...")
return
default:
i := 0

View file

@ -22,7 +22,8 @@ type BoltDBDriver struct {
}
type BoltDBDriverParameters struct {
RootDir, Name string
RootDir string
Name string
UseRelPaths bool
}
@ -206,10 +207,17 @@ func (d *BoltDBDriver) HasBlob(digest godigest.Digest, blob string) bool {
return errors.ErrCacheMiss
}
if origin.Get([]byte(blob)) == nil {
deduped := bucket.Bucket([]byte(constants.DuplicatesBucket))
if deduped == nil {
return errors.ErrCacheMiss
}
if origin.Get([]byte(blob)) == nil {
if deduped.Get([]byte(blob)) == nil {
return errors.ErrCacheMiss
}
}
return nil
}); err != nil {
return false

View file

@ -16,6 +16,7 @@ import (
"github.com/sigstore/cosign/pkg/oci/remote"
zerr "zotregistry.io/zot/errors"
"zotregistry.io/zot/pkg/scheduler"
storageConstants "zotregistry.io/zot/pkg/storage/constants"
)
@ -789,3 +790,88 @@ func CheckIsImageSignature(repoName string, manifestBlob []byte, reference strin
return false, "", "", nil
}
/*
DedupeTaskGenerator takes all blobs paths found in the imagestore and groups them by digest
for each digest and based on the dedupe value it will dedupe or restore deduped blobs to the original state(undeduped)\
by creating a task for each digest and pushing it to the task scheduler.
*/
type DedupeTaskGenerator struct {
ImgStore ImageStore
// storage dedupe value
Dedupe bool
// store blobs paths grouped by digest
digest godigest.Digest
duplicateBlobs []string
/* store processed digest, used for iterating duplicateBlobs one by one
and generating a task for each unprocessed one*/
lastDigests []godigest.Digest
done bool
Log zerolog.Logger
}
func (gen *DedupeTaskGenerator) GenerateTask() (scheduler.Task, error) {
var err error
// get all blobs from imageStore and group them by digest
gen.digest, gen.duplicateBlobs, err = gen.ImgStore.GetNextDigestWithBlobPaths(gen.lastDigests)
if err != nil {
gen.Log.Error().Err(err).Msg("dedupe rebuild: failed to get next digest")
return nil, err
}
// if no digests left, then mark the task generator as done
if gen.digest == "" {
gen.Log.Info().Msg("dedupe rebuild: finished")
gen.done = true
return nil, nil
}
// mark digest as processed before running its task
gen.lastDigests = append(gen.lastDigests, gen.digest)
// generate rebuild dedupe task for this digest
return newDedupeTask(gen.ImgStore, gen.digest, gen.Dedupe, gen.duplicateBlobs, gen.Log), nil
}
func (gen *DedupeTaskGenerator) IsDone() bool {
return gen.done
}
func (gen *DedupeTaskGenerator) Reset() {
gen.lastDigests = []godigest.Digest{}
gen.duplicateBlobs = []string{}
gen.digest = ""
gen.done = false
}
type dedupeTask struct {
imgStore ImageStore
// digest of duplicateBLobs
digest godigest.Digest
// blobs paths with the same digest ^
duplicateBlobs []string
dedupe bool
log zerolog.Logger
}
func newDedupeTask(imgStore ImageStore, digest godigest.Digest, dedupe bool,
duplicateBlobs []string, log zerolog.Logger,
) *dedupeTask {
return &dedupeTask{imgStore, digest, duplicateBlobs, dedupe, log}
}
func (dt *dedupeTask) DoWork() error {
// run task
err := dt.imgStore.RunDedupeForDigest(dt.digest, dt.dedupe, dt.duplicateBlobs)
if err != nil {
// log it
dt.log.Error().Err(err).Msgf("rebuild dedupe: failed to rebuild digest %s", dt.digest.String())
}
return err
}

View file

@ -6,6 +6,7 @@ import (
"errors"
"fmt"
"io"
"io/fs"
"os"
"path"
"path/filepath"
@ -1776,3 +1777,166 @@ func newGCTask(imgStore *ImageStoreLocal, repo string) *gcTask {
func (gcT *gcTask) DoWork() error {
return gcT.imgStore.RunGCRepo(gcT.repo)
}
func (is *ImageStoreLocal) GetNextDigestWithBlobPaths(lastDigests []godigest.Digest,
) (godigest.Digest, []string, error) {
var lockLatency time.Time
dir := is.rootDir
is.RLock(&lockLatency)
defer is.RUnlock(&lockLatency)
var duplicateBlobs []string
var digest godigest.Digest
err := filepath.WalkDir(dir, func(path string, info fs.DirEntry, err error) error {
if err != nil {
is.log.Warn().Err(err).Msg("unable to walk dir, skipping it")
// skip files/dirs which can't be walked
return filepath.SkipDir
}
if info.IsDir() {
return nil
}
blobDigest := godigest.NewDigestFromEncoded("sha256", info.Name())
if err := blobDigest.Validate(); err != nil {
return nil //nolint:nilerr // ignore files which are not blobs
}
if digest == "" && !common.DContains(lastDigests, blobDigest) {
digest = blobDigest
}
if blobDigest == digest {
duplicateBlobs = append(duplicateBlobs, path)
}
return nil
})
return digest, duplicateBlobs, err
}
func (is *ImageStoreLocal) dedupeBlobs(digest godigest.Digest, duplicateBlobs []string) error {
if fmt.Sprintf("%v", is.cache) == fmt.Sprintf("%v", nil) {
is.log.Error().Err(zerr.ErrDedupeRebuild).Msg("no cache driver found, can not dedupe blobs")
return zerr.ErrDedupeRebuild
}
is.log.Info().Str("digest", digest.String()).Msgf("rebuild dedupe: deduping blobs for digest")
var originalBlob string
var originalBlobFi fs.FileInfo
var err error
// rebuild from dedupe false to true
for _, blobPath := range duplicateBlobs {
/* for local storage, because we use hard links, we can assume that any blob can be original
so we skip the first one and hard link the rest of them with the first*/
if originalBlob == "" {
originalBlob = blobPath
originalBlobFi, err = os.Stat(originalBlob)
if err != nil {
is.log.Error().Err(err).Str("path", originalBlob).Msg("rebuild dedupe: failed to stat blob")
return err
}
// cache it
if ok := is.cache.HasBlob(digest, blobPath); !ok {
if err := is.cache.PutBlob(digest, blobPath); err != nil {
return err
}
}
continue
}
binfo, err := os.Stat(blobPath)
if err != nil {
is.log.Error().Err(err).Str("path", blobPath).Msg("rebuild dedupe: failed to stat blob")
return err
}
// dedupe blob
if !os.SameFile(originalBlobFi, binfo) {
// we should link to a temp file instead of removing blob and then linking
// to make this more atomic
uuid, err := guuid.NewV4()
if err != nil {
return err
}
// put temp blob in <repo>/.uploads dir
tempLinkBlobDir := path.Join(strings.Replace(blobPath, path.Join("blobs/sha256", binfo.Name()), "", 1),
storageConstants.BlobUploadDir)
if err := os.MkdirAll(tempLinkBlobDir, DefaultDirPerms); err != nil {
is.log.Error().Err(err).Str("dir", tempLinkBlobDir).Msg("rebuild dedupe: unable to mkdir")
return err
}
tempLinkBlobPath := path.Join(tempLinkBlobDir, uuid.String())
if err := os.Link(originalBlob, tempLinkBlobPath); err != nil {
is.log.Error().Err(err).Str("src", originalBlob).
Str("dst", tempLinkBlobPath).Msg("rebuild dedupe: unable to hard link")
return err
}
if err := os.Rename(tempLinkBlobPath, blobPath); err != nil {
is.log.Error().Err(err).Str("blobPath", blobPath).Msg("rebuild dedupe: unable to rename temp link")
return err
}
// cache it
if ok := is.cache.HasBlob(digest, blobPath); !ok {
if err := is.cache.PutBlob(digest, blobPath); err != nil {
return err
}
}
}
}
is.log.Info().Str("digest", digest.String()).Msgf("rebuild dedupe: deduping blobs for digest finished successfully")
return nil
}
func (is *ImageStoreLocal) RunDedupeForDigest(digest godigest.Digest, dedupe bool, duplicateBlobs []string) error {
var lockLatency time.Time
is.Lock(&lockLatency)
defer is.Unlock(&lockLatency)
if dedupe {
return is.dedupeBlobs(digest, duplicateBlobs)
}
// otherwise noop
return nil
}
func (is *ImageStoreLocal) RunDedupeBlobs(interval time.Duration, sch *scheduler.Scheduler) {
// for local storage no need to undedupe blobs
if is.dedupe {
generator := &storage.DedupeTaskGenerator{
ImgStore: is,
Dedupe: is.dedupe,
Log: is.log,
}
sch.SubmitGenerator(generator, interval, scheduler.HighPriority)
}
}

View file

@ -2,6 +2,7 @@ package local_test
import (
"bytes"
"context"
"crypto/rand"
_ "crypto/sha256"
"encoding/json"
@ -28,11 +29,13 @@ import (
"zotregistry.io/zot/pkg/common"
"zotregistry.io/zot/pkg/extensions/monitoring"
"zotregistry.io/zot/pkg/log"
"zotregistry.io/zot/pkg/scheduler"
"zotregistry.io/zot/pkg/storage"
"zotregistry.io/zot/pkg/storage/cache"
storageConstants "zotregistry.io/zot/pkg/storage/constants"
"zotregistry.io/zot/pkg/storage/local"
"zotregistry.io/zot/pkg/test"
"zotregistry.io/zot/pkg/test/mocks"
)
const (
@ -40,6 +43,18 @@ const (
repoName = "test"
)
var errCache = errors.New("new cache error")
func runAndGetScheduler() (*scheduler.Scheduler, context.CancelFunc) {
taskScheduler := scheduler.NewScheduler(log.Logger{})
taskScheduler.RateLimit = 50 * time.Millisecond
ctx, cancel := context.WithCancel(context.Background())
taskScheduler.RunScheduler(ctx)
return taskScheduler, cancel
}
func TestStorageFSAPIs(t *testing.T) {
dir := t.TempDir()
@ -1066,19 +1081,36 @@ func FuzzRunGCRepo(f *testing.F) {
}
func TestDedupeLinks(t *testing.T) {
dir := t.TempDir()
testCases := []struct {
dedupe bool
expected bool
}{
{
dedupe: true,
expected: true,
},
{
dedupe: false,
expected: false,
},
}
log := log.Logger{Logger: zerolog.New(os.Stdout)}
metrics := monitoring.NewMetricsServer(false, log)
for _, testCase := range testCases {
dir := t.TempDir()
cacheDriver, _ := storage.Create("boltdb", cache.BoltDBDriverParameters{
RootDir: dir,
Name: "cache",
UseRelPaths: true,
}, log)
imgStore := local.NewImageStore(dir, false, storage.DefaultGCDelay,
true, true, log, metrics, nil, cacheDriver)
Convey("Dedupe", t, func(c C) {
imgStore := local.NewImageStore(dir, false, storage.DefaultGCDelay,
testCase.dedupe, true, log, metrics, nil, cacheDriver)
Convey(fmt.Sprintf("Dedupe %t", testCase.dedupe), t, func(c C) {
// manifest1
upload, err := imgStore.NewBlobUpload("dedupe1")
So(err, ShouldBeNil)
@ -1199,11 +1231,132 @@ func TestDedupeLinks(t *testing.T) {
So(err, ShouldBeNil)
// verify that dedupe with hard links happened
fi1, err := os.Stat(path.Join(dir, "dedupe2", "blobs", "sha256", blobDigest1))
fi1, err := os.Stat(path.Join(dir, "dedupe1", "blobs", "sha256", blobDigest1))
So(err, ShouldBeNil)
fi2, err := os.Stat(path.Join(dir, "dedupe2", "blobs", "sha256", blobDigest2))
So(err, ShouldBeNil)
So(os.SameFile(fi1, fi2), ShouldBeTrue)
So(os.SameFile(fi1, fi2), ShouldEqual, testCase.expected)
if !testCase.dedupe {
Convey("Intrerrupt rebuilding and restart, checking idempotency", func() {
for i := 0; i < 10; i++ {
taskScheduler, cancel := runAndGetScheduler()
// rebuild with dedupe true
imgStore := local.NewImageStore(dir, false, storage.DefaultGCDelay,
true, true, log, metrics, nil, cacheDriver)
imgStore.RunDedupeBlobs(time.Duration(0), taskScheduler)
sleepValue := i * 50
time.Sleep(time.Duration(sleepValue) * time.Millisecond)
cancel()
}
taskScheduler, cancel := runAndGetScheduler()
// rebuild with dedupe true
imgStore := local.NewImageStore(dir, false, storage.DefaultGCDelay,
true, true, log, metrics, nil, cacheDriver)
imgStore.RunDedupeBlobs(time.Duration(0), taskScheduler)
// wait until rebuild finishes
time.Sleep(10 * time.Second)
cancel()
fi1, err := os.Stat(path.Join(dir, "dedupe1", "blobs", "sha256", blobDigest1))
So(err, ShouldBeNil)
fi2, err := os.Stat(path.Join(dir, "dedupe2", "blobs", "sha256", blobDigest2))
So(err, ShouldBeNil)
So(os.SameFile(fi1, fi2), ShouldEqual, true)
})
Convey("rebuild dedupe index error cache nil", func() {
// switch dedupe to true from false
taskScheduler, cancel := runAndGetScheduler()
imgStore := local.NewImageStore(dir, false, storage.DefaultGCDelay,
true, true, log, metrics, nil, nil)
// rebuild with dedupe true
imgStore.RunDedupeBlobs(time.Duration(0), taskScheduler)
// wait until rebuild finishes
time.Sleep(3 * time.Second)
cancel()
fi1, err := os.Stat(path.Join(dir, "dedupe1", "blobs", "sha256", blobDigest1))
So(err, ShouldBeNil)
fi2, err := os.Stat(path.Join(dir, "dedupe2", "blobs", "sha256", blobDigest2))
So(err, ShouldBeNil)
So(os.SameFile(fi1, fi2), ShouldEqual, false)
})
Convey("rebuild dedupe index cache error on original blob", func() {
// switch dedupe to true from false
taskScheduler, cancel := runAndGetScheduler()
imgStore := local.NewImageStore(dir, false, storage.DefaultGCDelay,
true, true, log, metrics, nil, &mocks.CacheMock{
HasBlobFn: func(digest godigest.Digest, path string) bool {
return false
},
PutBlobFn: func(digest godigest.Digest, path string) error {
return errCache
},
})
// rebuild with dedupe true, should have samefile blobs
imgStore.RunDedupeBlobs(time.Duration(0), taskScheduler)
// wait until rebuild finishes
time.Sleep(10 * time.Second)
cancel()
fi1, err := os.Stat(path.Join(dir, "dedupe1", "blobs", "sha256", blobDigest1))
So(err, ShouldBeNil)
fi2, err := os.Stat(path.Join(dir, "dedupe2", "blobs", "sha256", blobDigest2))
So(err, ShouldBeNil)
So(os.SameFile(fi1, fi2), ShouldEqual, false)
})
Convey("rebuild dedupe index cache error on duplicate blob", func() {
// switch dedupe to true from false
taskScheduler, cancel := runAndGetScheduler()
imgStore := local.NewImageStore(dir, false, storage.DefaultGCDelay,
true, true, log, metrics, nil, &mocks.CacheMock{
HasBlobFn: func(digest godigest.Digest, path string) bool {
return false
},
PutBlobFn: func(digest godigest.Digest, path string) error {
if strings.Contains(path, "dedupe2") {
return errCache
}
return nil
},
})
// rebuild with dedupe true, should have samefile blobs
imgStore.RunDedupeBlobs(time.Duration(0), taskScheduler)
// wait until rebuild finishes
time.Sleep(15 * time.Second)
cancel()
fi1, err := os.Stat(path.Join(dir, "dedupe1", "blobs", "sha256", blobDigest1))
So(err, ShouldBeNil)
fi2, err := os.Stat(path.Join(dir, "dedupe2", "blobs", "sha256", blobDigest2))
So(err, ShouldBeNil)
// deduped happened, but didn't cached
So(os.SameFile(fi1, fi2), ShouldEqual, true)
})
}
Convey("storage and cache inconsistency", func() {
// delete blobs
@ -1233,6 +1386,7 @@ func TestDedupeLinks(t *testing.T) {
So(blob, ShouldEqual, buflen)
})
})
}
}
func TestDedupe(t *testing.T) {

View file

@ -24,6 +24,7 @@ import (
"github.com/rs/zerolog"
zerr "zotregistry.io/zot/errors"
"zotregistry.io/zot/pkg/common"
"zotregistry.io/zot/pkg/extensions/monitoring"
zlog "zotregistry.io/zot/pkg/log"
zreg "zotregistry.io/zot/pkg/regexp"
@ -1191,7 +1192,7 @@ func (is *ObjectStorage) GetBlob(repo string, digest godigest.Digest, mediaType
}
// is a 'deduped' blob?
if binfo.Size() == 0 && fmt.Sprintf("%v", is.cache) != fmt.Sprintf("%v", nil) {
if binfo.Size() == 0 {
// Check blobs in cache
dstRecord, err := is.checkCacheBlob(digest)
if err != nil {
@ -1244,7 +1245,7 @@ func (is *ObjectStorage) GetBlobContent(repo string, digest godigest.Digest) ([]
}
// is a 'deduped' blob?
if binfo.Size() == 0 && fmt.Sprintf("%v", is.cache) != fmt.Sprintf("%v", nil) {
if binfo.Size() == 0 {
// Check blobs in cache
dstRecord, err := is.checkCacheBlob(digest)
if err != nil {
@ -1395,3 +1396,230 @@ func writeFile(store driver.StorageDriver, filepath string, buf []byte) (int, er
return n, nil
}
func (is *ObjectStorage) GetNextDigestWithBlobPaths(lastDigests []godigest.Digest) (godigest.Digest, []string, error) {
var lockLatency time.Time
dir := is.rootDir
is.RLock(&lockLatency)
defer is.RUnlock(&lockLatency)
var duplicateBlobs []string
var digest godigest.Digest
err := is.store.Walk(context.Background(), dir, func(fileInfo driver.FileInfo) error {
if fileInfo.IsDir() {
return nil
}
blobDigest := godigest.NewDigestFromEncoded("sha256", path.Base(fileInfo.Path()))
if err := blobDigest.Validate(); err != nil {
return nil //nolint:nilerr // ignore files which are not blobs
}
if digest == "" && !common.DContains(lastDigests, blobDigest) {
digest = blobDigest
}
if blobDigest == digest {
duplicateBlobs = append(duplicateBlobs, fileInfo.Path())
}
return nil
})
// if the root directory is not yet created
var perr driver.PathNotFoundError
if errors.As(err, &perr) {
return digest, duplicateBlobs, nil
}
return digest, duplicateBlobs, err
}
func (is *ObjectStorage) getOriginalBlobFromDisk(duplicateBlobs []string) (string, error) {
for _, blobPath := range duplicateBlobs {
binfo, err := is.store.Stat(context.Background(), blobPath)
if err != nil {
is.log.Error().Err(err).Str("path", blobPath).Msg("rebuild dedupe: failed to stat blob")
return "", zerr.ErrBlobNotFound
}
if binfo.Size() > 0 {
return blobPath, nil
}
}
return "", zerr.ErrBlobNotFound
}
func (is *ObjectStorage) getOriginalBlob(digest godigest.Digest, duplicateBlobs []string) (string, error) {
originalBlob := ""
var err error
originalBlob, err = is.checkCacheBlob(digest)
if err != nil && !errors.Is(err, zerr.ErrBlobNotFound) && !errors.Is(err, zerr.ErrCacheMiss) {
is.log.Error().Err(err).Msg("rebuild dedupe: unable to find blob in cache")
return originalBlob, err
}
// if we still don't have, search it
if originalBlob == "" {
is.log.Warn().Msg("rebuild dedupe: failed to find blob in cache, searching it in s3...")
// a rebuild dedupe was attempted in the past
// get original blob, should be found otherwise exit with error
originalBlob, err = is.getOriginalBlobFromDisk(duplicateBlobs)
if err != nil {
return originalBlob, err
}
}
is.log.Info().Msgf("rebuild dedupe: found original blob %s", originalBlob)
return originalBlob, nil
}
func (is *ObjectStorage) dedupeBlobs(digest godigest.Digest, duplicateBlobs []string) error {
if fmt.Sprintf("%v", is.cache) == fmt.Sprintf("%v", nil) {
is.log.Error().Err(zerr.ErrDedupeRebuild).Msg("no cache driver found, can not dedupe blobs")
return zerr.ErrDedupeRebuild
}
is.log.Info().Str("digest", digest.String()).Msgf("rebuild dedupe: deduping blobs for digest")
var originalBlob string
// rebuild from dedupe false to true
for _, blobPath := range duplicateBlobs {
binfo, err := is.store.Stat(context.Background(), blobPath)
if err != nil {
is.log.Error().Err(err).Str("path", blobPath).Msg("rebuild dedupe: failed to stat blob")
return err
}
if binfo.Size() == 0 {
is.log.Warn().Msg("rebuild dedupe: found file without content, trying to find the original blob")
// a rebuild dedupe was attempted in the past
// get original blob, should be found otherwise exit with error
if originalBlob == "" {
originalBlob, err = is.getOriginalBlob(digest, duplicateBlobs)
if err != nil {
is.log.Error().Err(err).Msg("rebuild dedupe: unable to find original blob")
return zerr.ErrDedupeRebuild
}
// cache original blob
if ok := is.cache.HasBlob(digest, originalBlob); !ok {
if err := is.cache.PutBlob(digest, originalBlob); err != nil {
return err
}
}
}
// cache dedupe blob
if ok := is.cache.HasBlob(digest, blobPath); !ok {
if err := is.cache.PutBlob(digest, blobPath); err != nil {
return err
}
}
} else {
// cache it
if ok := is.cache.HasBlob(digest, blobPath); !ok {
if err := is.cache.PutBlob(digest, blobPath); err != nil {
return err
}
}
// if we have an original blob cached then we can safely dedupe the rest of them
if originalBlob != "" {
if err := is.store.PutContent(context.Background(), blobPath, []byte{}); err != nil {
is.log.Error().Err(err).Str("path", blobPath).Msg("rebuild dedupe: unable to dedupe blob")
return err
}
}
// mark blob as preserved
originalBlob = blobPath
}
}
is.log.Info().Str("digest", digest.String()).Msgf("rebuild dedupe: deduping blobs for digest finished successfully")
return nil
}
func (is *ObjectStorage) restoreDedupedBlobs(digest godigest.Digest, duplicateBlobs []string) error {
is.log.Info().Str("digest", digest.String()).Msgf("rebuild dedupe: restoring deduped blobs for digest")
// first we need to find the original blob, either in cache or by checking each blob size
originalBlob, err := is.getOriginalBlob(digest, duplicateBlobs)
if err != nil {
is.log.Error().Err(err).Msg("rebuild dedupe: unable to find original blob")
return zerr.ErrDedupeRebuild
}
for _, blobPath := range duplicateBlobs {
binfo, err := is.store.Stat(context.Background(), blobPath)
if err != nil {
is.log.Error().Err(err).Str("path", blobPath).Msg("rebuild dedupe: failed to stat blob")
return err
}
// if we find a deduped blob, then copy original blob content to deduped one
if binfo.Size() == 0 {
// move content from original blob to deduped one
buf, err := is.store.GetContent(context.Background(), originalBlob)
if err != nil {
is.log.Error().Err(err).Str("path", originalBlob).Msg("rebuild dedupe: failed to get original blob content")
return err
}
_, err = writeFile(is.store, blobPath, buf)
if err != nil {
return err
}
}
}
is.log.Info().Str("digest", digest.String()).
Msgf("rebuild dedupe: restoring deduped blobs for digest finished successfully")
return nil
}
func (is *ObjectStorage) RunDedupeForDigest(digest godigest.Digest, dedupe bool, duplicateBlobs []string) error {
var lockLatency time.Time
is.Lock(&lockLatency)
defer is.Unlock(&lockLatency)
if dedupe {
return is.dedupeBlobs(digest, duplicateBlobs)
}
return is.restoreDedupedBlobs(digest, duplicateBlobs)
}
func (is *ObjectStorage) RunDedupeBlobs(interval time.Duration, sch *scheduler.Scheduler) {
generator := &storage.DedupeTaskGenerator{
ImgStore: is,
Dedupe: is.dedupe,
Log: is.log,
}
sch.SubmitGenerator(generator, interval, scheduler.HighPriority)
}

File diff suppressed because it is too large Load diff

View file

@ -53,4 +53,7 @@ type ImageStore interface { //nolint:interfacebloat
GetOrasReferrers(repo string, digest godigest.Digest, artifactType string) ([]artifactspec.Descriptor, error)
RunGCRepo(repo string) error
RunGCPeriodically(interval time.Duration, sch *scheduler.Scheduler)
RunDedupeBlobs(interval time.Duration, sch *scheduler.Scheduler)
RunDedupeForDigest(digest godigest.Digest, dedupe bool, duplicateBlobs []string) error
GetNextDigestWithBlobPaths(lastDigests []godigest.Digest) (godigest.Digest, []string, error)
}

View file

@ -84,6 +84,7 @@ func createObjectsStore(rootDir string, cacheDir string) (driver.StorageDriver,
Name: "s3_cache",
UseRelPaths: false,
}, log)
il := s3.NewImageStore(rootDir, cacheDir, false, storage.DefaultGCDelay,
true, false, log, metrics, nil, store, cacheDriver,
)

View file

@ -0,0 +1,60 @@
package mocks
import godigest "github.com/opencontainers/go-digest"
type CacheMock struct {
// Returns the human-readable "name" of the driver.
NameFn func() string
// Retrieves the blob matching provided digest.
GetBlobFn func(digest godigest.Digest) (string, error)
// Uploads blob to cachedb.
PutBlobFn func(digest godigest.Digest, path string) error
// Check if blob exists in cachedb.
HasBlobFn func(digest godigest.Digest, path string) bool
// Delete a blob from the cachedb.
DeleteBlobFn func(digest godigest.Digest, path string) error
}
func (cacheMock CacheMock) Name() string {
if cacheMock.NameFn != nil {
return cacheMock.NameFn()
}
return "mock"
}
func (cacheMock CacheMock) GetBlob(digest godigest.Digest) (string, error) {
if cacheMock.GetBlobFn != nil {
return cacheMock.GetBlobFn(digest)
}
return "", nil
}
func (cacheMock CacheMock) PutBlob(digest godigest.Digest, path string) error {
if cacheMock.PutBlobFn != nil {
return cacheMock.PutBlobFn(digest, path)
}
return nil
}
func (cacheMock CacheMock) HasBlob(digest godigest.Digest, path string) bool {
if cacheMock.HasBlobFn != nil {
return cacheMock.HasBlobFn(digest, path)
}
return true
}
func (cacheMock CacheMock) DeleteBlob(digest godigest.Digest, path string) error {
if cacheMock.DeleteBlobFn != nil {
return cacheMock.DeleteBlobFn(digest, path)
}
return nil
}

View file

@ -41,10 +41,14 @@ type MockedImageStore struct {
GetIndexContentFn func(repo string) ([]byte, error)
GetBlobContentFn func(repo string, digest godigest.Digest) ([]byte, error)
GetReferrersFn func(repo string, digest godigest.Digest, artifactTypes []string) (ispec.Index, error)
GetOrasReferrersFn func(repo string, digest godigest.Digest, artifactType string) ([]artifactspec.Descriptor, error)
GetOrasReferrersFn func(repo string, digest godigest.Digest, artifactType string,
) ([]artifactspec.Descriptor, error)
URLForPathFn func(path string) (string, error)
RunGCRepoFn func(repo string) error
RunGCPeriodicallyFn func(interval time.Duration, sch *scheduler.Scheduler)
RunDedupeBlobsFn func(interval time.Duration, sch *scheduler.Scheduler)
RunDedupeForDigestFn func(digest godigest.Digest, dedupe bool, duplicateBlobs []string) error
GetNextDigestWithBlobPathsFn func(lastDigests []godigest.Digest) (godigest.Digest, []string, error)
}
func (is MockedImageStore) Lock(t *time.Time) {
@ -332,3 +336,26 @@ func (is MockedImageStore) RunGCPeriodically(interval time.Duration, sch *schedu
is.RunGCPeriodicallyFn(interval, sch)
}
}
func (is MockedImageStore) RunDedupeBlobs(interval time.Duration, sch *scheduler.Scheduler) {
if is.RunDedupeBlobsFn != nil {
is.RunDedupeBlobsFn(interval, sch)
}
}
func (is MockedImageStore) RunDedupeForDigest(digest godigest.Digest, dedupe bool, duplicateBlobs []string) error {
if is.RunDedupeForDigestFn != nil {
return is.RunDedupeForDigestFn(digest, dedupe, duplicateBlobs)
}
return nil
}
func (is MockedImageStore) GetNextDigestWithBlobPaths(lastDigests []godigest.Digest,
) (godigest.Digest, []string, error) {
if is.GetNextDigestWithBlobPathsFn != nil {
return is.GetNextDigestWithBlobPathsFn(lastDigests)
}
return "", []string{}, nil
}

View file

@ -1,8 +1,10 @@
ROOT_DIR=$(git rev-parse --show-toplevel)
TEST_DATA_DIR=${ROOT_DIR}/test/data/
OS="${OS:-linux}"
ARCH="${ARCH:-amd64}"
ZOT_PATH=${ROOT_DIR}/bin/zot-${OS}-${ARCH}
mkdir -p ${TEST_DATA_DIR}
function verify_prerequisites {
if [ ! -f ${ZOT_PATH} ]; then
@ -28,17 +30,35 @@ function zot_serve_strace() {
strace -o "strace.txt" -f -e trace=openat ${ZOT_PATH} serve ${config_file} &
}
function zot_serve() {
local config_file=${1}
${ZOT_PATH} serve ${config_file} &
}
function zot_stop() {
pkill zot
}
function wait_for_string() {
string=$1
filepath=$2
while [ ! -f $filepath ]
do sleep 2;
done
while ! grep "${string}" $filepath
do sleep 10;
done
}
function wait_zot_reachable() {
zot_url=${1}
curl --connect-timeout 3 \
--max-time 3 \
--max-time 10 \
--retry 10 \
--retry-delay 0 \
--retry-max-time 60 \
--retry-max-time 120 \
--retry-connrefused \
${zot_url}
}

View file

@ -0,0 +1,288 @@
load helpers_pushpull
function setup_file() {
# Verify prerequisites are available
if ! verify_prerequisites; then
exit 1
fi
# Download test data to folder common for the entire suite, not just this file
skopeo --insecure-policy copy --format=oci docker://ghcr.io/project-zot/golang:1.20 oci:${TEST_DATA_DIR}/golang:1.20
# Setup zot server
local zot_root_dir=${BATS_FILE_TMPDIR}/zot
local zot_config_file=${BATS_FILE_TMPDIR}/zot_config.json
local oci_data_dir=${BATS_FILE_TMPDIR}/oci
mkdir -p ${zot_root_dir}
mkdir -p ${oci_data_dir}
cat > ${zot_config_file}<<EOF
{
"distSpecVersion": "1.1.0",
"storage": {
"rootDirectory": "${zot_root_dir}",
"dedupe": false,
"gc": true,
"gcInterval": "30s"
},
"http": {
"address": "0.0.0.0",
"port": "8080"
},
"log": {
"level": "debug"
}
}
EOF
git -C ${BATS_FILE_TMPDIR} clone https://github.com/project-zot/helm-charts.git
setup_zot_file_level ${zot_config_file}
wait_zot_reachable "http://127.0.0.1:8080/v2/_catalog"
}
function teardown_file() {
local zot_root_dir=${BATS_FILE_TMPDIR}/zot
local oci_data_dir=${BATS_FILE_TMPDIR}/oci
teardown_zot_file_level
rm -rf ${zot_root_dir}
rm -rf ${oci_data_dir}
}
@test "push image - dedupe not running" {
start=`date +%s`
run skopeo --insecure-policy copy --dest-tls-verify=false \
oci:${TEST_DATA_DIR}/golang:1.20 \
docker://127.0.0.1:8080/golang:1.20
[ "$status" -eq 0 ]
end=`date +%s`
runtime=$((end-start))
echo "push image exec time: $runtime sec" >&3
run curl http://127.0.0.1:8080/v2/_catalog
[ "$status" -eq 0 ]
[ $(echo "${lines[-1]}" | jq '.repositories[]') = '"golang"' ]
run curl http://127.0.0.1:8080/v2/golang/tags/list
[ "$status" -eq 0 ]
[ $(echo "${lines[-1]}" | jq '.tags[]') = '"1.20"' ]
}
@test "pull image - dedupe not running" {
local oci_data_dir=${BATS_FILE_TMPDIR}/oci
start=`date +%s`
run skopeo --insecure-policy copy --src-tls-verify=false \
docker://127.0.0.1:8080/golang:1.20 \
oci:${oci_data_dir}/golang:1.20
[ "$status" -eq 0 ]
end=`date +%s`
runtime=$((end-start))
echo "pull image exec time: $runtime sec" >&3
run cat ${BATS_FILE_TMPDIR}/oci/golang/index.json
[ "$status" -eq 0 ]
[ $(echo "${lines[-1]}" | jq '.manifests[].annotations."org.opencontainers.image.ref.name"') = '"1.20"' ]
}
@test "push 50 images with dedupe disabled" {
for i in {1..50}
do
run skopeo --insecure-policy copy --dest-tls-verify=false \
oci:${TEST_DATA_DIR}/golang:1.20 \
docker://127.0.0.1:8080/golang${i}:1.20
[ "$status" -eq 0 ]
done
}
@test "restart zot with dedupe enabled" {
local zot_config_file=${BATS_FILE_TMPDIR}/zot_config.json
# stop server
teardown_zot_file_level
# enable dedupe
sed -i 's/false/true/g' ${zot_config_file}
setup_zot_file_level ${zot_config_file}
wait_zot_reachable "http://127.0.0.1:8080/v2/_catalog"
# deduping will now run in background (task scheduler) while we push images, shouldn't interfere
}
@test "push image - dedupe running" {
start=`date +%s`
run skopeo --insecure-policy copy --dest-tls-verify=false \
oci:${TEST_DATA_DIR}/golang:1.20 \
docker://127.0.0.1:8080/dedupe/golang:1.20
[ "$status" -eq 0 ]
end=`date +%s`
runtime=$((end-start))
echo "push image exec time: $runtime sec" >&3
}
@test "pull image - dedupe running" {
local oci_data_dir=${BATS_FILE_TMPDIR}/oci
mkdir -p ${oci_data_dir}/dedupe/
start=`date +%s`
run skopeo --insecure-policy copy --src-tls-verify=false \
docker://127.0.0.1:8080/dedupe/golang:1.20 \
oci:${oci_data_dir}/dedupe/golang:1.20
[ "$status" -eq 0 ]
end=`date +%s`
runtime=$((end-start))
echo "pull image exec time: $runtime sec" >&3
}
@test "pull deduped image - dedupe running" {
local oci_data_dir=${BATS_FILE_TMPDIR}/oci
mkdir -p ${oci_data_dir}/dedupe/
start=`date +%s`
run skopeo --insecure-policy copy --src-tls-verify=false \
docker://127.0.0.1:8080/golang2:1.20 \
oci:${oci_data_dir}/dedupe/golang2:1.20
[ "$status" -eq 0 ]
end=`date +%s`
runtime=$((end-start))
echo "pull image exec time: $runtime sec" >&3
}
@test "push image index - dedupe running" {
# --multi-arch below pushes an image index (containing many images) instead
# of an image manifest (single image)
start=`date +%s`
run skopeo --insecure-policy copy --format=oci --dest-tls-verify=false --multi-arch=all \
docker://public.ecr.aws/docker/library/busybox:latest \
docker://127.0.0.1:8080/busybox:latest
[ "$status" -eq 0 ]
end=`date +%s`
runtime=$((end-start))
echo "push image index exec time: $runtime sec" >&3
run curl http://127.0.0.1:8080/v2/_catalog
[ "$status" -eq 0 ]
[ $(echo "${lines[-1]}" | jq '.repositories[0]') = '"busybox"' ]
run curl http://127.0.0.1:8080/v2/busybox/tags/list
[ "$status" -eq 0 ]
[ $(echo "${lines[-1]}" | jq '.tags[]') = '"latest"' ]
}
@test "pull image index - dedupe running" {
local oci_data_dir=${BATS_FILE_TMPDIR}/oci
start=`date +%s`
run skopeo --insecure-policy copy --src-tls-verify=false --multi-arch=all \
docker://127.0.0.1:8080/busybox:latest \
oci:${oci_data_dir}/busybox:latest
[ "$status" -eq 0 ]
end=`date +%s`
runtime=$((end-start))
echo "pull image index exec time: $runtime sec" >&3
run cat ${BATS_FILE_TMPDIR}/oci/busybox/index.json
[ "$status" -eq 0 ]
[ $(echo "${lines[-1]}" | jq '.manifests[].annotations."org.opencontainers.image.ref.name"') = '"latest"' ]
run skopeo --insecure-policy --override-arch=arm64 --override-os=linux copy --src-tls-verify=false --multi-arch=all \
docker://127.0.0.1:8080/busybox:latest \
oci:${oci_data_dir}/busybox:latest
[ "$status" -eq 0 ]
run cat ${BATS_FILE_TMPDIR}/oci/busybox/index.json
[ "$status" -eq 0 ]
[ $(echo "${lines[-1]}" | jq '.manifests[].annotations."org.opencontainers.image.ref.name"') = '"latest"' ]
run curl -X DELETE http://127.0.0.1:8080/v2/busybox/manifests/latest
[ "$status" -eq 0 ]
}
@test "push oras artifact - dedupe running" {
echo "{\"name\":\"foo\",\"value\":\"bar\"}" > config.json
echo "hello world" > artifact.txt
start=`date +%s`
run oras push --plain-http 127.0.0.1:8080/hello-artifact:v2 \
--config config.json:application/vnd.acme.rocket.config.v1+json artifact.txt:text/plain -d -v
[ "$status" -eq 0 ]
end=`date +%s`
runtime=$((end-start))
echo "push oras artifact exec time: $runtime sec" >&3
rm -f artifact.txt
rm -f config.json
}
@test "pull oras artifact - dedupe running" {
start=`date +%s`
run oras pull --plain-http 127.0.0.1:8080/hello-artifact:v2 -d -v
[ "$status" -eq 0 ]
end=`date +%s`
runtime=$((end-start))
echo "pull oras artifact exec time: $runtime sec" >&3
grep -q "hello world" artifact.txt
rm -f artifact.txt
}
@test "attach oras artifacts - dedupe running" {
# attach signature
echo "{\"artifact\": \"\", \"signature\": \"pat hancock\"}" > signature.json
start=`date +%s`
run oras attach --plain-http 127.0.0.1:8080/golang:1.20 --artifact-type 'signature/example' ./signature.json:application/json
[ "$status" -eq 0 ]
end=`date +%s`
runtime=$((end-start))
echo "attach signature exec time: $runtime sec" >&3
# attach sbom
echo "{\"version\": \"0.0.0.0\", \"artifact\": \"'127.0.0.1:8080/golang:1.20'\", \"contents\": \"good\"}" > sbom.json
start=`date +%s`
run oras attach --plain-http 127.0.0.1:8080/golang:1.20 --artifact-type 'sbom/example' ./sbom.json:application/json
[ "$status" -eq 0 ]
end=`date +%s`
runtime=$((end-start))
echo "attach sbom exec time: $runtime sec" >&3
}
@test "discover oras artifacts - dedupe running" {
start=`date +%s`
run oras discover --plain-http -o json 127.0.0.1:8080/golang:1.20
[ "$status" -eq 0 ]
end=`date +%s`
runtime=$((end-start))
echo "discover oras artifacts exec time: $runtime sec" >&3
[ $(echo "$output" | jq -r ".manifests | length") -eq 2 ]
}
@test "push helm chart - dedupe running" {
run helm package ${BATS_FILE_TMPDIR}/helm-charts/charts/zot
[ "$status" -eq 0 ]
local chart_version=$(awk '/version/{printf $2}' ${BATS_FILE_TMPDIR}/helm-charts/charts/zot/Chart.yaml)
start=`date +%s`
run helm push zot-${chart_version}.tgz oci://localhost:8080/zot-chart
[ "$status" -eq 0 ]
end=`date +%s`
runtime=$((end-start))
echo "helm push exec time: $runtime sec" >&3
}
@test "pull helm chart - dedupe running" {
local chart_version=$(awk '/version/{printf $2}' ${BATS_FILE_TMPDIR}/helm-charts/charts/zot/Chart.yaml)
start=`date +%s`
run helm pull oci://localhost:8080/zot-chart/zot --version ${chart_version}
[ "$status" -eq 0 ]
end=`date +%s`
runtime=$((end-start))
echo "helm pull exec time: $runtime sec" >&3
}
@test "push image with regclient - dedupe running" {
run regctl registry set localhost:8080 --tls disabled
[ "$status" -eq 0 ]
start=`date +%s`
run regctl image copy ocidir://${TEST_DATA_DIR}/golang:1.20 localhost:8080/test-regclient
[ "$status" -eq 0 ]
end=`date +%s`
runtime=$((end-start))
echo "regclient push exec time: $runtime" >&3
}

View file

@ -0,0 +1,140 @@
load helpers_cloud
function setup_file() {
# Verify prerequisites are available
if ! verify_prerequisites; then
exit 1
fi
# Download test data to folder common for the entire suite, not just this file
skopeo --insecure-policy copy --format=oci docker://ghcr.io/project-zot/golang:1.20 oci:${TEST_DATA_DIR}/golang:1.20
# Setup zot server
local zot_root_dir=${BATS_FILE_TMPDIR}/zot
local zot_config_file_dedupe=${BATS_FILE_TMPDIR}/zot_config_dedupe.json
local zot_config_file_nodedupe=${BATS_FILE_TMPDIR}/zot_config_nodedupe.json
local ZOT_LOG_FILE=${zot_root_dir}/zot-log.json
mkdir -p ${zot_root_dir}
cat > ${zot_config_file_dedupe}<<EOF
{
"distSpecVersion": "1.1.0-dev",
"storage": {
"rootDirectory": "${zot_root_dir}",
"dedupe": true,
"remoteCache": true,
"storageDriver": {
"name": "s3",
"rootdirectory": "/zot",
"region": "us-east-2",
"regionendpoint": "localhost:4566",
"bucket": "zot-storage",
"secure": false,
"skipverify": false
},
"cacheDriver": {
"name": "dynamodb",
"endpoint": "http://localhost:4566",
"region": "us-east-2",
"cacheTablename": "BlobTable"
}
},
"http": {
"address": "127.0.0.1",
"port": "8080"
},
"log": {
"level": "debug"
}
}
EOF
cat > ${zot_config_file_nodedupe}<<EOF
{
"distSpecVersion": "1.1.0-dev",
"storage": {
"rootDirectory": "${zot_root_dir}",
"dedupe": false,
"storageDriver": {
"name": "s3",
"rootdirectory": "/zot",
"region": "us-east-2",
"regionendpoint": "localhost:4566",
"bucket": "zot-storage",
"secure": false,
"skipverify": false
}
},
"http": {
"address": "127.0.0.1",
"port": "8080"
},
"log": {
"level": "debug",
"output": "${ZOT_LOG_FILE}"
}
}
EOF
awslocal s3 --region "us-east-2" mb s3://zot-storage
awslocal dynamodb --region "us-east-2" create-table --table-name "BlobTable" --attribute-definitions AttributeName=Digest,AttributeType=S --key-schema AttributeName=Digest,KeyType=HASH --provisioned-throughput ReadCapacityUnits=10,WriteCapacityUnits=5
zot_serve ${zot_config_file_dedupe}
wait_zot_reachable "http://127.0.0.1:8080/v2/_catalog"
}
function teardown_file() {
local zot_root_dir=${BATS_FILE_TMPDIR}/zot
zot_stop
rm -rf ${zot_root_dir}
awslocal s3 rb s3://"zot-storage" --force
awslocal dynamodb --region "us-east-2" delete-table --table-name "BlobTable"
}
@test "push 50 images with dedupe enabled" {
for i in {1..50}
do
run skopeo --insecure-policy copy --dest-tls-verify=false \
oci:${TEST_DATA_DIR}/golang:1.20 \
docker://127.0.0.1:8080/golang${i}:1.20
[ "$status" -eq 0 ]
done
}
@test "restart zot with dedupe false and wait for restore blobs task to finish" {
local zot_config_file_nodedupe=${BATS_FILE_TMPDIR}/zot_config_nodedupe.json
local zot_root_dir=${BATS_FILE_TMPDIR}/zot
local ZOT_LOG_FILE=${zot_root_dir}/zot-log.json
# stop server
zot_stop
sleep 10
# start with dedupe disabled
zot_serve ${zot_config_file_nodedupe}
wait_zot_reachable "http://127.0.0.1:8080/v2/"
start=`date +%s`
echo "waiting for restoring blobs task to finish" >&3
wait_for_string "dedupe rebuild: finished" ${ZOT_LOG_FILE}
end=`date +%s`
runtime=$((end-start))
echo "restoring blobs finished in $runtime sec" >&3
sleep 10 # wait a bit more because dedupe runs in background.
}
@test "pulling a previous deduped image should work" {
# golang1 should have original blobs already
echo "pulling first image" >&3
run skopeo --insecure-policy copy --src-tls-verify=false \
docker://127.0.0.1:8080/golang1:1.20 \
oci:${TEST_DATA_DIR}/golang1:1.20
[ "$status" -eq 0 ]
echo "pulling second image" >&3
# golang2 should have original blobs after restoring blobs
run skopeo --insecure-policy copy --src-tls-verify=false \
docker://127.0.0.1:8080/golang2:1.20 \
oci:${TEST_DATA_DIR}/golang2:1.20
[ "$status" -eq 0 ]
}

View file

@ -23,7 +23,8 @@ function setup() {
{
"distSpecVersion": "1.1.0",
"storage": {
"rootDirectory": "${ZOT_ROOT_DIR}"
"rootDirectory": "${ZOT_ROOT_DIR}",
"dedupe": false
},
"http": {
"address": "0.0.0.0",