mirror of
https://github.com/goharbor/harbor.git
synced 2025-01-23 16:11:24 +01:00
refactor(scan): refactor scan/scan all job to task manager (#13684)
Signed-off-by: He Weiwei <hweiwei@vmware.com>
This commit is contained in:
parent
90f9bea965
commit
08580f9fec
@ -268,7 +268,18 @@ BEGIN
|
||||
UPDATE scanner_registration SET is_default = TRUE WHERE name = 'Trivy' AND immutable = TRUE;
|
||||
END IF;
|
||||
END $$;
|
||||
|
||||
ALTER TABLE execution ALTER COLUMN vendor_type type varchar(64);
|
||||
ALTER TABLE schedule ALTER COLUMN vendor_type type varchar(64);
|
||||
ALTER TABLE schedule ADD COLUMN IF NOT EXISTS extra_attrs JSON;
|
||||
ALTER TABLE task ALTER COLUMN vendor_type type varchar(64);
|
||||
|
||||
/* Remove these columns in scan_report because execution-task pattern will handle them */
|
||||
ALTER TABLE scan_report DROP COLUMN IF EXISTS job_id;
|
||||
ALTER TABLE scan_report DROP COLUMN IF EXISTS track_id;
|
||||
ALTER TABLE scan_report DROP COLUMN IF EXISTS requester;
|
||||
ALTER TABLE scan_report DROP COLUMN IF EXISTS status;
|
||||
ALTER TABLE scan_report DROP COLUMN IF EXISTS status_code;
|
||||
ALTER TABLE scan_report DROP COLUMN IF EXISTS status_rev;
|
||||
ALTER TABLE scan_report DROP COLUMN IF EXISTS start_time;
|
||||
ALTER TABLE scan_report DROP COLUMN IF EXISTS end_time;
|
||||
|
56
src/controller/artifact/helper.go
Normal file
56
src/controller/artifact/helper.go
Normal file
@ -0,0 +1,56 @@
|
||||
// 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 artifact
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/goharbor/harbor/src/lib/log"
|
||||
"github.com/goharbor/harbor/src/lib/q"
|
||||
)
|
||||
|
||||
// Iterator returns the iterator to fetch all artifacts with query
|
||||
func Iterator(ctx context.Context, chunkSize int, query *q.Query, option *Option) <-chan *Artifact {
|
||||
ch := make(chan *Artifact, chunkSize)
|
||||
|
||||
go func() {
|
||||
defer close(ch)
|
||||
|
||||
clone := q.MustClone(query)
|
||||
clone.PageNumber = 1
|
||||
clone.PageSize = int64(chunkSize)
|
||||
|
||||
for {
|
||||
artifacts, err := Ctl.List(ctx, clone, option)
|
||||
if err != nil {
|
||||
log.G(ctx).Errorf("list artifacts failed, error: %v", err)
|
||||
return
|
||||
}
|
||||
|
||||
for _, artifact := range artifacts {
|
||||
ch <- artifact
|
||||
}
|
||||
|
||||
if len(artifacts) < chunkSize {
|
||||
break
|
||||
}
|
||||
|
||||
query.PageNumber++
|
||||
}
|
||||
|
||||
}()
|
||||
|
||||
return ch
|
||||
}
|
64
src/controller/artifact/helper_test.go
Normal file
64
src/controller/artifact/helper_test.go
Normal file
@ -0,0 +1,64 @@
|
||||
// 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 artifact
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/goharbor/harbor/src/pkg/artifact"
|
||||
artifacttesting "github.com/goharbor/harbor/src/testing/pkg/artifact"
|
||||
"github.com/stretchr/testify/suite"
|
||||
)
|
||||
|
||||
type IteratorTestSuite struct {
|
||||
suite.Suite
|
||||
|
||||
artMgr *artifacttesting.FakeManager
|
||||
|
||||
ctl *controller
|
||||
originalCtl Controller
|
||||
}
|
||||
|
||||
func (suite *IteratorTestSuite) SetupSuite() {
|
||||
suite.artMgr = &artifacttesting.FakeManager{}
|
||||
|
||||
suite.originalCtl = Ctl
|
||||
suite.ctl = &controller{artMgr: suite.artMgr}
|
||||
Ctl = suite.ctl
|
||||
}
|
||||
|
||||
func (suite *IteratorTestSuite) TeardownSuite() {
|
||||
Ctl = suite.originalCtl
|
||||
}
|
||||
|
||||
func (suite *IteratorTestSuite) TestIterator() {
|
||||
suite.artMgr.On("List").Return([]*artifact.Artifact{
|
||||
{ID: 1},
|
||||
{ID: 2},
|
||||
{ID: 3},
|
||||
}, nil)
|
||||
|
||||
var artifacts []*Artifact
|
||||
for art := range Iterator(context.TODO(), 5, nil, nil) {
|
||||
artifacts = append(artifacts, art)
|
||||
}
|
||||
|
||||
suite.Len(artifacts, 3)
|
||||
}
|
||||
|
||||
func TestIteratorTestSuite(t *testing.T) {
|
||||
suite.Run(t, &IteratorTestSuite{})
|
||||
}
|
@ -2,6 +2,7 @@ package handler
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/goharbor/harbor/src/controller/event"
|
||||
"github.com/goharbor/harbor/src/controller/event/handler/auditlog"
|
||||
"github.com/goharbor/harbor/src/controller/event/handler/internal"
|
||||
@ -31,7 +32,7 @@ func init() {
|
||||
notifier.Subscribe(event.TopicQuotaWarning, "a.Handler{})
|
||||
notifier.Subscribe(event.TopicScanningFailed, &scan.Handler{})
|
||||
notifier.Subscribe(event.TopicScanningCompleted, &scan.Handler{})
|
||||
notifier.Subscribe(event.TopicDeleteArtifact, &scan.DelArtHandler{})
|
||||
notifier.Subscribe(event.TopicDeleteArtifact, &scan.DelArtHandler{Context: orm.Context})
|
||||
notifier.Subscribe(event.TopicReplication, &artifact.ReplicationHandler{})
|
||||
notifier.Subscribe(event.TopicTagRetention, &artifact.RetentionHandler{RetentionController: artifact.DefaultRetentionControllerFunc})
|
||||
|
||||
|
@ -16,18 +16,18 @@ package scan
|
||||
|
||||
import (
|
||||
"context"
|
||||
bo "github.com/astaxie/beego/orm"
|
||||
|
||||
"github.com/goharbor/harbor/src/controller/artifact"
|
||||
"github.com/goharbor/harbor/src/controller/event"
|
||||
"github.com/goharbor/harbor/src/controller/scan"
|
||||
"github.com/goharbor/harbor/src/lib/errors"
|
||||
"github.com/goharbor/harbor/src/lib/log"
|
||||
"github.com/goharbor/harbor/src/lib/orm"
|
||||
"github.com/goharbor/harbor/src/lib/q"
|
||||
)
|
||||
|
||||
// DelArtHandler is a handler to listen to the internal delete image event.
|
||||
type DelArtHandler struct {
|
||||
Context func() context.Context
|
||||
}
|
||||
|
||||
// Name ...
|
||||
@ -48,32 +48,20 @@ func (o *DelArtHandler) Handle(value interface{}) error {
|
||||
|
||||
log.Debugf("clear the scan reports as receiving event %s", evt.EventType)
|
||||
|
||||
digests := make([]string, 0)
|
||||
query := &q.Query{
|
||||
Keywords: make(map[string]interface{}),
|
||||
}
|
||||
|
||||
ctx := orm.NewContext(context.TODO(), bo.NewOrm())
|
||||
ctx := o.Context()
|
||||
// Check if it is safe to delete the reports.
|
||||
query.Keywords["digest"] = evt.Artifact.Digest
|
||||
l, err := artifact.Ctl.List(ctx, query, nil)
|
||||
|
||||
if err != nil && len(l) != 0 {
|
||||
count, err := artifact.Ctl.Count(ctx, q.New(q.KeyWords{"digest": evt.Artifact.Digest}))
|
||||
if err != nil {
|
||||
// Just logged
|
||||
log.Error(errors.Wrap(err, "delete image event handler"))
|
||||
// Passed for safe consideration
|
||||
} else {
|
||||
if len(l) == 0 {
|
||||
digests = append(digests, evt.Artifact.Digest)
|
||||
log.Debugf("prepare to remove the scan report linked with artifact: %s", evt.Artifact.Digest)
|
||||
} else if count == 0 {
|
||||
log.Debugf("prepare to remove the scan report linked with artifact: %s", evt.Artifact.Digest)
|
||||
|
||||
if err := scan.DefaultController.DeleteReports(ctx, evt.Artifact.Digest); err != nil {
|
||||
return errors.Wrap(err, "delete image event handler")
|
||||
}
|
||||
}
|
||||
|
||||
if err := scan.DefaultController.DeleteReports(digests...); err != nil {
|
||||
return errors.Wrap(err, "delete image event handler")
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
|
86
src/controller/event/handler/webhook/scan/delete_test.go
Normal file
86
src/controller/event/handler/webhook/scan/delete_test.go
Normal file
@ -0,0 +1,86 @@
|
||||
// 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 scan
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/goharbor/harbor/src/controller/artifact"
|
||||
"github.com/goharbor/harbor/src/controller/event"
|
||||
"github.com/goharbor/harbor/src/controller/scan"
|
||||
"github.com/goharbor/harbor/src/lib/q"
|
||||
artifacttesting "github.com/goharbor/harbor/src/testing/controller/artifact"
|
||||
scantesting "github.com/goharbor/harbor/src/testing/controller/scan"
|
||||
"github.com/stretchr/testify/suite"
|
||||
)
|
||||
|
||||
type DelArtHandlerTestSuite struct {
|
||||
suite.Suite
|
||||
|
||||
artifactCtl *artifacttesting.Controller
|
||||
originalArtifactCtl artifact.Controller
|
||||
|
||||
scanCtl *scantesting.Controller
|
||||
originalScanCtl scan.Controller
|
||||
}
|
||||
|
||||
func (suite *DelArtHandlerTestSuite) SetupSuite() {
|
||||
suite.artifactCtl = &artifacttesting.Controller{}
|
||||
suite.originalArtifactCtl = artifact.Ctl
|
||||
artifact.Ctl = suite.artifactCtl
|
||||
|
||||
suite.scanCtl = &scantesting.Controller{}
|
||||
suite.originalScanCtl = scan.DefaultController
|
||||
scan.DefaultController = suite.scanCtl
|
||||
}
|
||||
|
||||
func (suite *DelArtHandlerTestSuite) TeardownSuite() {
|
||||
artifact.Ctl = suite.originalArtifactCtl
|
||||
scan.DefaultController = suite.originalScanCtl
|
||||
}
|
||||
|
||||
func (suite *DelArtHandlerTestSuite) TestHandle() {
|
||||
o := DelArtHandler{Context: context.TODO}
|
||||
|
||||
suite.Error(o.Handle(nil))
|
||||
|
||||
suite.Error(o.Handle("string"))
|
||||
|
||||
art := &artifact.Artifact{}
|
||||
art.Digest = "digest"
|
||||
ev := &event.ArtifactEvent{Artifact: &art.Artifact}
|
||||
|
||||
value := &event.DeleteArtifactEvent{ArtifactEvent: ev}
|
||||
|
||||
suite.artifactCtl.On("Count", context.TODO(), q.New(q.KeyWords{"digest": "digest"})).Return(int64(0), fmt.Errorf("failed")).Once()
|
||||
suite.Require().NoError(o.Handle(value))
|
||||
|
||||
suite.artifactCtl.On("Count", context.TODO(), q.New(q.KeyWords{"digest": "digest"})).Return(int64(1), nil).Once()
|
||||
suite.Require().NoError(o.Handle(value))
|
||||
|
||||
suite.artifactCtl.On("Count", context.TODO(), q.New(q.KeyWords{"digest": "digest"})).Return(int64(0), nil).Once()
|
||||
suite.scanCtl.On("DeleteReports", context.TODO(), "digest").Return(fmt.Errorf("failed")).Once()
|
||||
suite.Require().Error(o.Handle(value))
|
||||
|
||||
suite.artifactCtl.On("Count", context.TODO(), q.New(q.KeyWords{"digest": "digest"})).Return(int64(0), nil).Once()
|
||||
suite.scanCtl.On("DeleteReports", context.TODO(), "digest").Return(nil).Once()
|
||||
suite.Require().NoError(o.Handle(value))
|
||||
}
|
||||
|
||||
func TestDelArtHandlerTestSuite(t *testing.T) {
|
||||
suite.Run(t, &DelArtHandlerTestSuite{})
|
||||
}
|
@ -1,12 +1,13 @@
|
||||
package metadata
|
||||
|
||||
import (
|
||||
"github.com/goharbor/harbor/src/common/models"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
event2 "github.com/goharbor/harbor/src/controller/event"
|
||||
"github.com/goharbor/harbor/src/lib/errors"
|
||||
"github.com/goharbor/harbor/src/jobservice/job"
|
||||
"github.com/goharbor/harbor/src/pkg/notifier/event"
|
||||
v1 "github.com/goharbor/harbor/src/pkg/scan/rest/v1"
|
||||
"time"
|
||||
)
|
||||
|
||||
const (
|
||||
@ -24,15 +25,15 @@ func (si *ScanImageMetaData) Resolve(evt *event.Event) error {
|
||||
var eventType string
|
||||
var topic string
|
||||
|
||||
switch si.Status {
|
||||
case models.JobFinished:
|
||||
switch job.Status(si.Status) {
|
||||
case job.SuccessStatus:
|
||||
eventType = event2.TopicScanningCompleted
|
||||
topic = event2.TopicScanningCompleted
|
||||
case models.JobError, models.JobStopped:
|
||||
case job.ErrorStatus, job.StoppedStatus:
|
||||
eventType = event2.TopicScanningFailed
|
||||
topic = event2.TopicScanningFailed
|
||||
default:
|
||||
return errors.New("not supported scan hook status")
|
||||
return fmt.Errorf("not supported scan hook status %s", si.Status)
|
||||
}
|
||||
|
||||
data := &event2.ScanImageEvent{
|
||||
|
@ -15,11 +15,13 @@
|
||||
package metadata
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
event2 "github.com/goharbor/harbor/src/controller/event"
|
||||
"github.com/goharbor/harbor/src/jobservice/job"
|
||||
"github.com/goharbor/harbor/src/pkg/notifier/event"
|
||||
v1 "github.com/goharbor/harbor/src/pkg/scan/rest/v1"
|
||||
"github.com/stretchr/testify/suite"
|
||||
"testing"
|
||||
)
|
||||
|
||||
type scanEventTestSuite struct {
|
||||
@ -36,7 +38,7 @@ func (r *scanEventTestSuite) TestResolveOfScanImageEventMetadata() {
|
||||
Digest: "sha256:absdfd87123",
|
||||
MimeType: "docker.chart",
|
||||
},
|
||||
Status: "finished",
|
||||
Status: job.SuccessStatus.String(),
|
||||
}
|
||||
err := metadata.Resolve(e)
|
||||
r.Require().Nil(err)
|
||||
|
@ -1,111 +0,0 @@
|
||||
// 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 scan
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/goharbor/harbor/src/common/models"
|
||||
"github.com/goharbor/harbor/src/controller/artifact"
|
||||
"github.com/goharbor/harbor/src/controller/repository"
|
||||
"github.com/goharbor/harbor/src/lib/errors"
|
||||
"github.com/goharbor/harbor/src/lib/log"
|
||||
"github.com/goharbor/harbor/src/lib/q"
|
||||
)
|
||||
|
||||
// HandleCheckIn handles the check in data of the scan all job
|
||||
func HandleCheckIn(ctx context.Context, checkIn string) {
|
||||
if len(checkIn) == 0 {
|
||||
// Nothing to handle, directly return
|
||||
return
|
||||
}
|
||||
|
||||
batchSize := 50
|
||||
for repo := range fetchRepositories(ctx, batchSize) {
|
||||
for artifact := range fetchArtifacts(ctx, repo.RepositoryID, batchSize) {
|
||||
if err := DefaultController.Scan(ctx, artifact, WithRequester(checkIn)); err != nil {
|
||||
// Just logged
|
||||
log.Error(errors.Wrap(err, "handle check in"))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func fetchArtifacts(ctx context.Context, repositoryID int64, chunkSize int) <-chan *artifact.Artifact {
|
||||
ch := make(chan *artifact.Artifact, chunkSize)
|
||||
go func() {
|
||||
defer close(ch)
|
||||
|
||||
query := &q.Query{
|
||||
Keywords: map[string]interface{}{
|
||||
"repository_id": repositoryID,
|
||||
},
|
||||
PageSize: int64(chunkSize),
|
||||
PageNumber: 1,
|
||||
}
|
||||
|
||||
for {
|
||||
artifacts, err := artifact.Ctl.List(ctx, query, nil)
|
||||
if err != nil {
|
||||
log.Errorf("[scan all]: list artifacts failed, error: %v", err)
|
||||
return
|
||||
}
|
||||
|
||||
for _, artifact := range artifacts {
|
||||
ch <- artifact
|
||||
}
|
||||
|
||||
if len(artifacts) < chunkSize {
|
||||
break
|
||||
}
|
||||
|
||||
query.PageNumber++
|
||||
}
|
||||
|
||||
}()
|
||||
|
||||
return ch
|
||||
}
|
||||
|
||||
func fetchRepositories(ctx context.Context, chunkSize int) <-chan *models.RepoRecord {
|
||||
ch := make(chan *models.RepoRecord, chunkSize)
|
||||
go func() {
|
||||
defer close(ch)
|
||||
|
||||
query := &q.Query{
|
||||
PageSize: int64(chunkSize),
|
||||
PageNumber: 1,
|
||||
}
|
||||
|
||||
for {
|
||||
repositories, err := repository.Ctl.List(ctx, query)
|
||||
if err != nil {
|
||||
log.Warningf("[scan all]: list repositories failed, error: %v", err)
|
||||
break
|
||||
}
|
||||
|
||||
for _, repo := range repositories {
|
||||
ch <- repo
|
||||
}
|
||||
|
||||
if len(repositories) < chunkSize {
|
||||
break
|
||||
}
|
||||
|
||||
query.PageNumber++
|
||||
}
|
||||
}()
|
||||
return ch
|
||||
}
|
@ -18,28 +18,29 @@ import (
|
||||
"bytes"
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/goharbor/harbor/src/controller/project"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
cj "github.com/goharbor/harbor/src/common/job"
|
||||
jm "github.com/goharbor/harbor/src/common/job/models"
|
||||
"github.com/goharbor/harbor/src/common/rbac"
|
||||
ar "github.com/goharbor/harbor/src/controller/artifact"
|
||||
"github.com/goharbor/harbor/src/controller/robot"
|
||||
sc "github.com/goharbor/harbor/src/controller/scanner"
|
||||
"github.com/goharbor/harbor/src/core/config"
|
||||
"github.com/goharbor/harbor/src/jobservice/job"
|
||||
"github.com/goharbor/harbor/src/lib"
|
||||
"github.com/goharbor/harbor/src/lib/errors"
|
||||
"github.com/goharbor/harbor/src/lib/log"
|
||||
"github.com/goharbor/harbor/src/lib/orm"
|
||||
"github.com/goharbor/harbor/src/lib/q"
|
||||
"github.com/goharbor/harbor/src/pkg/permission/types"
|
||||
"github.com/goharbor/harbor/src/pkg/robot2/model"
|
||||
sca "github.com/goharbor/harbor/src/pkg/scan"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/all"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/dao/scan"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/dao/scanner"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/report"
|
||||
v1 "github.com/goharbor/harbor/src/pkg/scan/rest/v1"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/vuln"
|
||||
"github.com/goharbor/harbor/src/pkg/task"
|
||||
"github.com/google/uuid"
|
||||
)
|
||||
|
||||
@ -49,6 +50,13 @@ var DefaultController = NewController()
|
||||
const (
|
||||
configRegistryEndpoint = "registryEndpoint"
|
||||
configCoreInternalAddr = "coreInternalAddr"
|
||||
|
||||
artfiactKey = "artifact"
|
||||
registrationKey = "registration"
|
||||
|
||||
artifactIDKey = "artifact_id"
|
||||
reportUUIDsKey = "report_uuids"
|
||||
robotIDKey = "robot_id"
|
||||
)
|
||||
|
||||
// uuidGenerator is a func template which is for generating UUID.
|
||||
@ -58,9 +66,6 @@ type uuidGenerator func() (string, error)
|
||||
// utility methods.
|
||||
type configGetter func(cfg string) (string, error)
|
||||
|
||||
// jcGetter is a func template which is used to get the job service client.
|
||||
type jcGetter func() cj.Client
|
||||
|
||||
// basicController is default implementation of api.Controller interface
|
||||
type basicController struct {
|
||||
// Manage the scan report records
|
||||
@ -71,14 +76,16 @@ type basicController struct {
|
||||
sc sc.Controller
|
||||
// Robot account controller
|
||||
rc robot.Controller
|
||||
// Project controller
|
||||
pro project.Controller
|
||||
// Job service client
|
||||
jc jcGetter
|
||||
// UUID generator
|
||||
uuid uuidGenerator
|
||||
// Configuration getter func
|
||||
config configGetter
|
||||
|
||||
cloneCtx func(context.Context) context.Context
|
||||
makeCtx func() context.Context
|
||||
|
||||
execMgr task.ExecutionManager
|
||||
taskMgr task.Manager
|
||||
}
|
||||
|
||||
// NewController news a scan API controller
|
||||
@ -92,12 +99,6 @@ func NewController() Controller {
|
||||
sc: sc.DefaultController,
|
||||
// Refer to the default robot account controller
|
||||
rc: robot.Ctl,
|
||||
// Refer to the default project controller
|
||||
pro: project.Ctl,
|
||||
// Refer to the default job service client
|
||||
jc: func() cj.Client {
|
||||
return cj.GlobalClient
|
||||
},
|
||||
// Generate UUID with uuid lib
|
||||
uuid: func() (string, error) {
|
||||
aUUID, err := uuid.NewUUID()
|
||||
@ -118,6 +119,12 @@ func NewController() Controller {
|
||||
return "", errors.Errorf("configuration option %s not defined", cfg)
|
||||
}
|
||||
},
|
||||
|
||||
cloneCtx: orm.Clone,
|
||||
makeCtx: orm.Context,
|
||||
|
||||
execMgr: task.ExecMgr,
|
||||
taskMgr: task.Mgr,
|
||||
}
|
||||
}
|
||||
|
||||
@ -189,16 +196,15 @@ func (bc *basicController) Scan(ctx context.Context, artifact *ar.Artifact, opti
|
||||
}
|
||||
|
||||
type Param struct {
|
||||
Artifact *ar.Artifact
|
||||
TrackID string
|
||||
ProducesMimes []string
|
||||
Artifact *ar.Artifact
|
||||
Reports []*scan.Report
|
||||
}
|
||||
|
||||
params := []*Param{}
|
||||
|
||||
var errs []error
|
||||
for _, art := range artifacts {
|
||||
trackID, producesMimes, err := bc.makeReportPlaceholder(ctx, r, art, options...)
|
||||
reports, err := bc.makeReportPlaceholder(ctx, r, art)
|
||||
if err != nil {
|
||||
if errors.IsConflictErr(err) {
|
||||
errs = append(errs, err)
|
||||
@ -207,8 +213,8 @@ func (bc *basicController) Scan(ctx context.Context, artifact *ar.Artifact, opti
|
||||
}
|
||||
}
|
||||
|
||||
if len(producesMimes) > 0 {
|
||||
params = append(params, &Param{Artifact: art, TrackID: trackID, ProducesMimes: producesMimes})
|
||||
if len(reports) > 0 {
|
||||
params = append(params, &Param{Artifact: art, Reports: reports})
|
||||
}
|
||||
}
|
||||
|
||||
@ -217,9 +223,36 @@ func (bc *basicController) Scan(ctx context.Context, artifact *ar.Artifact, opti
|
||||
return errs[0]
|
||||
}
|
||||
|
||||
// Parse options
|
||||
opts, err := parseOptions(options...)
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "scan controller: scan")
|
||||
}
|
||||
|
||||
if opts.ExecutionID == 0 {
|
||||
extraAttrs := map[string]interface{}{
|
||||
artfiactKey: map[string]interface{}{
|
||||
"id": artifact.ID,
|
||||
"project_id": artifact.ProjectID,
|
||||
"repository_name": artifact.RepositoryName,
|
||||
"digest": artifact.Digest,
|
||||
},
|
||||
registrationKey: map[string]interface{}{
|
||||
"id": r.ID,
|
||||
"name": r.Name,
|
||||
},
|
||||
}
|
||||
executionID, err := bc.execMgr.Create(ctx, job.ImageScanJob, r.ID, task.ExecutionTriggerManual, extraAttrs)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
opts.ExecutionID = executionID
|
||||
}
|
||||
|
||||
errs = errs[:0]
|
||||
for _, param := range params {
|
||||
if err := bc.scanArtifact(ctx, r, param.Artifact, param.TrackID, param.ProducesMimes); err != nil {
|
||||
if err := bc.launchScanJob(ctx, opts.ExecutionID, param.Artifact, r, param.Reports); err != nil {
|
||||
log.G(ctx).Warningf("scan artifact %s@%s failed, error: %v", artifact.RepositoryName, artifact.Digest, err)
|
||||
errs = append(errs, err)
|
||||
}
|
||||
@ -233,80 +266,128 @@ func (bc *basicController) Scan(ctx context.Context, artifact *ar.Artifact, opti
|
||||
return nil
|
||||
}
|
||||
|
||||
func (bc *basicController) makeReportPlaceholder(ctx context.Context, r *scanner.Registration, art *ar.Artifact, options ...Option) (string, []string, error) {
|
||||
trackID, err := bc.uuid()
|
||||
func (bc *basicController) ScanAll(ctx context.Context, trigger string, async bool) (int64, error) {
|
||||
extraAttrs := map[string]interface{}{}
|
||||
executionID, err := bc.execMgr.Create(ctx, job.ImageScanAllJob, 0, trigger, extraAttrs)
|
||||
if err != nil {
|
||||
return "", nil, errors.Wrap(err, "scan controller: scan")
|
||||
return 0, err
|
||||
}
|
||||
|
||||
// Parse options
|
||||
ops, err := parseOptions(options...)
|
||||
if err != nil {
|
||||
return "", nil, errors.Wrap(err, "scan controller: scan")
|
||||
}
|
||||
|
||||
create := func(ctx context.Context, digest, registrationUUID, mimeType, trackID string, status job.Status) error {
|
||||
reportPlaceholder := &scan.Report{
|
||||
Digest: digest,
|
||||
RegistrationUUID: registrationUUID,
|
||||
Status: status.String(),
|
||||
StatusCode: status.Code(),
|
||||
TrackID: trackID,
|
||||
MimeType: mimeType,
|
||||
}
|
||||
// Set requester if it is specified
|
||||
if len(ops.Requester) > 0 {
|
||||
reportPlaceholder.Requester = ops.Requester
|
||||
} else {
|
||||
// Use the trackID as the requester
|
||||
reportPlaceholder.Requester = trackID
|
||||
}
|
||||
|
||||
_, e := bc.manager.Create(reportPlaceholder)
|
||||
return e
|
||||
}
|
||||
|
||||
if hasCapability(r, art) {
|
||||
var producesMimes []string
|
||||
|
||||
for _, pm := range r.GetProducesMimeTypes(art.ManifestMediaType) {
|
||||
if err = create(ctx, art.Digest, r.UUID, pm, trackID, job.PendingStatus); err != nil {
|
||||
return "", nil, err
|
||||
if async {
|
||||
go func() {
|
||||
// if async, this is running in another goroutine ensure the execution exists in db
|
||||
err := lib.RetryUntil(func() error {
|
||||
_, err := bc.execMgr.Get(ctx, executionID)
|
||||
return err
|
||||
})
|
||||
if err != nil {
|
||||
log.Errorf("failed to get the execution %d for the scan all", executionID)
|
||||
return
|
||||
}
|
||||
|
||||
producesMimes = append(producesMimes, pm)
|
||||
}
|
||||
|
||||
if len(producesMimes) > 0 {
|
||||
return trackID, producesMimes, nil
|
||||
bc.startScanAll(bc.makeCtx(), executionID)
|
||||
}()
|
||||
} else {
|
||||
if err := bc.startScanAll(ctx, executionID); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
}
|
||||
|
||||
err = create(ctx, art.Digest, r.UUID, v1.MimeTypeNativeReport, trackID, job.ErrorStatus)
|
||||
return "", nil, err
|
||||
return executionID, nil
|
||||
}
|
||||
|
||||
func (bc *basicController) scanArtifact(ctx context.Context, r *scanner.Registration, artifact *ar.Artifact, trackID string, producesMimes []string) error {
|
||||
jobID, err := bc.launchScanJob(ctx, trackID, artifact, r, producesMimes)
|
||||
if err != nil {
|
||||
// Update the status to the concrete error
|
||||
// Change status code to normal error code
|
||||
if e := bc.manager.UpdateStatus(trackID, err.Error(), 0); e != nil {
|
||||
err = errors.Wrap(e, err.Error())
|
||||
func (bc *basicController) startScanAll(ctx context.Context, executionID int64) error {
|
||||
artifactCount := 0
|
||||
artifactScannedCount := 0
|
||||
|
||||
batchSize := 50
|
||||
for artifact := range ar.Iterator(ctx, batchSize, nil, nil) {
|
||||
artifactCount++
|
||||
|
||||
scan := func(ctx context.Context) error {
|
||||
return bc.Scan(ctx, artifact, WithExecutionID(executionID))
|
||||
}
|
||||
|
||||
return errors.Wrap(err, "scan controller: scan")
|
||||
if err := orm.WithTransaction(scan)(ctx); err != nil {
|
||||
// Just logged
|
||||
log.Errorf("failed to scan artifact %s, error %v", artifact, err)
|
||||
continue
|
||||
}
|
||||
|
||||
artifactScannedCount++
|
||||
}
|
||||
|
||||
// Insert the generated job ID now
|
||||
// It will not block the whole process. If any errors happened, just logged.
|
||||
if err := bc.manager.UpdateScanJobID(trackID, jobID); err != nil {
|
||||
log.G(ctx).Error(errors.Wrap(err, "scan controller: scan"))
|
||||
// not artifact found
|
||||
if artifactCount == 0 || artifactScannedCount == 0 {
|
||||
message := "no task found"
|
||||
if artifactCount == 0 {
|
||||
message = "no artifact found"
|
||||
}
|
||||
|
||||
if err := bc.execMgr.MarkDone(ctx, executionID, message); err != nil {
|
||||
log.Errorf("failed to mark the execution %d to be done, error: %v", executionID, err)
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (bc *basicController) makeReportPlaceholder(ctx context.Context, r *scanner.Registration, art *ar.Artifact) ([]*scan.Report, error) {
|
||||
mimeTypes := r.GetProducesMimeTypes(art.ManifestMediaType)
|
||||
|
||||
oldReports, err := bc.manager.GetBy(bc.cloneCtx(ctx), art.Digest, r.UUID, mimeTypes)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err := bc.assembleReports(ctx, oldReports...); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if len(oldReports) > 0 {
|
||||
for _, oldReport := range oldReports {
|
||||
if !job.Status(oldReport.Status).Final() {
|
||||
return nil, errors.ConflictError(nil).WithMessage("a previous scan process is %s", oldReport.Status)
|
||||
}
|
||||
}
|
||||
|
||||
for _, oldReport := range oldReports {
|
||||
if err := bc.manager.Delete(ctx, oldReport.UUID); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
var reports []*scan.Report
|
||||
|
||||
for _, pm := range r.GetProducesMimeTypes(art.ManifestMediaType) {
|
||||
report := &scan.Report{
|
||||
Digest: art.Digest,
|
||||
RegistrationUUID: r.UUID,
|
||||
MimeType: pm,
|
||||
}
|
||||
|
||||
create := func(ctx context.Context) error {
|
||||
reportUUID, err := bc.manager.Create(ctx, report)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
report.UUID = reportUUID
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
if err := orm.WithTransaction(create)(ctx); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
reports = append(reports, report)
|
||||
}
|
||||
|
||||
return reports, nil
|
||||
}
|
||||
|
||||
// GetReport ...
|
||||
func (bc *basicController) GetReport(ctx context.Context, artifact *ar.Artifact, mimeTypes []string) ([]*scan.Report, error) {
|
||||
if artifact == nil {
|
||||
@ -348,7 +429,7 @@ func (bc *basicController) GetReport(ctx context.Context, artifact *ar.Artifact,
|
||||
go func(i int, a *ar.Artifact) {
|
||||
defer wg.Done()
|
||||
|
||||
reports, err := bc.manager.GetBy(a.Digest, r.UUID, mimes)
|
||||
reports, err := bc.manager.GetBy(bc.cloneCtx(ctx), a.Digest, r.UUID, mimes)
|
||||
if err != nil {
|
||||
log.Warningf("get reports of %s@%s failed, error: %v", a.RepositoryName, a.Digest, err)
|
||||
return
|
||||
@ -370,6 +451,14 @@ func (bc *basicController) GetReport(ctx context.Context, artifact *ar.Artifact,
|
||||
}
|
||||
}
|
||||
|
||||
if len(reports) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
if err := bc.assembleReports(ctx, reports...); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return reports, nil
|
||||
}
|
||||
|
||||
@ -407,70 +496,66 @@ func (bc *basicController) GetSummary(ctx context.Context, artifact *ar.Artifact
|
||||
return summaries, nil
|
||||
}
|
||||
|
||||
func (bc *basicController) getScanLog(uuid string) ([]byte, error) {
|
||||
// Get by uuid
|
||||
sr, err := bc.manager.Get(uuid)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "scan controller: get scan log")
|
||||
}
|
||||
|
||||
if sr == nil {
|
||||
// Not found
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Not job error
|
||||
if sr.StatusCode == job.ErrorStatus.Code() {
|
||||
jst := job.Status(sr.Status)
|
||||
if jst.Code() == -1 {
|
||||
return []byte(sr.Status), nil
|
||||
}
|
||||
}
|
||||
|
||||
// Job log
|
||||
return bc.jc().GetJobLog(sr.JobID)
|
||||
}
|
||||
|
||||
// GetScanLog ...
|
||||
func (bc *basicController) GetScanLog(uuid string) ([]byte, error) {
|
||||
func (bc *basicController) GetScanLog(ctx context.Context, uuid string) ([]byte, error) {
|
||||
if len(uuid) == 0 {
|
||||
return nil, errors.New("empty uuid to get scan log")
|
||||
}
|
||||
|
||||
reportIDs := vuln.ParseReportIDs(uuid)
|
||||
reportUUIDs := vuln.ParseReportIDs(uuid)
|
||||
tasks, err := bc.listScanTasks(ctx, reportUUIDs)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if len(tasks) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
reportUUIDToTasks := map[string]*task.Task{}
|
||||
for _, task := range tasks {
|
||||
for _, reportUUID := range getReportUUIDs(task.ExtraAttrs) {
|
||||
reportUUIDToTasks[reportUUID] = task
|
||||
}
|
||||
}
|
||||
|
||||
errs := map[string]error{}
|
||||
logs := make(map[string][]byte, len(reportIDs))
|
||||
logs := make(map[string][]byte, len(tasks))
|
||||
|
||||
var (
|
||||
mu sync.Mutex
|
||||
wg sync.WaitGroup
|
||||
)
|
||||
for _, reportID := range reportIDs {
|
||||
for _, reportUUID := range reportUUIDs {
|
||||
wg.Add(1)
|
||||
|
||||
go func(reportID string) {
|
||||
go func(reportUUID string) {
|
||||
defer wg.Done()
|
||||
|
||||
log, err := bc.getScanLog(reportID)
|
||||
task, ok := reportUUIDToTasks[reportUUID]
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
|
||||
log, err := bc.taskMgr.GetLog(ctx, task.ID)
|
||||
|
||||
mu.Lock()
|
||||
defer mu.Unlock()
|
||||
|
||||
if err != nil {
|
||||
errs[reportID] = err
|
||||
errs[reportUUID] = err
|
||||
} else {
|
||||
logs[reportID] = log
|
||||
logs[reportUUID] = log
|
||||
}
|
||||
}(reportID)
|
||||
}(reportUUID)
|
||||
}
|
||||
wg.Wait()
|
||||
|
||||
if len(reportIDs) == 1 {
|
||||
return logs[reportIDs[0]], errs[reportIDs[0]]
|
||||
if len(reportUUIDs) == 1 {
|
||||
return logs[reportUUIDs[0]], errs[reportUUIDs[0]]
|
||||
}
|
||||
|
||||
if len(errs) == len(reportIDs) {
|
||||
if len(errs) == len(reportUUIDs) {
|
||||
for _, err := range errs {
|
||||
return nil, err
|
||||
}
|
||||
@ -479,8 +564,8 @@ func (bc *basicController) GetScanLog(uuid string) ([]byte, error) {
|
||||
var b bytes.Buffer
|
||||
|
||||
multiLogs := len(logs) > 1
|
||||
for _, reportID := range reportIDs {
|
||||
log, ok := logs[reportID]
|
||||
for _, reportUUID := range reportUUIDs {
|
||||
log, ok := logs[reportUUID]
|
||||
if !ok || len(log) == 0 {
|
||||
continue
|
||||
}
|
||||
@ -489,7 +574,7 @@ func (bc *basicController) GetScanLog(uuid string) ([]byte, error) {
|
||||
if b.Len() > 0 {
|
||||
b.WriteString("\n\n\n\n")
|
||||
}
|
||||
b.WriteString(fmt.Sprintf("---------- Logs of report %s ----------\n", reportID))
|
||||
b.WriteString(fmt.Sprintf("---------- Logs of report %s ----------\n", reportUUID))
|
||||
}
|
||||
|
||||
b.Write(log)
|
||||
@ -498,88 +583,30 @@ func (bc *basicController) GetScanLog(uuid string) ([]byte, error) {
|
||||
return b.Bytes(), nil
|
||||
}
|
||||
|
||||
// HandleJobHooks ...
|
||||
func (bc *basicController) HandleJobHooks(ctx context.Context, trackID string, change *job.StatusChange) error {
|
||||
if len(trackID) == 0 {
|
||||
return errors.New("empty track ID")
|
||||
func (bc *basicController) UpdateReport(ctx context.Context, report *sca.CheckInReport) error {
|
||||
rpl, err := bc.manager.GetBy(ctx, report.Digest, report.RegistrationUUID, []string{report.MimeType})
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "scan controller: handle job hook")
|
||||
}
|
||||
|
||||
if change == nil {
|
||||
return errors.New("nil change object")
|
||||
if len(rpl) == 0 {
|
||||
return errors.New("no report found to update data")
|
||||
}
|
||||
|
||||
// Clear robot account
|
||||
// Only when the job is successfully done!
|
||||
if change.Status == job.SuccessStatus.String() {
|
||||
if v, ok := change.Metadata.Parameters[sca.JobParameterRobot]; ok {
|
||||
if jsonData, y := v.(string); y {
|
||||
r := &model.Robot{}
|
||||
if err := r.FromJSON(jsonData); err != nil {
|
||||
log.Error(errors.Wrap(err, "scan controller: handle job hook"))
|
||||
}
|
||||
|
||||
if r.ID > 0 {
|
||||
if err := robot.Ctl.Delete(ctx, r.ID); err != nil {
|
||||
// Should not block the main flow, just logged
|
||||
log.Error(errors.Wrap(err, "scan controller: handle job hook"))
|
||||
} else {
|
||||
log.Debugf("Robot account with id %d for the scan %s is removed", r.ID, trackID)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Check in data
|
||||
if len(change.CheckIn) > 0 {
|
||||
checkInReport := &sca.CheckInReport{}
|
||||
if err := checkInReport.FromJSON(change.CheckIn); err != nil {
|
||||
return errors.Wrap(err, "scan controller: handle job hook")
|
||||
}
|
||||
|
||||
rpl, err := bc.manager.GetBy(
|
||||
checkInReport.Digest,
|
||||
checkInReport.RegistrationUUID,
|
||||
[]string{checkInReport.MimeType})
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "scan controller: handle job hook")
|
||||
}
|
||||
|
||||
if len(rpl) == 0 {
|
||||
return errors.New("no report found to update data")
|
||||
}
|
||||
|
||||
if err := bc.manager.UpdateReportData(
|
||||
rpl[0].UUID,
|
||||
checkInReport.RawReport,
|
||||
change.Metadata.Revision); err != nil {
|
||||
return errors.Wrap(err, "scan controller: handle job hook")
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
return bc.manager.UpdateStatus(trackID, change.Status, change.Metadata.Revision)
|
||||
}
|
||||
|
||||
// DeleteReports ...
|
||||
func (bc *basicController) DeleteReports(digests ...string) error {
|
||||
if err := bc.manager.DeleteByDigests(digests...); err != nil {
|
||||
return errors.Wrap(err, "scan controller: delete reports")
|
||||
if err := bc.manager.UpdateReportData(ctx, rpl[0].UUID, report.RawReport); err != nil {
|
||||
return errors.Wrap(err, "scan controller: handle job hook")
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// GetStats ...
|
||||
func (bc *basicController) GetStats(requester string) (*all.Stats, error) {
|
||||
sts, err := bc.manager.GetStats(requester)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "scan controller: delete reports")
|
||||
// DeleteReports ...
|
||||
func (bc *basicController) DeleteReports(ctx context.Context, digests ...string) error {
|
||||
if err := bc.manager.DeleteByDigests(ctx, digests...); err != nil {
|
||||
return errors.Wrap(err, "scan controller: delete reports")
|
||||
}
|
||||
|
||||
return sts, nil
|
||||
return nil
|
||||
}
|
||||
|
||||
// makeRobotAccount creates a robot account based on the arguments for scanning.
|
||||
@ -590,10 +617,7 @@ func (bc *basicController) makeRobotAccount(ctx context.Context, projectID int64
|
||||
return nil, errors.Wrap(err, "scan controller: make robot account")
|
||||
}
|
||||
|
||||
p, err := bc.pro.Get(ctx, projectID)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "scan controller: make robot account")
|
||||
}
|
||||
projectName := strings.Split(repository, "/")[0]
|
||||
|
||||
robotReq := &robot.Robot{
|
||||
Robot: model.Robot{
|
||||
@ -605,7 +629,7 @@ func (bc *basicController) makeRobotAccount(ctx context.Context, projectID int64
|
||||
Permissions: []*robot.Permission{
|
||||
{
|
||||
Kind: "project",
|
||||
Namespace: p.Name,
|
||||
Namespace: projectName,
|
||||
Access: []*types.Policy{
|
||||
{
|
||||
Resource: rbac.ResourceRepository,
|
||||
@ -634,7 +658,12 @@ func (bc *basicController) makeRobotAccount(ctx context.Context, projectID int64
|
||||
}
|
||||
|
||||
// launchScanJob launches a job to run scan
|
||||
func (bc *basicController) launchScanJob(ctx context.Context, trackID string, artifact *ar.Artifact, registration *scanner.Registration, mimes []string) (jobID string, err error) {
|
||||
func (bc *basicController) launchScanJob(ctx context.Context, executionID int64, artifact *ar.Artifact, registration *scanner.Registration, reports []*scan.Report) error {
|
||||
// don't launch scan job for the artifact which is not supported by the scanner
|
||||
if !hasCapability(registration, artifact) {
|
||||
return nil
|
||||
}
|
||||
|
||||
var ck string
|
||||
if registration.UseInternalAddr {
|
||||
ck = configCoreInternalAddr
|
||||
@ -644,12 +673,12 @@ func (bc *basicController) launchScanJob(ctx context.Context, trackID string, ar
|
||||
|
||||
registryAddr, err := bc.config(ck)
|
||||
if err != nil {
|
||||
return "", errors.Wrap(err, "scan controller: launch scan job")
|
||||
return errors.Wrap(err, "scan controller: launch scan job")
|
||||
}
|
||||
|
||||
robot, err := bc.makeRobotAccount(ctx, artifact.ProjectID, artifact.RepositoryName, registration)
|
||||
if err != nil {
|
||||
return "", errors.Wrap(err, "scan controller: launch scan job")
|
||||
return errors.Wrap(err, "scan controller: launch scan job")
|
||||
}
|
||||
|
||||
// Set job parameters
|
||||
@ -667,17 +696,24 @@ func (bc *basicController) launchScanJob(ctx context.Context, trackID string, ar
|
||||
|
||||
rJSON, err := registration.ToJSON()
|
||||
if err != nil {
|
||||
return "", errors.Wrap(err, "scan controller: launch scan job")
|
||||
return errors.Wrap(err, "scan controller: launch scan job")
|
||||
}
|
||||
|
||||
sJSON, err := scanReq.ToJSON()
|
||||
if err != nil {
|
||||
return "", errors.Wrap(err, "launch scan job")
|
||||
return errors.Wrap(err, "launch scan job")
|
||||
}
|
||||
|
||||
robotJSON, err := robot.ToJSON()
|
||||
if err != nil {
|
||||
return "", errors.Wrap(err, "launch scan job")
|
||||
return errors.Wrap(err, "launch scan job")
|
||||
}
|
||||
|
||||
mimes := make([]string, len(reports))
|
||||
reportUUIDs := make([]string, len(reports))
|
||||
for i, report := range reports {
|
||||
mimes[i] = report.MimeType
|
||||
reportUUIDs[i] = report.UUID
|
||||
}
|
||||
|
||||
params := make(map[string]interface{})
|
||||
@ -687,23 +723,161 @@ func (bc *basicController) launchScanJob(ctx context.Context, trackID string, ar
|
||||
params[sca.JobParameterMimes] = mimes
|
||||
params[sca.JobParameterRobot] = robotJSON
|
||||
|
||||
// Launch job
|
||||
callbackURL, err := bc.config(configCoreInternalAddr)
|
||||
if err != nil {
|
||||
return "", errors.Wrap(err, "launch scan job")
|
||||
}
|
||||
hookURL := fmt.Sprintf("%s/service/notifications/jobs/scan/%s", callbackURL, trackID)
|
||||
|
||||
j := &jm.JobData{
|
||||
j := &task.Job{
|
||||
Name: job.ImageScanJob,
|
||||
Metadata: &jm.JobMetadata{
|
||||
Metadata: &job.Metadata{
|
||||
JobKind: job.KindGeneric,
|
||||
},
|
||||
Parameters: params,
|
||||
StatusHook: hookURL,
|
||||
}
|
||||
|
||||
return bc.jc().SubmitJob(j)
|
||||
// keep the report uuids in array so that when ?| operator support by the FilterRaw method of beego's orm
|
||||
// we can list the tasks of the scan reports by one SQL
|
||||
extraAttrs := map[string]interface{}{
|
||||
artifactIDKey: artifact.ID,
|
||||
robotIDKey: robot.ID,
|
||||
reportUUIDsKey: reportUUIDs,
|
||||
}
|
||||
|
||||
// NOTE: due to the limitation of the beego's orm, the List method of the task manager not support ?! operator for the jsonb field,
|
||||
// we cann't list the tasks for scan reports of uuid1, uuid2 by SQL `SELECT * FROM task WHERE (extra_attrs->'report_uuids')::jsonb ?| array['uuid1', 'uuid2']`
|
||||
// or by `SELECT * FROM task WHERE id IN (SELECT id FROM task WHERE (extra_attrs->'report_uuids')::jsonb ?| array['uuid1', 'uuid2'])`
|
||||
// so save {"report:uuid1": "1", "report:uuid2": "2"} in the extra_attrs of the task, and then list it with
|
||||
// SQL `SELECT * FROM task WHERE extra_attrs->>'report:uuid1' = '1'` in loop
|
||||
for _, reportUUID := range reportUUIDs {
|
||||
extraAttrs["report:"+reportUUID] = "1"
|
||||
}
|
||||
|
||||
_, err = bc.taskMgr.Create(ctx, executionID, j, extraAttrs)
|
||||
return err
|
||||
}
|
||||
|
||||
// listScanTasks returns the tasks of the reports
|
||||
func (bc *basicController) listScanTasks(ctx context.Context, reportUUIDs []string) ([]*task.Task, error) {
|
||||
if len(reportUUIDs) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
tasks := make([]*task.Task, len(reportUUIDs))
|
||||
errs := make([]error, len(reportUUIDs))
|
||||
|
||||
var wg sync.WaitGroup
|
||||
for i, reportUUID := range reportUUIDs {
|
||||
wg.Add(1)
|
||||
|
||||
go func(ix int, reportUUID string) {
|
||||
defer wg.Done()
|
||||
|
||||
task, err := bc.getScanTask(bc.cloneCtx(ctx), reportUUID)
|
||||
if err == nil {
|
||||
tasks[ix] = task
|
||||
} else if !errors.IsNotFoundErr(err) {
|
||||
errs[ix] = err
|
||||
} else {
|
||||
log.G(ctx).Warningf("task for the scan report %s not found", reportUUID)
|
||||
}
|
||||
}(i, reportUUID)
|
||||
}
|
||||
wg.Wait()
|
||||
|
||||
for _, err := range errs {
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
var results []*task.Task
|
||||
for _, task := range tasks {
|
||||
if task != nil {
|
||||
results = append(results, task)
|
||||
}
|
||||
}
|
||||
|
||||
return results, nil
|
||||
}
|
||||
|
||||
func (bc *basicController) getScanTask(ctx context.Context, reportUUID string) (*task.Task, error) {
|
||||
query := q.New(q.KeyWords{"extra_attrs." + "report:" + reportUUID: "1"})
|
||||
tasks, err := bc.taskMgr.List(bc.cloneCtx(ctx), query)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if len(tasks) == 0 {
|
||||
return nil, errors.NotFoundError(nil).WithMessage("task for report %s not found", reportUUID)
|
||||
}
|
||||
|
||||
return tasks[0], nil
|
||||
}
|
||||
|
||||
func (bc *basicController) assembleReports(ctx context.Context, reports ...*scan.Report) error {
|
||||
reportUUIDs := make([]string, len(reports))
|
||||
for i, report := range reports {
|
||||
reportUUIDs[i] = report.UUID
|
||||
}
|
||||
|
||||
tasks, err := bc.listScanTasks(ctx, reportUUIDs)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
reportUUIDToTasks := map[string]*task.Task{}
|
||||
for _, task := range tasks {
|
||||
for _, reportUUID := range getReportUUIDs(task.ExtraAttrs) {
|
||||
reportUUIDToTasks[reportUUID] = task
|
||||
}
|
||||
}
|
||||
|
||||
for _, report := range reports {
|
||||
if task, ok := reportUUIDToTasks[report.UUID]; ok {
|
||||
report.Status = task.Status
|
||||
report.StartTime = task.StartTime
|
||||
report.EndTime = task.EndTime
|
||||
} else {
|
||||
report.Status = job.ErrorStatus.String()
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func getArtifactID(extraAttrs map[string]interface{}) int64 {
|
||||
var artifactID float64
|
||||
if extraAttrs != nil {
|
||||
if v, ok := extraAttrs[artifactIDKey]; ok {
|
||||
artifactID, _ = v.(float64) // int64 Unmarshal to float64
|
||||
}
|
||||
}
|
||||
|
||||
return int64(artifactID)
|
||||
}
|
||||
|
||||
func getReportUUIDs(extraAttrs map[string]interface{}) []string {
|
||||
var reportUUIDs []string
|
||||
|
||||
if extraAttrs != nil {
|
||||
value, ok := extraAttrs[reportUUIDsKey]
|
||||
if ok {
|
||||
arr, _ := value.([]interface{})
|
||||
for _, el := range arr {
|
||||
if s, ok := el.(string); ok {
|
||||
reportUUIDs = append(reportUUIDs, s)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return reportUUIDs
|
||||
}
|
||||
|
||||
func getRobotID(extraAttrs map[string]interface{}) int64 {
|
||||
var trackID float64
|
||||
if extraAttrs != nil {
|
||||
if v, ok := extraAttrs[robotIDKey]; ok {
|
||||
trackID, _ = v.(float64) // int64 Unmarshal to float64
|
||||
}
|
||||
}
|
||||
|
||||
return int64(trackID)
|
||||
}
|
||||
|
||||
func parseOptions(options ...Option) (*Options, error) {
|
||||
|
@ -19,18 +19,16 @@ import (
|
||||
"encoding/base64"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"github.com/goharbor/harbor/src/common"
|
||||
models "github.com/goharbor/harbor/src/common/models"
|
||||
"github.com/goharbor/harbor/src/controller/robot"
|
||||
"github.com/goharbor/harbor/src/core/config"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
cj "github.com/goharbor/harbor/src/common/job"
|
||||
jm "github.com/goharbor/harbor/src/common/job/models"
|
||||
"github.com/goharbor/harbor/src/common"
|
||||
"github.com/goharbor/harbor/src/common/rbac"
|
||||
"github.com/goharbor/harbor/src/controller/artifact"
|
||||
"github.com/goharbor/harbor/src/jobservice/job"
|
||||
"github.com/goharbor/harbor/src/controller/robot"
|
||||
"github.com/goharbor/harbor/src/core/config"
|
||||
"github.com/goharbor/harbor/src/lib/orm"
|
||||
"github.com/goharbor/harbor/src/lib/q"
|
||||
"github.com/goharbor/harbor/src/pkg/permission/types"
|
||||
"github.com/goharbor/harbor/src/pkg/robot2/model"
|
||||
sca "github.com/goharbor/harbor/src/pkg/scan"
|
||||
@ -38,14 +36,15 @@ import (
|
||||
"github.com/goharbor/harbor/src/pkg/scan/dao/scanner"
|
||||
v1 "github.com/goharbor/harbor/src/pkg/scan/rest/v1"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/vuln"
|
||||
"github.com/goharbor/harbor/src/pkg/task"
|
||||
artifacttesting "github.com/goharbor/harbor/src/testing/controller/artifact"
|
||||
projecttesting "github.com/goharbor/harbor/src/testing/controller/project"
|
||||
robottesting "github.com/goharbor/harbor/src/testing/controller/robot"
|
||||
scannertesting "github.com/goharbor/harbor/src/testing/controller/scanner"
|
||||
mocktesting "github.com/goharbor/harbor/src/testing/mock"
|
||||
ormtesting "github.com/goharbor/harbor/src/testing/lib/orm"
|
||||
"github.com/goharbor/harbor/src/testing/mock"
|
||||
reporttesting "github.com/goharbor/harbor/src/testing/pkg/scan/report"
|
||||
tasktesting "github.com/goharbor/harbor/src/testing/pkg/task"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/require"
|
||||
"github.com/stretchr/testify/suite"
|
||||
)
|
||||
@ -54,10 +53,15 @@ import (
|
||||
type ControllerTestSuite struct {
|
||||
suite.Suite
|
||||
|
||||
artifactCtl *artifacttesting.Controller
|
||||
originalArtifactCtl artifact.Controller
|
||||
|
||||
registration *scanner.Registration
|
||||
artifact *artifact.Artifact
|
||||
rawReport string
|
||||
|
||||
execMgr *tasktesting.ExecutionManager
|
||||
taskMgr *tasktesting.Manager
|
||||
reportMgr *reporttesting.Manager
|
||||
ar artifact.Controller
|
||||
c Controller
|
||||
@ -70,6 +74,10 @@ func TestController(t *testing.T) {
|
||||
|
||||
// SetupSuite ...
|
||||
func (suite *ControllerTestSuite) SetupSuite() {
|
||||
suite.originalArtifactCtl = artifact.Ctl
|
||||
suite.artifactCtl = &artifacttesting.Controller{}
|
||||
artifact.Ctl = suite.artifactCtl
|
||||
|
||||
suite.artifact = &artifact.Artifact{}
|
||||
suite.artifact.Type = "IMAGE"
|
||||
suite.artifact.ProjectID = 1
|
||||
@ -107,20 +115,15 @@ func (suite *ControllerTestSuite) SetupSuite() {
|
||||
}
|
||||
|
||||
sc := &scannertesting.Controller{}
|
||||
sc.On("GetRegistrationByProject", context.TODO(), suite.artifact.ProjectID).Return(suite.registration, nil)
|
||||
sc.On("GetRegistrationByProject", mock.Anything, suite.artifact.ProjectID).Return(suite.registration, nil)
|
||||
sc.On("Ping", suite.registration).Return(m, nil)
|
||||
|
||||
mgr := &reporttesting.Manager{}
|
||||
mgr.On("Create", &scan.Report{
|
||||
mgr.On("Create", mock.Anything, &scan.Report{
|
||||
Digest: "digest-code",
|
||||
RegistrationUUID: "uuid001",
|
||||
MimeType: "application/vnd.scanner.adapter.vuln.report.harbor+json; version=1.0",
|
||||
Status: "Pending",
|
||||
StatusCode: 0,
|
||||
TrackID: "the-uuid-123",
|
||||
Requester: "the-uuid-123",
|
||||
}).Return("r-uuid", nil)
|
||||
mgr.On("UpdateScanJobID", "the-uuid-123", "the-job-id").Return(nil)
|
||||
|
||||
rp := vuln.Report{
|
||||
GeneratedAt: time.Now().UTC().String(),
|
||||
@ -155,18 +158,14 @@ func (suite *ControllerTestSuite) SetupSuite() {
|
||||
RegistrationUUID: "uuid001",
|
||||
MimeType: "application/vnd.scanner.adapter.vuln.report.harbor+json; version=1.0",
|
||||
Status: "Success",
|
||||
StatusCode: 3,
|
||||
TrackID: "the-uuid-123",
|
||||
JobID: "the-job-id",
|
||||
StatusRevision: time.Now().Unix(),
|
||||
Report: suite.rawReport,
|
||||
StartTime: time.Now(),
|
||||
EndTime: time.Now().Add(2 * time.Second),
|
||||
},
|
||||
}
|
||||
|
||||
mgr.On("GetBy", suite.artifact.Digest, suite.registration.UUID, []string{v1.MimeTypeNativeReport}).Return(reports, nil)
|
||||
mgr.On("Get", "rp-uuid-001").Return(reports[0], nil)
|
||||
mgr.On("GetBy", mock.Anything, suite.artifact.Digest, suite.registration.UUID, []string{v1.MimeTypeNativeReport}).Return(reports, nil)
|
||||
mgr.On("Get", mock.Anything, "rp-uuid-001").Return(reports[0], nil)
|
||||
mgr.On("UpdateReportData", "rp-uuid-001", suite.rawReport, (int64)(10000)).Return(nil)
|
||||
mgr.On("UpdateStatus", "the-uuid-123", "Success", (int64)(10000)).Return(nil)
|
||||
suite.reportMgr = mgr
|
||||
@ -205,8 +204,8 @@ func (suite *ControllerTestSuite) SetupSuite() {
|
||||
},
|
||||
}
|
||||
|
||||
rc.On("Create", context.TODO(), account).Return(int64(1), "robot-account", nil)
|
||||
rc.On("Get", context.TODO(), int64(1), &robot.Option{
|
||||
rc.On("Create", mock.Anything, account).Return(int64(1), "robot-account", nil)
|
||||
rc.On("Get", mock.Anything, int64(1), &robot.Option{
|
||||
WithPermission: false,
|
||||
}).Return(&robot.Robot{
|
||||
Robot: model.Robot{
|
||||
@ -243,7 +242,6 @@ func (suite *ControllerTestSuite) SetupSuite() {
|
||||
robotJSON, err := rb.ToJSON()
|
||||
require.NoError(suite.T(), err)
|
||||
|
||||
jc := &MockJobServiceClient{}
|
||||
params := make(map[string]interface{})
|
||||
params[sca.JobParamRegistration] = regJSON
|
||||
params[sca.JobParameterRequest] = rJSON
|
||||
@ -251,38 +249,17 @@ func (suite *ControllerTestSuite) SetupSuite() {
|
||||
params[sca.JobParameterAuthType] = "Basic"
|
||||
params[sca.JobParameterRobot] = robotJSON
|
||||
|
||||
j := &jm.JobData{
|
||||
Name: job.ImageScanJob,
|
||||
Metadata: &jm.JobMetadata{
|
||||
JobKind: job.KindGeneric,
|
||||
},
|
||||
Parameters: params,
|
||||
StatusHook: fmt.Sprintf("%s/service/notifications/jobs/scan/%s", "http://core:8080", "the-uuid-123"),
|
||||
}
|
||||
jc.On("SubmitJob", j).Return("the-job-id", nil)
|
||||
jc.On("GetJobLog", "the-job-id").Return([]byte("job log"), nil)
|
||||
|
||||
suite.ar = &artifacttesting.Controller{}
|
||||
mocktesting.OnAnything(suite.ar, "Walk").Return(nil).Run(func(args mock.Arguments) {
|
||||
walkFn := args.Get(2).(func(*artifact.Artifact) error)
|
||||
walkFn(suite.artifact)
|
||||
})
|
||||
|
||||
proCtl := &projecttesting.Controller{}
|
||||
proCtl.On("Get", context.TODO(), suite.artifact.ProjectID).Return(&models.Project{
|
||||
ProjectID: suite.artifact.ProjectID,
|
||||
Name: "library",
|
||||
}, nil)
|
||||
suite.execMgr = &tasktesting.ExecutionManager{}
|
||||
|
||||
suite.taskMgr = &tasktesting.Manager{}
|
||||
|
||||
suite.c = &basicController{
|
||||
manager: mgr,
|
||||
ar: suite.ar,
|
||||
sc: sc,
|
||||
jc: func() cj.Client {
|
||||
return jc
|
||||
},
|
||||
rc: rc,
|
||||
pro: proCtl,
|
||||
rc: rc,
|
||||
uuid: func() (string, error) {
|
||||
return "the-uuid-123", nil
|
||||
},
|
||||
@ -296,20 +273,90 @@ func (suite *ControllerTestSuite) SetupSuite() {
|
||||
|
||||
return "", nil
|
||||
},
|
||||
|
||||
cloneCtx: func(ctx context.Context) context.Context { return ctx },
|
||||
makeCtx: func() context.Context { return context.TODO() },
|
||||
|
||||
execMgr: suite.execMgr,
|
||||
taskMgr: suite.taskMgr,
|
||||
}
|
||||
}
|
||||
|
||||
// TearDownSuite ...
|
||||
func (suite *ControllerTestSuite) TearDownSuite() {}
|
||||
func (suite *ControllerTestSuite) TearDownSuite() {
|
||||
artifact.Ctl = suite.originalArtifactCtl
|
||||
}
|
||||
|
||||
// TestScanControllerScan ...
|
||||
func (suite *ControllerTestSuite) TestScanControllerScan() {
|
||||
err := suite.c.Scan(context.TODO(), suite.artifact)
|
||||
require.NoError(suite.T(), err)
|
||||
{
|
||||
// artifact not provieded
|
||||
suite.Require().Error(suite.c.Scan(context.TODO(), nil))
|
||||
}
|
||||
|
||||
{
|
||||
// success
|
||||
mock.OnAnything(suite.ar, "Walk").Return(nil).Run(func(args mock.Arguments) {
|
||||
walkFn := args.Get(2).(func(*artifact.Artifact) error)
|
||||
walkFn(suite.artifact)
|
||||
}).Once()
|
||||
|
||||
mock.OnAnything(suite.taskMgr, "List").Return([]*task.Task{
|
||||
{ExtraAttrs: suite.makeExtraAttrs("rp-uuid-001"), Status: "Success"},
|
||||
}, nil).Once()
|
||||
|
||||
mock.OnAnything(suite.reportMgr, "Delete").Return(nil).Once()
|
||||
|
||||
mock.OnAnything(suite.execMgr, "Create").Return(int64(1), nil).Once()
|
||||
mock.OnAnything(suite.taskMgr, "Create").Return(int64(1), nil).Once()
|
||||
|
||||
ctx := orm.NewContext(nil, &ormtesting.FakeOrmer{})
|
||||
|
||||
suite.Require().NoError(suite.c.Scan(ctx, suite.artifact))
|
||||
}
|
||||
|
||||
{
|
||||
// delete old report failed
|
||||
mock.OnAnything(suite.ar, "Walk").Return(nil).Run(func(args mock.Arguments) {
|
||||
walkFn := args.Get(2).(func(*artifact.Artifact) error)
|
||||
walkFn(suite.artifact)
|
||||
}).Once()
|
||||
|
||||
mock.OnAnything(suite.taskMgr, "List").Return([]*task.Task{
|
||||
{ExtraAttrs: suite.makeExtraAttrs("rp-uuid-001"), Status: "Success"},
|
||||
}, nil).Once()
|
||||
|
||||
mock.OnAnything(suite.reportMgr, "Delete").Return(fmt.Errorf("delete failed")).Once()
|
||||
|
||||
suite.Require().Error(suite.c.Scan(context.TODO(), suite.artifact))
|
||||
}
|
||||
|
||||
{
|
||||
// a previous scan process is ongoing
|
||||
mock.OnAnything(suite.ar, "Walk").Return(nil).Run(func(args mock.Arguments) {
|
||||
walkFn := args.Get(2).(func(*artifact.Artifact) error)
|
||||
walkFn(suite.artifact)
|
||||
}).Once()
|
||||
|
||||
mock.OnAnything(suite.taskMgr, "List").Return([]*task.Task{
|
||||
{ExtraAttrs: suite.makeExtraAttrs("rp-uuid-001"), Status: "Running"},
|
||||
}, nil).Once()
|
||||
|
||||
suite.Require().Error(suite.c.Scan(context.TODO(), suite.artifact))
|
||||
}
|
||||
}
|
||||
|
||||
// TestScanControllerGetReport ...
|
||||
func (suite *ControllerTestSuite) TestScanControllerGetReport() {
|
||||
mock.OnAnything(suite.ar, "Walk").Return(nil).Run(func(args mock.Arguments) {
|
||||
walkFn := args.Get(2).(func(*artifact.Artifact) error)
|
||||
walkFn(suite.artifact)
|
||||
}).Once()
|
||||
|
||||
mock.OnAnything(suite.taskMgr, "List").Return([]*task.Task{
|
||||
{ExtraAttrs: suite.makeExtraAttrs("rp-uuid-001")},
|
||||
}, nil).Once()
|
||||
|
||||
rep, err := suite.c.GetReport(context.TODO(), suite.artifact, []string{v1.MimeTypeNativeReport})
|
||||
require.NoError(suite.T(), err)
|
||||
assert.Equal(suite.T(), 1, len(rep))
|
||||
@ -317,6 +364,12 @@ func (suite *ControllerTestSuite) TestScanControllerGetReport() {
|
||||
|
||||
// TestScanControllerGetSummary ...
|
||||
func (suite *ControllerTestSuite) TestScanControllerGetSummary() {
|
||||
mock.OnAnything(suite.ar, "Walk").Return(nil).Run(func(args mock.Arguments) {
|
||||
walkFn := args.Get(2).(func(*artifact.Artifact) error)
|
||||
walkFn(suite.artifact)
|
||||
}).Once()
|
||||
mock.OnAnything(suite.taskMgr, "List").Return(nil, nil).Once()
|
||||
|
||||
sum, err := suite.c.GetSummary(context.TODO(), suite.artifact, []string{v1.MimeTypeNativeReport})
|
||||
require.NoError(suite.T(), err)
|
||||
assert.Equal(suite.T(), 1, len(sum))
|
||||
@ -324,7 +377,16 @@ func (suite *ControllerTestSuite) TestScanControllerGetSummary() {
|
||||
|
||||
// TestScanControllerGetScanLog ...
|
||||
func (suite *ControllerTestSuite) TestScanControllerGetScanLog() {
|
||||
bytes, err := suite.c.GetScanLog("rp-uuid-001")
|
||||
mock.OnAnything(suite.taskMgr, "List").Return([]*task.Task{
|
||||
{
|
||||
ID: 1,
|
||||
ExtraAttrs: suite.makeExtraAttrs("rp-uuid-001"),
|
||||
},
|
||||
}, nil).Once()
|
||||
|
||||
mock.OnAnything(suite.taskMgr, "GetLog").Return([]byte("log"), nil).Once()
|
||||
|
||||
bytes, err := suite.c.GetScanLog(context.TODO(), "rp-uuid-001")
|
||||
require.NoError(suite.T(), err)
|
||||
assert.Condition(suite.T(), func() (success bool) {
|
||||
success = len(bytes) > 0
|
||||
@ -333,25 +395,27 @@ func (suite *ControllerTestSuite) TestScanControllerGetScanLog() {
|
||||
}
|
||||
|
||||
func (suite *ControllerTestSuite) TestScanControllerGetMultiScanLog() {
|
||||
kw1 := q.KeyWords{"extra_attrs.report:rp-uuid-001": "1"}
|
||||
suite.taskMgr.On("List", context.TODO(), q.New(kw1)).Return([]*task.Task{
|
||||
{
|
||||
ID: 1,
|
||||
ExtraAttrs: suite.makeExtraAttrs("rp-uuid-001"),
|
||||
},
|
||||
}, nil).Times(4)
|
||||
|
||||
kw2 := q.KeyWords{"extra_attrs.report:rp-uuid-002": "1"}
|
||||
suite.taskMgr.On("List", context.TODO(), q.New(kw2)).Return([]*task.Task{
|
||||
{
|
||||
ID: 2,
|
||||
ExtraAttrs: suite.makeExtraAttrs("rp-uuid-002"),
|
||||
},
|
||||
}, nil).Times(4)
|
||||
|
||||
{
|
||||
// Both success
|
||||
suite.reportMgr.On("Get", "rp-uuid-002").Return(&scan.Report{
|
||||
ID: 12,
|
||||
UUID: "rp-uuid-002",
|
||||
Digest: "digest-code",
|
||||
RegistrationUUID: "uuid001",
|
||||
MimeType: "application/vnd.scanner.adapter.vuln.report.harbor+json; version=1.0",
|
||||
Status: "Success",
|
||||
StatusCode: 3,
|
||||
TrackID: "the-uuid-124",
|
||||
JobID: "the-job-id",
|
||||
StatusRevision: time.Now().Unix(),
|
||||
Report: suite.rawReport,
|
||||
StartTime: time.Now(),
|
||||
EndTime: time.Now().Add(2 * time.Second),
|
||||
}, nil).Once()
|
||||
mock.OnAnything(suite.taskMgr, "GetLog").Return([]byte("log"), nil).Twice()
|
||||
|
||||
bytes, err := suite.c.GetScanLog(base64.StdEncoding.EncodeToString([]byte("rp-uuid-001|rp-uuid-002")))
|
||||
bytes, err := suite.c.GetScanLog(context.TODO(), base64.StdEncoding.EncodeToString([]byte("rp-uuid-001|rp-uuid-002")))
|
||||
suite.Nil(err)
|
||||
suite.NotEmpty(bytes)
|
||||
suite.Contains(string(bytes), "Logs of report rp-uuid-001")
|
||||
@ -360,8 +424,10 @@ func (suite *ControllerTestSuite) TestScanControllerGetMultiScanLog() {
|
||||
|
||||
{
|
||||
// One successfully, one failed
|
||||
suite.reportMgr.On("Get", "rp-uuid-002").Return(nil, fmt.Errorf("error")).Once()
|
||||
bytes, err := suite.c.GetScanLog(base64.StdEncoding.EncodeToString([]byte("rp-uuid-001|rp-uuid-002")))
|
||||
suite.taskMgr.On("GetLog", context.TODO(), int64(1)).Return([]byte("log"), nil).Once()
|
||||
suite.taskMgr.On("GetLog", context.TODO(), int64(2)).Return(nil, fmt.Errorf("failed")).Once()
|
||||
|
||||
bytes, err := suite.c.GetScanLog(context.TODO(), base64.StdEncoding.EncodeToString([]byte("rp-uuid-001|rp-uuid-002")))
|
||||
suite.Nil(err)
|
||||
suite.NotEmpty(bytes)
|
||||
suite.NotContains(string(bytes), "Logs of report rp-uuid-001")
|
||||
@ -369,84 +435,104 @@ func (suite *ControllerTestSuite) TestScanControllerGetMultiScanLog() {
|
||||
|
||||
{
|
||||
// Both failed
|
||||
suite.reportMgr.On("Get", "rp-uuid-002").Return(nil, fmt.Errorf("error")).Once()
|
||||
suite.reportMgr.On("Get", "rp-uuid-003").Return(nil, fmt.Errorf("error")).Once()
|
||||
bytes, err := suite.c.GetScanLog(base64.StdEncoding.EncodeToString([]byte("rp-uuid-002|rp-uuid-003")))
|
||||
mock.OnAnything(suite.taskMgr, "GetLog").Return(nil, fmt.Errorf("failed")).Twice()
|
||||
|
||||
bytes, err := suite.c.GetScanLog(context.TODO(), base64.StdEncoding.EncodeToString([]byte("rp-uuid-001|rp-uuid-002")))
|
||||
suite.Error(err)
|
||||
suite.Empty(bytes)
|
||||
}
|
||||
|
||||
{
|
||||
// Both empty
|
||||
suite.reportMgr.On("Get", "rp-uuid-002").Return(nil, nil).Once()
|
||||
suite.reportMgr.On("Get", "rp-uuid-003").Return(nil, nil).Once()
|
||||
bytes, err := suite.c.GetScanLog(base64.StdEncoding.EncodeToString([]byte("rp-uuid-002|rp-uuid-003")))
|
||||
mock.OnAnything(suite.taskMgr, "GetLog").Return(nil, nil).Twice()
|
||||
|
||||
bytes, err := suite.c.GetScanLog(context.TODO(), base64.StdEncoding.EncodeToString([]byte("rp-uuid-001|rp-uuid-002")))
|
||||
suite.Nil(err)
|
||||
suite.Empty(bytes)
|
||||
}
|
||||
}
|
||||
|
||||
// TestScanControllerHandleJobHooks ...
|
||||
func (suite *ControllerTestSuite) TestScanControllerHandleJobHooks() {
|
||||
cReport := &sca.CheckInReport{
|
||||
Digest: "digest-code",
|
||||
RegistrationUUID: suite.registration.UUID,
|
||||
MimeType: v1.MimeTypeNativeReport,
|
||||
RawReport: suite.rawReport,
|
||||
func (suite *ControllerTestSuite) TestUpdateReport() {
|
||||
{
|
||||
// get report failed
|
||||
suite.reportMgr.On("GetBy", context.TODO(), "digest", "ruuid", []string{"mime"}).Return(nil, fmt.Errorf("failed")).Once()
|
||||
report := &sca.CheckInReport{Digest: "digest", RegistrationUUID: "ruuid", MimeType: "mime"}
|
||||
suite.Error(suite.c.UpdateReport(context.TODO(), report))
|
||||
}
|
||||
|
||||
cRpJSON, err := cReport.ToJSON()
|
||||
require.NoError(suite.T(), err)
|
||||
{
|
||||
// report not found
|
||||
suite.reportMgr.On("GetBy", context.TODO(), "digest", "ruuid", []string{"mime"}).Return(nil, nil).Once()
|
||||
report := &sca.CheckInReport{Digest: "digest", RegistrationUUID: "ruuid", MimeType: "mime"}
|
||||
suite.Error(suite.c.UpdateReport(context.TODO(), report))
|
||||
}
|
||||
}
|
||||
|
||||
statusChange := &job.StatusChange{
|
||||
JobID: "the-job-id",
|
||||
Status: "Success",
|
||||
CheckIn: string(cRpJSON),
|
||||
Metadata: &job.StatsInfo{
|
||||
Revision: (int64)(10000),
|
||||
},
|
||||
func (suite *ControllerTestSuite) TestScanAll() {
|
||||
{
|
||||
// no artifacts found when scan all
|
||||
ctx := context.TODO()
|
||||
|
||||
executionID := int64(1)
|
||||
|
||||
suite.execMgr.On(
|
||||
"Create", ctx, "IMAGE_SCAN_ALL", int64(0), "SCHEDULE", map[string]interface{}{},
|
||||
).Return(executionID, nil).Once()
|
||||
|
||||
mock.OnAnything(suite.artifactCtl, "List").Return([]*artifact.Artifact{}, nil).Once()
|
||||
|
||||
suite.taskMgr.On("Count", ctx, q.New(q.KeyWords{"execution_id": executionID})).Return(int64(0), nil).Once()
|
||||
|
||||
suite.execMgr.On("MarkDone", ctx, executionID, "no artifact found").Return(nil).Once()
|
||||
|
||||
_, err := suite.c.ScanAll(ctx, "SCHEDULE", false)
|
||||
suite.NoError(err)
|
||||
}
|
||||
|
||||
err = suite.c.HandleJobHooks(context.TODO(), "the-uuid-123", statusChange)
|
||||
require.NoError(suite.T(), err)
|
||||
}
|
||||
{
|
||||
// artifacts found, but scan it failed when scan all
|
||||
ctx := orm.NewContext(nil, &ormtesting.FakeOrmer{})
|
||||
|
||||
// Mock things
|
||||
executionID := int64(1)
|
||||
|
||||
// MockJobServiceClient ...
|
||||
type MockJobServiceClient struct {
|
||||
mock.Mock
|
||||
}
|
||||
suite.execMgr.On(
|
||||
"Create", ctx, "IMAGE_SCAN_ALL", int64(0), "SCHEDULE", map[string]interface{}{},
|
||||
).Return(executionID, nil).Once()
|
||||
|
||||
// SubmitJob ...
|
||||
func (mjc *MockJobServiceClient) SubmitJob(jData *jm.JobData) (string, error) {
|
||||
args := mjc.Called(jData)
|
||||
mock.OnAnything(suite.artifactCtl, "List").Return([]*artifact.Artifact{suite.artifact}, nil).Once()
|
||||
mock.OnAnything(suite.ar, "Walk").Return(nil).Run(func(args mock.Arguments) {
|
||||
walkFn := args.Get(2).(func(*artifact.Artifact) error)
|
||||
walkFn(suite.artifact)
|
||||
}).Once()
|
||||
|
||||
return args.String(0), args.Error(1)
|
||||
}
|
||||
mock.OnAnything(suite.taskMgr, "List").Return(nil, nil).Once()
|
||||
|
||||
// GetJobLog ...
|
||||
func (mjc *MockJobServiceClient) GetJobLog(uuid string) ([]byte, error) {
|
||||
args := mjc.Called(uuid)
|
||||
if args.Get(0) == nil {
|
||||
return nil, args.Error(1)
|
||||
mock.OnAnything(suite.reportMgr, "Delete").Return(nil).Once()
|
||||
mock.OnAnything(suite.reportMgr, "Create").Return("uuid", nil).Once()
|
||||
mock.OnAnything(suite.taskMgr, "Create").Return(int64(0), fmt.Errorf("failed")).Once()
|
||||
|
||||
suite.execMgr.On("MarkDone", ctx, executionID, "no task found").Return(nil).Once()
|
||||
|
||||
_, err := suite.c.ScanAll(ctx, "SCHEDULE", false)
|
||||
suite.NoError(err)
|
||||
}
|
||||
|
||||
return args.Get(0).([]byte), args.Error(1)
|
||||
}
|
||||
|
||||
// PostAction ...
|
||||
func (mjc *MockJobServiceClient) PostAction(uuid, action string) error {
|
||||
args := mjc.Called(uuid, action)
|
||||
func (suite *ControllerTestSuite) TestDeleteReports() {
|
||||
suite.reportMgr.On("DeleteByDigests", context.TODO(), "digest").Return(nil).Once()
|
||||
|
||||
return args.Error(0)
|
||||
suite.NoError(suite.c.DeleteReports(context.TODO(), "digest"))
|
||||
|
||||
suite.reportMgr.On("DeleteByDigests", context.TODO(), "digest").Return(fmt.Errorf("failed")).Once()
|
||||
|
||||
suite.Error(suite.c.DeleteReports(context.TODO(), "digest"))
|
||||
}
|
||||
|
||||
func (mjc *MockJobServiceClient) GetExecutions(uuid string) ([]job.Stats, error) {
|
||||
args := mjc.Called(uuid)
|
||||
if args.Get(0) == nil {
|
||||
return nil, args.Error(1)
|
||||
}
|
||||
func (suite *ControllerTestSuite) makeExtraAttrs(reportUUIDs ...string) map[string]interface{} {
|
||||
b, _ := json.Marshal(map[string]interface{}{reportUUIDsKey: reportUUIDs})
|
||||
|
||||
return args.Get(0).([]job.Stats), args.Error(1)
|
||||
extraAttrs := map[string]interface{}{}
|
||||
json.Unmarshal(b, &extraAttrs)
|
||||
|
||||
return extraAttrs
|
||||
}
|
||||
|
118
src/controller/scan/callback.go
Normal file
118
src/controller/scan/callback.go
Normal file
@ -0,0 +1,118 @@
|
||||
// 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 scan
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/goharbor/harbor/src/controller/artifact"
|
||||
"github.com/goharbor/harbor/src/controller/event/metadata"
|
||||
"github.com/goharbor/harbor/src/controller/robot"
|
||||
"github.com/goharbor/harbor/src/jobservice/job"
|
||||
"github.com/goharbor/harbor/src/lib/log"
|
||||
"github.com/goharbor/harbor/src/pkg/notification"
|
||||
"github.com/goharbor/harbor/src/pkg/scan"
|
||||
v1 "github.com/goharbor/harbor/src/pkg/scan/rest/v1"
|
||||
"github.com/goharbor/harbor/src/pkg/scheduler"
|
||||
"github.com/goharbor/harbor/src/pkg/task"
|
||||
)
|
||||
|
||||
const (
|
||||
// ScanAllCallback the scheduler callback name of the scan all
|
||||
ScanAllCallback = "scanAll"
|
||||
)
|
||||
|
||||
func init() {
|
||||
if err := scheduler.RegisterCallbackFunc(ScanAllCallback, scanAllCallback); err != nil {
|
||||
log.Fatalf("failed to register the callback for the scan all schedule, error %v", err)
|
||||
}
|
||||
|
||||
// NOTE: the vendor type of execution for the scan job trigger by the scan all is job.ImageScanAllJob
|
||||
if err := task.RegisterCheckInProcessor(job.ImageScanAllJob, scanTaskCheckInProcessor); err != nil {
|
||||
log.Fatalf("failed to register the checkin processor for the scan all job, error %v", err)
|
||||
}
|
||||
|
||||
if err := task.RegisterCheckInProcessor(job.ImageScanJob, scanTaskCheckInProcessor); err != nil {
|
||||
log.Fatalf("failed to register the checkin processor for the scan job, error %v", err)
|
||||
}
|
||||
|
||||
if err := task.RegisterTaskStatusChangePostFunc(job.ImageScanJob, scanTaskStatusChange); err != nil {
|
||||
log.Fatalf("failed to register the task status change post for the scan job, error %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func scanAllCallback(ctx context.Context, param string) error {
|
||||
_, err := DefaultController.ScanAll(ctx, task.ExecutionTriggerSchedule, true)
|
||||
return err
|
||||
}
|
||||
|
||||
func scanTaskStatusChange(ctx context.Context, taskID int64, status string) (err error) {
|
||||
logger := log.G(ctx).WithFields(log.Fields{"task_id": taskID, "status": status})
|
||||
|
||||
js := job.Status(status)
|
||||
|
||||
if js.Final() {
|
||||
t, err := task.Mgr.Get(ctx, taskID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if js == job.SuccessStatus {
|
||||
robotID := getRobotID(t.ExtraAttrs)
|
||||
if robotID > 0 {
|
||||
if err := robot.Ctl.Delete(ctx, robotID); err != nil {
|
||||
// Should not block the main flow, just logged
|
||||
logger.WithFields(log.Fields{"robot_id": robotID, "error": err}).Error("delete robot account failed")
|
||||
} else {
|
||||
logger.WithField("robot_id", robotID).Debug("Robot account for the scan task is removed")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
artifactID := getArtifactID(t.ExtraAttrs)
|
||||
if artifactID > 0 {
|
||||
art, err := artifact.Ctl.Get(ctx, artifactID, nil)
|
||||
if err != nil {
|
||||
logger.WithFields(log.Fields{"artifact_id": artifactID, "error": err}).Errorf("failed to get artifact")
|
||||
} else {
|
||||
e := &metadata.ScanImageMetaData{
|
||||
Artifact: &v1.Artifact{
|
||||
NamespaceID: art.ProjectID,
|
||||
Repository: art.RepositoryName,
|
||||
Digest: art.Digest,
|
||||
MimeType: art.ManifestMediaType,
|
||||
},
|
||||
Status: status,
|
||||
}
|
||||
// fire event
|
||||
notification.AddEvent(ctx, e)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// scanTaskCheckInProcessor checkin processor handles the webhook of scan job
|
||||
func scanTaskCheckInProcessor(ctx context.Context, t *task.Task, data string) (err error) {
|
||||
checkInReport := &scan.CheckInReport{}
|
||||
if err := checkInReport.FromJSON(data); err != nil {
|
||||
log.G(ctx).WithField("error", err).Errorf("failed to convert data to report")
|
||||
return err
|
||||
}
|
||||
|
||||
return DefaultController.UpdateReport(ctx, checkInReport)
|
||||
}
|
218
src/controller/scan/callback_test.go
Normal file
218
src/controller/scan/callback_test.go
Normal file
@ -0,0 +1,218 @@
|
||||
// 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 scan
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/goharbor/harbor/src/controller/artifact"
|
||||
"github.com/goharbor/harbor/src/controller/robot"
|
||||
"github.com/goharbor/harbor/src/jobservice/job"
|
||||
"github.com/goharbor/harbor/src/lib/q"
|
||||
"github.com/goharbor/harbor/src/pkg/scan"
|
||||
dscan "github.com/goharbor/harbor/src/pkg/scan/dao/scan"
|
||||
"github.com/goharbor/harbor/src/pkg/task"
|
||||
artifacttesting "github.com/goharbor/harbor/src/testing/controller/artifact"
|
||||
robottesting "github.com/goharbor/harbor/src/testing/controller/robot"
|
||||
"github.com/goharbor/harbor/src/testing/mock"
|
||||
reporttesting "github.com/goharbor/harbor/src/testing/pkg/scan/report"
|
||||
tasktesting "github.com/goharbor/harbor/src/testing/pkg/task"
|
||||
"github.com/stretchr/testify/suite"
|
||||
)
|
||||
|
||||
type CallbackTestSuite struct {
|
||||
suite.Suite
|
||||
|
||||
artifactCtl *artifacttesting.Controller
|
||||
originalArtifactCtl artifact.Controller
|
||||
|
||||
execMgr *tasktesting.ExecutionManager
|
||||
|
||||
robotCtl *robottesting.Controller
|
||||
originalRobotCtl robot.Controller
|
||||
|
||||
reportMgr *reporttesting.Manager
|
||||
|
||||
scanCtl Controller
|
||||
originalScanCtl Controller
|
||||
|
||||
taskMgr *tasktesting.Manager
|
||||
originalTaskMgr task.Manager
|
||||
}
|
||||
|
||||
func (suite *CallbackTestSuite) SetupSuite() {
|
||||
suite.originalArtifactCtl = artifact.Ctl
|
||||
suite.artifactCtl = &artifacttesting.Controller{}
|
||||
artifact.Ctl = suite.artifactCtl
|
||||
|
||||
suite.execMgr = &tasktesting.ExecutionManager{}
|
||||
|
||||
suite.originalRobotCtl = robot.Ctl
|
||||
suite.robotCtl = &robottesting.Controller{}
|
||||
robot.Ctl = suite.robotCtl
|
||||
|
||||
suite.reportMgr = &reporttesting.Manager{}
|
||||
|
||||
suite.originalTaskMgr = task.Mgr
|
||||
suite.taskMgr = &tasktesting.Manager{}
|
||||
task.Mgr = suite.taskMgr
|
||||
|
||||
suite.originalScanCtl = DefaultController
|
||||
suite.scanCtl = &basicController{
|
||||
makeCtx: context.TODO,
|
||||
manager: suite.reportMgr,
|
||||
execMgr: suite.execMgr,
|
||||
taskMgr: suite.taskMgr,
|
||||
}
|
||||
DefaultController = suite.scanCtl
|
||||
}
|
||||
|
||||
func (suite *CallbackTestSuite) TearDownSuite() {
|
||||
DefaultController = suite.originalScanCtl
|
||||
|
||||
artifact.Ctl = suite.originalArtifactCtl
|
||||
robot.Ctl = suite.originalRobotCtl
|
||||
task.Mgr = suite.originalTaskMgr
|
||||
}
|
||||
|
||||
func (suite *CallbackTestSuite) TestScanTaskStatusChange() {
|
||||
{
|
||||
// get task failed
|
||||
suite.taskMgr.On("Get", context.TODO(), int64(1)).Return(nil, fmt.Errorf("not found")).Once()
|
||||
suite.Error(scanTaskStatusChange(context.TODO(), 1, job.SuccessStatus.String()))
|
||||
}
|
||||
|
||||
{
|
||||
// status success
|
||||
suite.taskMgr.On("Get", context.TODO(), int64(1)).Return(
|
||||
&task.Task{
|
||||
ExtraAttrs: suite.makeExtraAttrs(0, 1),
|
||||
},
|
||||
nil,
|
||||
).Once()
|
||||
suite.robotCtl.On("Delete", context.TODO(), int64(1)).Return(nil).Once()
|
||||
suite.NoError(scanTaskStatusChange(context.TODO(), 1, job.SuccessStatus.String()))
|
||||
}
|
||||
|
||||
{
|
||||
// status success, delete robot failed
|
||||
suite.taskMgr.On("Get", context.TODO(), int64(1)).Return(
|
||||
&task.Task{
|
||||
ExtraAttrs: suite.makeExtraAttrs(0, 1),
|
||||
},
|
||||
nil,
|
||||
).Once()
|
||||
suite.robotCtl.On("Delete", context.TODO(), int64(1)).Return(fmt.Errorf("failed")).Once()
|
||||
suite.NoError(scanTaskStatusChange(context.TODO(), 1, job.SuccessStatus.String()))
|
||||
}
|
||||
|
||||
{
|
||||
// status success, artifact not found
|
||||
suite.taskMgr.On("Get", context.TODO(), int64(1)).Return(
|
||||
&task.Task{
|
||||
ExtraAttrs: suite.makeExtraAttrs(1, 0),
|
||||
},
|
||||
nil,
|
||||
).Once()
|
||||
suite.artifactCtl.On("Get", context.TODO(), int64(1), (*artifact.Option)(nil)).Return(nil, fmt.Errorf("not found")).Once()
|
||||
suite.NoError(scanTaskStatusChange(context.TODO(), 1, job.SuccessStatus.String()))
|
||||
}
|
||||
|
||||
{
|
||||
// status success
|
||||
suite.taskMgr.On("Get", context.TODO(), int64(1)).Return(
|
||||
&task.Task{
|
||||
ExtraAttrs: suite.makeExtraAttrs(1, 0),
|
||||
},
|
||||
nil,
|
||||
).Once()
|
||||
suite.artifactCtl.On("Get", context.TODO(), int64(1), (*artifact.Option)(nil)).Return(&artifact.Artifact{}, nil).Once()
|
||||
suite.NoError(scanTaskStatusChange(context.TODO(), 1, job.SuccessStatus.String()))
|
||||
}
|
||||
}
|
||||
|
||||
func (suite *CallbackTestSuite) TestScanTaskCheckInProcessor() {
|
||||
{
|
||||
suite.Error(scanTaskCheckInProcessor(context.TODO(), &task.Task{}, "report"))
|
||||
}
|
||||
|
||||
{
|
||||
suite.reportMgr.On("GetBy", context.TODO(), "digest", "ruuid", []string{"mime_type"}).Return(
|
||||
[]*dscan.Report{
|
||||
{UUID: "uuid"},
|
||||
},
|
||||
nil,
|
||||
).Once()
|
||||
|
||||
suite.reportMgr.On("UpdateReportData", context.TODO(), "uuid", "raw_report").Return(nil)
|
||||
|
||||
report := scan.CheckInReport{
|
||||
Digest: "digest",
|
||||
RegistrationUUID: "ruuid",
|
||||
MimeType: "mime_type",
|
||||
RawReport: "raw_report",
|
||||
}
|
||||
|
||||
r, _ := json.Marshal(report)
|
||||
suite.NoError(scanTaskCheckInProcessor(context.TODO(), &task.Task{}, string(r)))
|
||||
}
|
||||
}
|
||||
|
||||
func (suite *CallbackTestSuite) TestScanAllCallback() {
|
||||
{
|
||||
// create execution failed
|
||||
suite.execMgr.On(
|
||||
"Create", context.TODO(), "IMAGE_SCAN_ALL", int64(0), "SCHEDULE", map[string]interface{}{},
|
||||
).Return(int64(0), fmt.Errorf("failed")).Once()
|
||||
|
||||
suite.Error(scanAllCallback(context.TODO(), ""))
|
||||
}
|
||||
|
||||
{
|
||||
executionID := int64(1)
|
||||
|
||||
suite.execMgr.On(
|
||||
"Create", context.TODO(), "IMAGE_SCAN_ALL", int64(0), "SCHEDULE", map[string]interface{}{},
|
||||
).Return(executionID, nil).Once()
|
||||
|
||||
suite.execMgr.On(
|
||||
"Get", context.TODO(), executionID,
|
||||
).Return(&task.Execution{}, nil)
|
||||
|
||||
mock.OnAnything(suite.artifactCtl, "List").Return([]*artifact.Artifact{}, nil).Once()
|
||||
|
||||
suite.taskMgr.On("Count", context.TODO(), q.New(q.KeyWords{"execution_id": executionID})).Return(int64(0), nil).Once()
|
||||
|
||||
suite.execMgr.On("MarkDone", context.TODO(), executionID, "no artifact found").Return(nil).Once()
|
||||
|
||||
suite.NoError(scanAllCallback(context.TODO(), ""))
|
||||
}
|
||||
}
|
||||
|
||||
func (suite *CallbackTestSuite) makeExtraAttrs(artifactID, robotID int64) map[string]interface{} {
|
||||
b, _ := json.Marshal(map[string]interface{}{artifactIDKey: artifactID, robotIDKey: robotID})
|
||||
|
||||
extraAttrs := map[string]interface{}{}
|
||||
json.Unmarshal(b, &extraAttrs)
|
||||
|
||||
return extraAttrs
|
||||
}
|
||||
|
||||
func TestCallbackTestSuite(t *testing.T) {
|
||||
suite.Run(t, &CallbackTestSuite{})
|
||||
}
|
@ -18,8 +18,7 @@ import (
|
||||
"context"
|
||||
|
||||
"github.com/goharbor/harbor/src/controller/artifact"
|
||||
"github.com/goharbor/harbor/src/jobservice/job"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/all"
|
||||
sca "github.com/goharbor/harbor/src/pkg/scan"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/dao/scan"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/report"
|
||||
)
|
||||
@ -31,6 +30,7 @@ type Controller interface {
|
||||
// Scan the given artifact
|
||||
//
|
||||
// Arguments:
|
||||
// ctx context.Context : the context for this method
|
||||
// artifact *artifact.Artifact : artifact to be scanned
|
||||
// options ...Option : options for triggering a scan
|
||||
//
|
||||
@ -41,6 +41,7 @@ type Controller interface {
|
||||
// GetReport gets the reports for the given artifact identified by the digest
|
||||
//
|
||||
// Arguments:
|
||||
// ctx context.Context : the context for this method
|
||||
// artifact *v1.Artifact : the scanned artifact
|
||||
// mimeTypes []string : the mime types of the reports
|
||||
//
|
||||
@ -52,6 +53,7 @@ type Controller interface {
|
||||
// GetSummary gets the summaries of the reports with given types.
|
||||
//
|
||||
// Arguments:
|
||||
// ctx context.Context : the context for this method
|
||||
// artifact *artifact.Artifact : the scanned artifact
|
||||
// mimeTypes []string : the mime types of the reports
|
||||
// options ...report.Option : optional report options, specify if needed
|
||||
@ -64,23 +66,13 @@ type Controller interface {
|
||||
// Get the scan log for the specified artifact with the given digest
|
||||
//
|
||||
// Arguments:
|
||||
// ctx context.Context : the context for this method
|
||||
// uuid string : the UUID of the scan report
|
||||
//
|
||||
// Returns:
|
||||
// []byte : the log text stream
|
||||
// error : non nil error if any errors occurred
|
||||
GetScanLog(uuid string) ([]byte, error)
|
||||
|
||||
// HandleJobHooks handle the hook events from the job service
|
||||
// e.g : status change of the scan job or scan result
|
||||
//
|
||||
// Arguments:
|
||||
// trackID string : UUID for the report record
|
||||
// change *job.StatusChange : change event from the job service
|
||||
//
|
||||
// Returns:
|
||||
// error : non nil error if any errors occurred
|
||||
HandleJobHooks(ctx context.Context, trackID string, change *job.StatusChange) error
|
||||
GetScanLog(ctx context.Context, uuid string) ([]byte, error)
|
||||
|
||||
// Delete the reports related with the specified digests
|
||||
//
|
||||
@ -89,15 +81,26 @@ type Controller interface {
|
||||
//
|
||||
// Returns:
|
||||
// error : non nil error if any errors occurred
|
||||
DeleteReports(digests ...string) error
|
||||
DeleteReports(ctx context.Context, digests ...string) error
|
||||
|
||||
// Get the stats of the scan reports requested by the given requester.
|
||||
// UpdateReport update the report
|
||||
//
|
||||
// Arguments:
|
||||
// requester string : requester identity
|
||||
// Arguments:
|
||||
// ctx context.Context : the context for this method
|
||||
// report *sca.CheckInReport : the scan report
|
||||
//
|
||||
// Returns:
|
||||
// *all.AllStats: stats object including the related metric data
|
||||
// error : non nil error if any errors occurred
|
||||
GetStats(requester string) (*all.Stats, error)
|
||||
// Returns:
|
||||
// error : non nil error if any errors occurred
|
||||
UpdateReport(ctx context.Context, report *sca.CheckInReport) error
|
||||
|
||||
// Scan all the artifacts
|
||||
//
|
||||
// Arguments:
|
||||
// ctx context.Context : the context for this method
|
||||
// trigger string : the trigger mode to start the scan all job
|
||||
// async bool : scan all the artifacts in background
|
||||
//
|
||||
// Returns:
|
||||
// error : non nil error if any errors occurred
|
||||
ScanAll(ctx context.Context, trigger string, async bool) (int64, error)
|
||||
}
|
||||
|
@ -16,9 +16,7 @@ package scan
|
||||
|
||||
// Options keep the settings/configurations for scanning.
|
||||
type Options struct {
|
||||
// Mark the scan triggered by who.
|
||||
// Identified by the UUID.
|
||||
Requester string
|
||||
ExecutionID int64 // The execution id to scan artifact
|
||||
}
|
||||
|
||||
// Option represents an option item by func template.
|
||||
@ -28,10 +26,10 @@ type Options struct {
|
||||
// then a non nil error should be returned at then.
|
||||
type Option func(options *Options) error
|
||||
|
||||
// WithRequester sets the requester option.
|
||||
func WithRequester(Requester string) Option {
|
||||
// WithExecutionID sets the execution id option.
|
||||
func WithExecutionID(executionID int64) Option {
|
||||
return func(options *Options) error {
|
||||
options.Requester = Requester
|
||||
options.ExecutionID = executionID
|
||||
|
||||
return nil
|
||||
}
|
||||
|
@ -25,7 +25,6 @@ import (
|
||||
common_http "github.com/goharbor/harbor/src/common/http"
|
||||
common_job "github.com/goharbor/harbor/src/common/job"
|
||||
common_models "github.com/goharbor/harbor/src/common/models"
|
||||
"github.com/goharbor/harbor/src/controller/scan"
|
||||
"github.com/goharbor/harbor/src/core/api/models"
|
||||
utils_core "github.com/goharbor/harbor/src/core/utils"
|
||||
"github.com/goharbor/harbor/src/lib/errors"
|
||||
@ -264,23 +263,6 @@ func (aj *AJAPI) submit(ajr *models.AdminJobReq) {
|
||||
aj.SendConflictError(errors.Wrap(err, "submit : AJAPI"))
|
||||
return
|
||||
}
|
||||
|
||||
// For scan all job, check more
|
||||
if jb.Name == common_job.ImageScanAllJob {
|
||||
// Get the overall stats with the ID of the previous job
|
||||
stats, err := scan.DefaultController.GetStats(fmt.Sprintf("%d", jb.ID))
|
||||
if err != nil {
|
||||
aj.SendInternalServerError(errors.Wrap(err, "submit : AJAPI"))
|
||||
return
|
||||
}
|
||||
|
||||
if stats.Total != stats.Completed {
|
||||
// Not all scan processes are completed
|
||||
err := errors.Errorf("scan processes started by %s job with ID %d is in progress: %s", jb.Name, jb.ID, progress(stats.Completed, stats.Total))
|
||||
aj.SendPreconditionFailedError(errors.Wrap(err, "submit : AJAPI"))
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -4,10 +4,7 @@ import (
|
||||
"fmt"
|
||||
"net/http"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
common_job "github.com/goharbor/harbor/src/common/job"
|
||||
cm "github.com/goharbor/harbor/src/common/models"
|
||||
"github.com/goharbor/harbor/src/controller/scan"
|
||||
"github.com/goharbor/harbor/src/controller/scanner"
|
||||
"github.com/goharbor/harbor/src/core/api/models"
|
||||
@ -15,11 +12,13 @@ import (
|
||||
"github.com/goharbor/harbor/src/lib/errors"
|
||||
"github.com/goharbor/harbor/src/lib/q"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/all"
|
||||
"github.com/goharbor/harbor/src/pkg/scheduler"
|
||||
"github.com/goharbor/harbor/src/pkg/task"
|
||||
)
|
||||
|
||||
// ScanAllAPI handles request of scan all images...
|
||||
type ScanAllAPI struct {
|
||||
AJAPI
|
||||
BaseController
|
||||
}
|
||||
|
||||
// Prepare validates the URL and parms, it needs the system admin permission.
|
||||
@ -68,9 +67,57 @@ func (sc *ScanAllAPI) Post() {
|
||||
sc.SendBadRequestError(err)
|
||||
return
|
||||
}
|
||||
ajr.Name = common_job.ImageScanAllJob
|
||||
sc.submit(&ajr)
|
||||
sc.Redirect(http.StatusCreated, strconv.FormatInt(ajr.ID, 10))
|
||||
|
||||
if ajr.Schedule == nil {
|
||||
sc.SendBadRequestError(fmt.Errorf("schedule is required"))
|
||||
return
|
||||
}
|
||||
|
||||
if ajr.Schedule.Type == models.ScheduleNone {
|
||||
return
|
||||
}
|
||||
|
||||
if ajr.IsPeriodic() {
|
||||
schedule, err := sc.getScanAllSchedule()
|
||||
if err != nil {
|
||||
sc.SendError(err)
|
||||
return
|
||||
}
|
||||
|
||||
if schedule != nil {
|
||||
err := errors.New("fail to set schedule for scan all as always had one, please delete it firstly then to re-schedule")
|
||||
sc.SendPreconditionFailedError(err)
|
||||
return
|
||||
}
|
||||
|
||||
scheduleID, err := sc.createOrUpdateScanAllSchedule(ajr.Schedule.Type, ajr.Schedule.Cron, nil)
|
||||
if err != nil {
|
||||
sc.SendError(err)
|
||||
return
|
||||
}
|
||||
|
||||
sc.Redirect(http.StatusCreated, strconv.FormatInt(scheduleID, 10))
|
||||
} else {
|
||||
execution, err := sc.getLatestScanAllExecution(task.ExecutionTriggerManual)
|
||||
if err != nil {
|
||||
sc.SendError(err)
|
||||
return
|
||||
}
|
||||
|
||||
if execution != nil && execution.IsOnGoing() {
|
||||
err := errors.Errorf("a previous scan all job aleady exits, its status is %s", execution.Status)
|
||||
sc.SendConflictError(err)
|
||||
return
|
||||
}
|
||||
|
||||
executionID, err := scan.DefaultController.ScanAll(sc.Context(), task.ExecutionTriggerManual, true)
|
||||
if err != nil {
|
||||
sc.SendError(err)
|
||||
return
|
||||
}
|
||||
|
||||
sc.Redirect(http.StatusCreated, strconv.FormatInt(executionID, 10))
|
||||
}
|
||||
}
|
||||
|
||||
// Put handles scan all cron schedule update/delete.
|
||||
@ -88,57 +135,138 @@ func (sc *ScanAllAPI) Put() {
|
||||
sc.SendBadRequestError(err)
|
||||
return
|
||||
}
|
||||
ajr.Name = common_job.ImageScanAllJob
|
||||
sc.updateSchedule(ajr)
|
||||
|
||||
if ajr.Schedule.Type == models.ScheduleManual {
|
||||
err := fmt.Errorf("fail to update scan all schedule as wrong schedule type: %s", ajr.Schedule.Type)
|
||||
sc.SendBadRequestError(err)
|
||||
return
|
||||
}
|
||||
|
||||
schedule, err := sc.getScanAllSchedule()
|
||||
if err != nil {
|
||||
sc.SendError(err)
|
||||
return
|
||||
}
|
||||
|
||||
if ajr.Schedule.Type == models.ScheduleNone {
|
||||
if schedule != nil {
|
||||
err = scheduler.Sched.UnScheduleByID(sc.Context(), schedule.ID)
|
||||
}
|
||||
} else {
|
||||
_, err = sc.createOrUpdateScanAllSchedule(ajr.Schedule.Type, ajr.Schedule.Cron, schedule)
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
sc.SendError(err)
|
||||
}
|
||||
}
|
||||
|
||||
// Get gets scan all schedule ...
|
||||
func (sc *ScanAllAPI) Get() {
|
||||
sc.getSchedule(common_job.ImageScanAllJob)
|
||||
}
|
||||
result := models.AdminJobRep{}
|
||||
|
||||
// List returns the top 10 executions of scan all which includes manual and cron.
|
||||
func (sc *ScanAllAPI) List() {
|
||||
sc.list(common_job.ImageScanAllJob)
|
||||
schedule, err := sc.getScanAllSchedule()
|
||||
if err != nil {
|
||||
sc.SendError(err)
|
||||
return
|
||||
}
|
||||
|
||||
if schedule != nil {
|
||||
result.ID = schedule.ID
|
||||
result.Status = schedule.Status
|
||||
result.CreationTime = schedule.CreationTime
|
||||
result.UpdateTime = schedule.UpdateTime
|
||||
result.Schedule = &models.ScheduleParam{
|
||||
Type: schedule.CRONType,
|
||||
Cron: schedule.CRON,
|
||||
}
|
||||
}
|
||||
|
||||
sc.Data["json"] = result
|
||||
sc.ServeJSON()
|
||||
}
|
||||
|
||||
// GetScheduleMetrics returns the progress metrics for the latest scheduled scan all job
|
||||
func (sc *ScanAllAPI) GetScheduleMetrics() {
|
||||
sc.getMetrics(common_job.JobKindPeriodic)
|
||||
sc.getMetrics(task.ExecutionTriggerSchedule)
|
||||
}
|
||||
|
||||
// GetScanAllMetrics returns the progress metrics for the latest manually triggered scan all job
|
||||
func (sc *ScanAllAPI) GetScanAllMetrics() {
|
||||
sc.getMetrics(common_job.JobKindGeneric)
|
||||
sc.getMetrics(task.ExecutionTriggerManual)
|
||||
}
|
||||
|
||||
func (sc *ScanAllAPI) getMetrics(kind string) {
|
||||
aj, err := sc.getLatestAdminJob(common_job.ImageScanAllJob, kind)
|
||||
func (sc *ScanAllAPI) getMetrics(trigger string) {
|
||||
execution, err := sc.getLatestScanAllExecution(trigger)
|
||||
if err != nil {
|
||||
sc.SendInternalServerError(errors.Wrap(err, "get metrics: scan all API"))
|
||||
sc.SendError(err)
|
||||
return
|
||||
}
|
||||
|
||||
var sts *all.Stats
|
||||
if aj != nil {
|
||||
sts, err = scan.DefaultController.GetStats(fmt.Sprintf("%d", aj.ID))
|
||||
if err != nil {
|
||||
sc.SendInternalServerError(errors.Wrap(err, "get metrics: scan all API"))
|
||||
return
|
||||
sts := &all.Stats{}
|
||||
if execution != nil && execution.Metrics != nil {
|
||||
metrics := execution.Metrics
|
||||
sts.Total = uint(metrics.TaskCount)
|
||||
sts.Completed = uint(metrics.SuccessTaskCount)
|
||||
sts.Metrics = map[string]uint{
|
||||
"Pending": uint(metrics.PendingTaskCount),
|
||||
"Running": uint(metrics.RunningTaskCount),
|
||||
"Success": uint(metrics.SuccessTaskCount),
|
||||
"Error": uint(metrics.ErrorTaskCount),
|
||||
"Stopped": uint(metrics.StoppedTaskCount),
|
||||
}
|
||||
|
||||
setOngoing(sts, aj.Status, kind)
|
||||
}
|
||||
|
||||
// Return empty
|
||||
if sts == nil {
|
||||
sts = &all.Stats{}
|
||||
sts.Ongoing = !job.Status(execution.Status).Final() || sts.Total != sts.Completed
|
||||
}
|
||||
|
||||
sc.Data["json"] = sts
|
||||
sc.ServeJSON()
|
||||
}
|
||||
|
||||
func (sc *ScanAllAPI) getScanAllSchedule() (*scheduler.Schedule, error) {
|
||||
query := q.New(q.KeyWords{"vendor_type": job.ImageScanAllJob})
|
||||
schedules, err := scheduler.Sched.ListSchedules(sc.Context(), query.First("-creation_time"))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if len(schedules) > 1 {
|
||||
msg := "found more than one scheduled scan all job, please ensure that only one schedule left"
|
||||
return nil, errors.BadRequestError(nil).WithMessage(msg)
|
||||
} else if len(schedules) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
return schedules[0], nil
|
||||
}
|
||||
|
||||
func (sc *ScanAllAPI) createOrUpdateScanAllSchedule(cronType, cron string, previous *scheduler.Schedule) (int64, error) {
|
||||
if previous != nil {
|
||||
if cronType == previous.CRONType && cron == previous.CRON {
|
||||
return previous.ID, nil
|
||||
}
|
||||
|
||||
if err := scheduler.Sched.UnScheduleByID(sc.Context(), previous.ID); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
}
|
||||
|
||||
return scheduler.Sched.Schedule(sc.Context(), job.ImageScanAllJob, 0, cronType, cron, scan.ScanAllCallback, nil, nil)
|
||||
}
|
||||
|
||||
func (sc *ScanAllAPI) getLatestScanAllExecution(trigger string) (*task.Execution, error) {
|
||||
query := q.New(q.KeyWords{"vendor_type": job.ImageScanAllJob, "trigger": trigger})
|
||||
executions, err := task.ExecMgr.List(sc.Context(), query.First("-start_time"))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if len(executions) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
return executions[0], nil
|
||||
}
|
||||
|
||||
func isScanEnabled() (bool, error) {
|
||||
kws := make(map[string]interface{})
|
||||
kws["is_default"] = true
|
||||
@ -154,19 +282,3 @@ func isScanEnabled() (bool, error) {
|
||||
|
||||
return len(l) > 0, nil
|
||||
}
|
||||
|
||||
func setOngoing(stats *all.Stats, jobStatus, jobKind string) {
|
||||
status := job.PendingStatus
|
||||
|
||||
if jobStatus == cm.JobFinished {
|
||||
status = job.SuccessStatus
|
||||
} else {
|
||||
status = job.Status(strings.Title(jobStatus))
|
||||
}
|
||||
|
||||
if jobKind == common_job.JobKindPeriodic {
|
||||
stats.Ongoing = (status == job.RunningStatus) || stats.Total != stats.Completed
|
||||
} else if jobKind == common_job.JobKindGeneric {
|
||||
stats.Ongoing = !status.Final() || stats.Total != stats.Completed
|
||||
}
|
||||
}
|
||||
|
@ -17,9 +17,6 @@ package api
|
||||
import (
|
||||
"testing"
|
||||
|
||||
common_job "github.com/goharbor/harbor/src/common/job"
|
||||
cm "github.com/goharbor/harbor/src/common/models"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/all"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/dao/scanner"
|
||||
sc "github.com/goharbor/harbor/src/pkg/scan/scanner"
|
||||
"github.com/goharbor/harbor/src/testing/apitests/apilib"
|
||||
@ -71,6 +68,7 @@ func (suite *ScanAllAPITestSuite) TestScanAllPost() {
|
||||
apiTest := newHarborAPI()
|
||||
|
||||
// case 1: add a new scan all job
|
||||
adminJob002.Schedule = &apilib.ScheduleParam{Type: "Manual"}
|
||||
code, err := apiTest.AddScanAll(*admin, adminJob002)
|
||||
require.NoError(suite.T(), err, "Error occurred while add a scan all job")
|
||||
suite.Equal(201, code, "Add scan all status should be 200")
|
||||
@ -83,53 +81,3 @@ func (suite *ScanAllAPITestSuite) TestScanAllGet() {
|
||||
require.NoError(suite.T(), err, "Error occurred while get a scan all job")
|
||||
suite.Equal(200, code, "Get scan all status should be 200")
|
||||
}
|
||||
|
||||
func (suite *ScanAllAPITestSuite) TestSetOngoing() {
|
||||
{
|
||||
stats := &all.Stats{}
|
||||
setOngoing(stats, cm.JobPending, common_job.JobKindPeriodic)
|
||||
suite.False(stats.Ongoing)
|
||||
}
|
||||
|
||||
{
|
||||
stats := &all.Stats{}
|
||||
setOngoing(stats, cm.JobRunning, common_job.JobKindPeriodic)
|
||||
suite.True(stats.Ongoing)
|
||||
}
|
||||
|
||||
{
|
||||
stats := &all.Stats{}
|
||||
setOngoing(stats, cm.JobFinished, common_job.JobKindPeriodic)
|
||||
suite.False(stats.Ongoing)
|
||||
}
|
||||
|
||||
{
|
||||
stats := &all.Stats{}
|
||||
setOngoing(stats, cm.JobError, common_job.JobKindPeriodic)
|
||||
suite.False(stats.Ongoing)
|
||||
}
|
||||
|
||||
{
|
||||
stats := &all.Stats{}
|
||||
setOngoing(stats, cm.JobPending, common_job.JobKindGeneric)
|
||||
suite.True(stats.Ongoing)
|
||||
}
|
||||
|
||||
{
|
||||
stats := &all.Stats{}
|
||||
setOngoing(stats, cm.JobRunning, common_job.JobKindGeneric)
|
||||
suite.True(stats.Ongoing)
|
||||
}
|
||||
|
||||
{
|
||||
stats := &all.Stats{}
|
||||
setOngoing(stats, cm.JobFinished, common_job.JobKindGeneric)
|
||||
suite.False(stats.Ongoing)
|
||||
}
|
||||
|
||||
{
|
||||
stats := &all.Stats{}
|
||||
setOngoing(stats, cm.JobError, common_job.JobKindGeneric)
|
||||
suite.False(stats.Ongoing)
|
||||
}
|
||||
}
|
||||
|
@ -24,7 +24,6 @@ import (
|
||||
job_model "github.com/goharbor/harbor/src/common/job/models"
|
||||
"github.com/goharbor/harbor/src/common/models"
|
||||
"github.com/goharbor/harbor/src/controller/quota"
|
||||
"github.com/goharbor/harbor/src/controller/scan"
|
||||
"github.com/goharbor/harbor/src/core/config"
|
||||
"github.com/goharbor/harbor/src/core/service/notifications"
|
||||
j "github.com/goharbor/harbor/src/jobservice/job"
|
||||
@ -108,10 +107,7 @@ func (h *Handler) HandleAdminJob() {
|
||||
return
|
||||
}
|
||||
|
||||
// For scan all job
|
||||
if h.jobName == job.ImageScanAllJob && h.checkIn != "" {
|
||||
go scan.HandleCheckIn(orm.NewContext(context.TODO(), o.NewOrm()), h.checkIn)
|
||||
} else if h.jobName == job.ImageGC && h.status == models.JobFinished {
|
||||
if h.jobName == job.ImageGC && h.status == models.JobFinished {
|
||||
go func() {
|
||||
if config.QuotaPerProjectEnable() {
|
||||
quota.RefreshForProjects(orm.NewContext(context.TODO(), o.NewOrm()))
|
||||
|
@ -16,13 +16,11 @@ package jobs
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"github.com/goharbor/harbor/src/lib/orm"
|
||||
"time"
|
||||
|
||||
"github.com/goharbor/harbor/src/common/job"
|
||||
"github.com/goharbor/harbor/src/common/models"
|
||||
"github.com/goharbor/harbor/src/controller/event/metadata"
|
||||
"github.com/goharbor/harbor/src/controller/scan"
|
||||
"github.com/goharbor/harbor/src/core/service/notifications"
|
||||
jjob "github.com/goharbor/harbor/src/jobservice/job"
|
||||
"github.com/goharbor/harbor/src/lib/errors"
|
||||
@ -31,7 +29,6 @@ import (
|
||||
"github.com/goharbor/harbor/src/pkg/notification"
|
||||
"github.com/goharbor/harbor/src/pkg/notifier/event"
|
||||
"github.com/goharbor/harbor/src/pkg/retention"
|
||||
sc "github.com/goharbor/harbor/src/pkg/scan"
|
||||
)
|
||||
|
||||
var statusMap = map[string]string{
|
||||
@ -94,49 +91,8 @@ func (h *Handler) Prepare() {
|
||||
|
||||
// HandleScan handles the webhook of scan job
|
||||
func (h *Handler) HandleScan() {
|
||||
log.Debugf(
|
||||
"Received scan job status update event: job UUID: %s, status: %s, track_id: %s, revision: %d, is checkin: %v",
|
||||
h.change.JobID,
|
||||
h.status,
|
||||
h.trackID,
|
||||
h.revision,
|
||||
len(h.checkIn) > 0,
|
||||
)
|
||||
|
||||
// Trigger image scan webhook event only for JobFinished and JobError status
|
||||
if h.status == models.JobFinished ||
|
||||
h.status == models.JobError ||
|
||||
h.status == models.JobStopped {
|
||||
// Get the required info from the job parameters
|
||||
req, err := sc.ExtractScanReq(h.change.Metadata.Parameters)
|
||||
if err != nil {
|
||||
log.Error(errors.Wrap(err, "scan job hook handler: event publish"))
|
||||
} else {
|
||||
log.Debugf("Scan %s for artifact: %#v", h.status, req.Artifact)
|
||||
|
||||
e := &event.Event{}
|
||||
metaData := &metadata.ScanImageMetaData{
|
||||
Artifact: req.Artifact,
|
||||
Status: h.status,
|
||||
}
|
||||
|
||||
if err := e.Build(metaData); err == nil {
|
||||
if err := e.Publish(); err != nil {
|
||||
log.Error(errors.Wrap(err, "scan job hook handler: event publish"))
|
||||
}
|
||||
} else {
|
||||
log.Error(errors.Wrap(err, "scan job hook handler: event publish"))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if err := scan.DefaultController.HandleJobHooks(orm.Context(), h.trackID, h.change); err != nil {
|
||||
err = errors.Wrap(err, "scan job hook handler")
|
||||
log.Error(err)
|
||||
h.SendInternalServerError(err)
|
||||
|
||||
return
|
||||
}
|
||||
// legacy handler for the scan job
|
||||
return
|
||||
}
|
||||
|
||||
// HandleRetentionTask handles the webhook of retention task
|
||||
|
@ -141,6 +141,11 @@ func (l *Logger) WithFields(fields Fields) *Logger {
|
||||
return r
|
||||
}
|
||||
|
||||
// WithField returns cloned logger which fields merged with field key=value
|
||||
func (l *Logger) WithField(key string, value interface{}) *Logger {
|
||||
return l.WithFields(Fields{key: value})
|
||||
}
|
||||
|
||||
// setOutput sets the output of Logger l
|
||||
func (l *Logger) setOutput(out io.Writer) {
|
||||
l.mu.Lock()
|
||||
|
@ -29,6 +29,17 @@ type Query struct {
|
||||
Sorting string
|
||||
}
|
||||
|
||||
// First make the query only fetch the first one record in the sorting order
|
||||
func (q *Query) First(sorting ...string) *Query {
|
||||
q.PageNumber = 1
|
||||
q.PageSize = 1
|
||||
if len(sorting) > 0 {
|
||||
q.Sorting = sorting[0]
|
||||
}
|
||||
|
||||
return q
|
||||
}
|
||||
|
||||
// New returns Query with keywords
|
||||
func New(kw KeyWords) *Query {
|
||||
return &Query{Keywords: kw}
|
||||
|
@ -16,6 +16,7 @@ package artifact
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/docker/distribution/manifest/manifestlist"
|
||||
@ -45,6 +46,10 @@ type Artifact struct {
|
||||
References []*Reference `json:"references"` // child artifacts referenced by the parent artifact if the artifact is an index
|
||||
}
|
||||
|
||||
func (a *Artifact) String() string {
|
||||
return fmt.Sprintf("%s@%s", a.RepositoryName, a.Digest)
|
||||
}
|
||||
|
||||
// IsImageIndex returns true when artifact is image index
|
||||
func (a *Artifact) IsImageIndex() bool {
|
||||
return a.ManifestMediaType == v1.MediaTypeImageIndex ||
|
||||
|
@ -19,20 +19,16 @@ import "time"
|
||||
// Report of the scan.
|
||||
// Identified by the `digest`, `registration_uuid` and `mime_type`.
|
||||
type Report struct {
|
||||
ID int64 `orm:"pk;auto;column(id)"`
|
||||
UUID string `orm:"unique;column(uuid)"`
|
||||
Digest string `orm:"column(digest)"`
|
||||
RegistrationUUID string `orm:"column(registration_uuid)"`
|
||||
MimeType string `orm:"column(mime_type)"`
|
||||
JobID string `orm:"column(job_id)"`
|
||||
TrackID string `orm:"column(track_id)"`
|
||||
Requester string `orm:"column(requester)"`
|
||||
Status string `orm:"column(status)"`
|
||||
StatusCode int `orm:"column(status_code)"`
|
||||
StatusRevision int64 `orm:"column(status_rev)"`
|
||||
Report string `orm:"column(report);type(json)"`
|
||||
StartTime time.Time `orm:"column(start_time);auto_now_add;type(datetime)"`
|
||||
EndTime time.Time `orm:"column(end_time);type(datetime)"`
|
||||
ID int64 `orm:"pk;auto;column(id)"`
|
||||
UUID string `orm:"unique;column(uuid)"`
|
||||
Digest string `orm:"column(digest)"`
|
||||
RegistrationUUID string `orm:"column(registration_uuid)"`
|
||||
MimeType string `orm:"column(mime_type)"`
|
||||
Report string `orm:"column(report);type(json)"`
|
||||
|
||||
Status string `orm:"-"`
|
||||
StartTime time.Time `orm:"-"`
|
||||
EndTime time.Time `orm:"-"`
|
||||
}
|
||||
|
||||
// TableName for Report
|
||||
|
@ -15,14 +15,10 @@
|
||||
package scan
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strconv"
|
||||
"time"
|
||||
"context"
|
||||
|
||||
"github.com/astaxie/beego/orm"
|
||||
"github.com/goharbor/harbor/src/common/dao"
|
||||
"github.com/goharbor/harbor/src/lib/errors"
|
||||
"github.com/goharbor/harbor/src/lib/log"
|
||||
"github.com/goharbor/harbor/src/lib/orm"
|
||||
"github.com/goharbor/harbor/src/lib/q"
|
||||
)
|
||||
|
||||
@ -30,156 +26,74 @@ func init() {
|
||||
orm.RegisterModel(new(Report))
|
||||
}
|
||||
|
||||
// CreateReport creates new report
|
||||
func CreateReport(r *Report) (int64, error) {
|
||||
o := dao.GetOrmer()
|
||||
// DAO is the data access object interface for report
|
||||
type DAO interface {
|
||||
// Create creates new report
|
||||
Create(ctx context.Context, r *Report) (int64, error)
|
||||
// Delete the reports according to the query
|
||||
DeleteMany(ctx context.Context, query q.Query) (int64, error)
|
||||
// List lists the reports with given query parameters.
|
||||
List(ctx context.Context, query *q.Query) ([]*Report, error)
|
||||
// UpdateReportData only updates the `report` column with conditions matched.
|
||||
UpdateReportData(ctx context.Context, uuid string, report string) error
|
||||
}
|
||||
|
||||
// New returns an instance of the default DAO
|
||||
func New() DAO {
|
||||
return &dao{}
|
||||
}
|
||||
|
||||
type dao struct{}
|
||||
|
||||
// Create creates new report
|
||||
func (d *dao) Create(ctx context.Context, r *Report) (int64, error) {
|
||||
o, err := orm.FromContext(ctx)
|
||||
if err != nil {
|
||||
return 0, orm.WrapConflictError(err, "a previous scan report found for artifact %s", r.Digest)
|
||||
}
|
||||
|
||||
return o.Insert(r)
|
||||
}
|
||||
|
||||
// DeleteReport deletes the given report
|
||||
func DeleteReport(uuid string) error {
|
||||
o := dao.GetOrmer()
|
||||
qt := o.QueryTable(new(Report))
|
||||
func (d *dao) DeleteMany(ctx context.Context, query q.Query) (int64, error) {
|
||||
if len(query.Keywords) == 0 {
|
||||
return 0, errors.New("delete all reports at once is not allowed")
|
||||
}
|
||||
|
||||
// Delete report with query way
|
||||
count, err := qt.Filter("uuid", uuid).Delete()
|
||||
qs, err := orm.QuerySetter(ctx, &Report{}, &query)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
return qs.Delete()
|
||||
}
|
||||
|
||||
func (d *dao) List(ctx context.Context, query *q.Query) ([]*Report, error) {
|
||||
qs, err := orm.QuerySetter(ctx, &Report{}, query)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
reports := []*Report{}
|
||||
if _, err = qs.All(&reports); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return reports, nil
|
||||
}
|
||||
|
||||
// UpdateReportData only updates the `report` column with conditions matched.
|
||||
func (d *dao) UpdateReportData(ctx context.Context, uuid string, report string) error {
|
||||
o, err := orm.FromContext(ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if count == 0 {
|
||||
return errors.Errorf("no report with uuid %s deleted", uuid)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// ListReports lists the reports with given query parameters.
|
||||
// Keywords in query here will be enforced with `exact` way.
|
||||
func ListReports(query *q.Query) ([]*Report, error) {
|
||||
o := dao.GetOrmer()
|
||||
qt := o.QueryTable(new(Report))
|
||||
|
||||
if query != nil {
|
||||
if len(query.Keywords) > 0 {
|
||||
for k, v := range query.Keywords {
|
||||
if vv, ok := v.([]interface{}); ok {
|
||||
qt = qt.Filter(fmt.Sprintf("%s__in", k), vv...)
|
||||
continue
|
||||
}
|
||||
|
||||
qt = qt.Filter(k, v)
|
||||
}
|
||||
}
|
||||
|
||||
if query.PageNumber > 0 && query.PageSize > 0 {
|
||||
qt = qt.Limit(query.PageSize, (query.PageNumber-1)*query.PageSize)
|
||||
}
|
||||
}
|
||||
|
||||
l := make([]*Report, 0)
|
||||
_, err := qt.All(&l)
|
||||
|
||||
return l, err
|
||||
}
|
||||
|
||||
// UpdateReportData only updates the `report` column with conditions matched.
|
||||
func UpdateReportData(uuid string, report string, statusRev int64) error {
|
||||
o := dao.GetOrmer()
|
||||
qt := o.QueryTable(new(Report))
|
||||
|
||||
data := make(orm.Params)
|
||||
data["report"] = report
|
||||
data["status_rev"] = statusRev
|
||||
|
||||
count, err := qt.Filter("uuid", uuid).
|
||||
Filter("status_rev__lte", statusRev).Update(data)
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Update has preconditions which may NOT be matched, and then count may equal 0.
|
||||
// Just need log, no error need to be returned.
|
||||
if count == 0 {
|
||||
log.Warningf("Data of report with uuid %s is not updated as preconditions may not be matched: status change revision %d", uuid, statusRev)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// UpdateReportStatus updates the report `status` with conditions matched.
|
||||
func UpdateReportStatus(trackID string, status string, statusCode int, statusRev int64) error {
|
||||
o := dao.GetOrmer()
|
||||
qt := o.QueryTable(new(Report))
|
||||
|
||||
data := make(orm.Params)
|
||||
data["status"] = status
|
||||
data["status_code"] = statusCode
|
||||
data["status_rev"] = statusRev
|
||||
|
||||
// Technically it is not correct, just to avoid changing interface and adding more code.
|
||||
// running==2
|
||||
if statusCode > 2 {
|
||||
data["end_time"] = time.Now().UTC()
|
||||
}
|
||||
|
||||
// qt generates sql statements:
|
||||
// UPDATE "scan_report" SET "end_time" = $1, "status" = $2, "status_code" = $3, "status_rev" = $4
|
||||
// WHERE "id" IN ( SELECT T0."id" FROM "scan_report" T0 WHERE ( T0."status_rev" = $5 AND T0."status_code" < $6
|
||||
// OR ( T0."status_rev" < $7 ) ) AND ( T0."track_id" = $8 )
|
||||
c1 := orm.NewCondition().And("status_rev", statusRev).And("status_code__lt", statusCode)
|
||||
c2 := orm.NewCondition().And("status_rev__lt", statusRev)
|
||||
c3 := orm.NewCondition().And("track_id", trackID)
|
||||
c := orm.NewCondition().AndCond(c1.OrCond(c2)).AndCond(c3)
|
||||
|
||||
count, err := qt.SetCond(c).Update(data)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Update has preconditions which may NOT be matched, and then count may equal 0.
|
||||
// Just need log, no error need to be returned.
|
||||
if count == 0 {
|
||||
log.Warningf("Status of report with track ID %s is not updated as preconditions may not be matched: status change revision %d, status code %d", trackID, statusRev, statusCode)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// UpdateJobID updates the report `job_id` column
|
||||
func UpdateJobID(trackID string, jobID string) error {
|
||||
o := dao.GetOrmer()
|
||||
qt := o.QueryTable(new(Report))
|
||||
|
||||
params := make(orm.Params, 1)
|
||||
params["job_id"] = jobID
|
||||
_, err := qt.Filter("track_id", trackID).Update(params)
|
||||
|
||||
_, err = qt.Filter("uuid", uuid).Update(data)
|
||||
return err
|
||||
}
|
||||
|
||||
// GetScanStats gets the scan stats organized by status
|
||||
func GetScanStats(requester string) (map[string]uint, error) {
|
||||
res := make(orm.Params)
|
||||
|
||||
o := dao.GetOrmer()
|
||||
if _, err := o.Raw("select status, count(status) from (select status from scan_report where requester=? group by track_id, status) as scan_status group by status").
|
||||
SetArgs(requester).
|
||||
RowsToMap(&res, "status", "count"); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
m := make(map[string]uint)
|
||||
for k, v := range res {
|
||||
vl, err := strconv.ParseInt(v.(string), 10, 32)
|
||||
if err != nil {
|
||||
log.Error(errors.Wrap(err, "get scan stats"))
|
||||
continue
|
||||
}
|
||||
|
||||
m[k] = uint(vl)
|
||||
}
|
||||
|
||||
return m, nil
|
||||
}
|
||||
|
@ -17,12 +17,10 @@ package scan
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/goharbor/harbor/src/common/dao"
|
||||
"github.com/goharbor/harbor/src/jobservice/job"
|
||||
"github.com/goharbor/harbor/src/lib/errors"
|
||||
common "github.com/goharbor/harbor/src/common/dao"
|
||||
"github.com/goharbor/harbor/src/lib/orm"
|
||||
"github.com/goharbor/harbor/src/lib/q"
|
||||
v1 "github.com/goharbor/harbor/src/pkg/scan/rest/v1"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"github.com/stretchr/testify/suite"
|
||||
)
|
||||
@ -30,6 +28,8 @@ import (
|
||||
// ReportTestSuite is test suite of testing report DAO.
|
||||
type ReportTestSuite struct {
|
||||
suite.Suite
|
||||
|
||||
dao DAO
|
||||
}
|
||||
|
||||
// TestReport is the entry of ReportTestSuite.
|
||||
@ -39,20 +39,18 @@ func TestReport(t *testing.T) {
|
||||
|
||||
// SetupSuite prepares env for test suite.
|
||||
func (suite *ReportTestSuite) SetupSuite() {
|
||||
dao.PrepareTestForPostgresSQL()
|
||||
common.PrepareTestForPostgresSQL()
|
||||
|
||||
suite.dao = New()
|
||||
}
|
||||
|
||||
// SetupTest prepares env for test case.
|
||||
func (suite *ReportTestSuite) SetupTest() {
|
||||
r := &Report{
|
||||
UUID: "uuid",
|
||||
TrackID: "track-uuid",
|
||||
Digest: "digest1001",
|
||||
RegistrationUUID: "ruuid",
|
||||
Requester: "requester",
|
||||
MimeType: v1.MimeTypeNativeReport,
|
||||
Status: job.PendingStatus.String(),
|
||||
StatusCode: job.PendingStatus.Code(),
|
||||
}
|
||||
|
||||
suite.create(r)
|
||||
@ -60,7 +58,7 @@ func (suite *ReportTestSuite) SetupTest() {
|
||||
|
||||
// TearDownTest clears enf for test case.
|
||||
func (suite *ReportTestSuite) TearDownTest() {
|
||||
err := DeleteReport("uuid")
|
||||
_, err := suite.dao.DeleteMany(orm.Context(), q.Query{Keywords: q.KeyWords{"uuid": "uuid"}})
|
||||
require.NoError(suite.T(), err)
|
||||
}
|
||||
|
||||
@ -75,9 +73,9 @@ func (suite *ReportTestSuite) TestReportList() {
|
||||
"mime_type": v1.MimeTypeNativeReport,
|
||||
},
|
||||
}
|
||||
l, err := ListReports(query1)
|
||||
require.NoError(suite.T(), err)
|
||||
require.Equal(suite.T(), 1, len(l))
|
||||
l, err := suite.dao.List(orm.Context(), query1)
|
||||
suite.Require().NoError(err)
|
||||
suite.Require().Equal(1, len(l))
|
||||
|
||||
query2 := &q.Query{
|
||||
PageSize: 1,
|
||||
@ -86,147 +84,30 @@ func (suite *ReportTestSuite) TestReportList() {
|
||||
"digest": "digest1002",
|
||||
},
|
||||
}
|
||||
l, err = ListReports(query2)
|
||||
require.NoError(suite.T(), err)
|
||||
require.Equal(suite.T(), 0, len(l))
|
||||
}
|
||||
|
||||
// TestReportUpdateJobID tests update job ID of the report.
|
||||
func (suite *ReportTestSuite) TestReportUpdateJobID() {
|
||||
err := UpdateJobID("track-uuid", "jobid001")
|
||||
require.NoError(suite.T(), err)
|
||||
|
||||
l, err := ListReports(nil)
|
||||
require.NoError(suite.T(), err)
|
||||
require.Equal(suite.T(), 1, len(l))
|
||||
assert.Equal(suite.T(), "jobid001", l[0].JobID)
|
||||
l, err = suite.dao.List(orm.Context(), query2)
|
||||
suite.Require().NoError(err)
|
||||
suite.Require().Equal(0, len(l))
|
||||
}
|
||||
|
||||
// TestReportUpdateReportData tests update the report data.
|
||||
func (suite *ReportTestSuite) TestReportUpdateReportData() {
|
||||
err := UpdateReportData("uuid", "{}", 1000)
|
||||
require.NoError(suite.T(), err)
|
||||
err := suite.dao.UpdateReportData(orm.Context(), "uuid", "{}")
|
||||
suite.Require().NoError(err)
|
||||
|
||||
l, err := ListReports(nil)
|
||||
require.NoError(suite.T(), err)
|
||||
require.Equal(suite.T(), 1, len(l))
|
||||
assert.Equal(suite.T(), "{}", l[0].Report)
|
||||
|
||||
err = UpdateReportData("uuid", "{\"a\": 900}", 900)
|
||||
require.NoError(suite.T(), err)
|
||||
}
|
||||
|
||||
// TestReportUpdateStatus tests update the report status.
|
||||
func (suite *ReportTestSuite) TestReportUpdateStatus() {
|
||||
err := UpdateReportStatus("track-uuid", job.RunningStatus.String(), job.RunningStatus.Code(), 1000)
|
||||
require.NoError(suite.T(), err)
|
||||
|
||||
err = checkStatus("track-uuid", job.RunningStatus.String())
|
||||
suite.NoError(err, "regular status update")
|
||||
|
||||
err = UpdateReportStatus("track-uuid", job.SuccessStatus.String(), job.SuccessStatus.Code(), 900)
|
||||
require.NoError(suite.T(), err)
|
||||
|
||||
err = checkStatus("track-uuid", job.RunningStatus.String())
|
||||
suite.NoError(err, "update with outdated revision")
|
||||
|
||||
err = UpdateReportStatus("track-uuid", job.PendingStatus.String(), job.PendingStatus.Code(), 1000)
|
||||
require.NoError(suite.T(), err)
|
||||
|
||||
err = checkStatus("track-uuid", job.RunningStatus.String())
|
||||
suite.NoError(err, "update with same revision and previous status")
|
||||
|
||||
err = UpdateReportStatus("track-uuid", job.PendingStatus.String(), job.PendingStatus.Code(), 1001)
|
||||
require.NoError(suite.T(), err)
|
||||
|
||||
err = checkStatus("track-uuid", job.PendingStatus.String())
|
||||
suite.NoError(err, "update latest revision and previous status")
|
||||
}
|
||||
|
||||
// TestReportGetStats ...
|
||||
func (suite *ReportTestSuite) TestReportGetStats() {
|
||||
// Two more for getting stats
|
||||
r2 := &Report{
|
||||
UUID: "uuid2",
|
||||
TrackID: "track-uuid2",
|
||||
Digest: "digest1003",
|
||||
RegistrationUUID: "ruuid",
|
||||
Requester: "requester",
|
||||
MimeType: v1.MimeTypeNativeReport,
|
||||
Status: job.RunningStatus.String(),
|
||||
StatusCode: job.RunningStatus.Code(),
|
||||
}
|
||||
suite.create(r2)
|
||||
|
||||
r3 := &Report{
|
||||
UUID: "uuid3",
|
||||
TrackID: "track-uuid2",
|
||||
Digest: "digest1003",
|
||||
RegistrationUUID: "ruuid",
|
||||
Requester: "requester",
|
||||
MimeType: v1.MimeTypeRawReport,
|
||||
Status: job.RunningStatus.String(),
|
||||
StatusCode: job.RunningStatus.Code(),
|
||||
}
|
||||
suite.create(r3)
|
||||
|
||||
defer func() {
|
||||
err := DeleteReport("uuid2")
|
||||
suite.NoError(err)
|
||||
|
||||
err = DeleteReport("uuid3")
|
||||
suite.NoError(err)
|
||||
}()
|
||||
|
||||
m, err := GetScanStats("requester")
|
||||
require.NoError(suite.T(), err)
|
||||
suite.Equal(2, len(m))
|
||||
suite.Condition(func() (success bool) {
|
||||
v, ok := m[job.RunningStatus.String()]
|
||||
vv, ook := m[job.PendingStatus.String()]
|
||||
|
||||
success = ok && ook && v == 1 && vv == 1
|
||||
|
||||
return
|
||||
})
|
||||
l, err := suite.dao.List(orm.Context(), nil)
|
||||
suite.Require().NoError(err)
|
||||
suite.Require().Equal(1, len(l))
|
||||
suite.Equal("{}", l[0].Report)
|
||||
|
||||
err = suite.dao.UpdateReportData(orm.Context(), "uuid", "{\"a\": 900}")
|
||||
suite.Require().NoError(err)
|
||||
}
|
||||
|
||||
func (suite *ReportTestSuite) create(r *Report) {
|
||||
id, err := CreateReport(r)
|
||||
require.NoError(suite.T(), err)
|
||||
require.Condition(suite.T(), func() (success bool) {
|
||||
id, err := suite.dao.Create(orm.Context(), r)
|
||||
suite.Require().NoError(err)
|
||||
suite.Require().Condition(func() (success bool) {
|
||||
success = id > 0
|
||||
return
|
||||
})
|
||||
}
|
||||
|
||||
func list(trackID string) ([]*Report, error) {
|
||||
kws := make(map[string]interface{})
|
||||
kws["track_id"] = trackID
|
||||
query := &q.Query{
|
||||
Keywords: kws,
|
||||
}
|
||||
|
||||
l, err := ListReports(query)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return l, nil
|
||||
}
|
||||
|
||||
func checkStatus(trackID string, status string) error {
|
||||
l, err := list(trackID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for _, r := range l {
|
||||
if r.Status != status {
|
||||
return errors.Errorf("status is not matched: current %s : expected %s", r.Status, status)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
@ -92,7 +92,7 @@ type Job struct{}
|
||||
|
||||
// MaxFails for defining the number of retries
|
||||
func (j *Job) MaxFails() uint {
|
||||
return 3
|
||||
return 1
|
||||
}
|
||||
|
||||
// MaxCurrency is implementation of same method in Interface.
|
||||
@ -102,7 +102,7 @@ func (j *Job) MaxCurrency() uint {
|
||||
|
||||
// ShouldRetry indicates if the job should be retried
|
||||
func (j *Job) ShouldRetry() bool {
|
||||
return true
|
||||
return false
|
||||
}
|
||||
|
||||
// Validate the parameters of this job
|
||||
|
@ -1,282 +0,0 @@
|
||||
// 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 report
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/goharbor/harbor/src/jobservice/job"
|
||||
"github.com/goharbor/harbor/src/lib/errors"
|
||||
"github.com/goharbor/harbor/src/lib/q"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/all"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/dao/scan"
|
||||
"github.com/google/uuid"
|
||||
)
|
||||
|
||||
const (
|
||||
reportTimeout = 1 * time.Hour
|
||||
)
|
||||
|
||||
// basicManager is a default implementation of report manager.
|
||||
type basicManager struct{}
|
||||
|
||||
// NewManager news basic manager.
|
||||
func NewManager() Manager {
|
||||
return &basicManager{}
|
||||
}
|
||||
|
||||
// Create ...
|
||||
func (bm *basicManager) Create(r *scan.Report) (string, error) {
|
||||
// Validate report object
|
||||
if r == nil {
|
||||
return "", errors.New("nil scan report object")
|
||||
}
|
||||
|
||||
if len(r.Digest) == 0 || len(r.RegistrationUUID) == 0 || len(r.MimeType) == 0 {
|
||||
return "", errors.New("malformed scan report object")
|
||||
}
|
||||
|
||||
// Check if there is existing report copy
|
||||
// Limit only one scanning performed by a given provider on the specified artifact can be there
|
||||
kws := make(map[string]interface{}, 3)
|
||||
kws["digest"] = r.Digest
|
||||
kws["registration_uuid"] = r.RegistrationUUID
|
||||
kws["mime_type"] = []interface{}{r.MimeType}
|
||||
|
||||
existingCopies, err := scan.ListReports(&q.Query{
|
||||
PageNumber: 1,
|
||||
PageSize: 1,
|
||||
Keywords: kws,
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
return "", errors.Wrap(err, "create report: check existence of report")
|
||||
}
|
||||
|
||||
// Delete existing copy
|
||||
if len(existingCopies) > 0 {
|
||||
theCopy := existingCopies[0]
|
||||
|
||||
theStatus := job.Status(theCopy.Status)
|
||||
// Status is an error message
|
||||
if theStatus.Code() == -1 && theCopy.StatusCode == job.ErrorStatus.Code() {
|
||||
// Mark as regular error status
|
||||
theStatus = job.ErrorStatus
|
||||
}
|
||||
|
||||
// Status conflict
|
||||
if theCopy.StartTime.Add(reportTimeout).After(time.Now()) {
|
||||
if theStatus.Compare(job.RunningStatus) <= 0 {
|
||||
return "", errors.ConflictError(nil).WithMessage("a previous scan process is %s", theCopy.Status)
|
||||
}
|
||||
}
|
||||
|
||||
// Otherwise it will be a completed report
|
||||
// Clear it before insert this new one
|
||||
if err := scan.DeleteReport(theCopy.UUID); err != nil {
|
||||
return "", errors.Wrap(err, "create report: clear old scan report")
|
||||
}
|
||||
}
|
||||
|
||||
r.UUID = uuid.New().String()
|
||||
|
||||
// Fill in / override the related properties
|
||||
r.StartTime = time.Now().UTC()
|
||||
if r.Status == "" {
|
||||
r.Status = job.PendingStatus.String()
|
||||
r.StatusCode = job.PendingStatus.Code()
|
||||
}
|
||||
|
||||
// Insert
|
||||
if _, err = scan.CreateReport(r); err != nil {
|
||||
return "", errors.Wrap(err, "create report: insert")
|
||||
}
|
||||
|
||||
return r.UUID, nil
|
||||
}
|
||||
|
||||
// Get ...
|
||||
func (bm *basicManager) Get(uuid string) (*scan.Report, error) {
|
||||
if len(uuid) == 0 {
|
||||
return nil, errors.New("empty uuid to get scan report")
|
||||
}
|
||||
|
||||
kws := make(map[string]interface{})
|
||||
kws["uuid"] = uuid
|
||||
|
||||
l, err := scan.ListReports(&q.Query{
|
||||
PageNumber: 1,
|
||||
PageSize: 1,
|
||||
Keywords: kws,
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "report manager: get")
|
||||
}
|
||||
|
||||
if len(l) == 0 {
|
||||
// Not found
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
return l[0], nil
|
||||
}
|
||||
|
||||
// GetBy ...
|
||||
func (bm *basicManager) GetBy(digest string, registrationUUID string, mimeTypes []string) ([]*scan.Report, error) {
|
||||
if len(digest) == 0 {
|
||||
return nil, errors.New("empty digest to get report data")
|
||||
}
|
||||
|
||||
kws := make(map[string]interface{})
|
||||
kws["digest"] = digest
|
||||
if len(registrationUUID) > 0 {
|
||||
kws["registration_uuid"] = registrationUUID
|
||||
}
|
||||
if len(mimeTypes) > 0 {
|
||||
kws["mime_type"] = mimeTypes
|
||||
}
|
||||
// Query all
|
||||
query := &q.Query{
|
||||
PageNumber: 0,
|
||||
Keywords: kws,
|
||||
}
|
||||
|
||||
return scan.ListReports(query)
|
||||
}
|
||||
|
||||
// UpdateScanJobID ...
|
||||
func (bm *basicManager) UpdateScanJobID(trackID string, jobID string) error {
|
||||
if len(trackID) == 0 || len(jobID) == 0 {
|
||||
return errors.New("bad arguments")
|
||||
}
|
||||
|
||||
return scan.UpdateJobID(trackID, jobID)
|
||||
}
|
||||
|
||||
// UpdateStatus ...
|
||||
func (bm *basicManager) UpdateStatus(trackID string, status string, rev int64) error {
|
||||
if len(trackID) == 0 {
|
||||
return errors.New("missing uuid")
|
||||
}
|
||||
|
||||
stCode := job.ErrorStatus.Code()
|
||||
st := job.Status(status)
|
||||
// Check if it is job valid status.
|
||||
// Probably an error happened before submitting jobs.
|
||||
if st.Code() != -1 {
|
||||
// Assign error code
|
||||
stCode = st.Code()
|
||||
}
|
||||
|
||||
return scan.UpdateReportStatus(trackID, status, stCode, rev)
|
||||
}
|
||||
|
||||
// UpdateReportData ...
|
||||
func (bm *basicManager) UpdateReportData(uuid string, report string, rev int64) error {
|
||||
if len(uuid) == 0 {
|
||||
return errors.New("missing uuid")
|
||||
}
|
||||
|
||||
if len(report) == 0 {
|
||||
return errors.New("missing report JSON data")
|
||||
}
|
||||
|
||||
return scan.UpdateReportData(uuid, report, rev)
|
||||
}
|
||||
|
||||
// DeleteByDigests ...
|
||||
func (bm *basicManager) DeleteByDigests(digests ...string) error {
|
||||
if len(digests) == 0 {
|
||||
// Nothing to do
|
||||
return nil
|
||||
}
|
||||
|
||||
kws := make(map[string]interface{})
|
||||
ds := make([]interface{}, 0)
|
||||
|
||||
for _, dig := range digests {
|
||||
ds = append(ds, dig)
|
||||
}
|
||||
|
||||
kws["digest"] = ds
|
||||
query := &q.Query{
|
||||
Keywords: kws,
|
||||
}
|
||||
|
||||
rs, err := scan.ListReports(query)
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "report manager: delete by digests")
|
||||
}
|
||||
|
||||
// Return the combined errors at last
|
||||
for _, r := range rs {
|
||||
if er := scan.DeleteReport(r.UUID); er != nil {
|
||||
if err == nil {
|
||||
err = er
|
||||
} else {
|
||||
err = errors.Wrap(er, err.Error())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
// GetStats ...
|
||||
func (bm *basicManager) GetStats(requester string) (*all.Stats, error) {
|
||||
if len(requester) == 0 {
|
||||
return nil, errors.New("empty requester")
|
||||
}
|
||||
|
||||
m, err := scan.GetScanStats(requester)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "report manager: get stats")
|
||||
}
|
||||
|
||||
sts := &all.Stats{
|
||||
Metrics: make(all.StatusMetrics),
|
||||
}
|
||||
|
||||
for k, v := range m {
|
||||
// Increase the total metrics
|
||||
sts.Total += v
|
||||
|
||||
s := job.Status(k)
|
||||
// Increase the completed metrics if the status is not predefined ones or
|
||||
// the status is the final status.
|
||||
if s.Validate() != nil || s.Final() {
|
||||
sts.Completed += v
|
||||
}
|
||||
|
||||
// Not standard error status.
|
||||
// Convert it to standard error status.
|
||||
if s.Validate() != nil {
|
||||
tv := v
|
||||
|
||||
if val, ok := sts.Metrics[job.ErrorStatus.String()]; ok {
|
||||
tv = val + v
|
||||
}
|
||||
sts.Metrics[job.ErrorStatus.String()] = tv
|
||||
|
||||
continue
|
||||
}
|
||||
|
||||
sts.Metrics[k] = v
|
||||
}
|
||||
sts.Requester = requester
|
||||
|
||||
return sts, nil
|
||||
}
|
@ -1,225 +0,0 @@
|
||||
// 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 report
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/goharbor/harbor/src/common/dao"
|
||||
"github.com/goharbor/harbor/src/jobservice/job"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/dao/scan"
|
||||
v1 "github.com/goharbor/harbor/src/pkg/scan/rest/v1"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"github.com/stretchr/testify/suite"
|
||||
)
|
||||
|
||||
// TestManagerSuite is a test suite for the report manager.
|
||||
type TestManagerSuite struct {
|
||||
suite.Suite
|
||||
|
||||
m Manager
|
||||
rpUUID string
|
||||
}
|
||||
|
||||
// TestManager is an entry of suite TestManagerSuite.
|
||||
func TestManager(t *testing.T) {
|
||||
suite.Run(t, &TestManagerSuite{})
|
||||
}
|
||||
|
||||
// SetupSuite prepares test env for suite TestManagerSuite.
|
||||
func (suite *TestManagerSuite) SetupSuite() {
|
||||
dao.PrepareTestForPostgresSQL()
|
||||
|
||||
suite.m = NewManager()
|
||||
}
|
||||
|
||||
// SetupTest prepares env for test cases.
|
||||
func (suite *TestManagerSuite) SetupTest() {
|
||||
rp := &scan.Report{
|
||||
Digest: "d1000",
|
||||
RegistrationUUID: "ruuid",
|
||||
MimeType: v1.MimeTypeNativeReport,
|
||||
TrackID: "tid001",
|
||||
Requester: "requester",
|
||||
}
|
||||
|
||||
uuid, err := suite.m.Create(rp)
|
||||
require.NoError(suite.T(), err)
|
||||
require.NotEmpty(suite.T(), uuid)
|
||||
suite.rpUUID = uuid
|
||||
}
|
||||
|
||||
// TearDownTest clears test env for test cases.
|
||||
func (suite *TestManagerSuite) TearDownTest() {
|
||||
// No delete method defined in manager as no requirement,
|
||||
// so, to clear env, call dao method here
|
||||
err := scan.DeleteReport(suite.rpUUID)
|
||||
require.NoError(suite.T(), err)
|
||||
}
|
||||
|
||||
// TestManagerCreateWithExisting tests the case that a copy already is there when creating report.
|
||||
func (suite *TestManagerSuite) TestManagerCreateWithExisting() {
|
||||
err := suite.m.UpdateStatus("tid001", job.SuccessStatus.String(), 2000)
|
||||
require.NoError(suite.T(), err)
|
||||
|
||||
rp := &scan.Report{
|
||||
Digest: "d1000",
|
||||
RegistrationUUID: "ruuid",
|
||||
MimeType: v1.MimeTypeNativeReport,
|
||||
TrackID: "tid002",
|
||||
}
|
||||
|
||||
uuid, err := suite.m.Create(rp)
|
||||
require.NoError(suite.T(), err)
|
||||
require.NotEmpty(suite.T(), uuid)
|
||||
|
||||
assert.NotEqual(suite.T(), suite.rpUUID, uuid)
|
||||
suite.rpUUID = uuid
|
||||
}
|
||||
|
||||
// TestManagerGet tests the get method.
|
||||
func (suite *TestManagerSuite) TestManagerGet() {
|
||||
sr, err := suite.m.Get(suite.rpUUID)
|
||||
|
||||
require.NoError(suite.T(), err)
|
||||
require.NotNil(suite.T(), sr)
|
||||
|
||||
assert.Equal(suite.T(), "d1000", sr.Digest)
|
||||
}
|
||||
|
||||
// TestManagerGetBy tests the get by method.
|
||||
func (suite *TestManagerSuite) TestManagerGetBy() {
|
||||
l, err := suite.m.GetBy("d1000", "ruuid", []string{v1.MimeTypeNativeReport})
|
||||
require.NoError(suite.T(), err)
|
||||
require.Equal(suite.T(), 1, len(l))
|
||||
assert.Equal(suite.T(), suite.rpUUID, l[0].UUID)
|
||||
|
||||
l, err = suite.m.GetBy("d1000", "ruuid", nil)
|
||||
require.NoError(suite.T(), err)
|
||||
require.Equal(suite.T(), 1, len(l))
|
||||
assert.Equal(suite.T(), suite.rpUUID, l[0].UUID)
|
||||
|
||||
l, err = suite.m.GetBy("d1000", "", nil)
|
||||
require.NoError(suite.T(), err)
|
||||
require.Equal(suite.T(), 1, len(l))
|
||||
assert.Equal(suite.T(), suite.rpUUID, l[0].UUID)
|
||||
}
|
||||
|
||||
// TestManagerUpdateJobID tests update job ID method.
|
||||
func (suite *TestManagerSuite) TestManagerUpdateJobID() {
|
||||
l, err := suite.m.GetBy("d1000", "ruuid", []string{v1.MimeTypeNativeReport})
|
||||
require.NoError(suite.T(), err)
|
||||
require.Equal(suite.T(), 1, len(l))
|
||||
|
||||
oldJID := l[0].JobID
|
||||
|
||||
err = suite.m.UpdateScanJobID("tid001", "jID1001")
|
||||
require.NoError(suite.T(), err)
|
||||
|
||||
l, err = suite.m.GetBy("d1000", "ruuid", []string{v1.MimeTypeNativeReport})
|
||||
require.NoError(suite.T(), err)
|
||||
require.Equal(suite.T(), 1, len(l))
|
||||
|
||||
assert.NotEqual(suite.T(), oldJID, l[0].JobID)
|
||||
assert.Equal(suite.T(), "jID1001", l[0].JobID)
|
||||
}
|
||||
|
||||
// TestManagerUpdateStatus tests update status method
|
||||
func (suite *TestManagerSuite) TestManagerUpdateStatus() {
|
||||
l, err := suite.m.GetBy("d1000", "ruuid", []string{v1.MimeTypeNativeReport})
|
||||
require.NoError(suite.T(), err)
|
||||
require.Equal(suite.T(), 1, len(l))
|
||||
|
||||
oldSt := l[0].Status
|
||||
|
||||
err = suite.m.UpdateStatus("tid001", job.SuccessStatus.String(), 10000)
|
||||
require.NoError(suite.T(), err)
|
||||
|
||||
l, err = suite.m.GetBy("d1000", "ruuid", []string{v1.MimeTypeNativeReport})
|
||||
require.NoError(suite.T(), err)
|
||||
require.Equal(suite.T(), 1, len(l))
|
||||
|
||||
assert.NotEqual(suite.T(), oldSt, l[0].Status)
|
||||
assert.Equal(suite.T(), job.SuccessStatus.String(), l[0].Status)
|
||||
}
|
||||
|
||||
// TestManagerUpdateReportData tests update job report data.
|
||||
func (suite *TestManagerSuite) TestManagerUpdateReportData() {
|
||||
err := suite.m.UpdateReportData(suite.rpUUID, "{\"a\":1000}", 1000)
|
||||
require.NoError(suite.T(), err)
|
||||
|
||||
l, err := suite.m.GetBy("d1000", "ruuid", []string{v1.MimeTypeNativeReport})
|
||||
require.NoError(suite.T(), err)
|
||||
require.Equal(suite.T(), 1, len(l))
|
||||
|
||||
assert.Equal(suite.T(), "{\"a\":1000}", l[0].Report)
|
||||
}
|
||||
|
||||
// TestManagerDeleteByDigests ...
|
||||
func (suite *TestManagerSuite) TestManagerDeleteByDigests() {
|
||||
// Mock new data
|
||||
rp := &scan.Report{
|
||||
Digest: "d2000",
|
||||
RegistrationUUID: "ruuid",
|
||||
MimeType: v1.MimeTypeNativeReport,
|
||||
TrackID: "tid002",
|
||||
}
|
||||
|
||||
uuid, err := suite.m.Create(rp)
|
||||
require.NoError(suite.T(), err)
|
||||
require.NotEmpty(suite.T(), uuid)
|
||||
|
||||
err = suite.m.DeleteByDigests("d2000")
|
||||
require.NoError(suite.T(), err)
|
||||
|
||||
r, err := suite.m.Get(uuid)
|
||||
suite.NoError(err)
|
||||
suite.Nil(r)
|
||||
}
|
||||
|
||||
// TestManagerGetStats ...
|
||||
func (suite *TestManagerSuite) TestManagerGetStats() {
|
||||
// Mock new data
|
||||
rp := &scan.Report{
|
||||
Digest: "d1001",
|
||||
RegistrationUUID: "ruuid",
|
||||
MimeType: v1.MimeTypeNativeReport,
|
||||
TrackID: "tid002",
|
||||
Requester: "requester",
|
||||
}
|
||||
|
||||
uuid, err := suite.m.Create(rp)
|
||||
require.NoError(suite.T(), err)
|
||||
require.NotEmpty(suite.T(), uuid)
|
||||
|
||||
defer func() {
|
||||
err := scan.DeleteReport(uuid)
|
||||
suite.NoError(err, "clear test data")
|
||||
}()
|
||||
|
||||
err = suite.m.UpdateStatus("tid002", job.SuccessStatus.String(), 1000)
|
||||
require.NoError(suite.T(), err)
|
||||
|
||||
st, err := suite.m.GetStats("requester")
|
||||
require.NoError(suite.T(), err)
|
||||
require.NotNil(suite.T(), st)
|
||||
|
||||
suite.Equal(uint(2), st.Total)
|
||||
suite.Equal(uint(1), st.Completed)
|
||||
suite.Equal(2, len(st.Metrics))
|
||||
suite.Equal(uint(1), st.Metrics[job.SuccessStatus.String()])
|
||||
suite.Equal(uint(1), st.Metrics[job.PendingStatus.String()])
|
||||
}
|
@ -15,8 +15,13 @@
|
||||
package report
|
||||
|
||||
import (
|
||||
"github.com/goharbor/harbor/src/pkg/scan/all"
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/goharbor/harbor/src/lib/errors"
|
||||
"github.com/goharbor/harbor/src/lib/q"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/dao/scan"
|
||||
"github.com/google/uuid"
|
||||
)
|
||||
|
||||
// Manager is used to manage the scan reports.
|
||||
@ -24,52 +29,42 @@ type Manager interface {
|
||||
// Create a new report record.
|
||||
//
|
||||
// Arguments:
|
||||
// ctx context.Context : the context for this method
|
||||
// r *scan.Report : report model to be created
|
||||
//
|
||||
// Returns:
|
||||
// string : uuid of the new report
|
||||
// error : non nil error if any errors occurred
|
||||
//
|
||||
Create(r *scan.Report) (string, error)
|
||||
Create(ctx context.Context, r *scan.Report) (string, error)
|
||||
|
||||
// Update the scan job ID of the given report.
|
||||
// Delete delete report by uuid
|
||||
//
|
||||
// Arguments:
|
||||
// trackID string : uuid to identify the report
|
||||
// jobID string : scan job ID
|
||||
// ctx context.Context : the context for this method
|
||||
// uuid string : uuid of the report to delete
|
||||
//
|
||||
// Returns:
|
||||
// error : non nil error if any errors occurred
|
||||
//
|
||||
UpdateScanJobID(trackID string, jobID string) error
|
||||
|
||||
// Update the status (mapping to the scan job status) of the given report.
|
||||
//
|
||||
// Arguments:
|
||||
// trackID string : uuid to identify the report
|
||||
// status string : status info
|
||||
// rev int64 : data revision info
|
||||
//
|
||||
// Returns:
|
||||
// error : non nil error if any errors occurred
|
||||
//
|
||||
UpdateStatus(trackID string, status string, rev int64) error
|
||||
Delete(ctx context.Context, uuid string) error
|
||||
|
||||
// Update the report data (with JSON format) of the given report.
|
||||
//
|
||||
// Arguments:
|
||||
// ctx context.Context : the context for this method
|
||||
// uuid string : uuid to identify the report
|
||||
// report string : report JSON data
|
||||
// rev int64 : data revision info
|
||||
//
|
||||
// Returns:
|
||||
// error : non nil error if any errors occurred
|
||||
//
|
||||
UpdateReportData(uuid string, report string, rev int64) error
|
||||
UpdateReportData(ctx context.Context, uuid string, report string) error
|
||||
|
||||
// Get the reports for the given digest by other properties.
|
||||
//
|
||||
// Arguments:
|
||||
// ctx context.Context : the context for this method
|
||||
// digest string : digest of the artifact
|
||||
// registrationUUID string : [optional] the report generated by which registration.
|
||||
// If it is empty, reports by all the registrations are retrieved.
|
||||
@ -79,34 +74,134 @@ type Manager interface {
|
||||
// Returns:
|
||||
// []*scan.Report : report list
|
||||
// error : non nil error if any errors occurred
|
||||
GetBy(digest string, registrationUUID string, mimeTypes []string) ([]*scan.Report, error)
|
||||
|
||||
// Get the report for the given uuid.
|
||||
//
|
||||
// Arguments:
|
||||
// uuid string : uuid of the scan report
|
||||
//
|
||||
// Returns:
|
||||
// *scan.Report : scan report
|
||||
// error : non nil error if any errors occurred
|
||||
Get(uuid string) (*scan.Report, error)
|
||||
GetBy(ctx context.Context, digest string, registrationUUID string, mimeTypes []string) ([]*scan.Report, error)
|
||||
|
||||
// Delete the reports related with the specified digests (one or more...)
|
||||
//
|
||||
// Arguments:
|
||||
// ctx context.Context : the context for this method
|
||||
// digests ...string : specify one or more digests whose reports will be deleted
|
||||
//
|
||||
// Returns:
|
||||
// error : non nil error if any errors occurred
|
||||
DeleteByDigests(digests ...string) error
|
||||
DeleteByDigests(ctx context.Context, digests ...string) error
|
||||
|
||||
// GetStats retrieves and calculates the overall report stats organized by status targeting the
|
||||
// given requester.
|
||||
// List reports according to the query
|
||||
//
|
||||
// Arguments:
|
||||
// requester string : the requester of the scan (all)
|
||||
// ctx context.Context : the context for this method
|
||||
// query *q.Query : the query to list the reports
|
||||
//
|
||||
// Returns:
|
||||
// *all.AllStats: stats object including the related metric data
|
||||
// []*scan.Report : report list
|
||||
// error : non nil error if any errors occurred
|
||||
GetStats(requester string) (*all.Stats, error)
|
||||
List(ctx context.Context, query *q.Query) ([]*scan.Report, error)
|
||||
}
|
||||
|
||||
const (
|
||||
reportTimeout = 1 * time.Hour
|
||||
)
|
||||
|
||||
// basicManager is a default implementation of report manager.
|
||||
type basicManager struct {
|
||||
dao scan.DAO
|
||||
}
|
||||
|
||||
// NewManager news basic manager.
|
||||
func NewManager() Manager {
|
||||
return &basicManager{
|
||||
dao: scan.New(),
|
||||
}
|
||||
}
|
||||
|
||||
// Create ...
|
||||
func (bm *basicManager) Create(ctx context.Context, r *scan.Report) (string, error) {
|
||||
// Validate report object
|
||||
if r == nil {
|
||||
return "", errors.New("nil scan report object")
|
||||
}
|
||||
|
||||
if len(r.Digest) == 0 || len(r.RegistrationUUID) == 0 || len(r.MimeType) == 0 {
|
||||
return "", errors.New("malformed scan report object")
|
||||
}
|
||||
|
||||
r.UUID = uuid.New().String()
|
||||
|
||||
// Insert
|
||||
if _, err := bm.dao.Create(ctx, r); err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
||||
return r.UUID, nil
|
||||
}
|
||||
|
||||
func (bm *basicManager) Delete(ctx context.Context, uuid string) error {
|
||||
query := q.Query{Keywords: q.KeyWords{"uuid": uuid}}
|
||||
count, err := bm.dao.DeleteMany(ctx, query)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if count == 0 {
|
||||
return errors.Errorf("no report with uuid %s deleted", uuid)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// GetBy ...
|
||||
func (bm *basicManager) GetBy(ctx context.Context, digest string, registrationUUID string, mimeTypes []string) ([]*scan.Report, error) {
|
||||
if len(digest) == 0 {
|
||||
return nil, errors.New("empty digest to get report data")
|
||||
}
|
||||
|
||||
kws := make(map[string]interface{})
|
||||
kws["digest"] = digest
|
||||
if len(registrationUUID) > 0 {
|
||||
kws["registration_uuid"] = registrationUUID
|
||||
}
|
||||
if len(mimeTypes) > 0 {
|
||||
kws["mime_type__in"] = mimeTypes
|
||||
}
|
||||
// Query all
|
||||
query := &q.Query{
|
||||
PageNumber: 0,
|
||||
Keywords: kws,
|
||||
}
|
||||
|
||||
return bm.dao.List(ctx, query)
|
||||
}
|
||||
|
||||
// UpdateReportData ...
|
||||
func (bm *basicManager) UpdateReportData(ctx context.Context, uuid string, report string) error {
|
||||
if len(uuid) == 0 {
|
||||
return errors.New("missing uuid")
|
||||
}
|
||||
|
||||
if len(report) == 0 {
|
||||
return errors.New("missing report JSON data")
|
||||
}
|
||||
|
||||
return bm.dao.UpdateReportData(ctx, uuid, report)
|
||||
}
|
||||
|
||||
// DeleteByDigests ...
|
||||
func (bm *basicManager) DeleteByDigests(ctx context.Context, digests ...string) error {
|
||||
if len(digests) == 0 {
|
||||
// Nothing to do
|
||||
return nil
|
||||
}
|
||||
|
||||
var ol q.OrList
|
||||
for _, digest := range digests {
|
||||
ol.Values = append(ol.Values, digest)
|
||||
}
|
||||
|
||||
query := q.Query{Keywords: q.KeyWords{"digest": &ol}}
|
||||
_, err := bm.dao.DeleteMany(ctx, query)
|
||||
return err
|
||||
}
|
||||
|
||||
func (bm *basicManager) List(ctx context.Context, query *q.Query) ([]*scan.Report, error) {
|
||||
return bm.dao.List(ctx, query)
|
||||
}
|
||||
|
133
src/pkg/scan/report/manager_test.go
Normal file
133
src/pkg/scan/report/manager_test.go
Normal file
@ -0,0 +1,133 @@
|
||||
// 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 report
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/goharbor/harbor/src/common/dao"
|
||||
"github.com/goharbor/harbor/src/lib/orm"
|
||||
"github.com/goharbor/harbor/src/lib/q"
|
||||
"github.com/goharbor/harbor/src/pkg/scan/dao/scan"
|
||||
v1 "github.com/goharbor/harbor/src/pkg/scan/rest/v1"
|
||||
"github.com/stretchr/testify/suite"
|
||||
)
|
||||
|
||||
// TestManagerSuite is a test suite for the report manager.
|
||||
type TestManagerSuite struct {
|
||||
suite.Suite
|
||||
|
||||
m Manager
|
||||
reportUUID string
|
||||
}
|
||||
|
||||
// TestManager is an entry of suite TestManagerSuite.
|
||||
func TestManager(t *testing.T) {
|
||||
suite.Run(t, &TestManagerSuite{})
|
||||
}
|
||||
|
||||
// SetupSuite prepares test env for suite TestManagerSuite.
|
||||
func (suite *TestManagerSuite) SetupSuite() {
|
||||
dao.PrepareTestForPostgresSQL()
|
||||
|
||||
suite.m = NewManager()
|
||||
}
|
||||
|
||||
// SetupTest prepares env for test cases.
|
||||
func (suite *TestManagerSuite) SetupTest() {
|
||||
rp := &scan.Report{
|
||||
Digest: "d1000",
|
||||
RegistrationUUID: "ruuid",
|
||||
MimeType: v1.MimeTypeNativeReport,
|
||||
}
|
||||
|
||||
uuid, err := suite.m.Create(orm.Context(), rp)
|
||||
suite.Require().NoError(err)
|
||||
suite.Require().NotEmpty(uuid)
|
||||
suite.reportUUID = uuid
|
||||
}
|
||||
|
||||
// TearDownTest clears test env for test cases.
|
||||
func (suite *TestManagerSuite) TearDownTest() {
|
||||
suite.Nil(suite.m.Delete(orm.Context(), suite.reportUUID))
|
||||
}
|
||||
|
||||
// TestManagerGetBy tests the get by method.
|
||||
func (suite *TestManagerSuite) TestManagerGetBy() {
|
||||
l, err := suite.m.GetBy(orm.Context(), "d1000", "ruuid", []string{v1.MimeTypeNativeReport})
|
||||
suite.Require().NoError(err)
|
||||
suite.Require().Equal(1, len(l))
|
||||
suite.Require().Equal(suite.reportUUID, l[0].UUID)
|
||||
|
||||
l, err = suite.m.GetBy(orm.Context(), "d1000", "ruuid", nil)
|
||||
suite.Require().NoError(err)
|
||||
suite.Require().Equal(1, len(l))
|
||||
suite.Require().Equal(suite.reportUUID, l[0].UUID)
|
||||
|
||||
l, err = suite.m.GetBy(orm.Context(), "d1000", "", nil)
|
||||
suite.Require().NoError(err)
|
||||
suite.Require().Equal(1, len(l))
|
||||
suite.Require().Equal(suite.reportUUID, l[0].UUID)
|
||||
}
|
||||
|
||||
// TestManagerUpdateReportData tests update job report data.
|
||||
func (suite *TestManagerSuite) TestManagerUpdateReportData() {
|
||||
err := suite.m.UpdateReportData(orm.Context(), suite.reportUUID, "{\"a\":1000}")
|
||||
suite.Require().NoError(err)
|
||||
|
||||
l, err := suite.m.GetBy(orm.Context(), "d1000", "ruuid", []string{v1.MimeTypeNativeReport})
|
||||
suite.Require().NoError(err)
|
||||
suite.Require().Equal(1, len(l))
|
||||
|
||||
suite.Equal("{\"a\":1000}", l[0].Report)
|
||||
}
|
||||
|
||||
// TestManagerDeleteByDigests ...
|
||||
func (suite *TestManagerSuite) TestManagerDeleteByDigests() {
|
||||
// Mock new data
|
||||
rp1 := &scan.Report{
|
||||
Digest: "d2000",
|
||||
RegistrationUUID: "ruuid1",
|
||||
MimeType: v1.MimeTypeNativeReport,
|
||||
}
|
||||
|
||||
rp2 := &scan.Report{
|
||||
Digest: "d2000",
|
||||
RegistrationUUID: "ruuid2",
|
||||
MimeType: v1.MimeTypeNativeReport,
|
||||
}
|
||||
|
||||
var reportUUIDs []string
|
||||
for _, rp := range []*scan.Report{rp1, rp2} {
|
||||
uuid, err := suite.m.Create(orm.Context(), rp)
|
||||
suite.Require().NoError(err)
|
||||
suite.Require().NotEmpty(uuid)
|
||||
reportUUIDs = append(reportUUIDs, uuid)
|
||||
}
|
||||
|
||||
l, err := suite.m.List(orm.Context(), q.New(q.KeyWords{"uuid__in": reportUUIDs}))
|
||||
suite.Require().NoError(err)
|
||||
suite.Require().Equal(2, len(l))
|
||||
|
||||
err = suite.m.DeleteByDigests(orm.Context())
|
||||
suite.Require().NoError(err)
|
||||
|
||||
err = suite.m.DeleteByDigests(orm.Context(), "d2000")
|
||||
suite.Require().NoError(err)
|
||||
|
||||
l, err = suite.m.List(orm.Context(), q.New(q.KeyWords{"uuid__in": reportUUIDs}))
|
||||
suite.Require().NoError(err)
|
||||
suite.Require().Equal(0, len(l))
|
||||
}
|
@ -80,11 +80,7 @@ func (suite *SummaryTestSuite) SetupSuite() {
|
||||
Digest: "digest-code",
|
||||
RegistrationUUID: "reg-uuid-001",
|
||||
MimeType: v1.MimeTypeNativeReport,
|
||||
JobID: "job-uuid-001",
|
||||
TrackID: "track-uuid-001",
|
||||
Status: "Success",
|
||||
StatusCode: 3,
|
||||
StatusRevision: 10000,
|
||||
Report: string(jsonData),
|
||||
}
|
||||
}
|
||||
|
@ -52,6 +52,16 @@ type Execution struct {
|
||||
EndTime time.Time `json:"end_time"`
|
||||
}
|
||||
|
||||
// IsOnGoing returns true when the execution is running
|
||||
func (exec *Execution) IsOnGoing() bool {
|
||||
switch job.Status(exec.Status) {
|
||||
case job.RunningStatus:
|
||||
return true
|
||||
default:
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
// Task is the unit for running. It stores the jobservice job records and related information
|
||||
type Task struct {
|
||||
ID int64 `json:"id"`
|
||||
|
@ -76,7 +76,7 @@ func (s *scanAPI) GetReportLog(ctx context.Context, params operation.GetReportLo
|
||||
return s.SendError(ctx, err)
|
||||
}
|
||||
|
||||
bytes, err := s.scanCtl.GetScanLog(params.ReportID)
|
||||
bytes, err := s.scanCtl.GetScanLog(ctx, params.ReportID)
|
||||
if err != nil {
|
||||
return s.SendError(ctx, err)
|
||||
}
|
||||
|
@ -3,17 +3,16 @@
|
||||
package scan
|
||||
|
||||
import (
|
||||
artifact "github.com/goharbor/harbor/src/controller/artifact"
|
||||
all "github.com/goharbor/harbor/src/pkg/scan/all"
|
||||
|
||||
context "context"
|
||||
|
||||
artifact "github.com/goharbor/harbor/src/controller/artifact"
|
||||
|
||||
daoscan "github.com/goharbor/harbor/src/pkg/scan/dao/scan"
|
||||
|
||||
job "github.com/goharbor/harbor/src/jobservice/job"
|
||||
|
||||
mock "github.com/stretchr/testify/mock"
|
||||
|
||||
pkgscan "github.com/goharbor/harbor/src/pkg/scan"
|
||||
|
||||
report "github.com/goharbor/harbor/src/pkg/scan/report"
|
||||
|
||||
scan "github.com/goharbor/harbor/src/controller/scan"
|
||||
@ -24,19 +23,20 @@ type Controller struct {
|
||||
mock.Mock
|
||||
}
|
||||
|
||||
// DeleteReports provides a mock function with given fields: digests
|
||||
func (_m *Controller) DeleteReports(digests ...string) error {
|
||||
// DeleteReports provides a mock function with given fields: ctx, digests
|
||||
func (_m *Controller) DeleteReports(ctx context.Context, digests ...string) error {
|
||||
_va := make([]interface{}, len(digests))
|
||||
for _i := range digests {
|
||||
_va[_i] = digests[_i]
|
||||
}
|
||||
var _ca []interface{}
|
||||
_ca = append(_ca, ctx)
|
||||
_ca = append(_ca, _va...)
|
||||
ret := _m.Called(_ca...)
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(...string) error); ok {
|
||||
r0 = rf(digests...)
|
||||
if rf, ok := ret.Get(0).(func(context.Context, ...string) error); ok {
|
||||
r0 = rf(ctx, digests...)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
@ -67,13 +67,13 @@ func (_m *Controller) GetReport(ctx context.Context, _a1 *artifact.Artifact, mim
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// GetScanLog provides a mock function with given fields: uuid
|
||||
func (_m *Controller) GetScanLog(uuid string) ([]byte, error) {
|
||||
ret := _m.Called(uuid)
|
||||
// GetScanLog provides a mock function with given fields: ctx, uuid
|
||||
func (_m *Controller) GetScanLog(ctx context.Context, uuid string) ([]byte, error) {
|
||||
ret := _m.Called(ctx, uuid)
|
||||
|
||||
var r0 []byte
|
||||
if rf, ok := ret.Get(0).(func(string) []byte); ok {
|
||||
r0 = rf(uuid)
|
||||
if rf, ok := ret.Get(0).(func(context.Context, string) []byte); ok {
|
||||
r0 = rf(ctx, uuid)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).([]byte)
|
||||
@ -81,31 +81,8 @@ func (_m *Controller) GetScanLog(uuid string) ([]byte, error) {
|
||||
}
|
||||
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(1).(func(string) error); ok {
|
||||
r1 = rf(uuid)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// GetStats provides a mock function with given fields: requester
|
||||
func (_m *Controller) GetStats(requester string) (*all.Stats, error) {
|
||||
ret := _m.Called(requester)
|
||||
|
||||
var r0 *all.Stats
|
||||
if rf, ok := ret.Get(0).(func(string) *all.Stats); ok {
|
||||
r0 = rf(requester)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*all.Stats)
|
||||
}
|
||||
}
|
||||
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(1).(func(string) error); ok {
|
||||
r1 = rf(requester)
|
||||
if rf, ok := ret.Get(1).(func(context.Context, string) error); ok {
|
||||
r1 = rf(ctx, uuid)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
@ -143,20 +120,6 @@ func (_m *Controller) GetSummary(ctx context.Context, _a1 *artifact.Artifact, mi
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// HandleJobHooks provides a mock function with given fields: ctx, trackID, change
|
||||
func (_m *Controller) HandleJobHooks(ctx context.Context, trackID string, change *job.StatusChange) error {
|
||||
ret := _m.Called(ctx, trackID, change)
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, string, *job.StatusChange) error); ok {
|
||||
r0 = rf(ctx, trackID, change)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// Scan provides a mock function with given fields: ctx, _a1, options
|
||||
func (_m *Controller) Scan(ctx context.Context, _a1 *artifact.Artifact, options ...scan.Option) error {
|
||||
_va := make([]interface{}, len(options))
|
||||
@ -177,3 +140,38 @@ func (_m *Controller) Scan(ctx context.Context, _a1 *artifact.Artifact, options
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// ScanAll provides a mock function with given fields: ctx, trigger, async
|
||||
func (_m *Controller) ScanAll(ctx context.Context, trigger string, async bool) (int64, error) {
|
||||
ret := _m.Called(ctx, trigger, async)
|
||||
|
||||
var r0 int64
|
||||
if rf, ok := ret.Get(0).(func(context.Context, string, bool) int64); ok {
|
||||
r0 = rf(ctx, trigger, async)
|
||||
} else {
|
||||
r0 = ret.Get(0).(int64)
|
||||
}
|
||||
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(1).(func(context.Context, string, bool) error); ok {
|
||||
r1 = rf(ctx, trigger, async)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// UpdateReport provides a mock function with given fields: ctx, _a1
|
||||
func (_m *Controller) UpdateReport(ctx context.Context, _a1 *pkgscan.CheckInReport) error {
|
||||
ret := _m.Called(ctx, _a1)
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *pkgscan.CheckInReport) error); ok {
|
||||
r0 = rf(ctx, _a1)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
@ -3,7 +3,9 @@
|
||||
package report
|
||||
|
||||
import (
|
||||
all "github.com/goharbor/harbor/src/pkg/scan/all"
|
||||
context "context"
|
||||
|
||||
q "github.com/goharbor/harbor/src/lib/q"
|
||||
mock "github.com/stretchr/testify/mock"
|
||||
|
||||
scan "github.com/goharbor/harbor/src/pkg/scan/dao/scan"
|
||||
@ -14,20 +16,20 @@ type Manager struct {
|
||||
mock.Mock
|
||||
}
|
||||
|
||||
// Create provides a mock function with given fields: r
|
||||
func (_m *Manager) Create(r *scan.Report) (string, error) {
|
||||
ret := _m.Called(r)
|
||||
// Create provides a mock function with given fields: ctx, r
|
||||
func (_m *Manager) Create(ctx context.Context, r *scan.Report) (string, error) {
|
||||
ret := _m.Called(ctx, r)
|
||||
|
||||
var r0 string
|
||||
if rf, ok := ret.Get(0).(func(*scan.Report) string); ok {
|
||||
r0 = rf(r)
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *scan.Report) string); ok {
|
||||
r0 = rf(ctx, r)
|
||||
} else {
|
||||
r0 = ret.Get(0).(string)
|
||||
}
|
||||
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(1).(func(*scan.Report) error); ok {
|
||||
r1 = rf(r)
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *scan.Report) error); ok {
|
||||
r1 = rf(ctx, r)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
@ -35,19 +37,13 @@ func (_m *Manager) Create(r *scan.Report) (string, error) {
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// DeleteByDigests provides a mock function with given fields: digests
|
||||
func (_m *Manager) DeleteByDigests(digests ...string) error {
|
||||
_va := make([]interface{}, len(digests))
|
||||
for _i := range digests {
|
||||
_va[_i] = digests[_i]
|
||||
}
|
||||
var _ca []interface{}
|
||||
_ca = append(_ca, _va...)
|
||||
ret := _m.Called(_ca...)
|
||||
// Delete provides a mock function with given fields: ctx, uuid
|
||||
func (_m *Manager) Delete(ctx context.Context, uuid string) error {
|
||||
ret := _m.Called(ctx, uuid)
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(...string) error); ok {
|
||||
r0 = rf(digests...)
|
||||
if rf, ok := ret.Get(0).(func(context.Context, string) error); ok {
|
||||
r0 = rf(ctx, uuid)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
@ -55,36 +51,34 @@ func (_m *Manager) DeleteByDigests(digests ...string) error {
|
||||
return r0
|
||||
}
|
||||
|
||||
// Get provides a mock function with given fields: uuid
|
||||
func (_m *Manager) Get(uuid string) (*scan.Report, error) {
|
||||
ret := _m.Called(uuid)
|
||||
// DeleteByDigests provides a mock function with given fields: ctx, digests
|
||||
func (_m *Manager) DeleteByDigests(ctx context.Context, digests ...string) error {
|
||||
_va := make([]interface{}, len(digests))
|
||||
for _i := range digests {
|
||||
_va[_i] = digests[_i]
|
||||
}
|
||||
var _ca []interface{}
|
||||
_ca = append(_ca, ctx)
|
||||
_ca = append(_ca, _va...)
|
||||
ret := _m.Called(_ca...)
|
||||
|
||||
var r0 *scan.Report
|
||||
if rf, ok := ret.Get(0).(func(string) *scan.Report); ok {
|
||||
r0 = rf(uuid)
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, ...string) error); ok {
|
||||
r0 = rf(ctx, digests...)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*scan.Report)
|
||||
}
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(1).(func(string) error); ok {
|
||||
r1 = rf(uuid)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
return r0
|
||||
}
|
||||
|
||||
// GetBy provides a mock function with given fields: digest, registrationUUID, mimeTypes
|
||||
func (_m *Manager) GetBy(digest string, registrationUUID string, mimeTypes []string) ([]*scan.Report, error) {
|
||||
ret := _m.Called(digest, registrationUUID, mimeTypes)
|
||||
// GetBy provides a mock function with given fields: ctx, digest, registrationUUID, mimeTypes
|
||||
func (_m *Manager) GetBy(ctx context.Context, digest string, registrationUUID string, mimeTypes []string) ([]*scan.Report, error) {
|
||||
ret := _m.Called(ctx, digest, registrationUUID, mimeTypes)
|
||||
|
||||
var r0 []*scan.Report
|
||||
if rf, ok := ret.Get(0).(func(string, string, []string) []*scan.Report); ok {
|
||||
r0 = rf(digest, registrationUUID, mimeTypes)
|
||||
if rf, ok := ret.Get(0).(func(context.Context, string, string, []string) []*scan.Report); ok {
|
||||
r0 = rf(ctx, digest, registrationUUID, mimeTypes)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).([]*scan.Report)
|
||||
@ -92,8 +86,8 @@ func (_m *Manager) GetBy(digest string, registrationUUID string, mimeTypes []str
|
||||
}
|
||||
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(1).(func(string, string, []string) error); ok {
|
||||
r1 = rf(digest, registrationUUID, mimeTypes)
|
||||
if rf, ok := ret.Get(1).(func(context.Context, string, string, []string) error); ok {
|
||||
r1 = rf(ctx, digest, registrationUUID, mimeTypes)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
@ -101,22 +95,22 @@ func (_m *Manager) GetBy(digest string, registrationUUID string, mimeTypes []str
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// GetStats provides a mock function with given fields: requester
|
||||
func (_m *Manager) GetStats(requester string) (*all.Stats, error) {
|
||||
ret := _m.Called(requester)
|
||||
// List provides a mock function with given fields: ctx, query
|
||||
func (_m *Manager) List(ctx context.Context, query *q.Query) ([]*scan.Report, error) {
|
||||
ret := _m.Called(ctx, query)
|
||||
|
||||
var r0 *all.Stats
|
||||
if rf, ok := ret.Get(0).(func(string) *all.Stats); ok {
|
||||
r0 = rf(requester)
|
||||
var r0 []*scan.Report
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *q.Query) []*scan.Report); ok {
|
||||
r0 = rf(ctx, query)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*all.Stats)
|
||||
r0 = ret.Get(0).([]*scan.Report)
|
||||
}
|
||||
}
|
||||
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(1).(func(string) error); ok {
|
||||
r1 = rf(requester)
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *q.Query) error); ok {
|
||||
r1 = rf(ctx, query)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
@ -124,41 +118,13 @@ func (_m *Manager) GetStats(requester string) (*all.Stats, error) {
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// UpdateReportData provides a mock function with given fields: uuid, _a1, rev
|
||||
func (_m *Manager) UpdateReportData(uuid string, _a1 string, rev int64) error {
|
||||
ret := _m.Called(uuid, _a1, rev)
|
||||
// UpdateReportData provides a mock function with given fields: ctx, uuid, _a2
|
||||
func (_m *Manager) UpdateReportData(ctx context.Context, uuid string, _a2 string) error {
|
||||
ret := _m.Called(ctx, uuid, _a2)
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(string, string, int64) error); ok {
|
||||
r0 = rf(uuid, _a1, rev)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// UpdateScanJobID provides a mock function with given fields: trackID, jobID
|
||||
func (_m *Manager) UpdateScanJobID(trackID string, jobID string) error {
|
||||
ret := _m.Called(trackID, jobID)
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(string, string) error); ok {
|
||||
r0 = rf(trackID, jobID)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// UpdateStatus provides a mock function with given fields: trackID, status, rev
|
||||
func (_m *Manager) UpdateStatus(trackID string, status string, rev int64) error {
|
||||
ret := _m.Called(trackID, status, rev)
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(string, string, int64) error); ok {
|
||||
r0 = rf(trackID, status, rev)
|
||||
if rf, ok := ret.Get(0).(func(context.Context, string, string) error); ok {
|
||||
r0 = rf(ctx, uuid, _a2)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user