mirror of
https://github.com/goharbor/harbor.git
synced 2025-01-12 19:00:50 +01:00
feat(repository): introduce cache manager for repository (#16741)
Implement cache layer for resource repository and migrate repository.Mgr to pkg.RepositoryMgr. Signed-off-by: chlins <chenyuzh@vmware.com>
This commit is contained in:
parent
f78047f169
commit
a82f1ba63d
@ -116,7 +116,7 @@ type Controller interface {
|
||||
func NewController() Controller {
|
||||
return &controller{
|
||||
tagCtl: tag.Ctl,
|
||||
repoMgr: repository.Mgr,
|
||||
repoMgr: pkg.RepositoryMgr,
|
||||
artMgr: pkg.ArtifactMgr,
|
||||
artrashMgr: artifactrash.Mgr,
|
||||
blobMgr: blob.Mgr,
|
||||
|
@ -27,7 +27,6 @@ import (
|
||||
"github.com/goharbor/harbor/src/pkg"
|
||||
"github.com/goharbor/harbor/src/pkg/artifact"
|
||||
_ "github.com/goharbor/harbor/src/pkg/config/db"
|
||||
repo "github.com/goharbor/harbor/src/pkg/repository"
|
||||
"github.com/goharbor/harbor/src/pkg/repository/model"
|
||||
"github.com/goharbor/harbor/src/pkg/tag"
|
||||
tagmodel "github.com/goharbor/harbor/src/pkg/tag/model/tag"
|
||||
@ -58,7 +57,7 @@ func (suite *ArtifactHandlerTestSuite) SetupSuite() {
|
||||
_, err := pkg.ArtifactMgr.Create(suite.ctx, &artifact.Artifact{ID: 1, RepositoryID: 1})
|
||||
suite.Nil(err)
|
||||
// mock repository
|
||||
_, err = repo.Mgr.Create(suite.ctx, &model.RepoRecord{RepositoryID: 1})
|
||||
_, err = pkg.RepositoryMgr.Create(suite.ctx, &model.RepoRecord{RepositoryID: 1})
|
||||
suite.Nil(err)
|
||||
// mock tag
|
||||
_, err = tag.Mgr.Create(suite.ctx, &tagmodel.Tag{ID: 1, RepositoryID: 1, ArtifactID: 1, Name: "latest"})
|
||||
@ -74,7 +73,7 @@ func (suite *ArtifactHandlerTestSuite) TearDownSuite() {
|
||||
err = pkg.ArtifactMgr.Delete(suite.ctx, 1)
|
||||
suite.Nil(err)
|
||||
// delete repository
|
||||
err = repo.Mgr.Delete(suite.ctx, 1)
|
||||
err = pkg.RepositoryMgr.Delete(suite.ctx, 1)
|
||||
suite.Nil(err)
|
||||
|
||||
}
|
||||
@ -113,7 +112,7 @@ func (suite *ArtifactHandlerTestSuite) TestOnPull() {
|
||||
suite.False(art.PullTime.IsZero(), "sync update pull_time")
|
||||
lastPullTime := art.PullTime
|
||||
// pull_count
|
||||
repository, err := repo.Mgr.Get(suite.ctx, 1)
|
||||
repository, err := pkg.RepositoryMgr.Get(suite.ctx, 1)
|
||||
suite.Nil(err)
|
||||
suite.Equal(int64(1), repository.PullCount, "sync update pull_count")
|
||||
|
||||
@ -127,7 +126,7 @@ func (suite *ArtifactHandlerTestSuite) TestOnPull() {
|
||||
suite.Nil(err)
|
||||
suite.Equal(lastPullTime, art.PullTime, "pull_time should not be updated immediately")
|
||||
// pull_count
|
||||
repository, err = repo.Mgr.Get(suite.ctx, 1)
|
||||
repository, err = pkg.RepositoryMgr.Get(suite.ctx, 1)
|
||||
suite.Nil(err)
|
||||
suite.Equal(int64(1), repository.PullCount, "pull_count should not be updated immediately")
|
||||
// wait for db update
|
||||
@ -138,7 +137,7 @@ func (suite *ArtifactHandlerTestSuite) TestOnPull() {
|
||||
}, 3*asyncFlushDuration, asyncFlushDuration/2, "wait for pull_time async update")
|
||||
|
||||
suite.Eventually(func() bool {
|
||||
repository, err = repo.Mgr.Get(suite.ctx, 1)
|
||||
repository, err = pkg.RepositoryMgr.Get(suite.ctx, 1)
|
||||
suite.Nil(err)
|
||||
return int64(2) == repository.PullCount
|
||||
}, 3*asyncFlushDuration, asyncFlushDuration/2, "wait for pull_count async update")
|
||||
|
@ -24,11 +24,11 @@ import (
|
||||
"github.com/goharbor/harbor/src/controller/project"
|
||||
"github.com/goharbor/harbor/src/lib/log"
|
||||
"github.com/goharbor/harbor/src/lib/orm"
|
||||
"github.com/goharbor/harbor/src/pkg"
|
||||
"github.com/goharbor/harbor/src/pkg/notification"
|
||||
"github.com/goharbor/harbor/src/pkg/notifier/model"
|
||||
notifyModel "github.com/goharbor/harbor/src/pkg/notifier/model"
|
||||
proModels "github.com/goharbor/harbor/src/pkg/project/models"
|
||||
"github.com/goharbor/harbor/src/pkg/repository"
|
||||
)
|
||||
|
||||
// Handler preprocess artifact event data
|
||||
@ -118,7 +118,7 @@ func (a *Handler) constructArtifactPayload(event *event.ArtifactEvent, project *
|
||||
}
|
||||
|
||||
ctx := orm.NewContext(context.Background(), beegorm.NewOrm())
|
||||
repoRecord, err := repository.Mgr.GetByName(ctx, repoName)
|
||||
repoRecord, err := pkg.RepositoryMgr.GetByName(ctx, repoName)
|
||||
if err != nil {
|
||||
log.Errorf("failed to get repository with name %s: %v", repoName, err)
|
||||
} else {
|
||||
|
@ -61,7 +61,7 @@ type Controller interface {
|
||||
func NewController() Controller {
|
||||
return &controller{
|
||||
proMgr: pkg.ProjectMgr,
|
||||
repoMgr: repository.Mgr,
|
||||
repoMgr: pkg.RepositoryMgr,
|
||||
artMgr: pkg.ArtifactMgr,
|
||||
artCtl: artifact.Ctl,
|
||||
}
|
||||
|
@ -391,14 +391,14 @@ func (r *defaultController) UpdateTaskInfo(ctx context.Context, taskID int64, to
|
||||
// NewController ...
|
||||
func NewController() Controller {
|
||||
retentionMgr := retention.NewManager()
|
||||
retentionLauncher := retention.NewLauncher(pkg.ProjectMgr, repository.Mgr, retentionMgr, task.ExecMgr, task.Mgr)
|
||||
retentionLauncher := retention.NewLauncher(pkg.ProjectMgr, pkg.RepositoryMgr, retentionMgr, task.ExecMgr, task.Mgr)
|
||||
return &defaultController{
|
||||
manager: retentionMgr,
|
||||
execMgr: task.ExecMgr,
|
||||
taskMgr: task.Mgr,
|
||||
launcher: retentionLauncher,
|
||||
projectManager: pkg.ProjectMgr,
|
||||
repositoryMgr: repository.Mgr,
|
||||
repositoryMgr: pkg.RepositoryMgr,
|
||||
scheduler: scheduler.Sched,
|
||||
}
|
||||
}
|
||||
|
@ -22,7 +22,6 @@ import (
|
||||
"github.com/goharbor/harbor/src/lib/q"
|
||||
"github.com/goharbor/harbor/src/pkg"
|
||||
"github.com/goharbor/harbor/src/pkg/artifact"
|
||||
"github.com/goharbor/harbor/src/pkg/repository"
|
||||
)
|
||||
|
||||
func abstractArtData(ctx context.Context) error {
|
||||
@ -32,7 +31,7 @@ func abstractArtData(ctx context.Context) error {
|
||||
return err
|
||||
}
|
||||
for _, pro := range pros {
|
||||
repos, err := repository.Mgr.List(ctx, &q.Query{
|
||||
repos, err := pkg.RepositoryMgr.List(ctx, &q.Query{
|
||||
Keywords: map[string]interface{}{
|
||||
"ProjectID": pro.ProjectID,
|
||||
},
|
||||
|
@ -27,6 +27,8 @@ const (
|
||||
ResourceTypeArtifact = "artifact"
|
||||
// ResourceTypeProject defines project type.
|
||||
ResourceTypeProject = "project"
|
||||
// ResourceTypeRepository defines repository type.
|
||||
ResourceTypeRepository = "repository"
|
||||
)
|
||||
|
||||
// Manager is the interface for resource cache manager.
|
||||
|
254
src/pkg/cached/repository/redis/manager.go
Normal file
254
src/pkg/cached/repository/redis/manager.go
Normal file
@ -0,0 +1,254 @@
|
||||
// Copyright Project Harbor 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 redis
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
libcache "github.com/goharbor/harbor/src/lib/cache"
|
||||
"github.com/goharbor/harbor/src/lib/config"
|
||||
"github.com/goharbor/harbor/src/lib/errors"
|
||||
"github.com/goharbor/harbor/src/lib/log"
|
||||
"github.com/goharbor/harbor/src/lib/q"
|
||||
"github.com/goharbor/harbor/src/lib/retry"
|
||||
"github.com/goharbor/harbor/src/pkg/cached"
|
||||
"github.com/goharbor/harbor/src/pkg/repository"
|
||||
"github.com/goharbor/harbor/src/pkg/repository/model"
|
||||
)
|
||||
|
||||
var _ CachedManager = &manager{}
|
||||
|
||||
// CachedManager is the interface combines raw resource manager and cached manager for better extension.
|
||||
type CachedManager interface {
|
||||
// Manager is the raw resource manager.
|
||||
repository.Manager
|
||||
// Manager is the common interface for resource cache.
|
||||
cached.Manager
|
||||
}
|
||||
|
||||
// manager is the cached manager implemented by redis.
|
||||
type manager struct {
|
||||
// delegator delegates the raw crud to DAO.
|
||||
delegator repository.Manager
|
||||
// client returns the redis cache client.
|
||||
client func() libcache.Cache
|
||||
// keyBuilder builds cache object key.
|
||||
keyBuilder *cached.ObjectKey
|
||||
// lifetime is the cache life time.
|
||||
lifetime time.Duration
|
||||
}
|
||||
|
||||
// NewManager returns the redis cache manager.
|
||||
func NewManager(m repository.Manager) *manager {
|
||||
return &manager{
|
||||
delegator: m,
|
||||
client: func() libcache.Cache { return libcache.Default() },
|
||||
keyBuilder: cached.NewObjectKey(cached.ResourceTypeRepository),
|
||||
lifetime: time.Duration(config.CacheExpireHours()) * time.Hour,
|
||||
}
|
||||
}
|
||||
|
||||
func (m *manager) Count(ctx context.Context, query *q.Query) (int64, error) {
|
||||
return m.delegator.Count(ctx, query)
|
||||
}
|
||||
|
||||
func (m *manager) List(ctx context.Context, query *q.Query) ([]*model.RepoRecord, error) {
|
||||
return m.delegator.List(ctx, query)
|
||||
}
|
||||
|
||||
func (m *manager) Create(ctx context.Context, repo *model.RepoRecord) (int64, error) {
|
||||
return m.delegator.Create(ctx, repo)
|
||||
}
|
||||
|
||||
func (m *manager) NonEmptyRepos(ctx context.Context) ([]*model.RepoRecord, error) {
|
||||
return m.delegator.NonEmptyRepos(ctx)
|
||||
}
|
||||
|
||||
func (m *manager) Get(ctx context.Context, id int64) (*model.RepoRecord, error) {
|
||||
key, err := m.keyBuilder.Format("id", id)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
repo := &model.RepoRecord{}
|
||||
if err = m.client().Fetch(ctx, key, repo); err == nil {
|
||||
return repo, nil
|
||||
}
|
||||
|
||||
log.Debugf("get repository %d from cache error: %v, will query from database.", id, err)
|
||||
|
||||
repo, err = m.delegator.Get(ctx, id)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err = m.client().Save(ctx, key, repo, m.lifetime); err != nil {
|
||||
// log error if save to cache failed
|
||||
log.Debugf("save repository %s to cache error: %v", repo.Name, err)
|
||||
}
|
||||
|
||||
return repo, nil
|
||||
}
|
||||
|
||||
func (m *manager) GetByName(ctx context.Context, name string) (*model.RepoRecord, error) {
|
||||
key, err := m.keyBuilder.Format("name", name)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
repo := &model.RepoRecord{}
|
||||
if err = m.client().Fetch(ctx, key, repo); err == nil {
|
||||
return repo, nil
|
||||
}
|
||||
|
||||
repo, err = m.delegator.GetByName(ctx, name)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err = m.client().Save(ctx, key, repo, m.lifetime); err != nil {
|
||||
// log error if save to cache failed
|
||||
log.Debugf("save repository %s to cache error: %v", repo.Name, err)
|
||||
}
|
||||
|
||||
return repo, nil
|
||||
}
|
||||
|
||||
func (m *manager) Delete(ctx context.Context, id int64) error {
|
||||
repo, err := m.Get(ctx, id)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// pass on delete operation
|
||||
if err := m.delegator.Delete(ctx, id); err != nil {
|
||||
return err
|
||||
}
|
||||
// clean cache
|
||||
m.cleanUp(ctx, repo)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *manager) Update(ctx context.Context, repo *model.RepoRecord, props ...string) error {
|
||||
// pass on update operation
|
||||
if err := m.delegator.Update(ctx, repo, props...); err != nil {
|
||||
return err
|
||||
}
|
||||
// clean cache
|
||||
m.cleanUp(ctx, repo)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *manager) AddPullCount(ctx context.Context, id int64, count uint64) error {
|
||||
repo, err := m.Get(ctx, id)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// pass on update operation
|
||||
if err = m.delegator.AddPullCount(ctx, id, count); err != nil {
|
||||
return err
|
||||
}
|
||||
// refresh cache
|
||||
m.refreshCache(ctx, repo)
|
||||
return nil
|
||||
}
|
||||
|
||||
// cleanUp cleans up data in cache.
|
||||
func (m *manager) cleanUp(ctx context.Context, repo *model.RepoRecord) {
|
||||
// clean index by id
|
||||
idIdx, err := m.keyBuilder.Format("id", repo.RepositoryID)
|
||||
if err != nil {
|
||||
log.Errorf("format repository id key error: %v", err)
|
||||
} else {
|
||||
// retry to avoid dirty data
|
||||
if err = retry.Retry(func() error { return m.client().Delete(ctx, idIdx) }); err != nil {
|
||||
log.Errorf("delete repository cache key %s error: %v", idIdx, err)
|
||||
}
|
||||
}
|
||||
|
||||
// clean index by name
|
||||
nameIdx, err := m.keyBuilder.Format("name", repo.Name)
|
||||
if err != nil {
|
||||
log.Errorf("format repository name key error: %v", err)
|
||||
} else {
|
||||
if err = retry.Retry(func() error { return m.client().Delete(ctx, nameIdx) }); err != nil {
|
||||
log.Errorf("delete repository cache key %s error: %v", nameIdx, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// refreshCache refreshes cache.
|
||||
func (m *manager) refreshCache(ctx context.Context, repo *model.RepoRecord) {
|
||||
// refreshCache used for AddPullCount, because we have a background goroutine to
|
||||
// update per repo's pull_count in period time, in that case, we don't want to lose
|
||||
// cache every fixed interval, so prefer to use refreshCache instead of cleanUp.
|
||||
// no need to consider lock because we only have one goroutine do this work one by one.
|
||||
|
||||
// refreshCache includes 2 steps:
|
||||
// 1. cleanUp
|
||||
// 2. re-get
|
||||
m.cleanUp(ctx, repo)
|
||||
|
||||
var err error
|
||||
// re-get by id
|
||||
_, err = m.Get(ctx, repo.RepositoryID)
|
||||
if err != nil {
|
||||
log.Errorf("refresh cache by repository id %d error: %v", repo.RepositoryID, err)
|
||||
}
|
||||
// re-get by name
|
||||
_, err = m.GetByName(ctx, repo.Name)
|
||||
if err != nil {
|
||||
log.Errorf("refresh cache by repository name %s error: %v", repo.Name, err)
|
||||
}
|
||||
}
|
||||
|
||||
func (m *manager) ResourceType(ctx context.Context) string {
|
||||
return cached.ResourceTypeRepository
|
||||
}
|
||||
|
||||
func (m *manager) CountCache(ctx context.Context) (int64, error) {
|
||||
// prefix is resource type
|
||||
keys, err := m.client().Keys(ctx, m.ResourceType(ctx))
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
return int64(len(keys)), nil
|
||||
}
|
||||
|
||||
func (m *manager) DeleteCache(ctx context.Context, key string) error {
|
||||
return m.client().Delete(ctx, key)
|
||||
}
|
||||
|
||||
func (m *manager) FlushAll(ctx context.Context) error {
|
||||
// prefix is resource type
|
||||
keys, err := m.client().Keys(ctx, m.ResourceType(ctx))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
var errs errors.Errors
|
||||
for _, key := range keys {
|
||||
if err = m.client().Delete(ctx, key); err != nil {
|
||||
errs = append(errs, err)
|
||||
}
|
||||
}
|
||||
|
||||
if errs.Len() > 0 {
|
||||
return errs
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
191
src/pkg/cached/repository/redis/manager_test.go
Normal file
191
src/pkg/cached/repository/redis/manager_test.go
Normal file
@ -0,0 +1,191 @@
|
||||
// Copyright Project Harbor 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 redis
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"testing"
|
||||
|
||||
"github.com/goharbor/harbor/src/lib/cache"
|
||||
"github.com/goharbor/harbor/src/pkg/repository/model"
|
||||
testcache "github.com/goharbor/harbor/src/testing/lib/cache"
|
||||
"github.com/goharbor/harbor/src/testing/mock"
|
||||
testRepo "github.com/goharbor/harbor/src/testing/pkg/repository"
|
||||
"github.com/stretchr/testify/suite"
|
||||
)
|
||||
|
||||
type managerTestSuite struct {
|
||||
suite.Suite
|
||||
cachedManager CachedManager
|
||||
repoMgr *testRepo.Manager
|
||||
cache *testcache.Cache
|
||||
ctx context.Context
|
||||
}
|
||||
|
||||
func (m *managerTestSuite) SetupTest() {
|
||||
m.repoMgr = &testRepo.Manager{}
|
||||
m.cache = &testcache.Cache{}
|
||||
m.cachedManager = NewManager(
|
||||
m.repoMgr,
|
||||
)
|
||||
m.cachedManager.(*manager).client = func() cache.Cache { return m.cache }
|
||||
m.ctx = context.TODO()
|
||||
}
|
||||
|
||||
func (m *managerTestSuite) TestGet() {
|
||||
// get from cache directly
|
||||
m.cache.On("Fetch", mock.Anything, mock.Anything, mock.Anything).Return(nil).Once()
|
||||
_, err := m.cachedManager.Get(m.ctx, 100)
|
||||
m.NoError(err, "should get from cache")
|
||||
m.repoMgr.AssertNotCalled(m.T(), "Get", mock.Anything, mock.Anything)
|
||||
|
||||
// not found in cache, read from dao
|
||||
m.cache.On("Fetch", mock.Anything, mock.Anything, mock.Anything).Return(cache.ErrNotFound).Once()
|
||||
m.cache.On("Save", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil).Once()
|
||||
m.repoMgr.On("Get", mock.Anything, mock.Anything).Return(&model.RepoRecord{}, nil).Once()
|
||||
_, err = m.cachedManager.Get(m.ctx, 100)
|
||||
m.NoError(err, "should get from repoMgr")
|
||||
m.repoMgr.AssertCalled(m.T(), "Get", mock.Anything, mock.Anything)
|
||||
}
|
||||
|
||||
func (m *managerTestSuite) TestGetByName() {
|
||||
// get from cache directly
|
||||
m.cache.On("Fetch", mock.Anything, mock.Anything, mock.Anything).Return(nil).Once()
|
||||
_, err := m.cachedManager.GetByName(m.ctx, "repo")
|
||||
m.NoError(err, "should get from cache")
|
||||
m.repoMgr.AssertNotCalled(m.T(), "Get", mock.Anything, mock.Anything)
|
||||
|
||||
// not found in cache, read from dao
|
||||
m.cache.On("Fetch", mock.Anything, mock.Anything, mock.Anything).Return(cache.ErrNotFound).Once()
|
||||
m.cache.On("Save", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil).Once()
|
||||
m.repoMgr.On("GetByName", mock.Anything, mock.Anything).Return(&model.RepoRecord{}, nil).Once()
|
||||
_, err = m.cachedManager.GetByName(m.ctx, "repo")
|
||||
m.NoError(err, "should get from repoMgr")
|
||||
m.repoMgr.AssertCalled(m.T(), "GetByName", mock.Anything, mock.Anything)
|
||||
}
|
||||
|
||||
func (m *managerTestSuite) TestDelete() {
|
||||
// delete from repoMgr error
|
||||
errDelete := errors.New("delete failed")
|
||||
m.repoMgr.On("Delete", mock.Anything, mock.Anything).Return(errDelete).Once()
|
||||
m.cache.On("Fetch", mock.Anything, mock.Anything, mock.Anything).Return(nil).Once()
|
||||
err := m.cachedManager.Delete(m.ctx, 100)
|
||||
m.ErrorIs(err, errDelete, "delete should error")
|
||||
m.cache.AssertNotCalled(m.T(), "Delete", mock.Anything, mock.Anything)
|
||||
|
||||
// delete from repoMgr success
|
||||
m.repoMgr.On("Delete", mock.Anything, mock.Anything).Return(nil).Once()
|
||||
m.cache.On("Fetch", mock.Anything, mock.Anything, mock.Anything).Return(nil).Once()
|
||||
m.cache.On("Delete", mock.Anything, mock.Anything).Return(nil).Twice()
|
||||
err = m.cachedManager.Delete(m.ctx, 100)
|
||||
m.NoError(err, "delete should success")
|
||||
m.cache.AssertCalled(m.T(), "Delete", mock.Anything, mock.Anything)
|
||||
}
|
||||
|
||||
func (m *managerTestSuite) TestUpdate() {
|
||||
// update from repoMgr error
|
||||
errUpdate := errors.New("update failed")
|
||||
m.repoMgr.On("Update", mock.Anything, mock.Anything).Return(errUpdate).Once()
|
||||
err := m.cachedManager.Update(m.ctx, &model.RepoRecord{})
|
||||
m.ErrorIs(err, errUpdate, "update should error")
|
||||
m.cache.AssertNotCalled(m.T(), "Delete", mock.Anything, mock.Anything)
|
||||
|
||||
// update from repoMgr success
|
||||
m.repoMgr.On("Update", mock.Anything, mock.Anything).Return(nil).Once()
|
||||
m.cache.On("Delete", mock.Anything, mock.Anything).Return(nil).Twice()
|
||||
err = m.cachedManager.Update(m.ctx, &model.RepoRecord{})
|
||||
m.NoError(err, "update should success")
|
||||
m.cache.AssertCalled(m.T(), "Delete", mock.Anything, mock.Anything)
|
||||
}
|
||||
|
||||
func (m *managerTestSuite) TestAddPullCount() {
|
||||
// update pull count from repoMgr error
|
||||
errUpdate := errors.New("update pull count failed")
|
||||
m.repoMgr.On("AddPullCount", mock.Anything, mock.Anything, mock.Anything).Return(errUpdate).Once()
|
||||
m.cache.On("Fetch", mock.Anything, mock.Anything, mock.Anything).Return(nil)
|
||||
err := m.cachedManager.AddPullCount(m.ctx, 100, 1)
|
||||
m.ErrorIs(err, errUpdate, "update pull count should error")
|
||||
m.cache.AssertNotCalled(m.T(), "Delete", mock.Anything, mock.Anything)
|
||||
|
||||
// update pull count from repoMgr success
|
||||
m.repoMgr.On("AddPullCount", mock.Anything, mock.Anything, mock.Anything).Return(nil).Once()
|
||||
m.cache.On("Delete", mock.Anything, mock.Anything).Return(nil).Twice()
|
||||
err = m.cachedManager.AddPullCount(m.ctx, 100, 1)
|
||||
m.NoError(err, "update pull count should success")
|
||||
m.cache.AssertCalled(m.T(), "Delete", mock.Anything, mock.Anything)
|
||||
}
|
||||
|
||||
func (m *managerTestSuite) TestCount() {
|
||||
m.repoMgr.On("Count", mock.Anything, mock.Anything).Return(int64(1), nil)
|
||||
c, err := m.cachedManager.Count(m.ctx, nil)
|
||||
m.NoError(err)
|
||||
m.Equal(int64(1), c)
|
||||
m.repoMgr.AssertCalled(m.T(), "Count", mock.Anything, mock.Anything)
|
||||
}
|
||||
|
||||
func (m *managerTestSuite) TestList() {
|
||||
repos := []*model.RepoRecord{}
|
||||
m.repoMgr.On("List", mock.Anything, mock.Anything).Return(repos, nil)
|
||||
as, err := m.cachedManager.List(m.ctx, nil)
|
||||
m.NoError(err)
|
||||
m.Equal(repos, as)
|
||||
m.repoMgr.AssertCalled(m.T(), "List", mock.Anything, mock.Anything)
|
||||
}
|
||||
|
||||
func (m *managerTestSuite) TestCreate() {
|
||||
m.repoMgr.On("Create", mock.Anything, mock.Anything).Return(int64(1), nil)
|
||||
id, err := m.cachedManager.Create(m.ctx, nil)
|
||||
m.NoError(err)
|
||||
m.Equal(int64(1), id)
|
||||
m.repoMgr.AssertCalled(m.T(), "Create", mock.Anything, mock.Anything)
|
||||
}
|
||||
|
||||
func (m *managerTestSuite) TestNonEmptyRepos() {
|
||||
repos := []*model.RepoRecord{}
|
||||
m.repoMgr.On("NonEmptyRepos", mock.Anything, mock.Anything).Return(repos, nil)
|
||||
rs, err := m.cachedManager.NonEmptyRepos(m.ctx)
|
||||
m.NoError(err)
|
||||
m.Equal(repos, rs)
|
||||
m.repoMgr.AssertCalled(m.T(), "NonEmptyRepos", mock.Anything, mock.Anything)
|
||||
}
|
||||
func (m *managerTestSuite) TestResourceType() {
|
||||
t := m.cachedManager.ResourceType(m.ctx)
|
||||
m.Equal("repository", t)
|
||||
}
|
||||
|
||||
func (m *managerTestSuite) TestCountCache() {
|
||||
m.cache.On("Keys", mock.Anything, mock.Anything).Return([]string{"1"}, nil).Once()
|
||||
c, err := m.cachedManager.CountCache(m.ctx)
|
||||
m.NoError(err)
|
||||
m.Equal(int64(1), c)
|
||||
}
|
||||
|
||||
func (m *managerTestSuite) TestDeleteCache() {
|
||||
m.cache.On("Delete", mock.Anything, mock.Anything).Return(nil).Once()
|
||||
err := m.cachedManager.DeleteCache(m.ctx, "key")
|
||||
m.NoError(err)
|
||||
}
|
||||
|
||||
func (m *managerTestSuite) TestFlushAll() {
|
||||
m.cache.On("Keys", mock.Anything, mock.Anything).Return([]string{"1"}, nil).Once()
|
||||
m.cache.On("Delete", mock.Anything, mock.Anything).Return(nil).Once()
|
||||
err := m.cachedManager.FlushAll(m.ctx)
|
||||
m.NoError(err)
|
||||
}
|
||||
|
||||
func TestManager(t *testing.T) {
|
||||
suite.Run(t, &managerTestSuite{})
|
||||
}
|
@ -21,7 +21,6 @@ import (
|
||||
"github.com/goharbor/harbor/src/pkg/member"
|
||||
memberModels "github.com/goharbor/harbor/src/pkg/member/models"
|
||||
qtypes "github.com/goharbor/harbor/src/pkg/quota/types"
|
||||
"github.com/goharbor/harbor/src/pkg/repository"
|
||||
"github.com/goharbor/harbor/src/pkg/repository/model"
|
||||
"github.com/goharbor/harbor/src/pkg/user"
|
||||
)
|
||||
@ -80,13 +79,13 @@ func setupTest(t *testing.T) {
|
||||
|
||||
// Add repo to project
|
||||
repo1.ProjectID = testPro1.ProjectID
|
||||
repo1ID, err := repository.Mgr.Create(ctx, &repo1)
|
||||
repo1ID, err := pkg.RepositoryMgr.Create(ctx, &repo1)
|
||||
if err != nil {
|
||||
t.Errorf("add repo error %v", err)
|
||||
}
|
||||
repo1.RepositoryID = repo1ID
|
||||
repo2.ProjectID = testPro2.ProjectID
|
||||
repo2ID, err := repository.Mgr.Create(ctx, &repo2)
|
||||
repo2ID, err := pkg.RepositoryMgr.Create(ctx, &repo2)
|
||||
repo2.RepositoryID = repo2ID
|
||||
if err != nil {
|
||||
t.Errorf("add repo error %v", err)
|
||||
|
@ -19,7 +19,9 @@ import (
|
||||
"github.com/goharbor/harbor/src/pkg/artifact"
|
||||
cachedArtifact "github.com/goharbor/harbor/src/pkg/cached/artifact/redis"
|
||||
cachedProject "github.com/goharbor/harbor/src/pkg/cached/project/redis"
|
||||
cachedRepo "github.com/goharbor/harbor/src/pkg/cached/repository/redis"
|
||||
"github.com/goharbor/harbor/src/pkg/project"
|
||||
"github.com/goharbor/harbor/src/pkg/repository"
|
||||
)
|
||||
|
||||
// Define global resource manager.
|
||||
@ -28,6 +30,8 @@ var (
|
||||
ArtifactMgr artifact.Manager
|
||||
// ProjectMgr is the manager for project.
|
||||
ProjectMgr project.Manager
|
||||
// RepositoryMgr is the manager for repository.
|
||||
RepositoryMgr repository.Manager
|
||||
)
|
||||
|
||||
// init initialize mananger for resources
|
||||
@ -35,7 +39,7 @@ func init() {
|
||||
cacheEnabled := config.CacheEnabled()
|
||||
initArtifactMgr(cacheEnabled)
|
||||
initProjectMgr(cacheEnabled)
|
||||
|
||||
initRepositoryMgr(cacheEnabled)
|
||||
}
|
||||
|
||||
func initArtifactMgr(cacheEnabled bool) {
|
||||
@ -57,3 +61,12 @@ func initProjectMgr(cacheEnabled bool) {
|
||||
ProjectMgr = projectMgr
|
||||
}
|
||||
}
|
||||
|
||||
func initRepositoryMgr(cacheEnabled bool) {
|
||||
repoMgr := repository.New()
|
||||
if cacheEnabled {
|
||||
RepositoryMgr = cachedRepo.NewManager(repoMgr)
|
||||
} else {
|
||||
RepositoryMgr = repoMgr
|
||||
}
|
||||
}
|
||||
|
@ -20,7 +20,9 @@ import (
|
||||
"github.com/goharbor/harbor/src/pkg/artifact"
|
||||
cachedArtifact "github.com/goharbor/harbor/src/pkg/cached/artifact/redis"
|
||||
cachedProject "github.com/goharbor/harbor/src/pkg/cached/project/redis"
|
||||
cachedRepo "github.com/goharbor/harbor/src/pkg/cached/repository/redis"
|
||||
"github.com/goharbor/harbor/src/pkg/project"
|
||||
"github.com/goharbor/harbor/src/pkg/repository"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
@ -45,3 +47,14 @@ func TestInitProjectMgr(t *testing.T) {
|
||||
assert.NotNil(t, ProjectMgr)
|
||||
assert.IsType(t, cachedProject.NewManager(project.New()), ProjectMgr)
|
||||
}
|
||||
|
||||
func TestInitRepositoryMgr(t *testing.T) {
|
||||
// cache not enable
|
||||
assert.NotNil(t, RepositoryMgr)
|
||||
assert.IsType(t, repository.New(), RepositoryMgr)
|
||||
|
||||
// cache enable
|
||||
initRepositoryMgr(true)
|
||||
assert.NotNil(t, RepositoryMgr)
|
||||
assert.IsType(t, cachedRepo.NewManager(repository.New()), RepositoryMgr)
|
||||
}
|
||||
|
@ -23,9 +23,6 @@ import (
|
||||
"github.com/goharbor/harbor/src/pkg/repository/model"
|
||||
)
|
||||
|
||||
// Mgr is the global repository manager instance
|
||||
var Mgr = New()
|
||||
|
||||
// Manager is used for repository management
|
||||
type Manager interface {
|
||||
// Count returns the total count of repositories according to the query
|
||||
|
@ -19,7 +19,6 @@ import (
|
||||
internal_orm "github.com/goharbor/harbor/src/lib/orm"
|
||||
"github.com/goharbor/harbor/src/pkg/artifact"
|
||||
immu_model "github.com/goharbor/harbor/src/pkg/immutable/model"
|
||||
"github.com/goharbor/harbor/src/pkg/repository"
|
||||
"github.com/goharbor/harbor/src/pkg/repository/model"
|
||||
"github.com/goharbor/harbor/src/pkg/tag"
|
||||
tag_model "github.com/goharbor/harbor/src/pkg/tag/model/tag"
|
||||
@ -103,7 +102,7 @@ func (suite *HandlerSuite) addRepo(ctx context.Context, pid int64, repo string)
|
||||
Name: repo,
|
||||
ProjectID: pid,
|
||||
}
|
||||
repoid, err := repository.Mgr.Create(ctx, repoRec)
|
||||
repoid, err := pkg.RepositoryMgr.Create(ctx, repoRec)
|
||||
suite.Nil(err, fmt.Sprintf("Add repository failed for %s", repo))
|
||||
return repoid
|
||||
}
|
||||
@ -164,7 +163,7 @@ func (suite *HandlerSuite) TestPutDeleteManifestCreated() {
|
||||
defer func() {
|
||||
pkg.ProjectMgr.Delete(ctx, projectID)
|
||||
pkg.ArtifactMgr.Delete(ctx, afID)
|
||||
repository.Mgr.Delete(ctx, repoID)
|
||||
pkg.RepositoryMgr.Delete(ctx, repoID)
|
||||
tag.Mgr.Delete(ctx, tagID)
|
||||
immutable.Ctr.DeleteImmutableRule(internal_orm.Context(), immuRuleID)
|
||||
}()
|
||||
|
@ -17,18 +17,20 @@ package registry
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"github.com/goharbor/harbor/src/lib/errors"
|
||||
lib_http "github.com/goharbor/harbor/src/lib/http"
|
||||
"github.com/goharbor/harbor/src/pkg/repository"
|
||||
"github.com/goharbor/harbor/src/server/registry/util"
|
||||
"net/http"
|
||||
"sort"
|
||||
"strconv"
|
||||
|
||||
"github.com/goharbor/harbor/src/lib/errors"
|
||||
lib_http "github.com/goharbor/harbor/src/lib/http"
|
||||
"github.com/goharbor/harbor/src/pkg"
|
||||
"github.com/goharbor/harbor/src/pkg/repository"
|
||||
"github.com/goharbor/harbor/src/server/registry/util"
|
||||
)
|
||||
|
||||
func newRepositoryHandler() http.Handler {
|
||||
return &repositoryHandler{
|
||||
repoMgr: repository.Mgr,
|
||||
repoMgr: pkg.RepositoryMgr,
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -16,14 +16,16 @@ package registry
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"testing"
|
||||
|
||||
"github.com/goharbor/harbor/src/pkg"
|
||||
"github.com/goharbor/harbor/src/pkg/repository"
|
||||
"github.com/goharbor/harbor/src/pkg/repository/model"
|
||||
"github.com/goharbor/harbor/src/testing/mock"
|
||||
repotesting "github.com/goharbor/harbor/src/testing/pkg/repository"
|
||||
"github.com/stretchr/testify/suite"
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"testing"
|
||||
)
|
||||
|
||||
type catalogTestSuite struct {
|
||||
@ -33,19 +35,19 @@ type catalogTestSuite struct {
|
||||
}
|
||||
|
||||
func (c *catalogTestSuite) SetupSuite() {
|
||||
c.originalRepoMgr = repository.Mgr
|
||||
c.originalRepoMgr = pkg.RepositoryMgr
|
||||
}
|
||||
|
||||
func (c *catalogTestSuite) SetupTest() {
|
||||
c.repoMgr = &repotesting.Manager{}
|
||||
repository.Mgr = c.repoMgr
|
||||
pkg.RepositoryMgr = c.repoMgr
|
||||
}
|
||||
|
||||
func (c *catalogTestSuite) TearDownTest() {
|
||||
}
|
||||
|
||||
func (c *catalogTestSuite) TearDownSuite() {
|
||||
repository.Mgr = c.originalRepoMgr
|
||||
pkg.RepositoryMgr = c.originalRepoMgr
|
||||
}
|
||||
|
||||
func (c *catalogTestSuite) TestCatalog() {
|
||||
|
Loading…
Reference in New Issue
Block a user