From c5768a3bdcf47a12f6d83ff29243da4851e2cd1a Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Mon, 30 Oct 2023 14:26:28 +0900 Subject: [PATCH 01/26] refactor: replace with go-redis Signed-off-by: knqyf263 --- cmd/scanner-trivy/main.go | 17 +-- pkg/http/api/v1/handler.go | 4 +- pkg/persistence/redis/store.go | 78 +++++------- pkg/persistence/store.go | 9 +- pkg/queue/enqueuer.go | 90 ++++++++++---- pkg/queue/worker.go | 106 +++++++++------- pkg/redisx/pool.go | 115 +++++++----------- pkg/redisx/pool_test.go | 4 +- pkg/scan/controller.go | 19 ++- .../persistence/redis/store_test.go | 2 +- 10 files changed, 223 insertions(+), 221 deletions(-) diff --git a/cmd/scanner-trivy/main.go b/cmd/scanner-trivy/main.go index 0f9ecd85..4a9a2c42 100644 --- a/cmd/scanner-trivy/main.go +++ b/cmd/scanner-trivy/main.go @@ -1,6 +1,7 @@ package main import ( + "context" "fmt" "log/slog" "os" @@ -37,13 +38,14 @@ func main() { Date: date, } - if err := run(info); err != nil { + ctx := context.Background() + if err := run(ctx, info); err != nil { slog.Error("Error: %v", err) os.Exit(1) } } -func run(info etc.BuildInfo) error { +func run(ctx context.Context, info etc.BuildInfo) error { slog.Info("Starting harbor-scanner-trivy", slog.String("version", info.Version), slog.String("commit", info.Commit), slog.String("built_at", info.Date), ) @@ -56,16 +58,16 @@ func run(info etc.BuildInfo) error { return fmt.Errorf("checking config: %w", err) } - pool, err := redisx.NewPool(config.RedisPool) + rdb, err := redisx.NewClient(config.RedisPool) if err != nil { return fmt.Errorf("constructing connection pool: %w", err) } wrapper := trivy.NewWrapper(config.Trivy, ext.DefaultAmbassador) - store := redis.NewStore(config.RedisStore, pool) + store := redis.NewStore(config.RedisStore, rdb) controller := scan.NewController(store, wrapper, scan.NewTransformer(&scan.SystemClock{})) - enqueuer := queue.NewEnqueuer(config.JobQueue, pool, store) - worker := queue.NewWorker(config.JobQueue, pool, controller) + enqueuer := queue.NewEnqueuer(config.JobQueue, rdb, store) + worker := queue.NewWorker(config.JobQueue, rdb, controller) apiHandler := v1.NewAPIHandler(info, config, enqueuer, store, wrapper) apiServer, err := api.NewServer(config.API, apiHandler) @@ -82,11 +84,12 @@ func run(info etc.BuildInfo) error { apiServer.Shutdown() worker.Stop() + _ = rdb.Close() close(shutdownComplete) }() - worker.Start() + worker.Start(ctx) apiServer.ListenAndServe() <-shutdownComplete diff --git a/pkg/http/api/v1/handler.go b/pkg/http/api/v1/handler.go index a7fc9794..9d6d5364 100644 --- a/pkg/http/api/v1/handler.go +++ b/pkg/http/api/v1/handler.go @@ -92,7 +92,7 @@ func (h *requestHandler) AcceptScanRequest(res http.ResponseWriter, req *http.Re return } - scanJob, err := h.enqueuer.Enqueue(scanRequest) + scanJob, err := h.enqueuer.Enqueue(req.Context(), scanRequest) if err != nil { slog.Error("Error while enqueuing scan job", slog.String("err", err.Error())) h.WriteJSONError(res, harbor.Error{ @@ -163,7 +163,7 @@ func (h *requestHandler) GetScanReport(res http.ResponseWriter, req *http.Reques reqLog := slog.With(slog.String("scan_job_id", scanJobID)) - scanJob, err := h.store.Get(scanJobID) + scanJob, err := h.store.Get(req.Context(), scanJobID) if err != nil { reqLog.Error("Error while getting scan job") h.WriteJSONError(res, harbor.Error{ diff --git a/pkg/persistence/redis/store.go b/pkg/persistence/redis/store.go index 07535788..e0c43d61 100644 --- a/pkg/persistence/redis/store.go +++ b/pkg/persistence/redis/store.go @@ -1,6 +1,7 @@ package redis import ( + "context" "encoding/json" "errors" "fmt" @@ -10,32 +11,26 @@ import ( "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" "github.com/aquasecurity/harbor-scanner-trivy/pkg/job" "github.com/aquasecurity/harbor-scanner-trivy/pkg/persistence" - "github.com/gomodule/redigo/redis" + redis "github.com/redis/go-redis/v9" "golang.org/x/xerrors" ) type store struct { - cfg etc.RedisStore - pool *redis.Pool + cfg etc.RedisStore + rdb *redis.Client } -func NewStore(cfg etc.RedisStore, pool *redis.Pool) persistence.Store { - return &store{ - cfg: cfg, - pool: pool, - } +func NewStore(cfg etc.RedisStore, rdb *redis.Client) persistence.Store { + return &store{cfg: cfg, rdb: rdb} } -func (s *store) Create(scanJob job.ScanJob) error { - conn := s.pool.Get() - defer s.close(conn) - +func (s *store) Create(ctx context.Context, scanJob job.ScanJob) error { bytes, err := json.Marshal(scanJob) if err != nil { return xerrors.Errorf("marshalling scan job: %w", err) } - key := s.getKeyForScanJob(scanJob.ID) + key := s.keyForScanJob(scanJob.ID) slog.Debug("Saving scan job", slog.String("scan_job_id", scanJob.ID), @@ -44,23 +39,20 @@ func (s *store) Create(scanJob job.ScanJob) error { slog.Duration("expire", s.cfg.ScanJobTTL), ) - if _, err = conn.Do("SET", key, string(bytes), "NX", "EX", int(s.cfg.ScanJobTTL.Seconds())); err != nil { + if err = s.rdb.SetNX(ctx, key, string(bytes), s.cfg.ScanJobTTL).Err(); err != nil { return xerrors.Errorf("creating scan job: %w", err) } return nil } -func (s *store) update(scanJob job.ScanJob) error { - conn := s.pool.Get() - defer s.close(conn) - +func (s *store) update(ctx context.Context, scanJob job.ScanJob) error { bytes, err := json.Marshal(scanJob) if err != nil { return xerrors.Errorf("marshalling scan job: %w", err) } - key := s.getKeyForScanJob(scanJob.ID) + key := s.keyForScanJob(scanJob.ID) slog.Debug("Updating scan job", slog.String("scan_job_id", scanJob.ID), @@ -69,42 +61,39 @@ func (s *store) update(scanJob job.ScanJob) error { slog.Duration("expire", s.cfg.ScanJobTTL), ) - if _, err = conn.Do("SET", key, string(bytes), "XX", "EX", int(s.cfg.ScanJobTTL.Seconds())); err != nil { + if err = s.rdb.SetXX(ctx, key, string(bytes), s.cfg.ScanJobTTL).Err(); err != nil { return xerrors.Errorf("updating scan job: %w", err) } return nil } -func (s *store) Get(scanJobID string) (*job.ScanJob, error) { - conn := s.pool.Get() - defer s.close(conn) - - key := s.getKeyForScanJob(scanJobID) - value, err := redis.String(conn.Do("GET", key)) - if err != nil { - if errors.Is(err, redis.ErrNil) { - return nil, nil - } +func (s *store) Get(ctx context.Context, scanJobID string) (*job.ScanJob, error) { + key := s.keyForScanJob(scanJobID) + value, err := s.rdb.Get(ctx, key).Result() + if errors.Is(err, redis.Nil) { + return nil, nil + } else if err != nil { return nil, err } var scanJob job.ScanJob - err = json.Unmarshal([]byte(value), &scanJob) - if err != nil { - return nil, err + if err = json.Unmarshal([]byte(value), &scanJob); err != nil { + return nil, xerrors.Errorf("unmarshalling scan job: %w", err) } return &scanJob, nil } -func (s *store) UpdateStatus(scanJobID string, newStatus job.ScanJobStatus, error ...string) error { +func (s *store) UpdateStatus(ctx context.Context, scanJobID string, newStatus job.ScanJobStatus, error ...string) error { slog.Debug("Updating status for scan job", slog.String("scan_job_id", scanJobID), slog.String("new_status", newStatus.String()), ) - scanJob, err := s.Get(scanJobID) - if err != nil { + scanJob, err := s.Get(ctx, scanJobID) + if scanJob == nil { + return xerrors.Errorf("scan job %s not found", scanJobID) + } else if err != nil { return err } @@ -113,28 +102,21 @@ func (s *store) UpdateStatus(scanJobID string, newStatus job.ScanJobStatus, erro scanJob.Error = error[0] } - return s.update(*scanJob) + return s.update(ctx, *scanJob) } -func (s *store) UpdateReport(scanJobID string, report harbor.ScanReport) error { +func (s *store) UpdateReport(ctx context.Context, scanJobID string, report harbor.ScanReport) error { slog.Debug("Updating reports for scan job", slog.String("scan_job_id", scanJobID)) - scanJob, err := s.Get(scanJobID) + scanJob, err := s.Get(ctx, scanJobID) if err != nil { return err } scanJob.Report = report - return s.update(*scanJob) + return s.update(ctx, *scanJob) } -func (s *store) getKeyForScanJob(scanJobID string) string { +func (s *store) keyForScanJob(scanJobID string) string { return fmt.Sprintf("%s:scan-job:%s", s.cfg.Namespace, scanJobID) } - -func (s *store) close(conn redis.Conn) { - err := conn.Close() - if err != nil { - slog.Error("Error while closing connection", slog.String("err", err.Error())) - } -} diff --git a/pkg/persistence/store.go b/pkg/persistence/store.go index 7224d01a..41e4ffa7 100644 --- a/pkg/persistence/store.go +++ b/pkg/persistence/store.go @@ -1,13 +1,14 @@ package persistence import ( + "context" "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" "github.com/aquasecurity/harbor-scanner-trivy/pkg/job" ) type Store interface { - Create(scanJob job.ScanJob) error - Get(scanJobID string) (*job.ScanJob, error) - UpdateStatus(scanJobID string, newStatus job.ScanJobStatus, error ...string) error - UpdateReport(scanJobID string, report harbor.ScanReport) error + Create(ctx context.Context, scanJob job.ScanJob) error + Get(ctx context.Context, scanJobID string) (*job.ScanJob, error) + UpdateStatus(ctx context.Context, scanJobID string, newStatus job.ScanJobStatus, error ...string) error + UpdateReport(ctx context.Context, scanJobID string, report harbor.ScanReport) error } diff --git a/pkg/queue/enqueuer.go b/pkg/queue/enqueuer.go index 4ff8ab30..24d66bd0 100644 --- a/pkg/queue/enqueuer.go +++ b/pkg/queue/enqueuer.go @@ -1,12 +1,15 @@ package queue import ( + "context" + "crypto/rand" "encoding/json" "fmt" + "io" "log/slog" - "github.com/gocraft/work" - "github.com/gomodule/redigo/redis" + "github.com/redis/go-redis/v9" + "golang.org/x/xerrors" "github.com/aquasecurity/harbor-scanner-trivy/pkg/etc" "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" @@ -14,51 +17,84 @@ import ( "github.com/aquasecurity/harbor-scanner-trivy/pkg/persistence" ) -const ( - scanArtifactJobName = "scan_artifact" - scanRequestJobArg = "scan_request" -) +const scanArtifactJobName = "scan_artifact" type Enqueuer interface { - Enqueue(request harbor.ScanRequest) (job.ScanJob, error) + Enqueue(ctx context.Context, request harbor.ScanRequest) (job.ScanJob, error) } type enqueuer struct { - enqueuer *work.Enqueuer - store persistence.Store + namespace string + rdb *redis.Client + store persistence.Store } -func NewEnqueuer(config etc.JobQueue, redisPool *redis.Pool, store persistence.Store) Enqueuer { - return &enqueuer{ - enqueuer: work.NewEnqueuer(config.Namespace, redisPool), - store: store, - } +type Job struct { + Name string + ID string + Args Args } -func (e *enqueuer) Enqueue(request harbor.ScanRequest) (job.ScanJob, error) { - slog.Debug("Enqueueing scan job") +type Args struct { + ScanRequest *harbor.ScanRequest `json:",omitempty"` +} - b, err := json.Marshal(request) - if err != nil { - return job.ScanJob{}, fmt.Errorf("marshalling scan request: %v", err) +func NewEnqueuer(config etc.JobQueue, rdb *redis.Client, store persistence.Store) Enqueuer { + return &enqueuer{ + namespace: config.Namespace, + rdb: rdb, + store: store, } +} - j, err := e.enqueuer.Enqueue(scanArtifactJobName, work.Q{ - scanRequestJobArg: string(b), - }) - if err != nil { - return job.ScanJob{}, fmt.Errorf("enqueuing scan artifact job: %v", err) +func (e *enqueuer) Enqueue(ctx context.Context, request harbor.ScanRequest) (job.ScanJob, error) { + slog.Debug("Enqueueing scan job") + j := Job{ + Name: scanArtifactJobName, + ID: makeIdentifier(), + Args: Args{ + ScanRequest: &request, + }, } - slog.Debug("Successfully enqueued scan job", slog.String("job_id", j.ID)) scanJob := job.ScanJob{ ID: j.ID, Status: job.Queued, } - if err = e.store.Create(scanJob); err != nil { - return job.ScanJob{}, fmt.Errorf("creating scan job %v", err) + // Save the job status to Redis + if err := e.store.Create(ctx, scanJob); err != nil { + return job.ScanJob{}, xerrors.Errorf("creating scan job %v", err) + } + + b, err := json.Marshal(j) + if err != nil { + return job.ScanJob{}, xerrors.Errorf("marshalling scan request: %v", err) + } + + // Publish the job to the workers + if err = e.rdb.Publish(ctx, e.redisJobChannel(), b).Err(); err != nil { + return job.ScanJob{}, xerrors.Errorf("enqueuing scan artifact job: %v", err) } + slog.Debug("Successfully enqueued scan job", slog.String("job_id", j.ID)) + return scanJob, nil } + +func (e *enqueuer) redisJobChannel() string { + return redisJobChannel(e.namespace) +} + +func makeIdentifier() string { + b := make([]byte, 12) + _, err := io.ReadFull(rand.Reader, b) + if err != nil { + return "" + } + return fmt.Sprintf("%x", b) +} + +func redisJobChannel(namespace string) string { + return namespace + "jobs:" + scanArtifactJobName +} diff --git a/pkg/queue/worker.go b/pkg/queue/worker.go index eefea8ac..18de8a38 100644 --- a/pkg/queue/worker.go +++ b/pkg/queue/worker.go @@ -1,86 +1,100 @@ package queue import ( + "context" "encoding/json" - "fmt" "log/slog" + "time" - "github.com/gocraft/work" - "github.com/gomodule/redigo/redis" + "github.com/redis/go-redis/v9" + "github.com/samber/lo" + "golang.org/x/xerrors" "github.com/aquasecurity/harbor-scanner-trivy/pkg/etc" - "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" "github.com/aquasecurity/harbor-scanner-trivy/pkg/scan" ) -const ( - scanJobDefaultPriority = 1 // The highest - scanJobMaxFailures = 1 -) - type Worker interface { - Start() + Start(ctx context.Context) Stop() } type worker struct { - workerPool *work.WorkerPool -} + namespace string + concurrency int -func NewWorker(config etc.JobQueue, redisPool *redis.Pool, controller scan.Controller) Worker { + rdb *redis.Client + pubsub *redis.PubSub - workerPool := work.NewWorkerPool(workerContext{}, uint(config.WorkerConcurrency), config.Namespace, redisPool) + controller scan.Controller +} - // Note: For each scan job a new instance of the workerContext struct is created. - // Therefore, the only way to do a proper dependency injection is to use such closure - // and the following middleware as the first step in the processing chain. - workerPool.Middleware(func(ctx *workerContext, job *work.Job, next work.NextMiddlewareFunc) error { - ctx.controller = controller - return next() - }) +func NewWorker(config etc.JobQueue, rdb *redis.Client, controller scan.Controller) Worker { + return &worker{ + namespace: config.Namespace, + concurrency: config.WorkerConcurrency, - workerPool.JobWithOptions(scanArtifactJobName, - work.JobOptions{ - Priority: scanJobDefaultPriority, - MaxFails: scanJobMaxFailures, - }, (*workerContext).ScanArtifact) + rdb: rdb, - return &worker{ - workerPool: workerPool, + controller: controller, } } -func (w *worker) Start() { - w.workerPool.Start() +func (w *worker) Start(ctx context.Context) { + w.pubsub = w.rdb.Subscribe(ctx, w.redisJobChannel()) + ch := w.pubsub.Channel() + + for i := 0; i < w.concurrency; i++ { + go func() { + w.subscribe(ctx, ch) + }() + } } func (w *worker) Stop() { slog.Debug("Job queue shutdown started") - w.workerPool.Stop() + _ = w.pubsub.Close() slog.Debug("Job queue shutdown completed") } -// workerContext is a context for running scan jobs. -type workerContext struct { - controller scan.Controller +func (w *worker) redisJobChannel() string { + return redisJobChannel(w.namespace) } -// ScanArtifact is a handler function for the specified scan Job with the given workerContext. -func (s *workerContext) ScanArtifact(job *work.Job) error { - slog.Debug("Executing enqueued scan job", slog.String("scan_job_id", job.ID)) +func (w *worker) subscribe(ctx context.Context, ch <-chan *redis.Message) { + for msg := range ch { + chLog := slog.With( + slog.String("channel", msg.Channel), + slog.String("payload", msg.Payload), + ) + chLog.Debug("Message subscribed") + + if err := w.scanArtifact(ctx, msg); err != nil { + chLog.Error("Failed to scan artifact", slog.String("err", err.Error())) + continue + } + } +} - request, err := s.unmarshalScanRequest(job) +func (w *worker) scanArtifact(ctx context.Context, msg *redis.Message) error { + var job Job + if err := json.Unmarshal([]byte(msg.Payload), &job); err != nil { + return xerrors.Errorf("unmarshalling scan request: %w", err) + } + + // Lock the job so that other workers won't process it. + nx, err := w.rdb.SetNX(ctx, redisLockKey(w.namespace, job.ID), "", 5*time.Minute).Result() if err != nil { - return err + return xerrors.Errorf("redis lock: %w", err) + } else if !nx { + slog.Debug("Skip the locked job", slog.String("scan_job_id", job.ID)) + return nil } - return s.controller.Scan(job.ID, request) + slog.Debug("Executing enqueued scan job", slog.String("scan_job_id", job.ID)) + return w.controller.Scan(ctx, job.ID, lo.FromPtr(job.Args.ScanRequest)) } -func (s *workerContext) unmarshalScanRequest(job *work.Job) (request harbor.ScanRequest, err error) { - // TODO Fail fast and assert that the scan_request arg was set by the enqueuer. - if err = json.Unmarshal([]byte(job.ArgString(scanRequestJobArg)), &request); err != nil { - return request, fmt.Errorf("unmarshalling scan request: %v", err) - } - return +func redisLockKey(namespace, jobID string) string { + return redisJobChannel(namespace) + ":lock:" + jobID } diff --git a/pkg/redisx/pool.go b/pkg/redisx/pool.go index c7dcb196..93fc667c 100644 --- a/pkg/redisx/pool.go +++ b/pkg/redisx/pool.go @@ -1,120 +1,87 @@ package redisx import ( - "errors" + "context" "fmt" "log/slog" "net/url" "strconv" "strings" - "time" - "github.com/FZambia/sentinel" - "github.com/gomodule/redigo/redis" + "github.com/redis/go-redis/v9" + "golang.org/x/xerrors" "github.com/aquasecurity/harbor-scanner-trivy/pkg/etc" ) -// NewPool constructs a redis.Pool with the specified configuration. +// NewClient constructs a redis.Client with the specified configuration. // // The URI scheme currently supports connections to a standalone Redis server, // i.e. `redis://user:password@host:port/db-number`. -func NewPool(config etc.RedisPool) (pool *redis.Pool, err error) { +func NewClient(config etc.RedisPool) (*redis.Client, error) { configURL, err := url.Parse(config.URL) if err != nil { - err = fmt.Errorf("invalid redis URL: %s", err) - return + return nil, xerrors.Errorf("invalid redis URL: %s", err) } switch configURL.Scheme { case "redis": - pool = newInstancePool(config) + return newInstancePool(config) case "redis+sentinel": return newSentinelPool(configURL, config) default: - err = fmt.Errorf("invalid redis URL scheme: %s", configURL.Scheme) + return nil, xerrors.Errorf("invalid redis URL scheme: %s", configURL.Scheme) } - return } // redis://user:password@host:port/db-number -func newInstancePool(config etc.RedisPool) *redis.Pool { - return &redis.Pool{ - Dial: func() (redis.Conn, error) { - slog.Debug("Connecting to Redis", slog.String("url", config.URL)) - return redis.DialURL(config.URL) - }, - MaxIdle: config.MaxIdle, - MaxActive: config.MaxActive, - IdleTimeout: config.IdleTimeout, - Wait: true, +func newInstancePool(config etc.RedisPool) (*redis.Client, error) { + // TODO: Ask the Harbor team about why they use "idle_timeout_seconds" instead of "idle_timeout". + config.URL = strings.ReplaceAll(config.URL, "idle_timeout_seconds", "idle_timeout") + + slog.Debug("Constructing connection pool for Redis", slog.String("url", config.URL)) + options, err := redis.ParseURL(config.URL) + if err != nil { + return nil, xerrors.Errorf("invalid redis URL: %s", err) } + + options.MaxIdleConns = config.MaxIdle + options.MaxActiveConns = config.MaxActive + options.ConnMaxIdleTime = config.IdleTimeout + options.OnConnect = func(ctx context.Context, cn *redis.Conn) error { + slog.Debug("Connecting to Redis", slog.String("connection", cn.String())) + return nil + } + + return redis.NewClient(options), nil } // redis+sentinel://user:password@sentinel_host1:port1,sentinel_host2:port2/monitor-name/db-number -func newSentinelPool(configURL *url.URL, config etc.RedisPool) (pool *redis.Pool, err error) { +func newSentinelPool(configURL *url.URL, config etc.RedisPool) (*redis.Client, error) { slog.Debug("Constructing connection pool for Redis Sentinel") sentinelURL, err := ParseSentinelURL(configURL) if err != nil { - return + return nil, xerrors.Errorf("invalid redis sentinel URL: %s", err) } - var commonOpts []redis.DialOption - if config.ConnectionTimeout > 0 { - commonOpts = append(commonOpts, redis.DialConnectTimeout(config.ConnectionTimeout)) - } - if config.ReadTimeout > 0 { - commonOpts = append(commonOpts, redis.DialReadTimeout(config.ReadTimeout)) - } - if config.WriteTimeout > 0 { - commonOpts = append(commonOpts, redis.DialWriteTimeout(config.WriteTimeout)) - } + return redis.NewFailoverClient(&redis.FailoverOptions{ + MasterName: sentinelURL.MonitorName, + SentinelAddrs: sentinelURL.Addrs, + DB: sentinelURL.Database, + Password: sentinelURL.Password, - sentinelOpts := commonOpts - - sntnl := &sentinel.Sentinel{ - Addrs: sentinelURL.Addrs, - MasterName: sentinelURL.MonitorName, - Dial: func(addr string) (conn redis.Conn, err error) { - slog.Debug("Connecting to Redis sentinel", slog.String("addr", addr)) - conn, err = redis.Dial("tcp", addr, sentinelOpts...) - if err != nil { - return - } - return - }, - } + DialTimeout: config.ConnectionTimeout, + ReadTimeout: config.ReadTimeout, + WriteTimeout: config.WriteTimeout, - redisOpts := commonOpts + MaxIdleConns: config.MaxIdle, + ConnMaxIdleTime: config.IdleTimeout, - redisOpts = append(redisOpts, redis.DialDatabase(sentinelURL.Database)) - redisOpts = append(redisOpts, redis.DialPassword(sentinelURL.Password)) - - pool = &redis.Pool{ - Dial: func() (conn redis.Conn, err error) { - masterAddr, err := sntnl.MasterAddr() - if err != nil { - return - } - slog.Debug("Connecting to Redis master", slog.String("addr", masterAddr)) - return redis.Dial("tcp", masterAddr, redisOpts...) - }, - TestOnBorrow: func(c redis.Conn, t time.Time) error { - if time.Since(t) < time.Minute { - return nil - } - slog.Debug("Testing connection to Redis master on borrow") - if !sentinel.TestRole(c, "master") { - return errors.New("role check failed") - } + OnConnect: func(ctx context.Context, cn *redis.Conn) error { + slog.Debug("Connecting to Redis sentinel", slog.String("connection", cn.String())) return nil }, - MaxIdle: config.MaxIdle, - MaxActive: config.MaxActive, - IdleTimeout: config.IdleTimeout, - Wait: true, - } - return + }), nil } type SentinelURL struct { diff --git a/pkg/redisx/pool_test.go b/pkg/redisx/pool_test.go index 16e006a2..82186fc6 100644 --- a/pkg/redisx/pool_test.go +++ b/pkg/redisx/pool_test.go @@ -13,14 +13,14 @@ import ( func TestGetPool(t *testing.T) { t.Run("Should return error when configured to connect to secure redis", func(t *testing.T) { - _, err := NewPool(etc.RedisPool{ + _, err := NewClient(etc.RedisPool{ URL: "rediss://hostname:6379", }) assert.EqualError(t, err, "invalid redis URL scheme: rediss") }) t.Run("Should return error when configured with unsupported url scheme", func(t *testing.T) { - _, err := NewPool(etc.RedisPool{ + _, err := NewClient(etc.RedisPool{ URL: "https://hostname:6379", }) assert.EqualError(t, err, "invalid redis URL scheme: https") diff --git a/pkg/scan/controller.go b/pkg/scan/controller.go index 3772c4cd..b44d7399 100644 --- a/pkg/scan/controller.go +++ b/pkg/scan/controller.go @@ -1,6 +1,7 @@ package scan import ( + "context" "encoding/base64" "log/slog" "strings" @@ -13,7 +14,7 @@ import ( ) type Controller interface { - Scan(scanJobID string, request harbor.ScanRequest) error + Scan(ctx context.Context, scanJobID string, request harbor.ScanRequest) error } type controller struct { @@ -30,24 +31,24 @@ func NewController(store persistence.Store, wrapper trivy.Wrapper, transformer T } } -func (c *controller) Scan(scanJobID string, request harbor.ScanRequest) error { - if err := c.scan(scanJobID, request); err != nil { +func (c *controller) Scan(ctx context.Context, scanJobID string, request harbor.ScanRequest) error { + if err := c.scan(ctx, scanJobID, request); err != nil { slog.Error("Scan failed", slog.String("err", err.Error())) - if err = c.store.UpdateStatus(scanJobID, job.Failed, err.Error()); err != nil { + if err = c.store.UpdateStatus(ctx, scanJobID, job.Failed, err.Error()); err != nil { return xerrors.Errorf("updating scan job as failed: %v", err) } } return nil } -func (c *controller) scan(scanJobID string, req harbor.ScanRequest) (err error) { +func (c *controller) scan(ctx context.Context, scanJobID string, req harbor.ScanRequest) (err error) { defer func() { if r := recover(); r != nil { err = r.(error) } }() - err = c.store.UpdateStatus(scanJobID, job.Pending) + err = c.store.UpdateStatus(ctx, scanJobID, job.Pending) if err != nil { return xerrors.Errorf("updating scan job status: %v", err) } @@ -67,13 +68,11 @@ func (c *controller) scan(scanJobID string, req harbor.ScanRequest) (err error) return xerrors.Errorf("running trivy wrapper: %v", err) } - err = c.store.UpdateReport(scanJobID, c.transformer.Transform(req.Artifact, scanReport)) - if err != nil { + if err = c.store.UpdateReport(ctx, scanJobID, c.transformer.Transform(req.Artifact, scanReport)); err != nil { return xerrors.Errorf("saving scan report: %v", err) } - err = c.store.UpdateStatus(scanJobID, job.Finished) - if err != nil { + if err = c.store.UpdateStatus(ctx, scanJobID, job.Finished); err != nil { return xerrors.Errorf("updating scan job status: %v", err) } diff --git a/test/integration/persistence/redis/store_test.go b/test/integration/persistence/redis/store_test.go index 5bfde086..16d89ec0 100644 --- a/test/integration/persistence/redis/store_test.go +++ b/test/integration/persistence/redis/store_test.go @@ -47,7 +47,7 @@ func TestStore(t *testing.T) { ScanJobTTL: parseDuration(t, "10s"), } - pool, err := redisx.NewPool(etc.RedisPool{ + pool, err := redisx.NewClient(etc.RedisPool{ URL: redisURL, }) require.NoError(t, err) From 2417e610c468ad2a99103a52530d8e31ea5e1c79 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Mon, 30 Oct 2023 14:26:37 +0900 Subject: [PATCH 02/26] chore(deps): go mod tidy Signed-off-by: knqyf263 --- go.mod | 7 +++---- go.sum | 18 ++++++++++-------- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/go.mod b/go.mod index 6dc31628..72edc3da 100644 --- a/go.mod +++ b/go.mod @@ -3,15 +3,14 @@ module github.com/aquasecurity/harbor-scanner-trivy go 1.21 require ( - github.com/FZambia/sentinel v1.1.1 github.com/caarlos0/env/v6 v6.10.1 github.com/docker/docker v24.0.6+incompatible github.com/docker/go-connections v0.4.0 - github.com/gocraft/work v0.5.1 - github.com/gomodule/redigo v1.8.9 github.com/gorilla/mux v1.8.0 github.com/opencontainers/go-digest v1.0.0 github.com/prometheus/client_golang v1.17.0 + github.com/redis/go-redis/v9 v9.2.1 + github.com/samber/lo v1.38.1 github.com/stretchr/testify v1.8.4 github.com/testcontainers/testcontainers-go v0.25.0 golang.org/x/net v0.17.0 @@ -29,6 +28,7 @@ require ( github.com/containerd/containerd v1.7.6 // indirect github.com/cpuguy83/dockercfg v0.3.1 // indirect github.com/davecgh/go-spew v1.1.1 // indirect + github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f // indirect github.com/docker/distribution v2.8.2+incompatible // indirect github.com/docker/go-units v0.5.0 // indirect github.com/go-ole/go-ole v1.2.6 // indirect @@ -51,7 +51,6 @@ require ( github.com/prometheus/client_model v0.4.1-0.20230718164431-9a2bf3000d16 // indirect github.com/prometheus/common v0.44.0 // indirect github.com/prometheus/procfs v0.11.1 // indirect - github.com/robfig/cron v1.2.0 // indirect github.com/shirou/gopsutil/v3 v3.23.8 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect github.com/sirupsen/logrus v1.9.3 // indirect diff --git a/go.sum b/go.sum index dfa208c3..2bca1e36 100644 --- a/go.sum +++ b/go.sum @@ -5,14 +5,16 @@ github.com/AdaLogics/go-fuzz-headers v0.0.0-20230811130428-ced1acdcaa24/go.mod h github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 h1:UQHMgLO+TxOElx5B5HZ4hJQsoJ/PvUvKRhJHDQXO8P8= github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1/go.mod h1:xomTg63KZ2rFqZQzSB4Vz2SUXa1BpHTVz9L5PTmPC4E= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/FZambia/sentinel v1.1.1 h1:0ovTimlR7Ldm+wR15GgO+8C2dt7kkn+tm3PQS+Qk3Ek= -github.com/FZambia/sentinel v1.1.1/go.mod h1:ytL1Am/RLlAoAXG6Kj5LNuw/TRRQrv2rt2FT26vP5gI= github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= github.com/Microsoft/hcsshim v0.11.0 h1:7EFNIY4igHEXUdj1zXgAyU3fLc7QfOKHbkldRVTBdiM= github.com/Microsoft/hcsshim v0.11.0/go.mod h1:OEthFdQv/AD2RAdzR6Mm1N1KPCztGKDurW1Z8b8VGMM= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/bsm/ginkgo/v2 v2.12.0 h1:Ny8MWAHyOepLGlLKYmXG4IEkioBysk6GpaRTLC8zwWs= +github.com/bsm/ginkgo/v2 v2.12.0/go.mod h1:SwYbGRRDovPVboqFv0tPTcG1sN61LM1Z4ARdbAV9g4c= +github.com/bsm/gomega v1.27.10 h1:yeMWxP2pV2fG3FgAODIY8EiRE3dy0aeFYt4l7wh6yKA= +github.com/bsm/gomega v1.27.10/go.mod h1:JyEr/xRbxbtgWNi8tIEVPUYZ5Dzef52k01W3YH0H+O0= github.com/caarlos0/env/v6 v6.10.1 h1:t1mPSxNpei6M5yAeu1qtRdPAK29Nbcf/n3G7x+b3/II= github.com/caarlos0/env/v6 v6.10.1/go.mod h1:hvp/ryKXKipEkcuYjs9mI4bBCg+UI0Yhgm5Zu0ddvwc= github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= @@ -34,6 +36,8 @@ github.com/cyphar/filepath-securejoin v0.2.3/go.mod h1:aPGpWjXOXUn2NCNjFvBE6aRxG github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f h1:lO4WD4F/rVNCu3HqELle0jiPLLBs70cWOduZpkS1E78= +github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f/go.mod h1:cuUVRXasLTGF7a8hSLbxyZXjz+1KgoB3wDUb6vlszIc= github.com/docker/distribution v2.8.2+incompatible h1:T3de5rq0dB1j30rp0sA2rER+m322EBzniBPB6ZIzuh8= github.com/docker/distribution v2.8.2+incompatible/go.mod h1:J2gT2udsDAN96Uj4KfcMRqY0/ypR+oyYUYmja8H+y+w= github.com/docker/docker v24.0.6+incompatible h1:hceabKCtUgDqPu+qm0NgsaXf28Ljf4/pWFL7xjWWDgE= @@ -46,8 +50,6 @@ github.com/docker/go-units v0.5.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDD github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= -github.com/gocraft/work v0.5.1 h1:3bRjMiOo6N4zcRgZWV3Y7uX7R22SF+A9bPTk4xRXr34= -github.com/gocraft/work v0.5.1/go.mod h1:pc3n9Pb5FAESPPGfM0nL+7Q1xtgtRnF8rr/azzhQVlM= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/godbus/dbus/v5 v5.0.6/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= @@ -56,8 +58,6 @@ github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5y github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= -github.com/gomodule/redigo v1.8.9 h1:Sl3u+2BI/kk+VEatbj0scLdrFhjPmbxOc1myhDP41ws= -github.com/gomodule/redigo v1.8.9/go.mod h1:7ArFNvsTjH8GMMzB4uy1snslv2BwmginuMs06a1uzZE= github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= @@ -115,11 +115,13 @@ github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdO github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= github.com/prometheus/procfs v0.11.1 h1:xRC8Iq1yyca5ypa9n1EZnWZkt7dwcoRPQwX/5gwaUuI= github.com/prometheus/procfs v0.11.1/go.mod h1:eesXgaPo1q7lBpVMoMy0ZOFTth9hBn4W/y0/p/ScXhY= -github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= -github.com/robfig/cron v1.2.0/go.mod h1:JGuDeoQd7Z6yL4zQhZ3OPEVHB7fL6Ka6skscFHfmt2k= +github.com/redis/go-redis/v9 v9.2.1 h1:WlYJg71ODF0dVspZZCpYmoF1+U1Jjk9Rwd7pq6QmlCg= +github.com/redis/go-redis/v9 v9.2.1/go.mod h1:hdY0cQFCN4fnSYT6TkisLufl/4W5UIXyv0b/CLO2V2M= github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/samber/lo v1.38.1 h1:j2XEAqXKb09Am4ebOg31SpvzUTTs6EN3VfgeLUhPdXM= +github.com/samber/lo v1.38.1/go.mod h1:+m/ZKRl6ClXCE2Lgf3MsQlWfh4bn1bz6CXEOxnEXnEA= github.com/seccomp/libseccomp-golang v0.9.2-0.20220502022130-f33da4d89646/go.mod h1:JA8cRccbGaA1s33RQf7Y1+q9gHmZX1yB/z9WDN1C6fg= github.com/shirou/gopsutil/v3 v3.23.8 h1:xnATPiybo6GgdRoC4YoGnxXZFRc3dqQTGi73oLvvBrE= github.com/shirou/gopsutil/v3 v3.23.8/go.mod h1:7hmCaBn+2ZwaZOr6jmPBZDfawwMGuo1id3C6aM8EDqQ= From f5e338ea003e3f1f222959a62be30e7cc5c5c015 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Mon, 30 Oct 2023 14:26:45 +0900 Subject: [PATCH 03/26] chore: add run target Signed-off-by: knqyf263 --- Makefile | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/Makefile b/Makefile index 9e307ef9..74c3f689 100644 --- a/Makefile +++ b/Makefile @@ -29,3 +29,11 @@ lint: .PHONY: setup setup: curl -sfL https://raw.githubusercontent.com/golangci/golangci-lint/master/install.sh| sh -s v1.21.0 + +.PHONY: run +run: export SCANNER_TRIVY_CACHE_DIR = $(TMPDIR)harbor-scanner-trivy/.cache/trivy +run: export SCANNER_TRIVY_REPORTS_DIR=$(TMPDIR)harbor-scanner-trivy/.cache/reports +run: export SCANNER_LOG_LEVEL=debug +run: + @mkdir -p $(SCANNER_TRIVY_CACHE_DIR) $(SCANNER_TRIVY_REPORTS_DIR) + @go run cmd/scanner-trivy/main.go From 5255848234d1e7dcf0ac21f2d4c4cb979a5e831b Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Mon, 30 Oct 2023 14:58:50 +0900 Subject: [PATCH 04/26] test: fix args Signed-off-by: knqyf263 --- pkg/http/api/v1/handler_test.go | 18 +++++++++--------- pkg/mock/enqueuer.go | 5 +++-- pkg/mock/expectation.go | 3 +++ pkg/mock/store.go | 17 +++++++++-------- pkg/scan/controller_test.go | 16 +++++++++------- test/integration/api/rest_api_test.go | 8 ++++---- .../persistence/redis/store_test.go | 16 ++++++++-------- 7 files changed, 45 insertions(+), 38 deletions(-) diff --git a/pkg/http/api/v1/handler_test.go b/pkg/http/api/v1/handler_test.go index 32c49aee..3d8aedd8 100644 --- a/pkg/http/api/v1/handler_test.go +++ b/pkg/http/api/v1/handler_test.go @@ -117,7 +117,7 @@ func TestRequestHandler_AcceptScanRequest(t *testing.T) { name: "Should accept scan request", enqueuerExpectation: &mock.Expectation{ Method: "Enqueue", - Args: []interface{}{validScanRequest}, + Args: []interface{}{mock.Anything, validScanRequest}, ReturnArgs: []interface{}{job.ScanJob{ID: "job:123"}, nil}, }, requestBody: validScanRequestJSON, @@ -151,7 +151,7 @@ func TestRequestHandler_AcceptScanRequest(t *testing.T) { name: "Should respond with error 500 when enqueuing scan request fails", enqueuerExpectation: &mock.Expectation{ Method: "Enqueue", - Args: []interface{}{validScanRequest}, + Args: []interface{}{mock.Anything, validScanRequest}, ReturnArgs: []interface{}{job.ScanJob{}, errors.New("queue is down")}, }, requestBody: validScanRequestJSON, @@ -203,7 +203,7 @@ func TestRequestHandler_GetScanReport(t *testing.T) { name: "Should respond with error 500 when retrieving scan job fails", storeExpectation: &mock.Expectation{ Method: "Get", - Args: []interface{}{"job:123"}, + Args: []interface{}{mock.Anything, "job:123"}, ReturnArgs: []interface{}{&job.ScanJob{}, errors.New("data store is down")}, }, expectedStatus: http.StatusInternalServerError, @@ -218,7 +218,7 @@ func TestRequestHandler_GetScanReport(t *testing.T) { name: "Should respond with error 404 when scan job cannot be found", storeExpectation: &mock.Expectation{ Method: "Get", - Args: []interface{}{"job:123"}, + Args: []interface{}{mock.Anything, "job:123"}, ReturnArgs: []interface{}{(*job.ScanJob)(nil), nil}, }, expectedStatus: http.StatusNotFound, @@ -233,7 +233,7 @@ func TestRequestHandler_GetScanReport(t *testing.T) { name: fmt.Sprintf("Should respond with found status 302 when scan job is %s", job.Queued), storeExpectation: &mock.Expectation{ Method: "Get", - Args: []interface{}{"job:123"}, + Args: []interface{}{mock.Anything, "job:123"}, ReturnArgs: []interface{}{&job.ScanJob{ ID: "job:123", Status: job.Queued, @@ -245,7 +245,7 @@ func TestRequestHandler_GetScanReport(t *testing.T) { name: fmt.Sprintf("Should respond with found status 302 when scan job is %s", job.Pending), storeExpectation: &mock.Expectation{ Method: "Get", - Args: []interface{}{"job:123"}, + Args: []interface{}{mock.Anything, "job:123"}, ReturnArgs: []interface{}{&job.ScanJob{ ID: "job:123", Status: job.Pending, @@ -257,7 +257,7 @@ func TestRequestHandler_GetScanReport(t *testing.T) { name: fmt.Sprintf("Should respond with error 500 when scan job is %s", job.Failed), storeExpectation: &mock.Expectation{ Method: "Get", - Args: []interface{}{"job:123"}, + Args: []interface{}{mock.Anything, "job:123"}, ReturnArgs: []interface{}{&job.ScanJob{ ID: "job:123", Status: job.Failed, @@ -276,7 +276,7 @@ func TestRequestHandler_GetScanReport(t *testing.T) { name: fmt.Sprintf("Should respond with error 500 when scan job is NOT %s", job.Finished), storeExpectation: &mock.Expectation{ Method: "Get", - Args: []interface{}{"job:123"}, + Args: []interface{}{mock.Anything, "job:123"}, ReturnArgs: []interface{}{&job.ScanJob{ ID: "job:123", Status: 666, @@ -295,7 +295,7 @@ func TestRequestHandler_GetScanReport(t *testing.T) { name: "Should respond with vulnerabilities report", storeExpectation: &mock.Expectation{ Method: "Get", - Args: []interface{}{"job:123"}, + Args: []interface{}{mock.Anything, "job:123"}, ReturnArgs: []interface{}{&job.ScanJob{ ID: "job:123", Status: job.Finished, diff --git a/pkg/mock/enqueuer.go b/pkg/mock/enqueuer.go index 952f0baf..612f4a1a 100644 --- a/pkg/mock/enqueuer.go +++ b/pkg/mock/enqueuer.go @@ -1,6 +1,7 @@ package mock import ( + "context" "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" "github.com/aquasecurity/harbor-scanner-trivy/pkg/job" "github.com/stretchr/testify/mock" @@ -14,7 +15,7 @@ func NewEnqueuer() *Enqueuer { return &Enqueuer{} } -func (em *Enqueuer) Enqueue(request harbor.ScanRequest) (job.ScanJob, error) { - args := em.Called(request) +func (em *Enqueuer) Enqueue(ctx context.Context, request harbor.ScanRequest) (job.ScanJob, error) { + args := em.Called(ctx, request) return args.Get(0).(job.ScanJob), args.Error(1) } diff --git a/pkg/mock/expectation.go b/pkg/mock/expectation.go index f7e27ae0..d00dec37 100644 --- a/pkg/mock/expectation.go +++ b/pkg/mock/expectation.go @@ -1,11 +1,14 @@ package mock import ( + "github.com/stretchr/testify/mock" "testing" "github.com/aquasecurity/harbor-scanner-trivy/pkg/trivy" ) +const Anything = mock.Anything + // Expectation represents an expectation of a method being called and its return values. type Expectation struct { Method string diff --git a/pkg/mock/store.go b/pkg/mock/store.go index 180a302d..8b0fc271 100644 --- a/pkg/mock/store.go +++ b/pkg/mock/store.go @@ -1,6 +1,7 @@ package mock import ( + "context" "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" "github.com/aquasecurity/harbor-scanner-trivy/pkg/job" "github.com/stretchr/testify/mock" @@ -14,22 +15,22 @@ func NewStore() *Store { return &Store{} } -func (s *Store) Create(scanJob job.ScanJob) error { - args := s.Called(scanJob) +func (s *Store) Create(ctx context.Context, scanJob job.ScanJob) error { + args := s.Called(ctx, scanJob) return args.Error(0) } -func (s *Store) Get(scanJobID string) (*job.ScanJob, error) { - args := s.Called(scanJobID) +func (s *Store) Get(ctx context.Context, scanJobID string) (*job.ScanJob, error) { + args := s.Called(ctx, scanJobID) return args.Get(0).(*job.ScanJob), args.Error(1) } -func (s *Store) UpdateStatus(scanJobID string, newStatus job.ScanJobStatus, error ...string) error { - args := s.Called(scanJobID, newStatus, error) +func (s *Store) UpdateStatus(ctx context.Context, scanJobID string, newStatus job.ScanJobStatus, error ...string) error { + args := s.Called(ctx, scanJobID, newStatus, error) return args.Error(0) } -func (s *Store) UpdateReport(scanJobID string, report harbor.ScanReport) error { - args := s.Called(scanJobID, report) +func (s *Store) UpdateReport(ctx context.Context, scanJobID string, report harbor.ScanReport) error { + args := s.Called(ctx, scanJobID, report) return args.Error(0) } diff --git a/pkg/scan/controller_test.go b/pkg/scan/controller_test.go index ed9de751..a3e5616c 100644 --- a/pkg/scan/controller_test.go +++ b/pkg/scan/controller_test.go @@ -1,6 +1,7 @@ package scan import ( + "context" "fmt" "testing" @@ -12,7 +13,8 @@ import ( "golang.org/x/xerrors" ) -func TestController_Scan(t *testing.T) { +func TestContoller_Scan(t *testing.T) { + ctx := context.Background() artifact := harbor.Artifact{ Repository: "library/mongo", Digest: "sha256:917f5b7f4bef1b35ee90f03033f33a81002511c1e0767fd44276d4bd9cd2fa8e", @@ -44,17 +46,17 @@ func TestController_Scan(t *testing.T) { storeExpectation: []*mock.Expectation{ { Method: "UpdateStatus", - Args: []interface{}{"job:123", job.Pending, []string(nil)}, + Args: []interface{}{ctx, "job:123", job.Pending, []string(nil)}, ReturnArgs: []interface{}{nil}, }, { Method: "UpdateReport", - Args: []interface{}{"job:123", harborReport}, + Args: []interface{}{ctx, "job:123", harborReport}, ReturnArgs: []interface{}{nil}, }, { Method: "UpdateStatus", - Args: []interface{}{"job:123", job.Finished, []string(nil)}, + Args: []interface{}{ctx, "job:123", job.Finished, []string(nil)}, ReturnArgs: []interface{}{nil}, }, }, @@ -96,12 +98,12 @@ func TestController_Scan(t *testing.T) { storeExpectation: []*mock.Expectation{ { Method: "UpdateStatus", - Args: []interface{}{"job:123", job.Pending, []string(nil)}, + Args: []interface{}{ctx, "job:123", job.Pending, []string(nil)}, ReturnArgs: []interface{}{nil}, }, { Method: "UpdateStatus", - Args: []interface{}{"job:123", job.Failed, []string{"running trivy wrapper: out of memory"}}, + Args: []interface{}{ctx, "job:123", job.Failed, []string{"running trivy wrapper: out of memory"}}, ReturnArgs: []interface{}{nil}, }, }, @@ -132,7 +134,7 @@ func TestController_Scan(t *testing.T) { mock.ApplyExpectations(t, wrapper, tc.wrapperExpectation) mock.ApplyExpectations(t, transformer, tc.transformerExpectation) - err := NewController(store, wrapper, transformer).Scan(tc.scanJobID, tc.scanRequest) + err := NewController(store, wrapper, transformer).Scan(ctx, tc.scanJobID, tc.scanRequest) assert.Equal(t, tc.expectedError, err) store.AssertExpectations(t) diff --git a/test/integration/api/rest_api_test.go b/test/integration/api/rest_api_test.go index 7fbfa2d0..02e87f86 100644 --- a/test/integration/api/rest_api_test.go +++ b/test/integration/api/rest_api_test.go @@ -1,10 +1,10 @@ //go:build integration -// +build integration package api import ( "fmt" + "io" "io/ioutil" "net/http" "net/http/httptest" @@ -57,7 +57,7 @@ func TestRestApi(t *testing.T) { t.Run("POST /api/v1/scan", func(t *testing.T) { // given - enqueuer.On("Enqueue", harbor.ScanRequest{ + enqueuer.On("Enqueue", mock.Anything, harbor.ScanRequest{ Registry: harbor.Registry{ URL: "https://core.harbor.domain", Authorization: "Bearer JWTTOKENGOESHERE", @@ -85,7 +85,7 @@ func TestRestApi(t *testing.T) { assert.Equal(t, http.StatusAccepted, rs.StatusCode) assert.Equal(t, "application/vnd.scanner.adapter.scan.response+json; version=1.0", rs.Header.Get("Content-Type")) - bodyBytes, err := ioutil.ReadAll(rs.Body) + bodyBytes, err := io.ReadAll(rs.Body) require.NoError(t, err) assert.JSONEq(t, `{"id": "job:123"}`, string(bodyBytes)) @@ -95,7 +95,7 @@ func TestRestApi(t *testing.T) { // given now := time.Now() - store.On("Get", "job:123").Return(&job.ScanJob{ + store.On("Get", mock.Anything, "job:123").Return(&job.ScanJob{ ID: "job:123", Status: job.Finished, Report: harbor.ScanReport{ diff --git a/test/integration/persistence/redis/store_test.go b/test/integration/persistence/redis/store_test.go index 16d89ec0..610ceec9 100644 --- a/test/integration/persistence/redis/store_test.go +++ b/test/integration/persistence/redis/store_test.go @@ -57,23 +57,23 @@ func TestStore(t *testing.T) { t.Run("CRUD", func(t *testing.T) { scanJobID := "123" - err := store.Create(job.ScanJob{ + err := store.Create(ctx, job.ScanJob{ ID: scanJobID, Status: job.Queued, }) require.NoError(t, err, "saving scan job should not fail") - j, err := store.Get(scanJobID) + j, err := store.Get(ctx, scanJobID) require.NoError(t, err, "getting scan job should not fail") assert.Equal(t, &job.ScanJob{ ID: scanJobID, Status: job.Queued, }, j) - err = store.UpdateStatus(scanJobID, job.Pending) + err = store.UpdateStatus(ctx, scanJobID, job.Pending) require.NoError(t, err, "updating scan job status should not fail") - j, err = store.Get(scanJobID) + j, err = store.Get(ctx, scanJobID) require.NoError(t, err, "getting scan job should not fail") assert.Equal(t, &job.ScanJob{ ID: scanJobID, @@ -89,20 +89,20 @@ func TestStore(t *testing.T) { }, } - err = store.UpdateReport(scanJobID, scanReport) + err = store.UpdateReport(ctx, scanJobID, scanReport) require.NoError(t, err, "updating scan job reports should not fail") - j, err = store.Get(scanJobID) + j, err = store.Get(ctx, scanJobID) require.NoError(t, err, "retrieving scan job should not fail") require.NotNil(t, j, "retrieved scan job must not be nil") assert.Equal(t, scanReport, j.Report) - err = store.UpdateStatus(scanJobID, job.Finished) + err = store.UpdateStatus(ctx, scanJobID, job.Finished) require.NoError(t, err) time.Sleep(parseDuration(t, "12s")) - j, err = store.Get(scanJobID) + j, err = store.Get(ctx, scanJobID) require.NoError(t, err, "retrieve scan job should not fail") require.Nil(t, j, "retrieved scan job should be nil, i.e. expired") }) From 9f204ff75c0b1bc5309aeb0a79e4322472fdaf8e Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Tue, 31 Oct 2023 14:24:13 +0900 Subject: [PATCH 05/26] feat: support SBOM Signed-off-by: knqyf263 --- pkg/harbor/model.go | 35 +++++- pkg/http/api/base_handler.go | 6 + pkg/http/api/v1/handler.go | 36 +++++- pkg/http/api/v1/handler_test.go | 8 +- pkg/mock/transformer.go | 4 +- pkg/redisx/pool.go | 4 +- pkg/scan/controller.go | 16 ++- pkg/scan/controller_test.go | 16 ++- pkg/scan/transformer.go | 62 +++++----- pkg/scan/transformer_test.go | 167 +++++++++++++------------- pkg/trivy/model.go | 5 + pkg/trivy/wrapper.go | 58 +++++++-- pkg/trivy/wrapper_mock.go | 6 +- pkg/trivy/wrapper_test.go | 48 ++++---- test/integration/api/rest_api_test.go | 3 + 15 files changed, 308 insertions(+), 166 deletions(-) diff --git a/pkg/harbor/model.go b/pkg/harbor/model.go index cea4e2bf..2d5562e5 100644 --- a/pkg/harbor/model.go +++ b/pkg/harbor/model.go @@ -60,6 +60,24 @@ func (s *Severity) UnmarshalJSON(b []byte) error { return nil } +type ScanType string +type MediaType string + +const ( + ScanTypeSBOM ScanType = "sbom" + ScanTypeVulnerability ScanType = "vulnerability" + + MediaTypeSPDX MediaType = "application/spdx+json" + MediaTypeCycloneDX MediaType = "application/vnd.cyclonedx+json" +) + +var SupportedSBOMMediaTypes = []MediaType{MediaTypeSPDX, MediaTypeCycloneDX} + +type Scan struct { + Type ScanType `json:"type,omitempty"` + SBOMMediaType MediaType `json:"sbom_media_type,omitempty"` +} + type Registry struct { URL string `json:"url"` Authorization string `json:"authorization"` @@ -72,6 +90,7 @@ type Artifact struct { } type ScanRequest struct { + Scan Scan `json:"scan"` // From HTTP header Registry Registry `json:"registry"` Artifact Artifact `json:"artifact"` } @@ -103,11 +122,17 @@ type ScanResponse struct { } type ScanReport struct { - GeneratedAt time.Time `json:"generated_at"` - Artifact Artifact `json:"artifact"` - Scanner Scanner `json:"scanner"` - Severity Severity `json:"severity"` - Vulnerabilities []VulnerabilityItem `json:"vulnerabilities"` + GeneratedAt time.Time `json:"generated_at"` + Artifact Artifact `json:"artifact"` + Scanner Scanner `json:"scanner"` + Severity Severity `json:"severity,omitempty"` + + // For SBOM + MediaType string `json:"media_type,omitempty"` + SBOM any `json:"sbom,omitempty"` + + // For vulnerabilities + Vulnerabilities []VulnerabilityItem `json:"vulnerabilities,omitempty"` } type Layer struct { diff --git a/pkg/http/api/base_handler.go b/pkg/http/api/base_handler.go index 30fb5b75..0f80078c 100644 --- a/pkg/http/api/base_handler.go +++ b/pkg/http/api/base_handler.go @@ -25,6 +25,7 @@ var MimeTypeDockerImageManifestV2 = MimeType{Type: "application", Subtype: "vnd. var MimeTypeScanResponse = MimeType{Type: "application", Subtype: "vnd.scanner.adapter.scan.response+json", Params: MimeTypeVersion} var MimeTypeSecurityVulnerabilityReport = MimeType{Type: "application", Subtype: "vnd.security.vulnerability.report", Params: map[string]string{"version": "1.1"}} +var MimeTypeSecuritySBOMReport = MimeType{Type: "application", Subtype: "vnd.security.sbom.report+json", Params: map[string]string{"version": "1.1"}} var MimeTypeMetadata = MimeType{Type: "application", Subtype: "vnd.scanner.adapter.metadata+json", Params: MimeTypeVersion} var MimeTypeError = MimeType{Type: "application", Subtype: "vnd.scanner.adapter.error", Params: MimeTypeVersion} @@ -58,6 +59,11 @@ func (mt *MimeType) FromAcceptHeader(value string) error { mt.Subtype = MimeTypeSecurityVulnerabilityReport.Subtype mt.Params = MimeTypeSecurityVulnerabilityReport.Params return nil + case MimeTypeSecuritySBOMReport.String(): + mt.Type = MimeTypeSecuritySBOMReport.Type + mt.Subtype = MimeTypeSecuritySBOMReport.Subtype + mt.Params = MimeTypeSecuritySBOMReport.Params + return nil } return fmt.Errorf("unsupported mime type: %s", value) } diff --git a/pkg/http/api/v1/handler.go b/pkg/http/api/v1/handler.go index 9d6d5364..420f201b 100644 --- a/pkg/http/api/v1/handler.go +++ b/pkg/http/api/v1/handler.go @@ -6,6 +6,7 @@ import ( "log/slog" "net/http" "net/url" + "slices" "strconv" "time" @@ -76,7 +77,7 @@ func (h *requestHandler) logRequest(next http.Handler) http.Handler { } func (h *requestHandler) AcceptScanRequest(res http.ResponseWriter, req *http.Request) { - scanRequest := harbor.ScanRequest{} + var scanRequest harbor.ScanRequest if err := json.NewDecoder(req.Body).Decode(&scanRequest); err != nil { slog.Error("Error while unmarshalling scan request", slog.String("err", err.Error())) h.WriteJSONError(res, harbor.Error{ @@ -86,6 +87,17 @@ func (h *requestHandler) AcceptScanRequest(res http.ResponseWriter, req *http.Re return } + // Extract SBOM related properties from HTTP header + scanType := getHeader(req.Header, "X-Scan-Type", harbor.ScanTypeVulnerability) + var sbomMediaType harbor.MediaType + if scanType == harbor.ScanTypeSBOM { + sbomMediaType = getHeader(req.Header, "X-Scan-Sbom-MediaType", harbor.MediaTypeCycloneDX) + } + scanRequest.Scan = harbor.Scan{ + Type: scanType, + SBOMMediaType: sbomMediaType, + } + if validationError := h.ValidateScanRequest(scanRequest); validationError != nil { slog.Error("Error while validating scan request", slog.String("err", validationError.Message)) h.WriteJSONError(res, *validationError) @@ -108,6 +120,21 @@ func (h *requestHandler) AcceptScanRequest(res http.ResponseWriter, req *http.Re } func (h *requestHandler) ValidateScanRequest(req harbor.ScanRequest) *harbor.Error { + if req.Scan.Type != harbor.ScanTypeVulnerability && req.Scan.Type != harbor.ScanTypeSBOM { + return &harbor.Error{ + HTTPCode: http.StatusUnprocessableEntity, + Message: "invalid scan type", + } + } + + if req.Scan.Type == harbor.ScanTypeSBOM && + !slices.Contains(harbor.SupportedSBOMMediaTypes, req.Scan.SBOMMediaType) { + return &harbor.Error{ + HTTPCode: http.StatusUnprocessableEntity, + Message: "invalid SBOM media type", + } + } + if req.Registry.URL == "" { return &harbor.Error{ HTTPCode: http.StatusUnprocessableEntity, @@ -270,3 +297,10 @@ func (h *requestHandler) GetHealthy(res http.ResponseWriter, req *http.Request) func (h *requestHandler) GetReady(res http.ResponseWriter, req *http.Request) { res.WriteHeader(http.StatusOK) } + +func getHeader[T ~string](header http.Header, key string, defaultValue T) T { + if v := header.Get(key); v != "" { + return T(v) + } + return defaultValue +} diff --git a/pkg/http/api/v1/handler_test.go b/pkg/http/api/v1/handler_test.go index 3d8aedd8..2e5aaa22 100644 --- a/pkg/http/api/v1/handler_test.go +++ b/pkg/http/api/v1/handler_test.go @@ -27,7 +27,7 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { }{ { Name: "Should return error when Registry URL is blank", - Request: harbor.ScanRequest{}, + Request: harbor.ScanRequest{Scan: harbor.Scan{Type: harbor.ScanTypeVulnerability}}, ExpectedError: &harbor.Error{ HTTPCode: http.StatusUnprocessableEntity, Message: "missing registry.url", @@ -36,6 +36,7 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { { Name: "Should return error when Registry URL is invalid", Request: harbor.ScanRequest{ + Scan: harbor.Scan{Type: harbor.ScanTypeVulnerability}, Registry: harbor.Registry{ URL: "INVALID URL", }, @@ -48,6 +49,7 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { { Name: "Should return error when artifact repository is blank", Request: harbor.ScanRequest{ + Scan: harbor.Scan{Type: harbor.ScanTypeVulnerability}, Registry: harbor.Registry{ URL: "https://core.harbor.domain", }, @@ -60,6 +62,7 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { { Name: "Should return error when artifact digest is blank", Request: harbor.ScanRequest{ + Scan: harbor.Scan{Type: harbor.ScanTypeVulnerability}, Registry: harbor.Registry{ URL: "https://core.harbor.domain", }, @@ -85,6 +88,9 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { func TestRequestHandler_AcceptScanRequest(t *testing.T) { validScanRequest := harbor.ScanRequest{ + Scan: harbor.Scan{ + Type: harbor.ScanTypeVulnerability, + }, Registry: harbor.Registry{ URL: "https://core.harbor.domain", Authorization: "Bearer JWTTOKENGOESHERE", diff --git a/pkg/mock/transformer.go b/pkg/mock/transformer.go index 195a500e..bbfe80b2 100644 --- a/pkg/mock/transformer.go +++ b/pkg/mock/transformer.go @@ -14,7 +14,7 @@ func NewTransformer() *Transformer { return &Transformer{} } -func (t *Transformer) Transform(artifact harbor.Artifact, source []trivy.Vulnerability) harbor.ScanReport { - args := t.Called(artifact, source) +func (t *Transformer) Transform(req harbor.ScanRequest, source trivy.Report) harbor.ScanReport { + args := t.Called(req, source) return args.Get(0).(harbor.ScanReport) } diff --git a/pkg/redisx/pool.go b/pkg/redisx/pool.go index 93fc667c..bf6e4250 100644 --- a/pkg/redisx/pool.go +++ b/pkg/redisx/pool.go @@ -36,8 +36,8 @@ func NewClient(config etc.RedisPool) (*redis.Client, error) { // redis://user:password@host:port/db-number func newInstancePool(config etc.RedisPool) (*redis.Client, error) { - // TODO: Ask the Harbor team about why they use "idle_timeout_seconds" instead of "idle_timeout". - config.URL = strings.ReplaceAll(config.URL, "idle_timeout_seconds", "idle_timeout") + // `idle_timeout_seconds` is used in Harbor for backward compatibility. + config.URL = strings.ReplaceAll(config.URL, "idle_timeout_seconds=", "idle_timeout=") slog.Debug("Constructing connection pool for Redis", slog.String("url", config.URL)) options, err := redis.ParseURL(config.URL) diff --git a/pkg/scan/controller.go b/pkg/scan/controller.go index b44d7399..30a667ac 100644 --- a/pkg/scan/controller.go +++ b/pkg/scan/controller.go @@ -63,12 +63,13 @@ func (c *controller) scan(ctx context.Context, scanJobID string, req harbor.Scan return err } - scanReport, err := c.wrapper.Scan(trivy.ImageRef{Name: imageRef, Auth: auth, Insecure: insecureRegistry}) + scanReport, err := c.wrapper.Scan(trivy.ImageRef{Name: imageRef, Auth: auth, Insecure: insecureRegistry}, + trivy.ScanOption{Format: determineFormat(req.Scan.SBOMMediaType)}) if err != nil { return xerrors.Errorf("running trivy wrapper: %v", err) } - if err = c.store.UpdateReport(ctx, scanJobID, c.transformer.Transform(req.Artifact, scanReport)); err != nil { + if err = c.store.UpdateReport(ctx, scanJobID, c.transformer.Transform(req, scanReport)); err != nil { return xerrors.Errorf("saving scan report: %v", err) } @@ -113,3 +114,14 @@ func (c *controller) decodeBasicAuth(value string) (auth trivy.RegistryAuth, err } return } + +func determineFormat(sbomMediaType harbor.MediaType) trivy.Format { + switch sbomMediaType { + case harbor.MediaTypeSPDX: + return trivy.FormatSPDX + case harbor.MediaTypeCycloneDX: + return trivy.FormatCycloneDX + default: + return trivy.FormatJSON + } +} diff --git a/pkg/scan/controller_test.go b/pkg/scan/controller_test.go index a3e5616c..c184c578 100644 --- a/pkg/scan/controller_test.go +++ b/pkg/scan/controller_test.go @@ -13,13 +13,13 @@ import ( "golang.org/x/xerrors" ) -func TestContoller_Scan(t *testing.T) { +func TestController_Scan(t *testing.T) { ctx := context.Background() artifact := harbor.Artifact{ Repository: "library/mongo", Digest: "sha256:917f5b7f4bef1b35ee90f03033f33a81002511c1e0767fd44276d4bd9cd2fa8e", } - trivyReport := []trivy.Vulnerability{} + trivyReport := trivy.Report{} harborReport := harbor.ScanReport{} testCases := []struct { @@ -68,6 +68,7 @@ func TestContoller_Scan(t *testing.T) { Auth: trivy.BasicAuth{Username: "user", Password: "password"}, Insecure: false, }, + trivy.ScanOption{Format: "json"}, }, ReturnArgs: []interface{}{ trivyReport, @@ -77,7 +78,13 @@ func TestContoller_Scan(t *testing.T) { transformerExpectation: &mock.Expectation{ Method: "Transform", Args: []interface{}{ - artifact, + harbor.ScanRequest{ + Registry: harbor.Registry{ + URL: "https://core.harbor.domain", + Authorization: "Basic dXNlcjpwYXNzd29yZA==", // user:password + }, + Artifact: artifact, + }, trivyReport, }, ReturnArgs: []interface{}{ @@ -115,9 +122,10 @@ func TestContoller_Scan(t *testing.T) { Auth: trivy.BasicAuth{Username: "user", Password: "password"}, Insecure: false, }, + trivy.ScanOption{Format: "json"}, }, ReturnArgs: []interface{}{ - []trivy.Vulnerability{}, + trivy.Report{}, xerrors.New("out of memory"), }, }, diff --git a/pkg/scan/transformer.go b/pkg/scan/transformer.go index d61610aa..d3a2956e 100644 --- a/pkg/scan/transformer.go +++ b/pkg/scan/transformer.go @@ -1,6 +1,7 @@ package scan import ( + "github.com/samber/lo" "log/slog" "time" @@ -25,7 +26,7 @@ func (c *SystemClock) Now() time.Time { // Transformer wraps the Transform method. // Transform transforms Trivy's scan report into Harbor's packages vulnerabilities report. type Transformer interface { - Transform(artifact harbor.Artifact, source []trivy.Vulnerability) harbor.ScanReport + Transform(req harbor.ScanRequest, source trivy.Report) harbor.ScanReport } type transformer struct { @@ -39,11 +40,32 @@ func NewTransformer(clock Clock) Transformer { } } -func (t *transformer) Transform(artifact harbor.Artifact, source []trivy.Vulnerability) harbor.ScanReport { - vulnerabilities := make([]harbor.VulnerabilityItem, len(source)) +func (t *transformer) Transform(req harbor.ScanRequest, source trivy.Report) harbor.ScanReport { + report := harbor.ScanReport{ + GeneratedAt: t.clock.Now(), + Scanner: etc.GetScannerMetadata(), + Artifact: req.Artifact, + } + + switch req.Scan.SBOMMediaType { + case harbor.MediaTypeSPDX, harbor.MediaTypeCycloneDX: + report.MediaType = string(req.Scan.SBOMMediaType) + report.SBOM = source.SBOM + default: + report.Vulnerabilities = t.transformVulnerabilities(source.Vulnerabilities) + report.Severity = t.toHighestSeverity(report.Vulnerabilities) + } + + return report +} - for i, v := range source { - vulnerabilities[i] = harbor.VulnerabilityItem{ +func (t *transformer) transformVulnerabilities(source []trivy.Vulnerability) []harbor.VulnerabilityItem { + if len(source) == 0 { + return nil + } + + return lo.Map(source, func(v trivy.Vulnerability, _ int) harbor.VulnerabilityItem { + return harbor.VulnerabilityItem{ ID: v.VulnerabilityID, Pkg: v.PkgName, Version: v.InstalledVersion, @@ -55,15 +77,7 @@ func (t *transformer) Transform(artifact harbor.Artifact, source []trivy.Vulnera CweIDs: v.CweIDs, VendorAttributes: t.toVendorAttributes(v.CVSS), } - } - - return harbor.ScanReport{ - GeneratedAt: t.clock.Now(), - Scanner: etc.GetScannerMetadata(), - Artifact: artifact, - Severity: t.toHighestSeverity(vulnerabilities), - Vulnerabilities: vulnerabilities, - } + }) } func (t *transformer) toLinks(primaryURL string, references []string) []string { @@ -113,19 +127,9 @@ func (t *transformer) toVendorAttributes(info map[string]trivy.CVSSInfo) map[str return attributes } -func (t *transformer) toHighestSeverity(vlns []harbor.VulnerabilityItem) (highest harbor.Severity) { - highest = harbor.SevUnknown - - for _, vln := range vlns { - if vln.Severity > highest { - highest = vln.Severity - - if highest == harbor.SevCritical { - break - } - } - - } - - return +func (t *transformer) toHighestSeverity(vulns []harbor.VulnerabilityItem) harbor.Severity { + highest := lo.MaxBy(vulns, func(a, b harbor.VulnerabilityItem) bool { + return a.Severity > b.Severity + }) + return highest.Severity } diff --git a/pkg/scan/transformer_test.go b/pkg/scan/transformer_test.go index eab16c15..ca500a38 100644 --- a/pkg/scan/transformer_test.go +++ b/pkg/scan/transformer_test.go @@ -27,95 +27,100 @@ func TestTransformer_Transform(t *testing.T) { fixedTime: fixedTime, }) - hr := tf.Transform(harbor.Artifact{ - Repository: "library/mongo", - Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", - }, []trivy.Vulnerability{ - { - VulnerabilityID: "CVE-0000-0001", - PkgName: "PKG-01", - InstalledVersion: "PKG-01-VER", - FixedVersion: "PKG-01-FIX-VER", - Severity: "CRITICAL", - Description: "CVE-0000-0001.DESC", - References: []string{ - "http://cve.com?id=CVE-0000-0001", - "http://vendor.com?id=CVE-0000-0001", - }, - Layer: &trivy.Layer{ - Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb10", - }, - CVSS: map[string]trivy.CVSSInfo{ - "nvd": { - V2Vector: "AV:L/AC:M/Au:N/C:P/I:N/A:N", - V3Vector: "CVSS:3.1/AV:L/AC:H/PR:L/UI:N/S:U/C:H/I:N/A:N", - V2Score: float32Ptr(1.9), - V3Score: float32Ptr(4.7), - }, - }, - CweIDs: []string{ - "CWE-20", - "CWE-1289", - }, + hr := tf.Transform(harbor.ScanRequest{ + Scan: harbor.Scan{Type: harbor.ScanTypeVulnerability}, + Artifact: harbor.Artifact{ + Repository: "library/mongo", + Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", }, - { - VulnerabilityID: "CVE-0000-0002", - PkgName: "PKG-02", - InstalledVersion: "PKG-02-VER", - FixedVersion: "", - Severity: "HIGH", - Description: "CVE-0000-0002.DESC", - References: []string{ - "http://cve.com?id=CVE-0000-0002", - }, - Layer: &trivy.Layer{ - Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb11", + }, trivy.Report{ + Vulnerabilities: []trivy.Vulnerability{ + { + VulnerabilityID: "CVE-0000-0001", + PkgName: "PKG-01", + InstalledVersion: "PKG-01-VER", + FixedVersion: "PKG-01-FIX-VER", + Severity: "CRITICAL", + Description: "CVE-0000-0001.DESC", + References: []string{ + "http://cve.com?id=CVE-0000-0001", + "http://vendor.com?id=CVE-0000-0001", + }, + Layer: &trivy.Layer{ + Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb10", + }, + CVSS: map[string]trivy.CVSSInfo{ + "nvd": { + V2Vector: "AV:L/AC:M/Au:N/C:P/I:N/A:N", + V3Vector: "CVSS:3.1/AV:L/AC:H/PR:L/UI:N/S:U/C:H/I:N/A:N", + V2Score: float32Ptr(1.9), + V3Score: float32Ptr(4.7), + }, + }, + CweIDs: []string{ + "CWE-20", + "CWE-1289", + }, }, - }, - { - VulnerabilityID: "CVE-0000-0003", - PkgName: "PKG-03", - InstalledVersion: "PKG-03-VER", - FixedVersion: "PKG-03-FIX-VER", - Severity: "MEDIUM", - Description: "CVE-0000-0003.DESC", - References: []string{ - "http://cve.com?id=CVE-0000-0003", + { + VulnerabilityID: "CVE-0000-0002", + PkgName: "PKG-02", + InstalledVersion: "PKG-02-VER", + FixedVersion: "", + Severity: "HIGH", + Description: "CVE-0000-0002.DESC", + References: []string{ + "http://cve.com?id=CVE-0000-0002", + }, + Layer: &trivy.Layer{ + Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb11", + }, }, - Layer: &trivy.Layer{ - Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb12", + { + VulnerabilityID: "CVE-0000-0003", + PkgName: "PKG-03", + InstalledVersion: "PKG-03-VER", + FixedVersion: "PKG-03-FIX-VER", + Severity: "MEDIUM", + Description: "CVE-0000-0003.DESC", + References: []string{ + "http://cve.com?id=CVE-0000-0003", + }, + Layer: &trivy.Layer{ + Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb12", + }, + PrimaryURL: "https://avd.aquasec.com/nvd/cve-0000-0003/", }, - PrimaryURL: "https://avd.aquasec.com/nvd/cve-0000-0003/", - }, - { - VulnerabilityID: "CVE-0000-0004", - PkgName: "PKG-04", - InstalledVersion: "PKG-04-VER", - FixedVersion: "PKG-04-FIX-VER", - Severity: "LOW", - Description: "CVE-0000-0004.DESC", - References: []string{ - "http://cve.com?id=CVE-0000-0004", + { + VulnerabilityID: "CVE-0000-0004", + PkgName: "PKG-04", + InstalledVersion: "PKG-04-VER", + FixedVersion: "PKG-04-FIX-VER", + Severity: "LOW", + Description: "CVE-0000-0004.DESC", + References: []string{ + "http://cve.com?id=CVE-0000-0004", + }, + Layer: &trivy.Layer{ + Digest: "UNKNOWN", + }, }, - Layer: &trivy.Layer{ - Digest: "UNKNOWN", + { + VulnerabilityID: "CVE-0000-0005", + PkgName: "PKG-05", + InstalledVersion: "PKG-05-VER", + Severity: "~~~UNKNOWN~~~", + Layer: &trivy.Layer{ + Digest: "", + }, }, - }, - { - VulnerabilityID: "CVE-0000-0005", - PkgName: "PKG-05", - InstalledVersion: "PKG-05-VER", - Severity: "~~~UNKNOWN~~~", - Layer: &trivy.Layer{ - Digest: "", + { + VulnerabilityID: "CVE-0000-0006", + PkgName: "PKG-06", + InstalledVersion: "PKG-06-VER", + Severity: "UNKNOWN", }, }, - { - VulnerabilityID: "CVE-0000-0006", - PkgName: "PKG-06", - InstalledVersion: "PKG-06-VER", - Severity: "UNKNOWN", - }, }) assert.Equal(t, harbor.ScanReport{ GeneratedAt: fixedTime, diff --git a/pkg/trivy/model.go b/pkg/trivy/model.go index 0d930086..9e5a6d00 100644 --- a/pkg/trivy/model.go +++ b/pkg/trivy/model.go @@ -38,6 +38,11 @@ type CVSSInfo struct { V3Score *float32 `json:"V3Score,omitempty"` } +type Report struct { + SBOM any + Vulnerabilities []Vulnerability +} + type Vulnerability struct { VulnerabilityID string `json:"VulnerabilityID"` PkgName string `json:"PkgName"` diff --git a/pkg/trivy/wrapper.go b/pkg/trivy/wrapper.go index 083eaabb..6a100cdb 100644 --- a/pkg/trivy/wrapper.go +++ b/pkg/trivy/wrapper.go @@ -8,12 +8,20 @@ import ( "os/exec" "strings" + "golang.org/x/xerrors" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/etc" "github.com/aquasecurity/harbor-scanner-trivy/pkg/ext" ) +type Format string + const ( trivyCmd = "trivy" + + FormatJSON Format = "json" + FormatSPDX Format = "spdx-json" + FormatCycloneDX Format = "cyclonedx" ) type ImageRef struct { @@ -22,6 +30,10 @@ type ImageRef struct { Insecure bool } +type ScanOption struct { + Format Format +} + // RegistryAuth wraps registry credentials. type RegistryAuth interface { } @@ -39,7 +51,7 @@ type BearerAuth struct { } type Wrapper interface { - Scan(imageRef ImageRef) ([]Vulnerability, error) + Scan(imageRef ImageRef, opt ScanOption) (Report, error) GetVersion() (VersionInfo, error) } @@ -55,13 +67,13 @@ func NewWrapper(config etc.Trivy, ambassador ext.Ambassador) Wrapper { } } -func (w *wrapper) Scan(imageRef ImageRef) ([]Vulnerability, error) { +func (w *wrapper) Scan(imageRef ImageRef, opt ScanOption) (Report, error) { logger := slog.With(slog.String("image_ref", imageRef.Name)) logger.Debug("Started scanning") reportFile, err := w.ambassador.TempFile(w.config.ReportsDir, "scan_report_*.json") if err != nil { - return nil, err + return Report{}, err } logger.Debug("Saving scan report to tmp file", slog.String("path", reportFile.Name())) defer func() { @@ -71,9 +83,9 @@ func (w *wrapper) Scan(imageRef ImageRef) ([]Vulnerability, error) { } }() - cmd, err := w.prepareScanCmd(imageRef, reportFile.Name()) + cmd, err := w.prepareScanCmd(imageRef, reportFile.Name(), opt) if err != nil { - return nil, err + return Report{}, err } logger.Debug("Exec command with args", slog.String("path", cmd.Path), @@ -85,7 +97,7 @@ func (w *wrapper) Scan(imageRef ImageRef) ([]Vulnerability, error) { slog.String("exit_code", fmt.Sprintf("%d", cmd.ProcessState.ExitCode())), slog.String("std_out", string(stdout)), ) - return nil, fmt.Errorf("running trivy: %v: %v", err, string(stdout)) + return Report{}, fmt.Errorf("running trivy: %v: %v", err, string(stdout)) } logger.Debug("Running trivy finished", @@ -93,17 +105,27 @@ func (w *wrapper) Scan(imageRef ImageRef) ([]Vulnerability, error) { slog.String("std_out", string(stdout)), ) - return w.parseVulnerabilities(reportFile) + return w.parseReport(opt.Format, reportFile) } -func (w *wrapper) parseVulnerabilities(reportFile io.Reader) ([]Vulnerability, error) { +func (w *wrapper) parseReport(format Format, reportFile io.Reader) (Report, error) { + switch format { + case FormatJSON: + return w.parseJSONReport(reportFile) + case FormatSPDX, FormatCycloneDX: + return w.parseSBOM(reportFile) + } + return Report{}, fmt.Errorf("unsupported format %s", format) +} + +func (w *wrapper) parseJSONReport(reportFile io.Reader) (Report, error) { var scanReport ScanReport if err := json.NewDecoder(reportFile).Decode(&scanReport); err != nil { - return nil, fmt.Errorf("decoding scan report from file: %w", err) + return Report{}, xerrors.Errorf("report json decode error: %w", err) } if scanReport.SchemaVersion != SchemaVersion { - return nil, fmt.Errorf("unsupported schema %d, expected %d", scanReport.SchemaVersion, SchemaVersion) + return Report{}, xerrors.Errorf("unsupported schema %d, expected %d", scanReport.SchemaVersion, SchemaVersion) } var vulnerabilities []Vulnerability @@ -112,16 +134,26 @@ func (w *wrapper) parseVulnerabilities(reportFile io.Reader) ([]Vulnerability, e vulnerabilities = append(vulnerabilities, scanResult.Vulnerabilities...) } - return vulnerabilities, nil + return Report{ + Vulnerabilities: vulnerabilities, + }, nil } -func (w *wrapper) prepareScanCmd(imageRef ImageRef, outputFile string) (*exec.Cmd, error) { +func (w *wrapper) parseSBOM(reportFile io.Reader) (Report, error) { + var doc any + if err := json.NewDecoder(reportFile).Decode(&doc); err != nil { + return Report{}, xerrors.Errorf("sbom json decode error: %w", err) + } + return Report{SBOM: doc}, nil +} + +func (w *wrapper) prepareScanCmd(imageRef ImageRef, outputFile string, opt ScanOption) (*exec.Cmd, error) { args := []string{ "--no-progress", "--severity", w.config.Severity, "--vuln-type", w.config.VulnType, "--scanners", w.config.SecurityChecks, - "--format", "json", + "--format", string(opt.Format), "--output", outputFile, imageRef.Name, } diff --git a/pkg/trivy/wrapper_mock.go b/pkg/trivy/wrapper_mock.go index b92309ff..19e53e58 100644 --- a/pkg/trivy/wrapper_mock.go +++ b/pkg/trivy/wrapper_mock.go @@ -17,7 +17,7 @@ func NewMockWrapper() *MockWrapper { return &MockWrapper{} } -func (w *MockWrapper) Scan(imageRef ImageRef) ([]Vulnerability, error) { - args := w.Called(imageRef) - return args.Get(0).([]Vulnerability), args.Error(1) +func (w *MockWrapper) Scan(imageRef ImageRef, opt ScanOption) (Report, error) { + args := w.Called(imageRef, opt) + return args.Get(0).(Report), args.Error(1) } diff --git a/pkg/trivy/wrapper_test.go b/pkg/trivy/wrapper_test.go index 7dd9b5ed..6a6f559e 100644 --- a/pkg/trivy/wrapper_test.go +++ b/pkg/trivy/wrapper_test.go @@ -47,29 +47,31 @@ var ( } ] }` - expectedReport = []Vulnerability{ - { - VulnerabilityID: "CVE-2018-6543", - PkgName: "binutils", - InstalledVersion: "2.30-r1", - FixedVersion: "2.30-r2", - Severity: "MEDIUM", - References: []string{ - "https://cve.mitre.org/cgi-bin/cvename.cgi?name=CVE-2018-6543", - }, - Layer: &Layer{Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb10"}, - CVSS: map[string]CVSSInfo{ - "nvd": { - V2Vector: "AV:L/AC:M/Au:N/C:P/I:N/A:N", - V3Vector: "CVSS:3.1/AV:L/AC:H/PR:L/UI:N/S:U/C:H/I:N/A:N", - V2Score: float32Ptr(1.9), - V3Score: float32Ptr(4.7), + expectedReport = Report{ + Vulnerabilities: []Vulnerability{ + { + VulnerabilityID: "CVE-2018-6543", + PkgName: "binutils", + InstalledVersion: "2.30-r1", + FixedVersion: "2.30-r2", + Severity: "MEDIUM", + References: []string{ + "https://cve.mitre.org/cgi-bin/cvename.cgi?name=CVE-2018-6543", }, - "redhat": { - V2Vector: "", - V3Vector: "CVSS:3.0/AV:L/AC:L/PR:L/UI:N/S:U/C:H/I:N/A:N", - V2Score: nil, - V3Score: float32Ptr(5.5), + Layer: &Layer{Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb10"}, + CVSS: map[string]CVSSInfo{ + "nvd": { + V2Vector: "AV:L/AC:M/Au:N/C:P/I:N/A:N", + V3Vector: "CVSS:3.1/AV:L/AC:H/PR:L/UI:N/S:U/C:H/I:N/A:N", + V2Score: float32Ptr(1.9), + V3Score: float32Ptr(4.7), + }, + "redhat": { + V2Vector: "", + V3Vector: "CVSS:3.0/AV:L/AC:L/PR:L/UI:N/S:U/C:H/I:N/A:N", + V2Score: nil, + V3Score: float32Ptr(5.5), + }, }, }, }, @@ -154,7 +156,7 @@ func TestWrapper_Scan(t *testing.T) { Args: expectedCmdArgs}, ).Return([]byte{}, nil) - report, err := NewWrapper(config, ambassador).Scan(imageRef) + report, err := NewWrapper(config, ambassador).Scan(imageRef, ScanOption{Format: FormatJSON}) require.NoError(t, err) require.Equal(t, expectedReport, report) diff --git a/test/integration/api/rest_api_test.go b/test/integration/api/rest_api_test.go index 02e87f86..7d817205 100644 --- a/test/integration/api/rest_api_test.go +++ b/test/integration/api/rest_api_test.go @@ -58,6 +58,9 @@ func TestRestApi(t *testing.T) { t.Run("POST /api/v1/scan", func(t *testing.T) { // given enqueuer.On("Enqueue", mock.Anything, harbor.ScanRequest{ + Scan: harbor.Scan{ + Type: harbor.ScanTypeVulnerability, + }, Registry: harbor.Registry{ URL: "https://core.harbor.domain", Authorization: "Bearer JWTTOKENGOESHERE", From fb83feba969e2a2c2c835b8518b9a2f5e3f6f641 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Wed, 1 Nov 2023 11:12:56 +0900 Subject: [PATCH 06/26] fix: move header to body Signed-off-by: knqyf263 --- pkg/harbor/model.go | 8 ++++++-- pkg/http/api/v1/handler.go | 14 ++++---------- pkg/scan/controller.go | 2 +- pkg/scan/transformer.go | 4 ++-- 4 files changed, 13 insertions(+), 15 deletions(-) diff --git a/pkg/harbor/model.go b/pkg/harbor/model.go index 2d5562e5..35bdd276 100644 --- a/pkg/harbor/model.go +++ b/pkg/harbor/model.go @@ -74,7 +74,11 @@ const ( var SupportedSBOMMediaTypes = []MediaType{MediaTypeSPDX, MediaTypeCycloneDX} type Scan struct { - Type ScanType `json:"type,omitempty"` + Type ScanType `json:"type,omitempty"` + Parameters ScanParameters `json:"parameters,omitempty"` +} + +type ScanParameters struct { SBOMMediaType MediaType `json:"sbom_media_type,omitempty"` } @@ -90,7 +94,7 @@ type Artifact struct { } type ScanRequest struct { - Scan Scan `json:"scan"` // From HTTP header + Scan Scan `json:"scan"` Registry Registry `json:"registry"` Artifact Artifact `json:"artifact"` } diff --git a/pkg/http/api/v1/handler.go b/pkg/http/api/v1/handler.go index 420f201b..8254934b 100644 --- a/pkg/http/api/v1/handler.go +++ b/pkg/http/api/v1/handler.go @@ -87,15 +87,9 @@ func (h *requestHandler) AcceptScanRequest(res http.ResponseWriter, req *http.Re return } - // Extract SBOM related properties from HTTP header - scanType := getHeader(req.Header, "X-Scan-Type", harbor.ScanTypeVulnerability) - var sbomMediaType harbor.MediaType - if scanType == harbor.ScanTypeSBOM { - sbomMediaType = getHeader(req.Header, "X-Scan-Sbom-MediaType", harbor.MediaTypeCycloneDX) - } - scanRequest.Scan = harbor.Scan{ - Type: scanType, - SBOMMediaType: sbomMediaType, + // Set the default value for scan type if it is not specified. + if scanRequest.Scan.Type == "" { + scanRequest.Scan.Type = harbor.ScanTypeVulnerability } if validationError := h.ValidateScanRequest(scanRequest); validationError != nil { @@ -128,7 +122,7 @@ func (h *requestHandler) ValidateScanRequest(req harbor.ScanRequest) *harbor.Err } if req.Scan.Type == harbor.ScanTypeSBOM && - !slices.Contains(harbor.SupportedSBOMMediaTypes, req.Scan.SBOMMediaType) { + !slices.Contains(harbor.SupportedSBOMMediaTypes, req.Scan.Parameters.SBOMMediaType) { return &harbor.Error{ HTTPCode: http.StatusUnprocessableEntity, Message: "invalid SBOM media type", diff --git a/pkg/scan/controller.go b/pkg/scan/controller.go index 30a667ac..ee88249c 100644 --- a/pkg/scan/controller.go +++ b/pkg/scan/controller.go @@ -64,7 +64,7 @@ func (c *controller) scan(ctx context.Context, scanJobID string, req harbor.Scan } scanReport, err := c.wrapper.Scan(trivy.ImageRef{Name: imageRef, Auth: auth, Insecure: insecureRegistry}, - trivy.ScanOption{Format: determineFormat(req.Scan.SBOMMediaType)}) + trivy.ScanOption{Format: determineFormat(req.Scan.Parameters.SBOMMediaType)}) if err != nil { return xerrors.Errorf("running trivy wrapper: %v", err) } diff --git a/pkg/scan/transformer.go b/pkg/scan/transformer.go index d3a2956e..0b4004e8 100644 --- a/pkg/scan/transformer.go +++ b/pkg/scan/transformer.go @@ -47,9 +47,9 @@ func (t *transformer) Transform(req harbor.ScanRequest, source trivy.Report) har Artifact: req.Artifact, } - switch req.Scan.SBOMMediaType { + switch req.Scan.Parameters.SBOMMediaType { case harbor.MediaTypeSPDX, harbor.MediaTypeCycloneDX: - report.MediaType = string(req.Scan.SBOMMediaType) + report.MediaType = string(req.Scan.Parameters.SBOMMediaType) report.SBOM = source.SBOM default: report.Vulnerabilities = t.transformVulnerabilities(source.Vulnerabilities) From 20a84b7445e7b0871675397dd0a85d5d5fa45edc Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Wed, 1 Nov 2023 13:42:11 +0900 Subject: [PATCH 07/26] feat: scan SBOM [skip ci] Signed-off-by: knqyf263 --- go.mod | 15 +++- go.sum | 32 ++++++-- pkg/harbor/model.go | 4 +- pkg/scan/controller.go | 4 +- pkg/scan/controller_test.go | 12 +-- pkg/trivy/target.go | 141 ++++++++++++++++++++++++++++++++++++ pkg/trivy/wrapper.go | 65 +++++++++-------- pkg/trivy/wrapper_test.go | 6 +- 8 files changed, 227 insertions(+), 52 deletions(-) create mode 100644 pkg/trivy/target.go diff --git a/go.mod b/go.mod index 72edc3da..aa5101b6 100644 --- a/go.mod +++ b/go.mod @@ -6,6 +6,7 @@ require ( github.com/caarlos0/env/v6 v6.10.1 github.com/docker/docker v24.0.6+incompatible github.com/docker/go-connections v0.4.0 + github.com/google/go-containerregistry v0.14.0 github.com/gorilla/mux v1.8.0 github.com/opencontainers/go-digest v1.0.0 github.com/prometheus/client_golang v1.17.0 @@ -26,19 +27,23 @@ require ( github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/containerd/containerd v1.7.6 // indirect + github.com/containerd/stargz-snapshotter/estargz v0.14.3 // indirect github.com/cpuguy83/dockercfg v0.3.1 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f // indirect + github.com/docker/cli v24.0.0+incompatible // indirect github.com/docker/distribution v2.8.2+incompatible // indirect + github.com/docker/docker-credential-helpers v0.7.0 // indirect github.com/docker/go-units v0.5.0 // indirect github.com/go-ole/go-ole v1.2.6 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/protobuf v1.5.3 // indirect github.com/google/uuid v1.3.1 // indirect - github.com/klauspost/compress v1.16.0 // indirect + github.com/klauspost/compress v1.16.5 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/magiconair/properties v1.8.7 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect + github.com/mitchellh/go-homedir v1.1.0 // indirect github.com/moby/patternmatcher v0.5.0 // indirect github.com/moby/sys/sequential v0.5.0 // indirect github.com/moby/term v0.5.0 // indirect @@ -57,13 +62,17 @@ require ( github.com/stretchr/objx v0.5.0 // indirect github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect + github.com/vbatts/tar-split v0.11.3 // indirect github.com/yusufpapurcu/wmi v1.2.3 // indirect golang.org/x/exp v0.0.0-20230510235704-dd950f8aeaea // indirect - golang.org/x/mod v0.9.0 // indirect + golang.org/x/mod v0.10.0 // indirect + golang.org/x/sync v0.3.0 // indirect golang.org/x/sys v0.13.0 // indirect - golang.org/x/tools v0.7.0 // indirect + golang.org/x/tools v0.9.1 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19 // indirect google.golang.org/grpc v1.57.1 // indirect google.golang.org/protobuf v1.31.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) + +replace github.com/google/go-containerregistry => github.com/knqyf263/go-containerregistry v0.16.2-0.20231101014841-fd95d0f749dd diff --git a/go.sum b/go.sum index 2bca1e36..078c15e5 100644 --- a/go.sum +++ b/go.sum @@ -5,6 +5,7 @@ github.com/AdaLogics/go-fuzz-headers v0.0.0-20230811130428-ced1acdcaa24/go.mod h github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 h1:UQHMgLO+TxOElx5B5HZ4hJQsoJ/PvUvKRhJHDQXO8P8= github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1/go.mod h1:xomTg63KZ2rFqZQzSB4Vz2SUXa1BpHTVz9L5PTmPC4E= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/BurntSushi/toml v1.2.1/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= github.com/Microsoft/hcsshim v0.11.0 h1:7EFNIY4igHEXUdj1zXgAyU3fLc7QfOKHbkldRVTBdiM= @@ -26,10 +27,13 @@ github.com/cilium/ebpf v0.7.0/go.mod h1:/oI2+1shJiTGAMgl6/RgJr36Eo1jzrRcAWbcXO2u github.com/containerd/console v1.0.3/go.mod h1:7LqA/THxQ86k76b8c/EMSiaJ3h1eZkMkXar0TQ1gf3U= github.com/containerd/containerd v1.7.6 h1:oNAVsnhPoy4BTPQivLgTzI9Oleml9l/+eYIDYXRCYo8= github.com/containerd/containerd v1.7.6/go.mod h1:SY6lrkkuJT40BVNO37tlYTSnKJnP5AXBc0fhx0q+TJ4= +github.com/containerd/stargz-snapshotter/estargz v0.14.3 h1:OqlDCK3ZVUO6C3B/5FSkDwbkEETK84kQgEeFwDC+62k= +github.com/containerd/stargz-snapshotter/estargz v0.14.3/go.mod h1:KY//uOCIkSuNAHhJogcZtrNHdKrA99/FCCRjE3HD36o= github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cpuguy83/dockercfg v0.3.1 h1:/FpZ+JaygUR/lZP2NlFI2DVfrOEMAIKP5wWEJdoYe9E= github.com/cpuguy83/dockercfg v0.3.1/go.mod h1:sugsbF4//dDlL/i+S+rtpIWp+5h0BHJHfjj5/jFyUJc= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= +github.com/cpuguy83/go-md2man/v2 v2.0.2/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.18 h1:n56/Zwd5o6whRC5PMGretI4IdRLlmBXYNjScPaBgsbY= github.com/creack/pty v1.1.18/go.mod h1:MOBLtS5ELjhRRrroQr9kyvTxUAFNvYEK993ew/Vr4O4= github.com/cyphar/filepath-securejoin v0.2.3/go.mod h1:aPGpWjXOXUn2NCNjFvBE6aRxGGx79pTxQpKOJNYHHl4= @@ -38,10 +42,14 @@ github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f h1:lO4WD4F/rVNCu3HqELle0jiPLLBs70cWOduZpkS1E78= github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f/go.mod h1:cuUVRXasLTGF7a8hSLbxyZXjz+1KgoB3wDUb6vlszIc= +github.com/docker/cli v24.0.0+incompatible h1:0+1VshNwBQzQAx9lOl+OYCTCEAD8fKs/qeXMx3O0wqM= +github.com/docker/cli v24.0.0+incompatible/go.mod h1:JLrzqnKDaYBop7H2jaqPtU4hHvMKP+vjCwu2uszcLI8= github.com/docker/distribution v2.8.2+incompatible h1:T3de5rq0dB1j30rp0sA2rER+m322EBzniBPB6ZIzuh8= github.com/docker/distribution v2.8.2+incompatible/go.mod h1:J2gT2udsDAN96Uj4KfcMRqY0/ypR+oyYUYmja8H+y+w= github.com/docker/docker v24.0.6+incompatible h1:hceabKCtUgDqPu+qm0NgsaXf28Ljf4/pWFL7xjWWDgE= github.com/docker/docker v24.0.6+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= +github.com/docker/docker-credential-helpers v0.7.0 h1:xtCHsjxogADNZcdv1pKUHXryefjlVRqWqIhk/uXJp0A= +github.com/docker/docker-credential-helpers v0.7.0/go.mod h1:rETQfLdHNT3foU5kuNkFR1R1V12OJRRO5lzt2D1b5X0= github.com/docker/go-connections v0.4.0 h1:El9xVISelRB7BuFusrZozjnkIM5YnzCViNKohAFqRJQ= github.com/docker/go-connections v0.4.0/go.mod h1:Gbd7IOopHjR8Iph03tsViu4nIes5XhDvyHbTtUxmeec= github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= @@ -69,8 +77,10 @@ github.com/gorilla/mux v1.8.0 h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI= github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/klauspost/compress v1.16.0 h1:iULayQNOReoYUe+1qtKOqw9CwJv3aNQu8ivo7lw1HU4= -github.com/klauspost/compress v1.16.0/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +github.com/klauspost/compress v1.16.5 h1:IFV2oUNUzZaz+XyusxpLzpzS8Pt5rh0Z16For/djlyI= +github.com/klauspost/compress v1.16.5/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +github.com/knqyf263/go-containerregistry v0.16.2-0.20231101014841-fd95d0f749dd h1:SpYc4OYxtOjd/MbBOcdwFfj3foqy8NV9V77KBAOC0bA= +github.com/knqyf263/go-containerregistry v0.16.2-0.20231101014841-fd95d0f749dd/go.mod h1:u0qB2l7mvtWVR5kNcbFIhFY1hLbf8eeGapA+vbFDCtQ= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= @@ -83,6 +93,8 @@ github.com/magiconair/properties v1.8.7 h1:IeQXZAiQcpL9mgcAe1Nu6cX9LLw6ExEHKjN0V github.com/magiconair/properties v1.8.7/go.mod h1:Dhd985XPs7jluiymwWYZ0G4Z61jb3vdS329zhj2hYo0= github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= +github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y= +github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= github.com/moby/patternmatcher v0.5.0 h1:YCZgJOeULcxLw1Q+sVR636pmS7sPEn1Qo2iAN6M7DBo= github.com/moby/patternmatcher v0.5.0/go.mod h1:hDPoyOpDY7OrrMDLaYoY3hf52gNCR/YOUYxkhApJIxc= github.com/moby/sys/mountinfo v0.5.0/go.mod h1:3bMD3Rg+zkqx8MRYPi7Pyb0Ie97QEBmdxbhnCLlSvSU= @@ -120,6 +132,7 @@ github.com/redis/go-redis/v9 v9.2.1/go.mod h1:hdY0cQFCN4fnSYT6TkisLufl/4W5UIXyv0 github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/samber/lo v1.38.1 h1:j2XEAqXKb09Am4ebOg31SpvzUTTs6EN3VfgeLUhPdXM= github.com/samber/lo v1.38.1/go.mod h1:+m/ZKRl6ClXCE2Lgf3MsQlWfh4bn1bz6CXEOxnEXnEA= github.com/seccomp/libseccomp-golang v0.9.2-0.20220502022130-f33da4d89646/go.mod h1:JA8cRccbGaA1s33RQf7Y1+q9gHmZX1yB/z9WDN1C6fg= @@ -131,6 +144,7 @@ github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= github.com/shoenig/test v0.6.4/go.mod h1:byHiCGXqrVaflBLAMq/srcZIHynQPQgeyvkvXnjqq0k= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= +github.com/sirupsen/logrus v1.9.0/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= @@ -141,6 +155,7 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/syndtr/gocapability v0.0.0-20200815063812-42c35b437635/go.mod h1:hkRG7XYTFWNJGYcbNJQlaLq0fg1yr4J4t/NcTQtrfww= @@ -151,6 +166,9 @@ github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0h github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= github.com/tklauser/numcpus v0.6.1/go.mod h1:1XfjsgE2zo8GVw7POkMbHENHzVg3GzmoZ9fESEdAacY= github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= +github.com/urfave/cli v1.22.12/go.mod h1:sSBEIC79qR6OvcmsD4U3KABeOTxDqQtdDnaFuUN30b8= +github.com/vbatts/tar-split v0.11.3 h1:hLFqsOLQ1SsppQNTMpkpPXClLDfC2A3Zgy9OUU+RVck= +github.com/vbatts/tar-split v0.11.3/go.mod h1:9QlHN18E+fEH7RdG+QAJJcuya3rqT7eXSTY7wGrAokY= github.com/vishvananda/netlink v1.1.0/go.mod h1:cTgwzPIzzgDAYoQrMm0EdrjRUBkTqKYppBueQtXaqoE= github.com/vishvananda/netns v0.0.0-20191106174202-0a2b9b5464df/go.mod h1:JP3t17pCcGlemwknint6hfoeCVQrEMVwxRLRjXpq+BU= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= @@ -164,8 +182,8 @@ golang.org/x/exp v0.0.0-20230510235704-dd950f8aeaea h1:vLCWI/yYrdEHyN2JzIzPO3aaQ golang.org/x/exp v0.0.0-20230510235704-dd950f8aeaea/go.mod h1:V1LtkGg67GoY2N1AnLN78QLrzxkLyJw7RJb1gzOOz9w= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.9.0 h1:KENHtAZL2y3NLMYZeHY9DW8HW8V+kQyJsY/V9JlKvCs= -golang.org/x/mod v0.9.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.10.0 h1:lFO9qtOdlre5W1jxS3r/4szv2/6iXxScdzjoBMXNhYk= +golang.org/x/mod v0.10.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -194,6 +212,7 @@ golang.org/x/sys v0.0.0-20210906170528-6f6e22806c34/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211116061358-0a5406a5449c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220906165534-d0df966e6959/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE= @@ -209,8 +228,8 @@ golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGm golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.7.0 h1:W4OVu8VVOaIO0yzWMNdepAulS7YfoS3Zabrm8DOXXU4= -golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= +golang.org/x/tools v0.9.1 h1:8WMNJAz3zrtPmnYC7ISf5dEn3MT0gY7jBJfw27yrrLo= +golang.org/x/tools v0.9.1/go.mod h1:owI94Op576fPu3cIGQeHs3joujW/2Oc6MtlxbF5dfNc= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -229,6 +248,7 @@ gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8 gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= diff --git a/pkg/harbor/model.go b/pkg/harbor/model.go index 35bdd276..e128a437 100644 --- a/pkg/harbor/model.go +++ b/pkg/harbor/model.go @@ -101,7 +101,7 @@ type ScanRequest struct { // GetImageRef returns Docker image reference for this ScanRequest. // Example: core.harbor.domain/scanners/mysql@sha256:3b00a364fb74246ca119d16111eb62f7302b2ff66d51e373c2bb209f8a1f3b9e -func (c ScanRequest) GetImageRef() (imageRef string, insecureRegistry bool, err error) { +func (c ScanRequest) GetImageRef() (imageRef string, nonSSL bool, err error) { registryURL, err := url.Parse(c.Registry.URL) if err != nil { err = fmt.Errorf("parsing registry URL: %w", err) @@ -117,7 +117,7 @@ func (c ScanRequest) GetImageRef() (imageRef string, insecureRegistry bool, err } imageRef = fmt.Sprintf("%s:%s/%s@%s", registryURL.Hostname(), port, c.Artifact.Repository, c.Artifact.Digest) - insecureRegistry = "http" == registryURL.Scheme + nonSSL = "http" == registryURL.Scheme return } diff --git a/pkg/scan/controller.go b/pkg/scan/controller.go index ee88249c..ba726264 100644 --- a/pkg/scan/controller.go +++ b/pkg/scan/controller.go @@ -53,7 +53,7 @@ func (c *controller) scan(ctx context.Context, scanJobID string, req harbor.Scan return xerrors.Errorf("updating scan job status: %v", err) } - imageRef, insecureRegistry, err := req.GetImageRef() + imageRef, nonSSL, err := req.GetImageRef() if err != nil { return err } @@ -63,7 +63,7 @@ func (c *controller) scan(ctx context.Context, scanJobID string, req harbor.Scan return err } - scanReport, err := c.wrapper.Scan(trivy.ImageRef{Name: imageRef, Auth: auth, Insecure: insecureRegistry}, + scanReport, err := c.wrapper.Scan(trivy.ImageRef{Name: imageRef, Auth: auth, NonSSL: nonSSL}, trivy.ScanOption{Format: determineFormat(req.Scan.Parameters.SBOMMediaType)}) if err != nil { return xerrors.Errorf("running trivy wrapper: %v", err) diff --git a/pkg/scan/controller_test.go b/pkg/scan/controller_test.go index c184c578..b93abf7f 100644 --- a/pkg/scan/controller_test.go +++ b/pkg/scan/controller_test.go @@ -64,9 +64,9 @@ func TestController_Scan(t *testing.T) { Method: "Scan", Args: []interface{}{ trivy.ImageRef{ - Name: "core.harbor.domain:443/library/mongo@sha256:917f5b7f4bef1b35ee90f03033f33a81002511c1e0767fd44276d4bd9cd2fa8e", - Auth: trivy.BasicAuth{Username: "user", Password: "password"}, - Insecure: false, + Name: "core.harbor.domain:443/library/mongo@sha256:917f5b7f4bef1b35ee90f03033f33a81002511c1e0767fd44276d4bd9cd2fa8e", + Auth: trivy.BasicAuth{Username: "user", Password: "password"}, + NonSSL: false, }, trivy.ScanOption{Format: "json"}, }, @@ -118,9 +118,9 @@ func TestController_Scan(t *testing.T) { Method: "Scan", Args: []interface{}{ trivy.ImageRef{ - Name: "core.harbor.domain:443/library/mongo@sha256:917f5b7f4bef1b35ee90f03033f33a81002511c1e0767fd44276d4bd9cd2fa8e", - Auth: trivy.BasicAuth{Username: "user", Password: "password"}, - Insecure: false, + Name: "core.harbor.domain:443/library/mongo@sha256:917f5b7f4bef1b35ee90f03033f33a81002511c1e0767fd44276d4bd9cd2fa8e", + Auth: trivy.BasicAuth{Username: "user", Password: "password"}, + NonSSL: false, }, trivy.ScanOption{Format: "json"}, }, diff --git a/pkg/trivy/target.go b/pkg/trivy/target.go new file mode 100644 index 00000000..dd0fbcc6 --- /dev/null +++ b/pkg/trivy/target.go @@ -0,0 +1,141 @@ +package trivy + +import ( + "crypto/tls" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/etc" + "github.com/google/go-containerregistry/pkg/authn" + "github.com/google/go-containerregistry/pkg/name" + v1 "github.com/google/go-containerregistry/pkg/v1" + "github.com/google/go-containerregistry/pkg/v1/remote" + "golang.org/x/xerrors" + "io" + "log/slog" + "net/http" + "os" +) + +type Target string + +const ( + TargetSBOM Target = "sbom" + TargetImage Target = "image" +) + +type ScanTarget struct { + v1.Image + + ref ImageRef + Type Target +} + +func newTarget(imageRef ImageRef, config etc.Trivy) (ScanTarget, error) { + var nameOpts []name.Option + slog.Info("newTarget", + slog.Bool("nonssl", imageRef.NonSSL), + slog.Bool("insecure", config.Insecure), + ) + if imageRef.NonSSL { + nameOpts = append(nameOpts, name.Insecure) + } + ref, err := name.ParseReference(imageRef.Name, nameOpts...) + if err != nil { + return ScanTarget{}, xerrors.Errorf("parsing image reference: %w", err) + } + + authOpt := remote.WithAuthFromKeychain(authn.DefaultKeychain) + switch a := imageRef.Auth.(type) { + case NoAuth: + case BasicAuth: + authOpt = remote.WithAuth(&authn.Basic{ + Username: a.Username, + Password: a.Password, + }) + case BearerAuth: + authOpt = remote.WithAuth(&authn.Bearer{ + Token: a.Token, + }) + default: + return ScanTarget{}, xerrors.Errorf("invalid auth type %T", a) + } + + tr := http.DefaultTransport.(*http.Transport).Clone() + tr.TLSClientConfig = &tls.Config{InsecureSkipVerify: config.Insecure} + trOpt := remote.WithTransport(tr) + + img, err := remote.Image(ref, authOpt, trOpt) + if err != nil { + return ScanTarget{}, xerrors.Errorf("fetching image: %w", err) + } + + target := ScanTarget{ + Image: img, + ref: imageRef, + } + + m, err := target.Manifest() + if err != nil { + return ScanTarget{}, xerrors.Errorf("getting image manifest: %w", err) + } + + switch m.ArtifactType { + case "application/vnd.goharbor.harbor.sbom.v1": + target.Type = TargetSBOM + default: + target.Type = TargetImage + } + + return target, nil +} + +func (t ScanTarget) Name() (string, error) { + switch t.Type { + case TargetSBOM: + return t.downloadSBOM() + case TargetImage: + return t.ref.Name, nil + default: + return "", xerrors.Errorf("invalid target type %s", t.Type) + } +} + +func (t ScanTarget) NonSSL() bool { + return t.ref.NonSSL +} + +func (t ScanTarget) Auth() RegistryAuth { + switch t.Type { + case TargetSBOM: + return NoAuth{} + case TargetImage: + return t.ref.Auth + default: + return NoAuth{} + } +} + +func (t ScanTarget) downloadSBOM() (string, error) { + layers, err := t.Layers() + if err != nil { + return "", xerrors.Errorf("get image layers: %w", err) + } else if len(layers) != 1 { + return "", xerrors.Errorf("invalid number of layers: %d", len(layers)) + } + + r, err := layers[0].Uncompressed() + if err != nil { + return "", xerrors.Errorf("uncompress layer: %w", err) + } + defer r.Close() + + sbomFile, err := os.CreateTemp("", "sbom_*.json") + if err != nil { + return "", xerrors.Errorf("create temp file: %w", err) + } + defer sbomFile.Close() + + if _, err = io.Copy(sbomFile, r); err != nil { + return "", xerrors.Errorf("copy layer to temp file: %w", err) + } + + return sbomFile.Name(), nil +} diff --git a/pkg/trivy/wrapper.go b/pkg/trivy/wrapper.go index 6a100cdb..ab51c814 100644 --- a/pkg/trivy/wrapper.go +++ b/pkg/trivy/wrapper.go @@ -25,9 +25,9 @@ const ( ) type ImageRef struct { - Name string - Auth RegistryAuth - Insecure bool + Name string + Auth RegistryAuth + NonSSL bool } type ScanOption struct { @@ -71,6 +71,11 @@ func (w *wrapper) Scan(imageRef ImageRef, opt ScanOption) (Report, error) { logger := slog.With(slog.String("image_ref", imageRef.Name)) logger.Debug("Started scanning") + target, err := newTarget(imageRef, w.config) + if err != nil { + return Report{}, xerrors.Errorf("creating scan target: %w", err) + } + reportFile, err := w.ambassador.TempFile(w.config.ReportsDir, "scan_report_*.json") if err != nil { return Report{}, err @@ -83,7 +88,7 @@ func (w *wrapper) Scan(imageRef ImageRef, opt ScanOption) (Report, error) { } }() - cmd, err := w.prepareScanCmd(imageRef, reportFile.Name(), opt) + cmd, err := w.prepareScanCmd(target, reportFile.Name(), opt) if err != nil { return Report{}, err } @@ -147,54 +152,62 @@ func (w *wrapper) parseSBOM(reportFile io.Reader) (Report, error) { return Report{SBOM: doc}, nil } -func (w *wrapper) prepareScanCmd(imageRef ImageRef, outputFile string, opt ScanOption) (*exec.Cmd, error) { +func (w *wrapper) prepareScanCmd(target ScanTarget, outputFile string, opt ScanOption) (*exec.Cmd, error) { args := []string{ + string(target.Type), // subcommand "--no-progress", "--severity", w.config.Severity, "--vuln-type", w.config.VulnType, - "--scanners", w.config.SecurityChecks, "--format", string(opt.Format), "--output", outputFile, - imageRef.Name, + "--cache-dir", w.config.CacheDir, + "--timeout", w.config.Timeout.String(), + } + + if target.Type == TargetImage { + args = append(args, "--scanners", w.config.SecurityChecks) } if w.config.IgnoreUnfixed { - args = append([]string{"--ignore-unfixed"}, args...) + args = append(args, "--ignore-unfixed") } if w.config.SkipUpdate { - args = append([]string{"--skip-db-update"}, args...) + args = append(args, "--skip-db-update") } if w.config.OfflineScan { - args = append([]string{"--offline-scan"}, args...) + args = append(args, "--offline-scan") } if w.config.IgnorePolicy != "" { - args = append([]string{"--ignore-policy", w.config.IgnorePolicy}, args...) + args = append(args, "--ignore-policy") } - name, err := w.ambassador.LookPath(trivyCmd) - if err != nil { - return nil, err + if w.config.DebugMode { + args = append(args, "--debug") } - globalArgs := []string{"--cache-dir", w.config.CacheDir} + if w.config.Insecure || target.NonSSL() { + args = append(args, "--insecure") + } - if w.config.DebugMode { - globalArgs = append(globalArgs, "--debug") + targetName, err := target.Name() + if err != nil { + return nil, xerrors.Errorf("get target name: %w", err) } - globalArgs = append(globalArgs, "image") + args = append(args, targetName) - args = append(globalArgs, args...) + name, err := w.ambassador.LookPath(trivyCmd) + if err != nil { + return nil, err + } cmd := exec.Command(name, args...) cmd.Env = w.ambassador.Environ() - cmd.Env = append(cmd.Env, fmt.Sprintf("TRIVY_TIMEOUT=%s", w.config.Timeout.String())) - - switch a := imageRef.Auth.(type) { + switch a := target.Auth().(type) { case NoAuth: case BasicAuth: cmd.Env = append(cmd.Env, @@ -207,18 +220,10 @@ func (w *wrapper) prepareScanCmd(imageRef ImageRef, outputFile string, opt ScanO return nil, fmt.Errorf("invalid auth type %T", a) } - if imageRef.Insecure { - cmd.Env = append(cmd.Env, "TRIVY_NON_SSL=true") - } - if strings.TrimSpace(w.config.GitHubToken) != "" { cmd.Env = append(cmd.Env, fmt.Sprintf("GITHUB_TOKEN=%s", w.config.GitHubToken)) } - if w.config.Insecure { - cmd.Env = append(cmd.Env, "TRIVY_INSECURE=true") - } - return cmd, nil } diff --git a/pkg/trivy/wrapper_test.go b/pkg/trivy/wrapper_test.go index 6a6f559e..e51b9724 100644 --- a/pkg/trivy/wrapper_test.go +++ b/pkg/trivy/wrapper_test.go @@ -107,9 +107,9 @@ func TestWrapper_Scan(t *testing.T) { } imageRef := ImageRef{ - Name: "alpine:3.10.2", - Auth: BasicAuth{Username: "dave.loper", Password: "s3cret"}, - Insecure: true, + Name: "alpine:3.10.2", + Auth: BasicAuth{Username: "dave.loper", Password: "s3cret"}, + NonSSL: true, } expectedCmdArgs := []string{ From 146f247eb9ee1bb0933d23f3a269b1d2ec60cf30 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Tue, 9 Jan 2024 12:00:20 +0400 Subject: [PATCH 08/26] feat: add SBOM capability Signed-off-by: knqyf263 --- pkg/harbor/model.go | 6 +- pkg/http/api/base_handler.go | 47 +++++- pkg/http/api/v1/handler.go | 11 ++ pkg/http/api/v1/handler_test.go | 285 ++++++++++++++++++++++---------- 4 files changed, 248 insertions(+), 101 deletions(-) diff --git a/pkg/harbor/model.go b/pkg/harbor/model.go index e128a437..3f14d8a0 100644 --- a/pkg/harbor/model.go +++ b/pkg/harbor/model.go @@ -71,7 +71,10 @@ const ( MediaTypeCycloneDX MediaType = "application/vnd.cyclonedx+json" ) -var SupportedSBOMMediaTypes = []MediaType{MediaTypeSPDX, MediaTypeCycloneDX} +var SupportedSBOMMediaTypes = []MediaType{ + MediaTypeSPDX, + MediaTypeCycloneDX, +} type Scan struct { Type ScanType `json:"type,omitempty"` @@ -179,6 +182,7 @@ type Scanner struct { } type Capability struct { + Type string `json:"type"` ConsumesMIMETypes []string `json:"consumes_mime_types"` ProducesMIMETypes []string `json:"produces_mime_types"` } diff --git a/pkg/http/api/base_handler.go b/pkg/http/api/base_handler.go index 0f80078c..e6b12836 100644 --- a/pkg/http/api/base_handler.go +++ b/pkg/http/api/base_handler.go @@ -17,17 +17,46 @@ const ( type MimeTypeParams map[string]string -var MimeTypeVersion = map[string]string{"version": "1.0"} +var ( + MimeTypeVersion = map[string]string{"version": "1.0"} -var MimeTypeOCIImageManifest = MimeType{Type: "application", Subtype: "vnd.oci.image.manifest.v1+json"} -var MimeTypeDockerImageManifestV2 = MimeType{Type: "application", Subtype: "vnd.docker.distribution.manifest.v2+json"} + CapabilityTypeVulnerability = "vulnerability" + CapabilityTypeSBOM = "sbom" -var MimeTypeScanResponse = MimeType{Type: "application", Subtype: "vnd.scanner.adapter.scan.response+json", Params: MimeTypeVersion} - -var MimeTypeSecurityVulnerabilityReport = MimeType{Type: "application", Subtype: "vnd.security.vulnerability.report", Params: map[string]string{"version": "1.1"}} -var MimeTypeSecuritySBOMReport = MimeType{Type: "application", Subtype: "vnd.security.sbom.report+json", Params: map[string]string{"version": "1.1"}} -var MimeTypeMetadata = MimeType{Type: "application", Subtype: "vnd.scanner.adapter.metadata+json", Params: MimeTypeVersion} -var MimeTypeError = MimeType{Type: "application", Subtype: "vnd.scanner.adapter.error", Params: MimeTypeVersion} + MimeTypeOCIImageManifest = MimeType{ + Type: "application", + Subtype: "vnd.oci.image.manifest.v1+json", + } + MimeTypeDockerImageManifestV2 = MimeType{ + Type: "application", + Subtype: "vnd.docker.distribution.manifest.v2+json", + } + MimeTypeScanResponse = MimeType{ + Type: "application", + Subtype: "vnd.scanner.adapter.scan.response+json", + Params: MimeTypeVersion, + } + MimeTypeSecurityVulnerabilityReport = MimeType{ + Type: "application", + Subtype: "vnd.security.vulnerability.report", + Params: map[string]string{"version": "1.1"}, + } + MimeTypeSecuritySBOMReport = MimeType{ + Type: "application", + Subtype: "vnd.security.sbom.report+json", + Params: map[string]string{"version": "1.0"}, + } + MimeTypeMetadata = MimeType{ + Type: "application", + Subtype: "vnd.scanner.adapter.metadata+json", + Params: MimeTypeVersion, + } + MimeTypeError = MimeType{ + Type: "application", + Subtype: "vnd.scanner.adapter.error", + Params: MimeTypeVersion, + } +) type MimeType struct { Type string diff --git a/pkg/http/api/v1/handler.go b/pkg/http/api/v1/handler.go index 7d9b0954..f2e12276 100644 --- a/pkg/http/api/v1/handler.go +++ b/pkg/http/api/v1/handler.go @@ -277,6 +277,7 @@ func (h *requestHandler) GetMetadata(res http.ResponseWriter, _ *http.Request) { Scanner: etc.GetScannerMetadata(), Capabilities: []harbor.Capability{ { + Type: api.CapabilityTypeVulnerability, ConsumesMIMETypes: []string{ api.MimeTypeOCIImageManifest.String(), api.MimeTypeDockerImageManifestV2.String(), @@ -285,6 +286,16 @@ func (h *requestHandler) GetMetadata(res http.ResponseWriter, _ *http.Request) { api.MimeTypeSecurityVulnerabilityReport.String(), }, }, + { + Type: api.CapabilityTypeSBOM, + ConsumesMIMETypes: []string{ + api.MimeTypeOCIImageManifest.String(), + api.MimeTypeDockerImageManifestV2.String(), + }, + ProducesMIMETypes: []string{ + api.MimeTypeSecuritySBOMReport.String(), + }, + }, }, Properties: properties, } diff --git a/pkg/http/api/v1/handler_test.go b/pkg/http/api/v1/handler_test.go index 2abe37d0..3f1a129d 100644 --- a/pkg/http/api/v1/handler_test.go +++ b/pkg/http/api/v1/handler_test.go @@ -122,9 +122,15 @@ func TestRequestHandler_AcceptScanRequest(t *testing.T) { { name: "Should accept scan request", enqueuerExpectation: &mock.Expectation{ - Method: "Enqueue", - Args: []interface{}{mock.Anything, validScanRequest}, - ReturnArgs: []interface{}{job.ScanJob{ID: "job:123"}, nil}, + Method: "Enqueue", + Args: []interface{}{ + mock.Anything, + validScanRequest, + }, + ReturnArgs: []interface{}{ + job.ScanJob{ID: "job:123"}, + nil, + }, }, requestBody: validScanRequestJSON, expectedStatus: http.StatusAccepted, @@ -156,9 +162,15 @@ func TestRequestHandler_AcceptScanRequest(t *testing.T) { { name: "Should respond with error 500 when enqueuing scan request fails", enqueuerExpectation: &mock.Expectation{ - Method: "Enqueue", - Args: []interface{}{mock.Anything, validScanRequest}, - ReturnArgs: []interface{}{job.ScanJob{}, errors.New("queue is down")}, + Method: "Enqueue", + Args: []interface{}{ + mock.Anything, + validScanRequest, + }, + ReturnArgs: []interface{}{ + job.ScanJob{}, + errors.New("queue is down"), + }, }, requestBody: validScanRequestJSON, expectedStatus: http.StatusInternalServerError, @@ -208,9 +220,15 @@ func TestRequestHandler_GetScanReport(t *testing.T) { { name: "Should respond with error 500 when retrieving scan job fails", storeExpectation: &mock.Expectation{ - Method: "Get", - Args: []interface{}{mock.Anything, "job:123"}, - ReturnArgs: []interface{}{&job.ScanJob{}, errors.New("data store is down")}, + Method: "Get", + Args: []interface{}{ + mock.Anything, + "job:123", + }, + ReturnArgs: []interface{}{ + &job.ScanJob{}, + errors.New("data store is down"), + }, }, expectedStatus: http.StatusInternalServerError, expectedContentType: "application/vnd.scanner.adapter.error; version=1.0", @@ -223,9 +241,15 @@ func TestRequestHandler_GetScanReport(t *testing.T) { { name: "Should respond with error 404 when scan job cannot be found", storeExpectation: &mock.Expectation{ - Method: "Get", - Args: []interface{}{mock.Anything, "job:123"}, - ReturnArgs: []interface{}{(*job.ScanJob)(nil), nil}, + Method: "Get", + Args: []interface{}{ + mock.Anything, + "job:123", + }, + ReturnArgs: []interface{}{ + (*job.ScanJob)(nil), + nil, + }, }, expectedStatus: http.StatusNotFound, expectedContentType: "application/vnd.scanner.adapter.error; version=1.0", @@ -239,11 +263,17 @@ func TestRequestHandler_GetScanReport(t *testing.T) { name: fmt.Sprintf("Should respond with found status 302 when scan job is %s", job.Queued), storeExpectation: &mock.Expectation{ Method: "Get", - Args: []interface{}{mock.Anything, "job:123"}, - ReturnArgs: []interface{}{&job.ScanJob{ - ID: "job:123", - Status: job.Queued, - }, nil}, + Args: []interface{}{ + mock.Anything, + "job:123", + }, + ReturnArgs: []interface{}{ + &job.ScanJob{ + ID: "job:123", + Status: job.Queued, + }, + nil, + }, }, expectedStatus: http.StatusFound, }, @@ -251,11 +281,17 @@ func TestRequestHandler_GetScanReport(t *testing.T) { name: fmt.Sprintf("Should respond with found status 302 when scan job is %s", job.Pending), storeExpectation: &mock.Expectation{ Method: "Get", - Args: []interface{}{mock.Anything, "job:123"}, - ReturnArgs: []interface{}{&job.ScanJob{ - ID: "job:123", - Status: job.Pending, - }, nil}, + Args: []interface{}{ + mock.Anything, + "job:123", + }, + ReturnArgs: []interface{}{ + &job.ScanJob{ + ID: "job:123", + Status: job.Pending, + }, + nil, + }, }, expectedStatus: http.StatusFound, }, @@ -263,12 +299,18 @@ func TestRequestHandler_GetScanReport(t *testing.T) { name: fmt.Sprintf("Should respond with error 500 when scan job is %s", job.Failed), storeExpectation: &mock.Expectation{ Method: "Get", - Args: []interface{}{mock.Anything, "job:123"}, - ReturnArgs: []interface{}{&job.ScanJob{ - ID: "job:123", - Status: job.Failed, - Error: "queue worker failed", - }, nil}, + Args: []interface{}{ + mock.Anything, + "job:123", + }, + ReturnArgs: []interface{}{ + &job.ScanJob{ + ID: "job:123", + Status: job.Failed, + Error: "queue worker failed", + }, + nil, + }, }, expectedStatus: http.StatusInternalServerError, expectedContentType: "application/vnd.scanner.adapter.error; version=1.0", @@ -282,12 +324,18 @@ func TestRequestHandler_GetScanReport(t *testing.T) { name: fmt.Sprintf("Should respond with error 500 when scan job is NOT %s", job.Finished), storeExpectation: &mock.Expectation{ Method: "Get", - Args: []interface{}{mock.Anything, "job:123"}, - ReturnArgs: []interface{}{&job.ScanJob{ - ID: "job:123", - Status: 666, - Error: "queue worker failed", - }, nil}, + Args: []interface{}{ + mock.Anything, + "job:123", + }, + ReturnArgs: []interface{}{ + &job.ScanJob{ + ID: "job:123", + Status: 666, + Error: "queue worker failed", + }, + nil, + }, }, expectedStatus: http.StatusInternalServerError, expectedContentType: "application/vnd.scanner.adapter.error; version=1.0", @@ -301,40 +349,46 @@ func TestRequestHandler_GetScanReport(t *testing.T) { name: "Should respond with vulnerabilities report", storeExpectation: &mock.Expectation{ Method: "Get", - Args: []interface{}{mock.Anything, "job:123"}, - ReturnArgs: []interface{}{&job.ScanJob{ - ID: "job:123", - Status: job.Finished, - Report: harbor.ScanReport{ - GeneratedAt: now, - Artifact: harbor.Artifact{ - Repository: "library/mongo", - Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", - }, - Scanner: harbor.Scanner{ - Name: "Trivy", - Vendor: "Aqua Security", - Version: "0.1.6", - }, - Severity: harbor.SevCritical, - Vulnerabilities: []harbor.VulnerabilityItem{ - { - ID: "CVE-2019-1111", - Pkg: "openssl", - Version: "2.0-rc1", - FixVersion: "2.1", - Severity: harbor.SevCritical, - Description: "You'd better upgrade your server", - Links: []string{ - "http://cve.com?id=CVE-2019-1111", - }, - Layer: &harbor.Layer{ - Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb10", + Args: []interface{}{ + mock.Anything, + "job:123", + }, + ReturnArgs: []interface{}{ + &job.ScanJob{ + ID: "job:123", + Status: job.Finished, + Report: harbor.ScanReport{ + GeneratedAt: now, + Artifact: harbor.Artifact{ + Repository: "library/mongo", + Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", + }, + Scanner: harbor.Scanner{ + Name: "Trivy", + Vendor: "Aqua Security", + Version: "0.1.6", + }, + Severity: harbor.SevCritical, + Vulnerabilities: []harbor.VulnerabilityItem{ + { + ID: "CVE-2019-1111", + Pkg: "openssl", + Version: "2.0-rc1", + FixVersion: "2.1", + Severity: harbor.SevCritical, + Description: "You'd better upgrade your server", + Links: []string{ + "http://cve.com?id=CVE-2019-1111", + }, + Layer: &harbor.Layer{ + Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb10", + }, }, }, }, }, - }, nil}, + nil, + }, }, expectedStatus: http.StatusOK, expectedContentType: "application/vnd.security.vulnerability.report; version=1.1", @@ -443,8 +497,12 @@ func TestRequestHandler_GetMetadata(t *testing.T) { expectedError error }{ { - name: "Should respond with a valid Metadata JSON and HTTP 200 OK", - buildInfo: etc.BuildInfo{Version: "0.1", Commit: "abc", Date: "2019-01-03T13:40"}, + name: "Should respond with a valid Metadata JSON and HTTP 200 OK", + buildInfo: etc.BuildInfo{ + Version: "0.1", + Commit: "abc", + Date: "2019-01-03T13:40", + }, version: trivy.VersionInfo{ Version: "v0.5.2-17-g3c9af62", VulnerabilityDB: &trivy.Metadata{ @@ -452,17 +510,19 @@ func TestRequestHandler_GetMetadata(t *testing.T) { UpdatedAt: time.Unix(1584517644, 0).UTC(), }, }, - config: etc.Config{Trivy: etc.Trivy{ - SkipUpdate: false, - SkipJavaDBUpdate: false, - IgnoreUnfixed: true, - DebugMode: true, - Insecure: true, - VulnType: "os,library", - SecurityChecks: "vuln", - Severity: "UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL", - Timeout: 5 * time.Minute, - }}, + config: etc.Config{ + Trivy: etc.Trivy{ + SkipUpdate: false, + SkipJavaDBUpdate: false, + IgnoreUnfixed: true, + DebugMode: true, + Insecure: true, + VulnType: "os,library", + SecurityChecks: "vuln", + Severity: "UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL", + Timeout: 5 * time.Minute, + }, + }, expectedHTTPCode: http.StatusOK, expectedResp: `{ "scanner":{ @@ -472,6 +532,7 @@ func TestRequestHandler_GetMetadata(t *testing.T) { }, "capabilities":[ { + "type": "vulnerability", "consumes_mime_types":[ "application/vnd.oci.image.manifest.v1+json", "application/vnd.docker.distribution.manifest.v2+json" @@ -479,6 +540,16 @@ func TestRequestHandler_GetMetadata(t *testing.T) { "produces_mime_types":[ "application/vnd.security.vulnerability.report; version=1.1" ] + }, + { + "type": "sbom", + "consumes_mime_types":[ + "application/vnd.oci.image.manifest.v1+json", + "application/vnd.docker.distribution.manifest.v2+json" + ], + "produces_mime_types":[ + "application/vnd.security.sbom.report+json; version=1.0" + ] } ], "properties":{ @@ -503,22 +574,28 @@ func TestRequestHandler_GetMetadata(t *testing.T) { }`, }, { - name: "Should respond with a valid Metadata JSON and HTTP 200 OK, when there's no trivy Metadata present", - buildInfo: etc.BuildInfo{Version: "0.1", Commit: "abc", Date: "2019-01-03T13:40"}, + name: "Should respond with a valid Metadata JSON and HTTP 200 OK, when there's no trivy Metadata present", + buildInfo: etc.BuildInfo{ + Version: "0.1", + Commit: "abc", + Date: "2019-01-03T13:40", + }, version: trivy.VersionInfo{ Version: "v0.5.2-17-g3c9af62", }, - config: etc.Config{Trivy: etc.Trivy{ - SkipUpdate: false, - SkipJavaDBUpdate: false, - IgnoreUnfixed: true, - DebugMode: true, - Insecure: true, - VulnType: "os,library", - SecurityChecks: "vuln", - Severity: "UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL", - Timeout: 5 * time.Minute, - }}, + config: etc.Config{ + Trivy: etc.Trivy{ + SkipUpdate: false, + SkipJavaDBUpdate: false, + IgnoreUnfixed: true, + DebugMode: true, + Insecure: true, + VulnType: "os,library", + SecurityChecks: "vuln", + Severity: "UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL", + Timeout: 5 * time.Minute, + }, + }, expectedHTTPCode: http.StatusOK, expectedResp: `{ "scanner":{ @@ -528,6 +605,7 @@ func TestRequestHandler_GetMetadata(t *testing.T) { }, "capabilities":[ { + "type": "vulnerability", "consumes_mime_types":[ "application/vnd.oci.image.manifest.v1+json", "application/vnd.docker.distribution.manifest.v2+json" @@ -535,6 +613,16 @@ func TestRequestHandler_GetMetadata(t *testing.T) { "produces_mime_types":[ "application/vnd.security.vulnerability.report; version=1.1" ] + }, + { + "type": "sbom", + "consumes_mime_types":[ + "application/vnd.oci.image.manifest.v1+json", + "application/vnd.docker.distribution.manifest.v2+json" + ], + "produces_mime_types":[ + "application/vnd.security.sbom.report+json; version=1.0" + ] } ], "properties":{ @@ -559,7 +647,11 @@ func TestRequestHandler_GetMetadata(t *testing.T) { { name: "Should respond with a valid Metadata JSON and HTTP 200 OK when GetVersion fails", mockedError: errors.New("get version failed"), - buildInfo: etc.BuildInfo{Version: "0.1", Commit: "abc", Date: "2019-01-03T13:40"}, + buildInfo: etc.BuildInfo{ + Version: "0.1", + Commit: "abc", + Date: "2019-01-03T13:40", + }, config: etc.Config{ Trivy: etc.Trivy{ VulnType: "os,library", @@ -578,6 +670,7 @@ func TestRequestHandler_GetMetadata(t *testing.T) { }, "capabilities":[ { + "type": "vulnerability", "consumes_mime_types":[ "application/vnd.oci.image.manifest.v1+json", "application/vnd.docker.distribution.manifest.v2+json" @@ -585,6 +678,16 @@ func TestRequestHandler_GetMetadata(t *testing.T) { "produces_mime_types":[ "application/vnd.security.vulnerability.report; version=1.1" ] + }, + { + "type": "sbom", + "consumes_mime_types":[ + "application/vnd.oci.image.manifest.v1+json", + "application/vnd.docker.distribution.manifest.v2+json" + ], + "produces_mime_types":[ + "application/vnd.security.sbom.report+json; version=1.0" + ] } ], "properties":{ From c04fb14c3dd0fe92c63e38f4b83166323316fe13 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Tue, 9 Jan 2024 21:46:40 +0400 Subject: [PATCH 09/26] feat: add support for enabled capabilities Signed-off-by: knqyf263 --- pkg/etc/config.go | 2 +- pkg/etc/config_test.go | 31 ++-- pkg/ext/ambassador.go | 23 ++- pkg/ext/ambassador_mock.go | 48 ++--- pkg/harbor/model.go | 32 ++-- pkg/http/api/v1/handler.go | 55 +++--- pkg/http/api/v1/handler_test.go | 36 +++- pkg/scan/controller.go | 14 +- pkg/scan/controller_test.go | 75 ++++++-- pkg/scan/transformer.go | 4 +- pkg/scan/transformer_test.go | 6 +- pkg/trivy/target.go | 52 ++++-- pkg/trivy/wrapper.go | 53 ++++-- pkg/trivy/wrapper_test.go | 245 ++++++++++++++++++-------- test/integration/api/rest_api_test.go | 4 +- 15 files changed, 433 insertions(+), 247 deletions(-) diff --git a/pkg/etc/config.go b/pkg/etc/config.go index 1ed0f58c..f1449a8a 100644 --- a/pkg/etc/config.go +++ b/pkg/etc/config.go @@ -33,7 +33,7 @@ type Trivy struct { Severity string `env:"SCANNER_TRIVY_SEVERITY" envDefault:"UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL"` IgnoreUnfixed bool `env:"SCANNER_TRIVY_IGNORE_UNFIXED" envDefault:"false"` IgnorePolicy string `env:"SCANNER_TRIVY_IGNORE_POLICY"` - SkipUpdate bool `env:"SCANNER_TRIVY_SKIP_UPDATE" envDefault:"false"` + SkipDBUpdate bool `env:"SCANNER_TRIVY_SKIP_UPDATE" envDefault:"false"` SkipJavaDBUpdate bool `env:"SCANNER_TRIVY_SKIP_JAVA_DB_UPDATE" envDefault:"false"` OfflineScan bool `env:"SCANNER_TRIVY_OFFLINE_SCAN" envDefault:"false"` GitHubToken string `env:"SCANNER_TRIVY_GITHUB_TOKEN"` diff --git a/pkg/etc/config_test.go b/pkg/etc/config_test.go index 86cf167b..83ba887b 100644 --- a/pkg/etc/config_test.go +++ b/pkg/etc/config_test.go @@ -176,10 +176,13 @@ func TestGetConfig(t *testing.T) { Addr: ":4200", TLSCertificate: "/certs/tls.crt", TLSKey: "/certs/tls.key", - ClientCAs: []string{"/certs/tls1.crt", "/certs/tls2.crt"}, - ReadTimeout: parseDuration(t, "1h"), - WriteTimeout: parseDuration(t, "2m"), - IdleTimeout: parseDuration(t, "3m10s"), + ClientCAs: []string{ + "/certs/tls1.crt", + "/certs/tls2.crt", + }, + ReadTimeout: parseDuration(t, "1h"), + WriteTimeout: parseDuration(t, "2m"), + IdleTimeout: parseDuration(t, "3m10s"), }, Trivy: Trivy{ CacheDir: "/home/scanner/trivy-cache", @@ -189,7 +192,7 @@ func TestGetConfig(t *testing.T) { SecurityChecks: "vuln", Severity: "CRITICAL", IgnoreUnfixed: true, - SkipUpdate: true, + SkipDBUpdate: true, SkipJavaDBUpdate: false, OfflineScan: true, Insecure: true, @@ -234,13 +237,21 @@ func TestGetScannerMetadata(t *testing.T) { expectedScanner harbor.Scanner }{ { - name: "Should return version set via env", - envs: Envs{"TRIVY_VERSION": "0.1.6"}, - expectedScanner: harbor.Scanner{Name: "Trivy", Vendor: "Aqua Security", Version: "0.1.6"}, + name: "Should return version set via env", + envs: Envs{"TRIVY_VERSION": "0.1.6"}, + expectedScanner: harbor.Scanner{ + Name: "Trivy", + Vendor: "Aqua Security", + Version: "0.1.6", + }, }, { - name: "Should return unknown version when it is not set via env", - expectedScanner: harbor.Scanner{Name: "Trivy", Vendor: "Aqua Security", Version: "Unknown"}, + name: "Should return unknown version when it is not set via env", + expectedScanner: harbor.Scanner{ + Name: "Trivy", + Vendor: "Aqua Security", + Version: "Unknown", + }, }, } for _, tc := range testCases { diff --git a/pkg/ext/ambassador.go b/pkg/ext/ambassador.go index b89e736c..d874c76c 100644 --- a/pkg/ext/ambassador.go +++ b/pkg/ext/ambassador.go @@ -1,6 +1,9 @@ package ext import ( + "github.com/google/go-containerregistry/pkg/name" + v1 "github.com/google/go-containerregistry/pkg/v1" + "github.com/google/go-containerregistry/pkg/v1/remote" "os" "os/exec" ) @@ -9,20 +12,14 @@ var ( DefaultAmbassador = &ambassador{} ) -// File abstracts the few methods we need, so we can test without real files. -type File interface { - Name() string - Read([]byte) (int, error) -} - // Ambassador the ambassador to the outside "world". Wraps methods that modify global state and hence make the code that // use them very hard to test. type Ambassador interface { Environ() []string LookPath(string) (string, error) + TempFile(string, string) (*os.File, error) RunCmd(cmd *exec.Cmd) ([]byte, error) - TempFile(dir, pattern string) (File, error) - Remove(name string) error + RemoteImage(name.Reference, ...remote.Option) (v1.Image, error) } type ambassador struct { @@ -36,14 +33,14 @@ func (a *ambassador) RunCmd(cmd *exec.Cmd) ([]byte, error) { return cmd.CombinedOutput() } -func (a *ambassador) TempFile(dir, pattern string) (File, error) { +func (a *ambassador) TempFile(dir, pattern string) (*os.File, error) { return os.CreateTemp(dir, pattern) } -func (a *ambassador) Remove(name string) error { - return os.Remove(name) -} - func (a *ambassador) LookPath(file string) (string, error) { return exec.LookPath(file) } + +func (a *ambassador) RemoteImage(ref name.Reference, options ...remote.Option) (v1.Image, error) { + return remote.Image(ref, options...) +} diff --git a/pkg/ext/ambassador_mock.go b/pkg/ext/ambassador_mock.go index 450f8683..d04b9f0f 100644 --- a/pkg/ext/ambassador_mock.go +++ b/pkg/ext/ambassador_mock.go @@ -1,36 +1,14 @@ package ext import ( - "io" - "os/exec" - "strings" - + "github.com/google/go-containerregistry/pkg/name" + v1 "github.com/google/go-containerregistry/pkg/v1" + "github.com/google/go-containerregistry/pkg/v1/remote" "github.com/stretchr/testify/mock" + "os" + "os/exec" ) -type FakeFile struct { - name string - content string - reader io.Reader -} - -// NewFakeFile constructs a new FakeFile with the given name and content. -func NewFakeFile(name, content string) *FakeFile { - return &FakeFile{ - name: name, - content: content, - reader: strings.NewReader(content), - } -} - -func (ff *FakeFile) Name() string { - return ff.name -} - -func (ff *FakeFile) Read(p []byte) (int, error) { - return ff.reader.Read(p) -} - type MockAmbassador struct { mock.Mock } @@ -49,17 +27,17 @@ func (m *MockAmbassador) LookPath(file string) (string, error) { return args.String(0), args.Error(1) } +func (m *MockAmbassador) TempFile(dir, pattern string) (*os.File, error) { + args := m.Called(dir, pattern) + return args.Get(0).(*os.File), args.Error(1) +} + func (m *MockAmbassador) RunCmd(cmd *exec.Cmd) ([]byte, error) { args := m.Called(cmd) return args.Get(0).([]byte), args.Error(1) } -func (m *MockAmbassador) TempFile(dir, pattern string) (File, error) { - args := m.Called(dir, pattern) - return args.Get(0).(File), args.Error(1) -} - -func (m *MockAmbassador) Remove(name string) error { - args := m.Called(name) - return args.Error(0) +func (m *MockAmbassador) RemoteImage(ref name.Reference, options ...remote.Option) (v1.Image, error) { + args := m.Called(ref, options) + return args.Get(0).(v1.Image), args.Error(1) } diff --git a/pkg/harbor/model.go b/pkg/harbor/model.go index 3f14d8a0..5e77b9a3 100644 --- a/pkg/harbor/model.go +++ b/pkg/harbor/model.go @@ -60,12 +60,12 @@ func (s *Severity) UnmarshalJSON(b []byte) error { return nil } -type ScanType string +type CapabilityType string type MediaType string const ( - ScanTypeSBOM ScanType = "sbom" - ScanTypeVulnerability ScanType = "vulnerability" + CapabilityTypeSBOM CapabilityType = "sbom" + CapabilityTypeVulnerability CapabilityType = "vulnerability" MediaTypeSPDX MediaType = "application/spdx+json" MediaTypeCycloneDX MediaType = "application/vnd.cyclonedx+json" @@ -76,15 +76,6 @@ var SupportedSBOMMediaTypes = []MediaType{ MediaTypeCycloneDX, } -type Scan struct { - Type ScanType `json:"type,omitempty"` - Parameters ScanParameters `json:"parameters,omitempty"` -} - -type ScanParameters struct { - SBOMMediaType MediaType `json:"sbom_media_type,omitempty"` -} - type Registry struct { URL string `json:"url"` Authorization string `json:"authorization"` @@ -97,9 +88,9 @@ type Artifact struct { } type ScanRequest struct { - Scan Scan `json:"scan"` - Registry Registry `json:"registry"` - Artifact Artifact `json:"artifact"` + Registry Registry `json:"registry"` + Artifact Artifact `json:"artifact"` + Capabilities []Capability `json:"enabled_capabilities"` } // GetImageRef returns Docker image reference for this ScanRequest. @@ -182,9 +173,14 @@ type Scanner struct { } type Capability struct { - Type string `json:"type"` - ConsumesMIMETypes []string `json:"consumes_mime_types"` - ProducesMIMETypes []string `json:"produces_mime_types"` + Type CapabilityType `json:"type"` + ConsumesMIMETypes []string `json:"consumes_mime_types"` + ProducesMIMETypes []string `json:"produces_mime_types"` + Parameters *CapabilityParameters `json:"parameters,omitempty"` +} + +type CapabilityParameters struct { + MediaType MediaType `json:"accept_media_type,omitempty"` } // Error holds the information about an error, including metadata about its JSON structure. diff --git a/pkg/http/api/v1/handler.go b/pkg/http/api/v1/handler.go index f2e12276..30c17ccc 100644 --- a/pkg/http/api/v1/handler.go +++ b/pkg/http/api/v1/handler.go @@ -89,17 +89,19 @@ func (h *requestHandler) AcceptScanRequest(res http.ResponseWriter, req *http.Re return } - // Set the default value for scan type if it is not specified. - if scanRequest.Scan.Type == "" { - scanRequest.Scan.Type = harbor.ScanTypeVulnerability - } - if validationError := h.ValidateScanRequest(scanRequest); validationError != nil { slog.Error("Error while validating scan request", slog.String("err", validationError.Message)) h.WriteJSONError(res, *validationError) return } + // Set the default value for capability type if not specified. + if len(scanRequest.Capabilities) == 0 { + scanRequest.Capabilities = append(scanRequest.Capabilities, harbor.Capability{ + Type: harbor.CapabilityTypeVulnerability, + }) + } + scanJob, err := h.enqueuer.Enqueue(req.Context(), scanRequest) if err != nil { slog.Error("Error while enqueuing scan job", slog.String("err", err.Error())) @@ -116,19 +118,8 @@ func (h *requestHandler) AcceptScanRequest(res http.ResponseWriter, req *http.Re } func (h *requestHandler) ValidateScanRequest(req harbor.ScanRequest) *harbor.Error { - if req.Scan.Type != harbor.ScanTypeVulnerability && req.Scan.Type != harbor.ScanTypeSBOM { - return &harbor.Error{ - HTTPCode: http.StatusUnprocessableEntity, - Message: "invalid scan type", - } - } - - if req.Scan.Type == harbor.ScanTypeSBOM && - !slices.Contains(harbor.SupportedSBOMMediaTypes, req.Scan.Parameters.SBOMMediaType) { - return &harbor.Error{ - HTTPCode: http.StatusUnprocessableEntity, - Message: "invalid SBOM media type", - } + if err := h.validateCapabilities(req.Capabilities); err != nil { + return err } if req.Registry.URL == "" { @@ -162,6 +153,26 @@ func (h *requestHandler) ValidateScanRequest(req harbor.ScanRequest) *harbor.Err return nil } +func (h *requestHandler) validateCapabilities(capabilities []harbor.Capability) *harbor.Error { + for _, c := range capabilities { + if c.Type != harbor.CapabilityTypeVulnerability && c.Type != harbor.CapabilityTypeSBOM { + return &harbor.Error{ + HTTPCode: http.StatusUnprocessableEntity, + Message: "invalid scan type", + } + } + + if c.Type == harbor.CapabilityTypeSBOM && + !slices.Contains(harbor.SupportedSBOMMediaTypes, c.Parameters.MediaType) { + return &harbor.Error{ + HTTPCode: http.StatusUnprocessableEntity, + Message: "invalid SBOM media type", + } + } + } + return nil +} + func (h *requestHandler) GetScanReport(res http.ResponseWriter, req *http.Request) { var reportMimeType api.MimeType @@ -244,7 +255,7 @@ func (h *requestHandler) GetMetadata(res http.ResponseWriter, _ *http.Request) { "org.label-schema.vcs-ref": h.info.Commit, "org.label-schema.vcs": "https://github.com/aquasecurity/harbor-scanner-trivy", - "env.SCANNER_TRIVY_SKIP_UPDATE": strconv.FormatBool(h.config.Trivy.SkipUpdate), + "env.SCANNER_TRIVY_SKIP_UPDATE": strconv.FormatBool(h.config.Trivy.SkipDBUpdate), "env.SCANNER_TRIVY_SKIP_JAVA_DB_UPDATE": strconv.FormatBool(h.config.Trivy.SkipJavaDBUpdate), "env.SCANNER_TRIVY_OFFLINE_SCAN": strconv.FormatBool(h.config.Trivy.OfflineScan), "env.SCANNER_TRIVY_IGNORE_UNFIXED": strconv.FormatBool(h.config.Trivy.IgnoreUnfixed), @@ -265,7 +276,7 @@ func (h *requestHandler) GetMetadata(res http.ResponseWriter, _ *http.Request) { properties[propertyDBUpdatedAt] = vi.VulnerabilityDB.UpdatedAt.Format(time.RFC3339) } - if err == nil && vi.VulnerabilityDB != nil && !h.config.Trivy.SkipUpdate { + if err == nil && vi.VulnerabilityDB != nil && !h.config.Trivy.SkipDBUpdate { properties[propertyDBNextUpdateAt] = vi.VulnerabilityDB.NextUpdate.Format(time.RFC3339) } @@ -277,7 +288,7 @@ func (h *requestHandler) GetMetadata(res http.ResponseWriter, _ *http.Request) { Scanner: etc.GetScannerMetadata(), Capabilities: []harbor.Capability{ { - Type: api.CapabilityTypeVulnerability, + Type: harbor.CapabilityTypeVulnerability, ConsumesMIMETypes: []string{ api.MimeTypeOCIImageManifest.String(), api.MimeTypeDockerImageManifestV2.String(), @@ -287,7 +298,7 @@ func (h *requestHandler) GetMetadata(res http.ResponseWriter, _ *http.Request) { }, }, { - Type: api.CapabilityTypeSBOM, + Type: harbor.CapabilityTypeSBOM, ConsumesMIMETypes: []string{ api.MimeTypeOCIImageManifest.String(), api.MimeTypeDockerImageManifestV2.String(), diff --git a/pkg/http/api/v1/handler_test.go b/pkg/http/api/v1/handler_test.go index 3f1a129d..5c97d8a1 100644 --- a/pkg/http/api/v1/handler_test.go +++ b/pkg/http/api/v1/handler_test.go @@ -26,8 +26,12 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { ExpectedError *harbor.Error }{ { - Name: "Should return error when Registry URL is blank", - Request: harbor.ScanRequest{Scan: harbor.Scan{Type: harbor.ScanTypeVulnerability}}, + Name: "Should return error when Registry URL is blank", + Request: harbor.ScanRequest{ + Capabilities: []harbor.Capability{ + {Type: harbor.CapabilityTypeVulnerability}, + }, + }, ExpectedError: &harbor.Error{ HTTPCode: http.StatusUnprocessableEntity, Message: "missing registry.url", @@ -36,7 +40,11 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { { Name: "Should return error when Registry URL is invalid", Request: harbor.ScanRequest{ - Scan: harbor.Scan{Type: harbor.ScanTypeVulnerability}, + Capabilities: []harbor.Capability{ + { + Type: harbor.CapabilityTypeVulnerability, + }, + }, Registry: harbor.Registry{ URL: "INVALID URL", }, @@ -49,7 +57,11 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { { Name: "Should return error when artifact repository is blank", Request: harbor.ScanRequest{ - Scan: harbor.Scan{Type: harbor.ScanTypeVulnerability}, + Capabilities: []harbor.Capability{ + { + Type: harbor.CapabilityTypeVulnerability, + }, + }, Registry: harbor.Registry{ URL: "https://core.harbor.domain", }, @@ -62,7 +74,11 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { { Name: "Should return error when artifact digest is blank", Request: harbor.ScanRequest{ - Scan: harbor.Scan{Type: harbor.ScanTypeVulnerability}, + Capabilities: []harbor.Capability{ + { + Type: harbor.CapabilityTypeVulnerability, + }, + }, Registry: harbor.Registry{ URL: "https://core.harbor.domain", }, @@ -88,8 +104,10 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { func TestRequestHandler_AcceptScanRequest(t *testing.T) { validScanRequest := harbor.ScanRequest{ - Scan: harbor.Scan{ - Type: harbor.ScanTypeVulnerability, + Capabilities: []harbor.Capability{ + { + Type: harbor.CapabilityTypeVulnerability, + }, }, Registry: harbor.Registry{ URL: "https://core.harbor.domain", @@ -512,7 +530,7 @@ func TestRequestHandler_GetMetadata(t *testing.T) { }, config: etc.Config{ Trivy: etc.Trivy{ - SkipUpdate: false, + SkipDBUpdate: false, SkipJavaDBUpdate: false, IgnoreUnfixed: true, DebugMode: true, @@ -585,7 +603,7 @@ func TestRequestHandler_GetMetadata(t *testing.T) { }, config: etc.Config{ Trivy: etc.Trivy{ - SkipUpdate: false, + SkipDBUpdate: false, SkipJavaDBUpdate: false, IgnoreUnfixed: true, DebugMode: true, diff --git a/pkg/scan/controller.go b/pkg/scan/controller.go index 0eeaa906..cf6128e9 100644 --- a/pkg/scan/controller.go +++ b/pkg/scan/controller.go @@ -3,6 +3,7 @@ package scan import ( "context" "encoding/base64" + "github.com/samber/lo" "log/slog" "strings" @@ -63,12 +64,15 @@ func (c *controller) scan(ctx context.Context, scanJobID string, req harbor.Scan return err } - scanReport, err := c.wrapper.Scan(trivy.ImageRef{ + ref := trivy.ImageRef{ Name: imageRef, Auth: auth, NonSSL: nonSSL, - }, - trivy.ScanOption{Format: determineFormat(req.Scan.Parameters.SBOMMediaType)}) + } + + scanReport, err := c.wrapper.Scan(ref, trivy.ScanOption{ + Format: determineFormat(lo.FromPtr(req.Capabilities[0].Parameters).MediaType), + }) if err != nil { return xerrors.Errorf("running trivy wrapper: %v", err) } @@ -119,8 +123,8 @@ func (c *controller) decodeBasicAuth(value string) (auth trivy.RegistryAuth, err return } -func determineFormat(sbomMediaType harbor.MediaType) trivy.Format { - switch sbomMediaType { +func determineFormat(m harbor.MediaType) trivy.Format { + switch m { case harbor.MediaTypeSPDX: return trivy.FormatSPDX case harbor.MediaTypeCycloneDX: diff --git a/pkg/scan/controller_test.go b/pkg/scan/controller_test.go index b93abf7f..eb630ceb 100644 --- a/pkg/scan/controller_test.go +++ b/pkg/scan/controller_test.go @@ -13,6 +13,12 @@ import ( "golang.org/x/xerrors" ) +var capabilities = []harbor.Capability{ + { + Type: harbor.CapabilityTypeVulnerability, + }, +} + func TestController_Scan(t *testing.T) { ctx := context.Background() artifact := harbor.Artifact{ @@ -41,22 +47,37 @@ func TestController_Scan(t *testing.T) { URL: "https://core.harbor.domain", Authorization: "Basic dXNlcjpwYXNzd29yZA==", // user:password }, - Artifact: artifact, + Artifact: artifact, + Capabilities: capabilities, }, storeExpectation: []*mock.Expectation{ { - Method: "UpdateStatus", - Args: []interface{}{ctx, "job:123", job.Pending, []string(nil)}, + Method: "UpdateStatus", + Args: []interface{}{ + ctx, + "job:123", + job.Pending, + []string(nil), + }, ReturnArgs: []interface{}{nil}, }, { - Method: "UpdateReport", - Args: []interface{}{ctx, "job:123", harborReport}, + Method: "UpdateReport", + Args: []interface{}{ + ctx, + "job:123", + harborReport, + }, ReturnArgs: []interface{}{nil}, }, { - Method: "UpdateStatus", - Args: []interface{}{ctx, "job:123", job.Finished, []string(nil)}, + Method: "UpdateStatus", + Args: []interface{}{ + ctx, + "job:123", + job.Finished, + []string(nil), + }, ReturnArgs: []interface{}{nil}, }, }, @@ -64,8 +85,11 @@ func TestController_Scan(t *testing.T) { Method: "Scan", Args: []interface{}{ trivy.ImageRef{ - Name: "core.harbor.domain:443/library/mongo@sha256:917f5b7f4bef1b35ee90f03033f33a81002511c1e0767fd44276d4bd9cd2fa8e", - Auth: trivy.BasicAuth{Username: "user", Password: "password"}, + Name: "core.harbor.domain:443/library/mongo@sha256:917f5b7f4bef1b35ee90f03033f33a81002511c1e0767fd44276d4bd9cd2fa8e", + Auth: trivy.BasicAuth{ + Username: "user", + Password: "password", + }, NonSSL: false, }, trivy.ScanOption{Format: "json"}, @@ -83,7 +107,8 @@ func TestController_Scan(t *testing.T) { URL: "https://core.harbor.domain", Authorization: "Basic dXNlcjpwYXNzd29yZA==", // user:password }, - Artifact: artifact, + Artifact: artifact, + Capabilities: capabilities, }, trivyReport, }, @@ -101,16 +126,31 @@ func TestController_Scan(t *testing.T) { Authorization: "Basic dXNlcjpwYXNzd29yZA==", // user:password }, Artifact: artifact, + Capabilities: []harbor.Capability{ + { + Type: harbor.CapabilityTypeVulnerability, + }, + }, }, storeExpectation: []*mock.Expectation{ { - Method: "UpdateStatus", - Args: []interface{}{ctx, "job:123", job.Pending, []string(nil)}, + Method: "UpdateStatus", + Args: []interface{}{ + ctx, + "job:123", + job.Pending, + []string(nil), + }, ReturnArgs: []interface{}{nil}, }, { - Method: "UpdateStatus", - Args: []interface{}{ctx, "job:123", job.Failed, []string{"running trivy wrapper: out of memory"}}, + Method: "UpdateStatus", + Args: []interface{}{ + ctx, + "job:123", + job.Failed, + []string{"running trivy wrapper: out of memory"}, + }, ReturnArgs: []interface{}{nil}, }, }, @@ -118,8 +158,11 @@ func TestController_Scan(t *testing.T) { Method: "Scan", Args: []interface{}{ trivy.ImageRef{ - Name: "core.harbor.domain:443/library/mongo@sha256:917f5b7f4bef1b35ee90f03033f33a81002511c1e0767fd44276d4bd9cd2fa8e", - Auth: trivy.BasicAuth{Username: "user", Password: "password"}, + Name: "core.harbor.domain:443/library/mongo@sha256:917f5b7f4bef1b35ee90f03033f33a81002511c1e0767fd44276d4bd9cd2fa8e", + Auth: trivy.BasicAuth{ + Username: "user", + Password: "password", + }, NonSSL: false, }, trivy.ScanOption{Format: "json"}, diff --git a/pkg/scan/transformer.go b/pkg/scan/transformer.go index 0b4004e8..dab60727 100644 --- a/pkg/scan/transformer.go +++ b/pkg/scan/transformer.go @@ -47,9 +47,9 @@ func (t *transformer) Transform(req harbor.ScanRequest, source trivy.Report) har Artifact: req.Artifact, } - switch req.Scan.Parameters.SBOMMediaType { + switch lo.FromPtr(req.Capabilities[0].Parameters).MediaType { case harbor.MediaTypeSPDX, harbor.MediaTypeCycloneDX: - report.MediaType = string(req.Scan.Parameters.SBOMMediaType) + report.MediaType = string(req.Capabilities[0].Parameters.MediaType) report.SBOM = source.SBOM default: report.Vulnerabilities = t.transformVulnerabilities(source.Vulnerabilities) diff --git a/pkg/scan/transformer_test.go b/pkg/scan/transformer_test.go index ca500a38..5907618f 100644 --- a/pkg/scan/transformer_test.go +++ b/pkg/scan/transformer_test.go @@ -28,7 +28,11 @@ func TestTransformer_Transform(t *testing.T) { }) hr := tf.Transform(harbor.ScanRequest{ - Scan: harbor.Scan{Type: harbor.ScanTypeVulnerability}, + Capabilities: []harbor.Capability{ + { + Type: harbor.CapabilityTypeVulnerability, + }, + }, Artifact: harbor.Artifact{ Repository: "library/mongo", Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", diff --git a/pkg/trivy/target.go b/pkg/trivy/target.go index dd0fbcc6..1dcd4083 100644 --- a/pkg/trivy/target.go +++ b/pkg/trivy/target.go @@ -3,6 +3,7 @@ package trivy import ( "crypto/tls" "github.com/aquasecurity/harbor-scanner-trivy/pkg/etc" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/ext" "github.com/google/go-containerregistry/pkg/authn" "github.com/google/go-containerregistry/pkg/name" v1 "github.com/google/go-containerregistry/pkg/v1" @@ -22,15 +23,15 @@ const ( ) type ScanTarget struct { - v1.Image - - ref ImageRef - Type Target + img v1.Image + ref ImageRef + kind Target + filePath string // For SBOM } -func newTarget(imageRef ImageRef, config etc.Trivy) (ScanTarget, error) { +func newTarget(imageRef ImageRef, config etc.Trivy, ambassador ext.Ambassador) (ScanTarget, error) { var nameOpts []name.Option - slog.Info("newTarget", + slog.Debug("newTarget", slog.Bool("nonssl", imageRef.NonSSL), slog.Bool("insecure", config.Insecure), ) @@ -62,39 +63,42 @@ func newTarget(imageRef ImageRef, config etc.Trivy) (ScanTarget, error) { tr.TLSClientConfig = &tls.Config{InsecureSkipVerify: config.Insecure} trOpt := remote.WithTransport(tr) - img, err := remote.Image(ref, authOpt, trOpt) + img, err := ambassador.RemoteImage(ref, authOpt, trOpt) if err != nil { return ScanTarget{}, xerrors.Errorf("fetching image: %w", err) } target := ScanTarget{ - Image: img, - ref: imageRef, + img: img, + ref: imageRef, } - m, err := target.Manifest() + m, err := target.img.Manifest() if err != nil { return ScanTarget{}, xerrors.Errorf("getting image manifest: %w", err) } switch m.ArtifactType { case "application/vnd.goharbor.harbor.sbom.v1": - target.Type = TargetSBOM + target.kind = TargetSBOM + if target.filePath, err = downloadSBOM(img, config.CacheDir, ambassador); err != nil { + return ScanTarget{}, xerrors.Errorf("downloading SBOM: %w", err) + } default: - target.Type = TargetImage + target.kind = TargetImage } return target, nil } func (t ScanTarget) Name() (string, error) { - switch t.Type { + switch t.kind { case TargetSBOM: - return t.downloadSBOM() + return t.filePath, nil case TargetImage: return t.ref.Name, nil default: - return "", xerrors.Errorf("invalid target type %s", t.Type) + return "", xerrors.Errorf("invalid target type %s", t.kind) } } @@ -103,7 +107,7 @@ func (t ScanTarget) NonSSL() bool { } func (t ScanTarget) Auth() RegistryAuth { - switch t.Type { + switch t.kind { case TargetSBOM: return NoAuth{} case TargetImage: @@ -113,8 +117,18 @@ func (t ScanTarget) Auth() RegistryAuth { } } -func (t ScanTarget) downloadSBOM() (string, error) { - layers, err := t.Layers() +func (t ScanTarget) Clean() error { + switch t.kind { + case TargetSBOM: + return os.Remove(t.filePath) + default: + return nil + } +} + +// downloadSBOM downloads the SBOM from the registry and returns the path to the downloaded file. +func downloadSBOM(img v1.Image, cacheDir string, ambassador ext.Ambassador) (string, error) { + layers, err := img.Layers() if err != nil { return "", xerrors.Errorf("get image layers: %w", err) } else if len(layers) != 1 { @@ -127,7 +141,7 @@ func (t ScanTarget) downloadSBOM() (string, error) { } defer r.Close() - sbomFile, err := os.CreateTemp("", "sbom_*.json") + sbomFile, err := ambassador.TempFile(cacheDir, "sbom_*.json") if err != nil { return "", xerrors.Errorf("create temp file: %w", err) } diff --git a/pkg/trivy/wrapper.go b/pkg/trivy/wrapper.go index 564e099f..81afbe3e 100644 --- a/pkg/trivy/wrapper.go +++ b/pkg/trivy/wrapper.go @@ -5,6 +5,7 @@ import ( "fmt" "io" "log/slog" + "os" "os/exec" "strings" @@ -71,26 +72,34 @@ func (w *wrapper) Scan(imageRef ImageRef, opt ScanOption) (Report, error) { logger := slog.With(slog.String("image_ref", imageRef.Name)) logger.Debug("Started scanning") - target, err := newTarget(imageRef, w.config) + target, err := newTarget(imageRef, w.config, w.ambassador) if err != nil { return Report{}, xerrors.Errorf("creating scan target: %w", err) } + defer func() { + if err = target.Clean(); err != nil { + logger.Warn("Error while removing sbom tmp file", slog.String("err", err.Error())) + } + }() reportFile, err := w.ambassador.TempFile(w.config.ReportsDir, "scan_report_*.json") if err != nil { - return Report{}, err + return Report{}, xerrors.Errorf("creating scan report tmp file: %w", err) } logger.Debug("Saving scan report to tmp file", slog.String("path", reportFile.Name())) defer func() { + if err = reportFile.Close(); err != nil { + logger.Warn("Error while closing scan report tmp file", slog.String("err", err.Error())) + } logger.Debug("Removing scan report tmp file", slog.String("path", reportFile.Name())) - if err = w.ambassador.Remove(reportFile.Name()); err != nil { + if err = os.Remove(reportFile.Name()); err != nil { logger.Warn("Error while removing scan report tmp file", slog.String("err", err.Error())) } }() cmd, err := w.prepareScanCmd(target, reportFile.Name(), opt) if err != nil { - return Report{}, err + return Report{}, xerrors.Errorf("preparing scan command: %w", err) } logger.Debug("Exec command with args", slog.String("path", cmd.Path), @@ -120,7 +129,7 @@ func (w *wrapper) parseReport(format Format, reportFile io.Reader) (Report, erro case FormatSPDX, FormatCycloneDX: return w.parseSBOM(reportFile) } - return Report{}, fmt.Errorf("unsupported format %s", format) + return Report{}, xerrors.Errorf("unsupported format %s", format) } func (w *wrapper) parseJSONReport(reportFile io.Reader) (Report, error) { @@ -154,17 +163,23 @@ func (w *wrapper) parseSBOM(reportFile io.Reader) (Report, error) { func (w *wrapper) prepareScanCmd(target ScanTarget, outputFile string, opt ScanOption) (*exec.Cmd, error) { args := []string{ - string(target.Type), // subcommand + string(target.kind), // subcommand "--no-progress", - "--severity", w.config.Severity, - "--vuln-type", w.config.VulnType, - "--format", string(opt.Format), - "--output", outputFile, - "--cache-dir", w.config.CacheDir, - "--timeout", w.config.Timeout.String(), + "--severity", + w.config.Severity, + "--vuln-type", + w.config.VulnType, + "--format", + string(opt.Format), + "--output", + outputFile, + "--cache-dir", + w.config.CacheDir, + "--timeout", + w.config.Timeout.String(), } - if target.Type == TargetImage { + if target.kind == TargetImage { args = append(args, "--scanners", w.config.SecurityChecks) } @@ -172,12 +187,12 @@ func (w *wrapper) prepareScanCmd(target ScanTarget, outputFile string, opt ScanO args = append(args, "--ignore-unfixed") } - if w.config.SkipUpdate { + if w.config.SkipDBUpdate { args = append(args, "--skip-db-update") } if w.config.SkipJavaDBUpdate { - args = append([]string{"--skip-java-db-update"}, args...) + args = append(args, "--skip-java-db-update") } if w.config.OfflineScan { @@ -185,7 +200,7 @@ func (w *wrapper) prepareScanCmd(target ScanTarget, outputFile string, opt ScanO } if w.config.IgnorePolicy != "" { - args = append(args, "--ignore-policy") + args = append(args, "--ignore-policy", w.config.IgnorePolicy) } if w.config.DebugMode { @@ -253,9 +268,11 @@ func (w *wrapper) GetVersion() (VersionInfo, error) { func (w *wrapper) prepareVersionCmd() (*exec.Cmd, error) { args := []string{ - "--cache-dir", w.config.CacheDir, + "--cache-dir", + w.config.CacheDir, "version", - "--format", "json", + "--format", + "json", } name, err := w.ambassador.LookPath(trivyCmd) diff --git a/pkg/trivy/wrapper_test.go b/pkg/trivy/wrapper_test.go index 8cbecc27..c5dba510 100644 --- a/pkg/trivy/wrapper_test.go +++ b/pkg/trivy/wrapper_test.go @@ -2,7 +2,15 @@ package trivy import ( "encoding/json" + v1 "github.com/google/go-containerregistry/pkg/v1" + "github.com/google/go-containerregistry/pkg/v1/fake" + "github.com/google/go-containerregistry/pkg/v1/random" + "github.com/google/go-containerregistry/pkg/v1/types" + "github.com/samber/lo" + "github.com/stretchr/testify/mock" + "os" "os/exec" + "path/filepath" "testing" "time" @@ -63,20 +71,19 @@ var ( "nvd": { V2Vector: "AV:L/AC:M/Au:N/C:P/I:N/A:N", V3Vector: "CVSS:3.1/AV:L/AC:H/PR:L/UI:N/S:U/C:H/I:N/A:N", - V2Score: float32Ptr(1.9), - V3Score: float32Ptr(4.7), + V2Score: lo.ToPtr[float32](1.9), + V3Score: lo.ToPtr[float32](4.7), }, "redhat": { V2Vector: "", V3Vector: "CVSS:3.0/AV:L/AC:L/PR:L/UI:N/S:U/C:H/I:N/A:N", V2Score: nil, - V3Score: float32Ptr(5.5), + V3Score: lo.ToPtr[float32](5.5), }, }, }, }, } - expectedVersion = VersionInfo{ Version: "v0.5.2-17-g3c9af62", VulnerabilityDB: &Metadata{ @@ -87,83 +94,162 @@ var ( ) func TestWrapper_Scan(t *testing.T) { - ambassador := ext.NewMockAmbassador() - ambassador.On("Environ").Return([]string{"HTTP_PROXY=http://someproxy:7777"}) - ambassador.On("LookPath", "trivy").Return("/usr/local/bin/trivy", nil) + t.Run("vulnerability", func(t *testing.T) { + ambassador := ext.NewMockAmbassador() + ambassador.On("Environ").Return([]string{"HTTP_PROXY=http://someproxy:7777"}) + ambassador.On("LookPath", "trivy").Return("/usr/local/bin/trivy", nil) - config := etc.Trivy{ - CacheDir: "/home/scanner/.cache/trivy", - ReportsDir: "/home/scanner/.cache/reports", - DebugMode: true, - VulnType: "os,library", - SecurityChecks: "vuln", - Severity: "CRITICAL,MEDIUM", - IgnoreUnfixed: true, - IgnorePolicy: "/home/scanner/opa/policy.rego", - SkipUpdate: true, - SkipJavaDBUpdate: true, - GitHubToken: "", - Insecure: true, - Timeout: 5 * time.Minute, - } + fakeImage := &fake.FakeImage{} + fakeImage.ManifestReturns(&v1.Manifest{}, nil) + ambassador.On("RemoteImage", mock.Anything, mock.Anything).Return(fakeImage, nil) - imageRef := ImageRef{ - Name: "alpine:3.10.2", - Auth: BasicAuth{Username: "dave.loper", Password: "s3cret"}, - NonSSL: true, - } + reportsDir, cacheDir := tmpDirs(t) + config := etc.Trivy{ + CacheDir: cacheDir, + ReportsDir: reportsDir, + DebugMode: true, + VulnType: "os,library", + SecurityChecks: "vuln", + Severity: "CRITICAL,MEDIUM", + IgnoreUnfixed: true, + IgnorePolicy: "/home/scanner/opa/policy.rego", + SkipDBUpdate: true, + SkipJavaDBUpdate: true, + GitHubToken: "", + Insecure: true, + Timeout: 5 * time.Minute, + } - expectedCmdArgs := []string{ - "/usr/local/bin/trivy", - "--cache-dir", - "/home/scanner/.cache/trivy", - "--debug", - "image", - "--ignore-policy", - "/home/scanner/opa/policy.rego", - "--skip-java-db-update", - "--skip-db-update", - "--ignore-unfixed", - "--no-progress", - "--severity", - "CRITICAL,MEDIUM", - "--vuln-type", - "os,library", - "--scanners", - "vuln", - "--format", - "json", - "--output", - "/home/scanner/.cache/reports/scan_report_1234567890.json", - "alpine:3.10.2", - } + reportPath := filepath.Join(reportsDir, "scan_report_vuln.json") + require.NoError(t, os.WriteFile(reportPath, []byte(expectedReportJSON), 0644)) + ambassador.On("TempFile", reportsDir, mock.Anything).Return(os.Open(reportPath)) - expectedCmdEnvs := []string{ - "HTTP_PROXY=http://someproxy:7777", - "TRIVY_TIMEOUT=5m0s", - "TRIVY_USERNAME=dave.loper", - "TRIVY_PASSWORD=s3cret", - "TRIVY_NON_SSL=true", - "GITHUB_TOKEN=", - "TRIVY_INSECURE=true", - } + ambassador.On("RunCmd", &exec.Cmd{ + Path: "/usr/local/bin/trivy", + Env: []string{ + "HTTP_PROXY=http://someproxy:7777", + "TRIVY_USERNAME=dave.loper", + "TRIVY_PASSWORD=s3cret", + "GITHUB_TOKEN=", + }, + Args: []string{ + "/usr/local/bin/trivy", + "image", + "--no-progress", + "--severity", + "CRITICAL,MEDIUM", + "--vuln-type", + "os,library", + "--format", + "json", + "--output", + reportPath, + "--cache-dir", + cacheDir, + "--timeout", + "5m0s", + "--scanners", + "vuln", + "--ignore-unfixed", + "--skip-db-update", + "--skip-java-db-update", + "--ignore-policy", + "/home/scanner/opa/policy.rego", + "--debug", + "--insecure", + "alpine:3.10.2", + }, + }, + ).Return([]byte{}, nil) - ambassador.On("TempFile", "/home/scanner/.cache/reports", "scan_report_*.json"). - Return(ext.NewFakeFile("/home/scanner/.cache/reports/scan_report_1234567890.json", expectedReportJSON), nil) - ambassador.On("Remove", "/home/scanner/.cache/reports/scan_report_1234567890.json"). - Return(nil) - ambassador.On("RunCmd", &exec.Cmd{ - Path: "/usr/local/bin/trivy", - Env: expectedCmdEnvs, - Args: expectedCmdArgs}, - ).Return([]byte{}, nil) + imageRef := ImageRef{ + Name: "alpine:3.10.2", + Auth: BasicAuth{ + Username: "dave.loper", + Password: "s3cret", + }, + NonSSL: true, + } - report, err := NewWrapper(config, ambassador).Scan(imageRef, ScanOption{Format: FormatJSON}) + got, err := NewWrapper(config, ambassador).Scan(imageRef, ScanOption{Format: FormatJSON}) + require.NoError(t, err) + require.Equal(t, expectedReport, got) - require.NoError(t, err) - require.Equal(t, expectedReport, report) + ambassador.AssertExpectations(t) + }) - ambassador.AssertExpectations(t) + t.Run("sbom", func(t *testing.T) { + ambassador := ext.NewMockAmbassador() + ambassador.On("Environ").Return([]string{"HTTP_PROXY=http://someproxy:7777"}) + ambassador.On("LookPath", "trivy").Return("/usr/local/bin/trivy", nil) + + fakeImage := &fake.FakeImage{} + fakeImage.ManifestReturns(&v1.Manifest{ + ArtifactType: "application/vnd.goharbor.harbor.sbom.v1", + }, nil) + fakeLayer, err := random.Layer(1024, types.DockerLayer) + require.NoError(t, err, "failed to create fake layer") + fakeImage.LayersReturns([]v1.Layer{fakeLayer}, nil) + ambassador.On("RemoteImage", mock.Anything, mock.Anything).Return(fakeImage, nil) + + reportsDir, cacheDir := tmpDirs(t) + config := etc.Trivy{ + CacheDir: cacheDir, + ReportsDir: reportsDir, + SecurityChecks: "vuln", + VulnType: "library", + Severity: "CRITICAL", + SkipDBUpdate: true, + SkipJavaDBUpdate: true, + Timeout: 10 * time.Second, + } + + reportPath := filepath.Join(reportsDir, "scan_report_vuln.json") + require.NoError(t, os.WriteFile(reportPath, []byte(expectedReportJSON), 0644)) + ambassador.On("TempFile", reportsDir, mock.Anything).Return(os.Open(reportPath)) + + sbomPath := filepath.Join(cacheDir, "sbom.json") + ambassador.On("TempFile", cacheDir, mock.Anything).Return(os.Create(sbomPath)) + + ambassador.On("RunCmd", &exec.Cmd{ + Path: "/usr/local/bin/trivy", + Env: []string{ + "HTTP_PROXY=http://someproxy:7777", + }, + Args: []string{ + "/usr/local/bin/trivy", + "sbom", + "--no-progress", + "--severity", + "CRITICAL", + "--vuln-type", + "library", + "--format", + "json", + "--output", + reportPath, + "--cache-dir", + cacheDir, + "--timeout", + "10s", + "--skip-db-update", + "--skip-java-db-update", + sbomPath, + }, + }, + ).Return([]byte{}, nil) + + imageRef := ImageRef{ + Name: "alpine@sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb10", + Auth: NoAuth{}, + } + + got, err := NewWrapper(config, ambassador).Scan(imageRef, ScanOption{Format: FormatJSON}) + require.NoError(t, err) + require.Equal(t, expectedReport, got) + + ambassador.AssertExpectations(t) + }) } func TestWrapper_GetVersion(t *testing.T) { @@ -187,7 +273,8 @@ func TestWrapper_GetVersion(t *testing.T) { b, _ := json.Marshal(expectedVersion) ambassador.On("RunCmd", &exec.Cmd{ Path: "/usr/local/bin/trivy", - Args: expectedCmdArgs}, + Args: expectedCmdArgs, + }, ).Return(b, nil) vi, err := NewWrapper(config, ambassador).GetVersion() @@ -197,6 +284,12 @@ func TestWrapper_GetVersion(t *testing.T) { ambassador.AssertExpectations(t) } -func float32Ptr(f float32) *float32 { - return &f +func tmpDirs(t *testing.T) (string, string) { + tmpDir := t.TempDir() + cacheDir := filepath.Join(tmpDir, "cache") + require.NoError(t, os.MkdirAll(cacheDir, 0700)) + reportsDir := filepath.Join(tmpDir, "reports") + require.NoError(t, os.MkdirAll(reportsDir, 0700)) + + return cacheDir, reportsDir } diff --git a/test/integration/api/rest_api_test.go b/test/integration/api/rest_api_test.go index 13907f16..fbe55f99 100644 --- a/test/integration/api/rest_api_test.go +++ b/test/integration/api/rest_api_test.go @@ -41,7 +41,7 @@ func TestRestApi(t *testing.T) { }, etc.Config{ Trivy: etc.Trivy{ - SkipUpdate: false, + SkipDBUpdate: false, SkipJavaDBUpdate: false, IgnoreUnfixed: true, DebugMode: true, @@ -60,7 +60,7 @@ func TestRestApi(t *testing.T) { // given enqueuer.On("Enqueue", mock.Anything, harbor.ScanRequest{ Scan: harbor.Scan{ - Type: harbor.ScanTypeVulnerability, + Type: harbor.CapabilityTypeVulnerability, }, Registry: harbor.Registry{ URL: "https://core.harbor.domain", From 19908e095ec138af12fe74c3455b5d49588b00f0 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Tue, 9 Jan 2024 22:19:21 +0400 Subject: [PATCH 10/26] test(integration): fix Signed-off-by: knqyf263 --- test/integration/api/rest_api_test.go | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/test/integration/api/rest_api_test.go b/test/integration/api/rest_api_test.go index fbe55f99..621dcff6 100644 --- a/test/integration/api/rest_api_test.go +++ b/test/integration/api/rest_api_test.go @@ -59,8 +59,10 @@ func TestRestApi(t *testing.T) { t.Run("POST /api/v1/scan", func(t *testing.T) { // given enqueuer.On("Enqueue", mock.Anything, harbor.ScanRequest{ - Scan: harbor.Scan{ - Type: harbor.CapabilityTypeVulnerability, + Capabilities: []harbor.Capability{ + { + Type: harbor.CapabilityTypeVulnerability, + }, }, Registry: harbor.Registry{ URL: "https://core.harbor.domain", @@ -199,6 +201,7 @@ func TestRestApi(t *testing.T) { }, "capabilities": [ { + "type": "vulnerability", "consumes_mime_types": [ "application/vnd.oci.image.manifest.v1+json", "application/vnd.docker.distribution.manifest.v2+json" @@ -206,6 +209,16 @@ func TestRestApi(t *testing.T) { "produces_mime_types": [ "application/vnd.security.vulnerability.report; version=1.1" ] + }, + { + "type": "sbom", + "consumes_mime_types": [ + "application/vnd.oci.image.manifest.v1+json", + "application/vnd.docker.distribution.manifest.v2+json" + ], + "produces_mime_types": [ + "application/vnd.security.sbom.report+json; version=1.0" + ] } ], "properties": { From c175279ae5e2738afb74f095d5cd11d64679cdb4 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Thu, 11 Jan 2024 09:20:05 +0400 Subject: [PATCH 11/26] feat: add support for multiple capabilities Signed-off-by: knqyf263 --- pkg/etc/config.go | 13 -- pkg/etc/config_test.go | 33 ---- pkg/harbor/model.go | 31 +-- pkg/harbor/model_test.go | 39 +++- pkg/http/api/base_handler.go | 58 +++--- pkg/http/api/base_handler_test.go | 20 +- pkg/http/api/v1/handler.go | 69 ++++--- pkg/http/api/v1/handler_test.go | 42 ++-- pkg/job/model.go | 28 ++- pkg/mock/enqueuer.go | 5 +- pkg/mock/store.go | 12 +- pkg/mock/transformer.go | 5 +- pkg/persistence/redis/store.go | 49 +++-- pkg/persistence/store.go | 6 +- pkg/queue/enqueuer.go | 70 +++++-- pkg/queue/worker.go | 9 +- pkg/scan/controller.go | 28 +-- pkg/scan/controller_test.go | 36 ++-- pkg/scan/transformer.go | 15 +- pkg/scan/transformer_test.go | 183 +++++++++--------- .../persistence/redis/store_test.go | 23 ++- 21 files changed, 431 insertions(+), 343 deletions(-) diff --git a/pkg/etc/config.go b/pkg/etc/config.go index f1449a8a..3b106f77 100644 --- a/pkg/etc/config.go +++ b/pkg/etc/config.go @@ -6,7 +6,6 @@ import ( "strings" "time" - "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" "github.com/caarlos0/env/v6" ) @@ -107,15 +106,3 @@ func GetConfig() (Config, error) { return cfg, nil } - -func GetScannerMetadata() harbor.Scanner { - version, ok := os.LookupEnv("TRIVY_VERSION") - if !ok { - version = "Unknown" - } - return harbor.Scanner{ - Name: "Trivy", - Vendor: "Aqua Security", - Version: version, - } -} diff --git a/pkg/etc/config_test.go b/pkg/etc/config_test.go index 83ba887b..1108e4a7 100644 --- a/pkg/etc/config_test.go +++ b/pkg/etc/config_test.go @@ -5,7 +5,6 @@ import ( "testing" "time" - "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -230,38 +229,6 @@ func TestGetConfig(t *testing.T) { } } -func TestGetScannerMetadata(t *testing.T) { - testCases := []struct { - name string - envs Envs - expectedScanner harbor.Scanner - }{ - { - name: "Should return version set via env", - envs: Envs{"TRIVY_VERSION": "0.1.6"}, - expectedScanner: harbor.Scanner{ - Name: "Trivy", - Vendor: "Aqua Security", - Version: "0.1.6", - }, - }, - { - name: "Should return unknown version when it is not set via env", - expectedScanner: harbor.Scanner{ - Name: "Trivy", - Vendor: "Aqua Security", - Version: "Unknown", - }, - }, - } - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - setEnvs(t, tc.envs) - assert.Equal(t, tc.expectedScanner, GetScannerMetadata()) - }) - } -} - func setEnvs(t *testing.T, envs Envs) { for k, v := range envs { t.Setenv(k, v) diff --git a/pkg/harbor/model.go b/pkg/harbor/model.go index 5e77b9a3..75df610c 100644 --- a/pkg/harbor/model.go +++ b/pkg/harbor/model.go @@ -5,7 +5,10 @@ import ( "encoding/json" "fmt" "net/url" + "os" "time" + + "github.com/aquasecurity/harbor-scanner-trivy/pkg/http/api" ) // Severity represents the severity of a image/component in terms of vulnerability. @@ -61,19 +64,15 @@ func (s *Severity) UnmarshalJSON(b []byte) error { } type CapabilityType string -type MediaType string const ( CapabilityTypeSBOM CapabilityType = "sbom" CapabilityTypeVulnerability CapabilityType = "vulnerability" - - MediaTypeSPDX MediaType = "application/spdx+json" - MediaTypeCycloneDX MediaType = "application/vnd.cyclonedx+json" ) -var SupportedSBOMMediaTypes = []MediaType{ - MediaTypeSPDX, - MediaTypeCycloneDX, +var SupportedSBOMMediaTypes = []api.MediaType{ + api.MediaTypeSPDX, + api.MediaTypeCycloneDX, } type Registry struct { @@ -175,16 +174,22 @@ type Scanner struct { type Capability struct { Type CapabilityType `json:"type"` ConsumesMIMETypes []string `json:"consumes_mime_types"` - ProducesMIMETypes []string `json:"produces_mime_types"` + ProducesMIMETypes []api.MIMEType `json:"produces_mime_types"` Parameters *CapabilityParameters `json:"parameters,omitempty"` } type CapabilityParameters struct { - MediaType MediaType `json:"accept_media_type,omitempty"` + MediaType api.MediaType `json:"accept_media_type,omitempty"` } -// Error holds the information about an error, including metadata about its JSON structure. -type Error struct { - HTTPCode int `json:"-"` - Message string `json:"message"` +func GetScannerMetadata() Scanner { + version, ok := os.LookupEnv("TRIVY_VERSION") + if !ok { + version = "Unknown" + } + return Scanner{ + Name: "Trivy", + Vendor: "Aqua Security", + Version: version, + } } diff --git a/pkg/harbor/model_test.go b/pkg/harbor/model_test.go index 5946018b..fc995a09 100644 --- a/pkg/harbor/model_test.go +++ b/pkg/harbor/model_test.go @@ -62,8 +62,9 @@ func TestScanRequest_GetImageRef(t *testing.T) { Registry: Registry{ URL: "https://core.harbor.domain:8443", }, - Artifact: Artifact{Repository: "library/nginx", - Digest: "test:DEF", + Artifact: Artifact{ + Repository: "library/nginx", + Digest: "test:DEF", }, }, expectedImageRef: "core.harbor.domain:8443/library/nginx@test:DEF", @@ -166,3 +167,37 @@ func TestSeverity_UnmarshalJSON(t *testing.T) { } } + +func TestGetScannerMetadata(t *testing.T) { + tests := []struct { + name string + envs map[string]string + expectedScanner Scanner + }{ + { + name: "Should return version set via env", + envs: map[string]string{"TRIVY_VERSION": "0.1.6"}, + expectedScanner: Scanner{ + Name: "Trivy", + Vendor: "Aqua Security", + Version: "0.1.6", + }, + }, + { + name: "Should return unknown version when it is not set via env", + expectedScanner: Scanner{ + Name: "Trivy", + Vendor: "Aqua Security", + Version: "Unknown", + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + for k, v := range tt.envs { + t.Setenv(k, v) + } + assert.Equal(t, tt.expectedScanner, GetScannerMetadata()) + }) + } +} diff --git a/pkg/http/api/base_handler.go b/pkg/http/api/base_handler.go index e6b12836..ad7c31ac 100644 --- a/pkg/http/api/base_handler.go +++ b/pkg/http/api/base_handler.go @@ -6,8 +6,6 @@ import ( "log/slog" "net/http" "strings" - - "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" ) const ( @@ -16,55 +14,74 @@ const ( ) type MimeTypeParams map[string]string +type MediaType string + +// Error holds the information about an error, including metadata about its JSON structure. +type Error struct { + HTTPCode int `json:"-"` + Message string `json:"message"` +} var ( MimeTypeVersion = map[string]string{"version": "1.0"} - CapabilityTypeVulnerability = "vulnerability" - CapabilityTypeSBOM = "sbom" - - MimeTypeOCIImageManifest = MimeType{ + MimeTypeOCIImageManifest = MIMEType{ Type: "application", Subtype: "vnd.oci.image.manifest.v1+json", } - MimeTypeDockerImageManifestV2 = MimeType{ + MimeTypeDockerImageManifestV2 = MIMEType{ Type: "application", Subtype: "vnd.docker.distribution.manifest.v2+json", } - MimeTypeScanResponse = MimeType{ + MimeTypeScanResponse = MIMEType{ Type: "application", Subtype: "vnd.scanner.adapter.scan.response+json", Params: MimeTypeVersion, } - MimeTypeSecurityVulnerabilityReport = MimeType{ + MimeTypeSecurityVulnerabilityReport = MIMEType{ Type: "application", Subtype: "vnd.security.vulnerability.report", Params: map[string]string{"version": "1.1"}, } - MimeTypeSecuritySBOMReport = MimeType{ + MimeTypeSecuritySBOMReport = MIMEType{ Type: "application", Subtype: "vnd.security.sbom.report+json", Params: map[string]string{"version": "1.0"}, } - MimeTypeMetadata = MimeType{ + MimeTypeMetadata = MIMEType{ Type: "application", Subtype: "vnd.scanner.adapter.metadata+json", Params: MimeTypeVersion, } - MimeTypeError = MimeType{ + MimeTypeError = MIMEType{ Type: "application", Subtype: "vnd.scanner.adapter.error", Params: MimeTypeVersion, } + + MediaTypeSPDX MediaType = "application/spdx+json" + MediaTypeCycloneDX MediaType = "application/vnd.cyclonedx+json" ) -type MimeType struct { +type MIMEType struct { Type string Subtype string Params MimeTypeParams } -func (mt MimeType) String() string { +func (mt MIMEType) MarshalJSON() ([]byte, error) { + return json.Marshal(mt.String()) +} + +func (mt *MIMEType) UnmarshalJSON(b []byte) error { + var s string + if err := json.Unmarshal(b, &s); err != nil { + return err + } + return mt.Parse(s) +} + +func (mt *MIMEType) String() string { s := fmt.Sprintf("%s/%s", mt.Type, mt.Subtype) if len(mt.Params) == 0 { return s @@ -76,18 +93,13 @@ func (mt MimeType) String() string { return fmt.Sprintf("%s; %s", s, strings.Join(params, ";")) } -func (mt *MimeType) FromAcceptHeader(value string) error { +func (mt *MIMEType) Parse(value string) error { switch value { case "", "*/*", MimeTypeSecurityVulnerabilityReport.String(): mt.Type = MimeTypeSecurityVulnerabilityReport.Type mt.Subtype = MimeTypeSecurityVulnerabilityReport.Subtype mt.Params = MimeTypeSecurityVulnerabilityReport.Params return nil - case MimeTypeSecurityVulnerabilityReport.String(): - mt.Type = MimeTypeSecurityVulnerabilityReport.Type - mt.Subtype = MimeTypeSecurityVulnerabilityReport.Subtype - mt.Params = MimeTypeSecurityVulnerabilityReport.Params - return nil case MimeTypeSecuritySBOMReport.String(): mt.Type = MimeTypeSecuritySBOMReport.Type mt.Subtype = MimeTypeSecuritySBOMReport.Subtype @@ -100,7 +112,7 @@ func (mt *MimeType) FromAcceptHeader(value string) error { type BaseHandler struct { } -func (h *BaseHandler) WriteJSON(res http.ResponseWriter, data interface{}, mimeType MimeType, statusCode int) { +func (h *BaseHandler) WriteJSON(res http.ResponseWriter, data interface{}, mimeType MIMEType, statusCode int) { res.Header().Set(HeaderContentType, mimeType.String()) res.WriteHeader(statusCode) @@ -112,9 +124,9 @@ func (h *BaseHandler) WriteJSON(res http.ResponseWriter, data interface{}, mimeT } } -func (h *BaseHandler) WriteJSONError(res http.ResponseWriter, err harbor.Error) { +func (h *BaseHandler) WriteJSONError(res http.ResponseWriter, err Error) { data := struct { - Err harbor.Error `json:"error"` + Err Error `json:"error"` }{err} h.WriteJSON(res, data, MimeTypeError, err.HTTPCode) diff --git a/pkg/http/api/base_handler_test.go b/pkg/http/api/base_handler_test.go index e3df9a60..c746296a 100644 --- a/pkg/http/api/base_handler_test.go +++ b/pkg/http/api/base_handler_test.go @@ -5,24 +5,30 @@ import ( "net/http/httptest" "testing" - "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" "github.com/stretchr/testify/assert" ) func TestMimeType_String(t *testing.T) { testCases := []struct { Name string - MimeType MimeType + MimeType MIMEType ExpectedString string }{ { - Name: "A", - MimeType: MimeType{Type: "application", Subtype: "vnd.scanner.adapter.scan.request+json"}, + Name: "A", + MimeType: MIMEType{ + Type: "application", + Subtype: "vnd.scanner.adapter.scan.request+json", + }, ExpectedString: "application/vnd.scanner.adapter.scan.request+json", }, { - Name: "B", - MimeType: MimeType{Type: "application", Subtype: "vnd.scanner.adapter.scan.request+json", Params: map[string]string{"version": "1.0"}}, + Name: "B", + MimeType: MIMEType{ + Type: "application", + Subtype: "vnd.scanner.adapter.scan.request+json", + Params: map[string]string{"version": "1.0"}, + }, ExpectedString: "application/vnd.scanner.adapter.scan.request+json; version=1.0", }, } @@ -40,7 +46,7 @@ func TestBaseHandler_WriteJSONError(t *testing.T) { handler := &BaseHandler{} // when - handler.WriteJSONError(recorder, harbor.Error{ + handler.WriteJSONError(recorder, Error{ HTTPCode: http.StatusBadRequest, Message: "Invalid request", }) diff --git a/pkg/http/api/v1/handler.go b/pkg/http/api/v1/handler.go index 30c17ccc..2446229d 100644 --- a/pkg/http/api/v1/handler.go +++ b/pkg/http/api/v1/handler.go @@ -82,7 +82,7 @@ func (h *requestHandler) AcceptScanRequest(res http.ResponseWriter, req *http.Re var scanRequest harbor.ScanRequest if err := json.NewDecoder(req.Body).Decode(&scanRequest); err != nil { slog.Error("Error while unmarshalling scan request", slog.String("err", err.Error())) - h.WriteJSONError(res, harbor.Error{ + h.WriteJSONError(res, api.Error{ HTTPCode: http.StatusBadRequest, Message: fmt.Sprintf("unmarshalling scan request: %s", err.Error()), }) @@ -102,49 +102,49 @@ func (h *requestHandler) AcceptScanRequest(res http.ResponseWriter, req *http.Re }) } - scanJob, err := h.enqueuer.Enqueue(req.Context(), scanRequest) + scanJobID, err := h.enqueuer.Enqueue(req.Context(), scanRequest) if err != nil { slog.Error("Error while enqueuing scan job", slog.String("err", err.Error())) - h.WriteJSONError(res, harbor.Error{ + h.WriteJSONError(res, api.Error{ HTTPCode: http.StatusInternalServerError, Message: fmt.Sprintf("enqueuing scan job: %s", err.Error()), }) return } - scanResponse := harbor.ScanResponse{ID: scanJob.ID} + scanResponse := harbor.ScanResponse{ID: scanJobID} h.WriteJSON(res, scanResponse, api.MimeTypeScanResponse, http.StatusAccepted) } -func (h *requestHandler) ValidateScanRequest(req harbor.ScanRequest) *harbor.Error { +func (h *requestHandler) ValidateScanRequest(req harbor.ScanRequest) *api.Error { if err := h.validateCapabilities(req.Capabilities); err != nil { return err } if req.Registry.URL == "" { - return &harbor.Error{ + return &api.Error{ HTTPCode: http.StatusUnprocessableEntity, Message: "missing registry.url", } } if _, err := url.ParseRequestURI(req.Registry.URL); err != nil { - return &harbor.Error{ + return &api.Error{ HTTPCode: http.StatusUnprocessableEntity, Message: "invalid registry.url", } } if req.Artifact.Repository == "" { - return &harbor.Error{ + return &api.Error{ HTTPCode: http.StatusUnprocessableEntity, Message: "missing artifact.repository", } } if req.Artifact.Digest == "" { - return &harbor.Error{ + return &api.Error{ HTTPCode: http.StatusUnprocessableEntity, Message: "missing artifact.digest", } @@ -153,10 +153,10 @@ func (h *requestHandler) ValidateScanRequest(req harbor.ScanRequest) *harbor.Err return nil } -func (h *requestHandler) validateCapabilities(capabilities []harbor.Capability) *harbor.Error { +func (h *requestHandler) validateCapabilities(capabilities []harbor.Capability) *api.Error { for _, c := range capabilities { if c.Type != harbor.CapabilityTypeVulnerability && c.Type != harbor.CapabilityTypeSBOM { - return &harbor.Error{ + return &api.Error{ HTTPCode: http.StatusUnprocessableEntity, Message: "invalid scan type", } @@ -164,7 +164,7 @@ func (h *requestHandler) validateCapabilities(capabilities []harbor.Capability) if c.Type == harbor.CapabilityTypeSBOM && !slices.Contains(harbor.SupportedSBOMMediaTypes, c.Parameters.MediaType) { - return &harbor.Error{ + return &api.Error{ HTTPCode: http.StatusUnprocessableEntity, Message: "invalid SBOM media type", } @@ -174,10 +174,10 @@ func (h *requestHandler) validateCapabilities(capabilities []harbor.Capability) } func (h *requestHandler) GetScanReport(res http.ResponseWriter, req *http.Request) { - var reportMimeType api.MimeType + var reportMIMEType api.MIMEType - if err := reportMimeType.FromAcceptHeader(req.Header.Get(api.HeaderAccept)); err != nil { - h.WriteJSONError(res, harbor.Error{ + if err := reportMIMEType.Parse(req.Header.Get(api.HeaderAccept)); err != nil { + h.WriteJSONError(res, api.Error{ HTTPCode: http.StatusUnsupportedMediaType, Message: fmt.Sprintf("unsupported media type %s", req.Header.Get(api.HeaderAccept)), }) @@ -188,19 +188,23 @@ func (h *requestHandler) GetScanReport(res http.ResponseWriter, req *http.Reques scanJobID, ok := vars[pathVarScanRequestID] if !ok { slog.Error("Error while parsing `scan_request_id` path variable") - h.WriteJSONError(res, harbor.Error{ + h.WriteJSONError(res, api.Error{ HTTPCode: http.StatusBadRequest, Message: "missing scan_request_id", }) return } - reqLog := slog.With(slog.String("scan_job_id", scanJobID)) + reqLog := slog.With(slog.String("scan_job_id", scanJobID), slog.String("mime_type", reportMIMEType.String())) - scanJob, err := h.store.Get(req.Context(), scanJobID) + scanJob, err := h.store.Get(req.Context(), job.ScanJobKey{ + ID: scanJobID, + MIMEType: reportMIMEType, + MediaType: "", // TODO: Harbor should pass MediaType + }) if err != nil { reqLog.Error("Error while getting scan job") - h.WriteJSONError(res, harbor.Error{ + h.WriteJSONError(res, api.Error{ HTTPCode: http.StatusInternalServerError, Message: fmt.Sprintf("getting scan job: %v", err), }) @@ -209,7 +213,7 @@ func (h *requestHandler) GetScanReport(res http.ResponseWriter, req *http.Reques if scanJob == nil { reqLog.Error("Cannot find scan job") - h.WriteJSONError(res, harbor.Error{ + h.WriteJSONError(res, api.Error{ HTTPCode: http.StatusNotFound, Message: fmt.Sprintf("cannot find scan job: %v", scanJobID), }) @@ -227,7 +231,7 @@ func (h *requestHandler) GetScanReport(res http.ResponseWriter, req *http.Reques if scanJob.Status == job.Failed { scanJobLog.Error("Scan job failed", slog.String("err", scanJob.Error)) - h.WriteJSONError(res, harbor.Error{ + h.WriteJSONError(res, api.Error{ HTTPCode: http.StatusInternalServerError, Message: scanJob.Error, }) @@ -236,14 +240,14 @@ func (h *requestHandler) GetScanReport(res http.ResponseWriter, req *http.Reques if scanJob.Status != job.Finished { scanJobLog.Error("Unexpected scan job status") - h.WriteJSONError(res, harbor.Error{ + h.WriteJSONError(res, api.Error{ HTTPCode: http.StatusInternalServerError, - Message: fmt.Sprintf("unexpected status %v of scan job %v", scanJob.Status, scanJob.ID), + Message: fmt.Sprintf("unexpected status %v of scan job %v", scanJob.Status, scanJob.Key.ID), }) return } - h.WriteJSON(res, scanJob.Report, reportMimeType, http.StatusOK) + h.WriteJSON(res, scanJob.Report, reportMIMEType, http.StatusOK) } func (h *requestHandler) GetMetadata(res http.ResponseWriter, _ *http.Request) { @@ -285,7 +289,7 @@ func (h *requestHandler) GetMetadata(res http.ResponseWriter, _ *http.Request) { } metadata := &harbor.ScannerAdapterMetadata{ - Scanner: etc.GetScannerMetadata(), + Scanner: harbor.GetScannerMetadata(), Capabilities: []harbor.Capability{ { Type: harbor.CapabilityTypeVulnerability, @@ -293,8 +297,8 @@ func (h *requestHandler) GetMetadata(res http.ResponseWriter, _ *http.Request) { api.MimeTypeOCIImageManifest.String(), api.MimeTypeDockerImageManifestV2.String(), }, - ProducesMIMETypes: []string{ - api.MimeTypeSecurityVulnerabilityReport.String(), + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecurityVulnerabilityReport, }, }, { @@ -303,8 +307,8 @@ func (h *requestHandler) GetMetadata(res http.ResponseWriter, _ *http.Request) { api.MimeTypeOCIImageManifest.String(), api.MimeTypeDockerImageManifestV2.String(), }, - ProducesMIMETypes: []string{ - api.MimeTypeSecuritySBOMReport.String(), + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecuritySBOMReport, }, }, }, @@ -320,10 +324,3 @@ func (h *requestHandler) GetHealthy(res http.ResponseWriter, req *http.Request) func (h *requestHandler) GetReady(res http.ResponseWriter, req *http.Request) { res.WriteHeader(http.StatusOK) } - -func getHeader[T ~string](header http.Header, key string, defaultValue T) T { - if v := header.Get(key); v != "" { - return T(v) - } - return defaultValue -} diff --git a/pkg/http/api/v1/handler_test.go b/pkg/http/api/v1/handler_test.go index 5c97d8a1..5801478f 100644 --- a/pkg/http/api/v1/handler_test.go +++ b/pkg/http/api/v1/handler_test.go @@ -23,7 +23,7 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { testCases := []struct { Name string Request harbor.ScanRequest - ExpectedError *harbor.Error + ExpectedError *api.Error }{ { Name: "Should return error when Registry URL is blank", @@ -32,7 +32,7 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { {Type: harbor.CapabilityTypeVulnerability}, }, }, - ExpectedError: &harbor.Error{ + ExpectedError: &api.Error{ HTTPCode: http.StatusUnprocessableEntity, Message: "missing registry.url", }, @@ -49,7 +49,7 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { URL: "INVALID URL", }, }, - ExpectedError: &harbor.Error{ + ExpectedError: &api.Error{ HTTPCode: http.StatusUnprocessableEntity, Message: "invalid registry.url", }, @@ -66,7 +66,7 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { URL: "https://core.harbor.domain", }, }, - ExpectedError: &harbor.Error{ + ExpectedError: &api.Error{ HTTPCode: http.StatusUnprocessableEntity, Message: "missing artifact.repository", }, @@ -86,7 +86,7 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { Repository: "library/mongo", }, }, - ExpectedError: &harbor.Error{ + ExpectedError: &api.Error{ HTTPCode: http.StatusUnprocessableEntity, Message: "missing artifact.digest", }, @@ -146,7 +146,7 @@ func TestRequestHandler_AcceptScanRequest(t *testing.T) { validScanRequest, }, ReturnArgs: []interface{}{ - job.ScanJob{ID: "job:123"}, + "job:123", nil, }, }, @@ -186,7 +186,7 @@ func TestRequestHandler_AcceptScanRequest(t *testing.T) { validScanRequest, }, ReturnArgs: []interface{}{ - job.ScanJob{}, + "", errors.New("queue is down"), }, }, @@ -227,6 +227,10 @@ func TestRequestHandler_AcceptScanRequest(t *testing.T) { func TestRequestHandler_GetScanReport(t *testing.T) { now := time.Now() + scanJobKey := job.ScanJobKey{ + ID: "job:123", + MIMEType: api.MimeTypeSecurityVulnerabilityReport, + } testCases := []struct { name string @@ -241,7 +245,7 @@ func TestRequestHandler_GetScanReport(t *testing.T) { Method: "Get", Args: []interface{}{ mock.Anything, - "job:123", + scanJobKey, }, ReturnArgs: []interface{}{ &job.ScanJob{}, @@ -262,7 +266,7 @@ func TestRequestHandler_GetScanReport(t *testing.T) { Method: "Get", Args: []interface{}{ mock.Anything, - "job:123", + scanJobKey, }, ReturnArgs: []interface{}{ (*job.ScanJob)(nil), @@ -283,11 +287,11 @@ func TestRequestHandler_GetScanReport(t *testing.T) { Method: "Get", Args: []interface{}{ mock.Anything, - "job:123", + scanJobKey, }, ReturnArgs: []interface{}{ &job.ScanJob{ - ID: "job:123", + Key: scanJobKey, Status: job.Queued, }, nil, @@ -301,11 +305,11 @@ func TestRequestHandler_GetScanReport(t *testing.T) { Method: "Get", Args: []interface{}{ mock.Anything, - "job:123", + scanJobKey, }, ReturnArgs: []interface{}{ &job.ScanJob{ - ID: "job:123", + Key: scanJobKey, Status: job.Pending, }, nil, @@ -319,11 +323,11 @@ func TestRequestHandler_GetScanReport(t *testing.T) { Method: "Get", Args: []interface{}{ mock.Anything, - "job:123", + scanJobKey, }, ReturnArgs: []interface{}{ &job.ScanJob{ - ID: "job:123", + Key: scanJobKey, Status: job.Failed, Error: "queue worker failed", }, @@ -344,11 +348,11 @@ func TestRequestHandler_GetScanReport(t *testing.T) { Method: "Get", Args: []interface{}{ mock.Anything, - "job:123", + scanJobKey, }, ReturnArgs: []interface{}{ &job.ScanJob{ - ID: "job:123", + Key: scanJobKey, Status: 666, Error: "queue worker failed", }, @@ -369,11 +373,11 @@ func TestRequestHandler_GetScanReport(t *testing.T) { Method: "Get", Args: []interface{}{ mock.Anything, - "job:123", + scanJobKey, }, ReturnArgs: []interface{}{ &job.ScanJob{ - ID: "job:123", + Key: scanJobKey, Status: job.Finished, Report: harbor.ScanReport{ GeneratedAt: now, diff --git a/pkg/job/model.go b/pkg/job/model.go index 89e5eafb..7f1e3c65 100644 --- a/pkg/job/model.go +++ b/pkg/job/model.go @@ -1,7 +1,10 @@ package job import ( + "fmt" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/http/api" ) type ScanJobStatus int @@ -17,12 +20,33 @@ func (s ScanJobStatus) String() string { if s < 0 || s > 3 { return "Unknown" } - return [...]string{"Queued", "Pending", "Finished", "Failed"}[s] + return [...]string{ + "Queued", + "Pending", + "Finished", + "Failed", + }[s] +} + +// ScanJobKey uniquely identifies a scan job. +// If MIMEType indicates SBOM, MediaType is not empty. +type ScanJobKey struct { + ID string `json:"id"` + MIMEType api.MIMEType `json:"mime_type"` + MediaType api.MediaType `json:"media_type"` // it can be empty +} + +func (s *ScanJobKey) String() string { + return fmt.Sprintf("%s:%s", s.ID, s.MIMEType.String()) // TODO: add MediaType } type ScanJob struct { - ID string `json:"id"` + Key ScanJobKey `json:"key"` // Must be unique Status ScanJobStatus `json:"status"` Error string `json:"error"` Report harbor.ScanReport `json:"report"` } + +func (s *ScanJob) ID() string { + return s.Key.String() +} diff --git a/pkg/mock/enqueuer.go b/pkg/mock/enqueuer.go index 612f4a1a..e3176179 100644 --- a/pkg/mock/enqueuer.go +++ b/pkg/mock/enqueuer.go @@ -3,7 +3,6 @@ package mock import ( "context" "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" - "github.com/aquasecurity/harbor-scanner-trivy/pkg/job" "github.com/stretchr/testify/mock" ) @@ -15,7 +14,7 @@ func NewEnqueuer() *Enqueuer { return &Enqueuer{} } -func (em *Enqueuer) Enqueue(ctx context.Context, request harbor.ScanRequest) (job.ScanJob, error) { +func (em *Enqueuer) Enqueue(ctx context.Context, request harbor.ScanRequest) (string, error) { args := em.Called(ctx, request) - return args.Get(0).(job.ScanJob), args.Error(1) + return args.String(0), args.Error(1) } diff --git a/pkg/mock/store.go b/pkg/mock/store.go index 8b0fc271..f6d7426a 100644 --- a/pkg/mock/store.go +++ b/pkg/mock/store.go @@ -20,17 +20,17 @@ func (s *Store) Create(ctx context.Context, scanJob job.ScanJob) error { return args.Error(0) } -func (s *Store) Get(ctx context.Context, scanJobID string) (*job.ScanJob, error) { - args := s.Called(ctx, scanJobID) +func (s *Store) Get(ctx context.Context, scanJobKey job.ScanJobKey) (*job.ScanJob, error) { + args := s.Called(ctx, scanJobKey) return args.Get(0).(*job.ScanJob), args.Error(1) } -func (s *Store) UpdateStatus(ctx context.Context, scanJobID string, newStatus job.ScanJobStatus, error ...string) error { - args := s.Called(ctx, scanJobID, newStatus, error) +func (s *Store) UpdateStatus(ctx context.Context, scanJobKey job.ScanJobKey, newStatus job.ScanJobStatus, error ...string) error { + args := s.Called(ctx, scanJobKey, newStatus, error) return args.Error(0) } -func (s *Store) UpdateReport(ctx context.Context, scanJobID string, report harbor.ScanReport) error { - args := s.Called(ctx, scanJobID, report) +func (s *Store) UpdateReport(ctx context.Context, scanJobKey job.ScanJobKey, report harbor.ScanReport) error { + args := s.Called(ctx, scanJobKey, report) return args.Error(0) } diff --git a/pkg/mock/transformer.go b/pkg/mock/transformer.go index bbfe80b2..ccbf1134 100644 --- a/pkg/mock/transformer.go +++ b/pkg/mock/transformer.go @@ -2,6 +2,7 @@ package mock import ( "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/http/api" "github.com/aquasecurity/harbor-scanner-trivy/pkg/trivy" "github.com/stretchr/testify/mock" ) @@ -14,7 +15,7 @@ func NewTransformer() *Transformer { return &Transformer{} } -func (t *Transformer) Transform(req harbor.ScanRequest, source trivy.Report) harbor.ScanReport { - args := t.Called(req, source) +func (t *Transformer) Transform(mediaType api.MediaType, req harbor.ScanRequest, source trivy.Report) harbor.ScanReport { + args := t.Called(mediaType, req, source) return args.Get(0).(harbor.ScanReport) } diff --git a/pkg/persistence/redis/store.go b/pkg/persistence/redis/store.go index e0c43d61..822b4d9c 100644 --- a/pkg/persistence/redis/store.go +++ b/pkg/persistence/redis/store.go @@ -21,7 +21,10 @@ type store struct { } func NewStore(cfg etc.RedisStore, rdb *redis.Client) persistence.Store { - return &store{cfg: cfg, rdb: rdb} + return &store{ + cfg: cfg, + rdb: rdb, + } } func (s *store) Create(ctx context.Context, scanJob job.ScanJob) error { @@ -30,10 +33,10 @@ func (s *store) Create(ctx context.Context, scanJob job.ScanJob) error { return xerrors.Errorf("marshalling scan job: %w", err) } - key := s.keyForScanJob(scanJob.ID) + key := s.keyForScanJob(scanJob.Key) - slog.Debug("Saving scan job", - slog.String("scan_job_id", scanJob.ID), + logger := storeLogger(scanJob.Key) + logger.Debug("Saving scan job", slog.String("scan_job_status", scanJob.Status.String()), slog.String("redis_key", key), slog.Duration("expire", s.cfg.ScanJobTTL), @@ -52,10 +55,10 @@ func (s *store) update(ctx context.Context, scanJob job.ScanJob) error { return xerrors.Errorf("marshalling scan job: %w", err) } - key := s.keyForScanJob(scanJob.ID) + key := s.keyForScanJob(scanJob.Key) - slog.Debug("Updating scan job", - slog.String("scan_job_id", scanJob.ID), + logger := storeLogger(scanJob.Key) + logger.Debug("Updating scan job", slog.String("scan_job_status", scanJob.Status.String()), slog.String("redis_key", key), slog.Duration("expire", s.cfg.ScanJobTTL), @@ -68,8 +71,8 @@ func (s *store) update(ctx context.Context, scanJob job.ScanJob) error { return nil } -func (s *store) Get(ctx context.Context, scanJobID string) (*job.ScanJob, error) { - key := s.keyForScanJob(scanJobID) +func (s *store) Get(ctx context.Context, scanJobKey job.ScanJobKey) (*job.ScanJob, error) { + key := s.keyForScanJob(scanJobKey) value, err := s.rdb.Get(ctx, key).Result() if errors.Is(err, redis.Nil) { return nil, nil @@ -85,14 +88,13 @@ func (s *store) Get(ctx context.Context, scanJobID string) (*job.ScanJob, error) return &scanJob, nil } -func (s *store) UpdateStatus(ctx context.Context, scanJobID string, newStatus job.ScanJobStatus, error ...string) error { - slog.Debug("Updating status for scan job", slog.String("scan_job_id", scanJobID), - slog.String("new_status", newStatus.String()), - ) +func (s *store) UpdateStatus(ctx context.Context, scanJobKey job.ScanJobKey, newStatus job.ScanJobStatus, error ...string) error { + logger := storeLogger(scanJobKey) + logger.Debug("Updating status for scan job", slog.String("new_status", newStatus.String())) - scanJob, err := s.Get(ctx, scanJobID) + scanJob, err := s.Get(ctx, scanJobKey) if scanJob == nil { - return xerrors.Errorf("scan job %s not found", scanJobID) + return xerrors.Errorf("scan job (%s) not found", scanJobKey) } else if err != nil { return err } @@ -105,10 +107,11 @@ func (s *store) UpdateStatus(ctx context.Context, scanJobID string, newStatus jo return s.update(ctx, *scanJob) } -func (s *store) UpdateReport(ctx context.Context, scanJobID string, report harbor.ScanReport) error { - slog.Debug("Updating reports for scan job", slog.String("scan_job_id", scanJobID)) +func (s *store) UpdateReport(ctx context.Context, scanJobKey job.ScanJobKey, report harbor.ScanReport) error { + logger := storeLogger(scanJobKey) + logger.Debug("Updating reports for scan job") - scanJob, err := s.Get(ctx, scanJobID) + scanJob, err := s.Get(ctx, scanJobKey) if err != nil { return err } @@ -117,6 +120,12 @@ func (s *store) UpdateReport(ctx context.Context, scanJobID string, report harbo return s.update(ctx, *scanJob) } -func (s *store) keyForScanJob(scanJobID string) string { - return fmt.Sprintf("%s:scan-job:%s", s.cfg.Namespace, scanJobID) +func (s *store) keyForScanJob(scanJobKey job.ScanJobKey) string { + return fmt.Sprintf("%s:scan-job:%s", s.cfg.Namespace, scanJobKey.String()) +} + +func storeLogger(scanJobKey job.ScanJobKey) *slog.Logger { + return slog.With( + slog.String("scan_job_id", scanJobKey.ID), + slog.String("mime_type", scanJobKey.MIMEType.String())) } diff --git a/pkg/persistence/store.go b/pkg/persistence/store.go index 41e4ffa7..ce335ae8 100644 --- a/pkg/persistence/store.go +++ b/pkg/persistence/store.go @@ -8,7 +8,7 @@ import ( type Store interface { Create(ctx context.Context, scanJob job.ScanJob) error - Get(ctx context.Context, scanJobID string) (*job.ScanJob, error) - UpdateStatus(ctx context.Context, scanJobID string, newStatus job.ScanJobStatus, error ...string) error - UpdateReport(ctx context.Context, scanJobID string, report harbor.ScanReport) error + Get(ctx context.Context, scanJobKey job.ScanJobKey) (*job.ScanJob, error) + UpdateStatus(ctx context.Context, scanJobKey job.ScanJobKey, newStatus job.ScanJobStatus, error ...string) error + UpdateReport(ctx context.Context, scanJobKey job.ScanJobKey, report harbor.ScanReport) error } diff --git a/pkg/queue/enqueuer.go b/pkg/queue/enqueuer.go index 24d66bd0..808b816a 100644 --- a/pkg/queue/enqueuer.go +++ b/pkg/queue/enqueuer.go @@ -9,6 +9,7 @@ import ( "log/slog" "github.com/redis/go-redis/v9" + "github.com/samber/lo" "golang.org/x/xerrors" "github.com/aquasecurity/harbor-scanner-trivy/pkg/etc" @@ -20,7 +21,7 @@ import ( const scanArtifactJobName = "scan_artifact" type Enqueuer interface { - Enqueue(ctx context.Context, request harbor.ScanRequest) (job.ScanJob, error) + Enqueue(ctx context.Context, request harbor.ScanRequest) (string, error) } type enqueuer struct { @@ -31,10 +32,14 @@ type enqueuer struct { type Job struct { Name string - ID string + Key job.ScanJobKey Args Args } +func (s *Job) ID() string { + return s.Key.String() +} + type Args struct { ScanRequest *harbor.ScanRequest `json:",omitempty"` } @@ -47,39 +52,64 @@ func NewEnqueuer(config etc.JobQueue, rdb *redis.Client, store persistence.Store } } -func (e *enqueuer) Enqueue(ctx context.Context, request harbor.ScanRequest) (job.ScanJob, error) { - slog.Debug("Enqueueing scan job") - j := Job{ - Name: scanArtifactJobName, - ID: makeIdentifier(), - Args: Args{ - ScanRequest: &request, - }, +func (e *enqueuer) Enqueue(ctx context.Context, request harbor.ScanRequest) (string, error) { + if len(request.Capabilities) == 0 { + return "", xerrors.Errorf("no capabilities provided") } - scanJob := job.ScanJob{ - ID: j.ID, - Status: job.Queued, + jobID := makeIdentifier() + + for _, c := range request.Capabilities { + mediaType := lo.FromPtr(c.Parameters).MediaType + for _, m := range c.ProducesMIMETypes { + jobKey := job.ScanJobKey{ + ID: jobID, + MIMEType: m, + MediaType: mediaType, + } + + j := Job{ + Name: scanArtifactJobName, + Key: jobKey, + Args: Args{ + ScanRequest: &request, + }, + } + scanJob := job.ScanJob{ + Key: jobKey, + Status: job.Queued, + } + + if err := e.enqueue(ctx, j, scanJob); err != nil { + return "", xerrors.Errorf("enqueuing scan job: %v", err) + } + } } + return jobID, nil +} + +func (e *enqueuer) enqueue(ctx context.Context, j Job, scanJob job.ScanJob) error { + logger := slog.With(slog.String("job_id", j.Key.ID), slog.String("mime_type", j.Key.MIMEType.String())) + logger.Debug("Enqueueing scan job") + // Save the job status to Redis if err := e.store.Create(ctx, scanJob); err != nil { - return job.ScanJob{}, xerrors.Errorf("creating scan job %v", err) + return xerrors.Errorf("creating scan job %v", err) } b, err := json.Marshal(j) if err != nil { - return job.ScanJob{}, xerrors.Errorf("marshalling scan request: %v", err) + return xerrors.Errorf("marshalling scan request: %v", err) } // Publish the job to the workers if err = e.rdb.Publish(ctx, e.redisJobChannel(), b).Err(); err != nil { - return job.ScanJob{}, xerrors.Errorf("enqueuing scan artifact job: %v", err) + return xerrors.Errorf("enqueuing scan artifact job: %v", err) } - slog.Debug("Successfully enqueued scan job", slog.String("job_id", j.ID)) - - return scanJob, nil + logger.Debug("Successfully enqueued scan job") + return nil } func (e *enqueuer) redisJobChannel() string { @@ -96,5 +126,5 @@ func makeIdentifier() string { } func redisJobChannel(namespace string) string { - return namespace + "jobs:" + scanArtifactJobName + return namespace + ":jobs:" + scanArtifactJobName } diff --git a/pkg/queue/worker.go b/pkg/queue/worker.go index 18de8a38..a9469f3d 100644 --- a/pkg/queue/worker.go +++ b/pkg/queue/worker.go @@ -7,7 +7,6 @@ import ( "time" "github.com/redis/go-redis/v9" - "github.com/samber/lo" "golang.org/x/xerrors" "github.com/aquasecurity/harbor-scanner-trivy/pkg/etc" @@ -83,16 +82,16 @@ func (w *worker) scanArtifact(ctx context.Context, msg *redis.Message) error { } // Lock the job so that other workers won't process it. - nx, err := w.rdb.SetNX(ctx, redisLockKey(w.namespace, job.ID), "", 5*time.Minute).Result() + nx, err := w.rdb.SetNX(ctx, redisLockKey(w.namespace, job.ID()), "", 5*time.Minute).Result() if err != nil { return xerrors.Errorf("redis lock: %w", err) } else if !nx { - slog.Debug("Skip the locked job", slog.String("scan_job_id", job.ID)) + slog.Debug("Skip the locked job", slog.String("scan_job_id", job.Key.ID)) return nil } - slog.Debug("Executing enqueued scan job", slog.String("scan_job_id", job.ID)) - return w.controller.Scan(ctx, job.ID, lo.FromPtr(job.Args.ScanRequest)) + slog.Debug("Executing enqueued scan job", slog.String("scan_job_id", job.Key.ID)) + return w.controller.Scan(ctx, job.Key, job.Args.ScanRequest) } func redisLockKey(namespace, jobID string) string { diff --git a/pkg/scan/controller.go b/pkg/scan/controller.go index cf6128e9..43393bf8 100644 --- a/pkg/scan/controller.go +++ b/pkg/scan/controller.go @@ -4,18 +4,19 @@ import ( "context" "encoding/base64" "github.com/samber/lo" + "golang.org/x/xerrors" "log/slog" "strings" "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/http/api" "github.com/aquasecurity/harbor-scanner-trivy/pkg/job" "github.com/aquasecurity/harbor-scanner-trivy/pkg/persistence" "github.com/aquasecurity/harbor-scanner-trivy/pkg/trivy" - "golang.org/x/xerrors" ) type Controller interface { - Scan(ctx context.Context, scanJobID string, request harbor.ScanRequest) error + Scan(ctx context.Context, scanJobKey job.ScanJobKey, request *harbor.ScanRequest) error } type controller struct { @@ -32,24 +33,24 @@ func NewController(store persistence.Store, wrapper trivy.Wrapper, transformer T } } -func (c *controller) Scan(ctx context.Context, scanJobID string, request harbor.ScanRequest) error { - if err := c.scan(ctx, scanJobID, request); err != nil { +func (c *controller) Scan(ctx context.Context, scanJobKey job.ScanJobKey, request *harbor.ScanRequest) error { + if err := c.scan(ctx, scanJobKey, request); err != nil { slog.Error("Scan failed", slog.String("err", err.Error())) - if err = c.store.UpdateStatus(ctx, scanJobID, job.Failed, err.Error()); err != nil { + if err = c.store.UpdateStatus(ctx, scanJobKey, job.Failed, err.Error()); err != nil { return xerrors.Errorf("updating scan job as failed: %v", err) } } return nil } -func (c *controller) scan(ctx context.Context, scanJobID string, req harbor.ScanRequest) (err error) { +func (c *controller) scan(ctx context.Context, scanJobKey job.ScanJobKey, req *harbor.ScanRequest) (err error) { defer func() { if r := recover(); r != nil { err = r.(error) } }() - err = c.store.UpdateStatus(ctx, scanJobID, job.Pending) + err = c.store.UpdateStatus(ctx, scanJobKey, job.Pending) if err != nil { return xerrors.Errorf("updating scan job status: %v", err) } @@ -71,17 +72,18 @@ func (c *controller) scan(ctx context.Context, scanJobID string, req harbor.Scan } scanReport, err := c.wrapper.Scan(ref, trivy.ScanOption{ - Format: determineFormat(lo.FromPtr(req.Capabilities[0].Parameters).MediaType), + Format: determineFormat(scanJobKey.MediaType), }) if err != nil { return xerrors.Errorf("running trivy wrapper: %v", err) } - if err = c.store.UpdateReport(ctx, scanJobID, c.transformer.Transform(req, scanReport)); err != nil { + harborScanReport := c.transformer.Transform(scanJobKey.MediaType, lo.FromPtr(req), scanReport) + if err = c.store.UpdateReport(ctx, scanJobKey, harborScanReport); err != nil { return xerrors.Errorf("saving scan report: %v", err) } - if err = c.store.UpdateStatus(ctx, scanJobID, job.Finished); err != nil { + if err = c.store.UpdateStatus(ctx, scanJobKey, job.Finished); err != nil { return xerrors.Errorf("updating scan job status: %v", err) } @@ -123,11 +125,11 @@ func (c *controller) decodeBasicAuth(value string) (auth trivy.RegistryAuth, err return } -func determineFormat(m harbor.MediaType) trivy.Format { +func determineFormat(m api.MediaType) trivy.Format { switch m { - case harbor.MediaTypeSPDX: + case api.MediaTypeSPDX: return trivy.FormatSPDX - case harbor.MediaTypeCycloneDX: + case api.MediaTypeCycloneDX: return trivy.FormatCycloneDX default: return trivy.FormatJSON diff --git a/pkg/scan/controller_test.go b/pkg/scan/controller_test.go index eb630ceb..0abf72ac 100644 --- a/pkg/scan/controller_test.go +++ b/pkg/scan/controller_test.go @@ -5,17 +5,22 @@ import ( "fmt" "testing" + "github.com/stretchr/testify/assert" + "golang.org/x/xerrors" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/http/api" "github.com/aquasecurity/harbor-scanner-trivy/pkg/job" "github.com/aquasecurity/harbor-scanner-trivy/pkg/mock" "github.com/aquasecurity/harbor-scanner-trivy/pkg/trivy" - "github.com/stretchr/testify/assert" - "golang.org/x/xerrors" ) var capabilities = []harbor.Capability{ { Type: harbor.CapabilityTypeVulnerability, + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecurityVulnerabilityReport, + }, }, } @@ -25,13 +30,17 @@ func TestController_Scan(t *testing.T) { Repository: "library/mongo", Digest: "sha256:917f5b7f4bef1b35ee90f03033f33a81002511c1e0767fd44276d4bd9cd2fa8e", } + jobKey := job.ScanJobKey{ + ID: "job:123", + MIMEType: api.MimeTypeSecurityVulnerabilityReport, + } trivyReport := trivy.Report{} harborReport := harbor.ScanReport{} testCases := []struct { name string - scanJobID string + scanJobKey job.ScanJobKey scanRequest harbor.ScanRequest storeExpectation []*mock.Expectation wrapperExpectation *mock.Expectation @@ -40,8 +49,8 @@ func TestController_Scan(t *testing.T) { expectedError error }{ { - name: fmt.Sprintf("Should update job status to %s when everything is fine", job.Finished.String()), - scanJobID: "job:123", + name: fmt.Sprintf("Should update job status to %s when everything is fine", job.Finished.String()), + scanJobKey: jobKey, scanRequest: harbor.ScanRequest{ Registry: harbor.Registry{ URL: "https://core.harbor.domain", @@ -55,7 +64,7 @@ func TestController_Scan(t *testing.T) { Method: "UpdateStatus", Args: []interface{}{ ctx, - "job:123", + jobKey, job.Pending, []string(nil), }, @@ -65,7 +74,7 @@ func TestController_Scan(t *testing.T) { Method: "UpdateReport", Args: []interface{}{ ctx, - "job:123", + jobKey, harborReport, }, ReturnArgs: []interface{}{nil}, @@ -74,7 +83,7 @@ func TestController_Scan(t *testing.T) { Method: "UpdateStatus", Args: []interface{}{ ctx, - "job:123", + jobKey, job.Finished, []string(nil), }, @@ -102,6 +111,7 @@ func TestController_Scan(t *testing.T) { transformerExpectation: &mock.Expectation{ Method: "Transform", Args: []interface{}{ + api.MediaType(""), harbor.ScanRequest{ Registry: harbor.Registry{ URL: "https://core.harbor.domain", @@ -118,8 +128,8 @@ func TestController_Scan(t *testing.T) { }, }, { - name: fmt.Sprintf("Should update job status to %s when Trivy wrapper fails", job.Failed.String()), - scanJobID: "job:123", + name: fmt.Sprintf("Should update job status to %s when Trivy wrapper fails", job.Failed.String()), + scanJobKey: jobKey, scanRequest: harbor.ScanRequest{ Registry: harbor.Registry{ URL: "https://core.harbor.domain", @@ -137,7 +147,7 @@ func TestController_Scan(t *testing.T) { Method: "UpdateStatus", Args: []interface{}{ ctx, - "job:123", + jobKey, job.Pending, []string(nil), }, @@ -147,7 +157,7 @@ func TestController_Scan(t *testing.T) { Method: "UpdateStatus", Args: []interface{}{ ctx, - "job:123", + jobKey, job.Failed, []string{"running trivy wrapper: out of memory"}, }, @@ -185,7 +195,7 @@ func TestController_Scan(t *testing.T) { mock.ApplyExpectations(t, wrapper, tc.wrapperExpectation) mock.ApplyExpectations(t, transformer, tc.transformerExpectation) - err := NewController(store, wrapper, transformer).Scan(ctx, tc.scanJobID, tc.scanRequest) + err := NewController(store, wrapper, transformer).Scan(ctx, tc.scanJobKey, &tc.scanRequest) assert.Equal(t, tc.expectedError, err) store.AssertExpectations(t) diff --git a/pkg/scan/transformer.go b/pkg/scan/transformer.go index dab60727..65d66426 100644 --- a/pkg/scan/transformer.go +++ b/pkg/scan/transformer.go @@ -1,12 +1,13 @@ package scan import ( - "github.com/samber/lo" "log/slog" "time" - "github.com/aquasecurity/harbor-scanner-trivy/pkg/etc" + "github.com/samber/lo" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/http/api" "github.com/aquasecurity/harbor-scanner-trivy/pkg/trivy" ) @@ -26,7 +27,7 @@ func (c *SystemClock) Now() time.Time { // Transformer wraps the Transform method. // Transform transforms Trivy's scan report into Harbor's packages vulnerabilities report. type Transformer interface { - Transform(req harbor.ScanRequest, source trivy.Report) harbor.ScanReport + Transform(mediaType api.MediaType, req harbor.ScanRequest, source trivy.Report) harbor.ScanReport } type transformer struct { @@ -40,15 +41,15 @@ func NewTransformer(clock Clock) Transformer { } } -func (t *transformer) Transform(req harbor.ScanRequest, source trivy.Report) harbor.ScanReport { +func (t *transformer) Transform(mediaType api.MediaType, req harbor.ScanRequest, source trivy.Report) harbor.ScanReport { report := harbor.ScanReport{ GeneratedAt: t.clock.Now(), - Scanner: etc.GetScannerMetadata(), + Scanner: harbor.GetScannerMetadata(), Artifact: req.Artifact, } - switch lo.FromPtr(req.Capabilities[0].Parameters).MediaType { - case harbor.MediaTypeSPDX, harbor.MediaTypeCycloneDX: + switch mediaType { + case api.MediaTypeSPDX, api.MediaTypeCycloneDX: report.MediaType = string(req.Capabilities[0].Parameters.MediaType) report.SBOM = source.SBOM default: diff --git a/pkg/scan/transformer_test.go b/pkg/scan/transformer_test.go index 5907618f..70d25d3e 100644 --- a/pkg/scan/transformer_test.go +++ b/pkg/scan/transformer_test.go @@ -27,105 +27,106 @@ func TestTransformer_Transform(t *testing.T) { fixedTime: fixedTime, }) - hr := tf.Transform(harbor.ScanRequest{ - Capabilities: []harbor.Capability{ - { - Type: harbor.CapabilityTypeVulnerability, - }, - }, - Artifact: harbor.Artifact{ - Repository: "library/mongo", - Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", - }, - }, trivy.Report{ - Vulnerabilities: []trivy.Vulnerability{ - { - VulnerabilityID: "CVE-0000-0001", - PkgName: "PKG-01", - InstalledVersion: "PKG-01-VER", - FixedVersion: "PKG-01-FIX-VER", - Severity: "CRITICAL", - Description: "CVE-0000-0001.DESC", - References: []string{ - "http://cve.com?id=CVE-0000-0001", - "http://vendor.com?id=CVE-0000-0001", - }, - Layer: &trivy.Layer{ - Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb10", - }, - CVSS: map[string]trivy.CVSSInfo{ - "nvd": { - V2Vector: "AV:L/AC:M/Au:N/C:P/I:N/A:N", - V3Vector: "CVSS:3.1/AV:L/AC:H/PR:L/UI:N/S:U/C:H/I:N/A:N", - V2Score: float32Ptr(1.9), - V3Score: float32Ptr(4.7), - }, - }, - CweIDs: []string{ - "CWE-20", - "CWE-1289", + hr := tf.Transform("", + harbor.ScanRequest{ + Capabilities: []harbor.Capability{ + { + Type: harbor.CapabilityTypeVulnerability, }, }, - { - VulnerabilityID: "CVE-0000-0002", - PkgName: "PKG-02", - InstalledVersion: "PKG-02-VER", - FixedVersion: "", - Severity: "HIGH", - Description: "CVE-0000-0002.DESC", - References: []string{ - "http://cve.com?id=CVE-0000-0002", - }, - Layer: &trivy.Layer{ - Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb11", - }, + Artifact: harbor.Artifact{ + Repository: "library/mongo", + Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", }, - { - VulnerabilityID: "CVE-0000-0003", - PkgName: "PKG-03", - InstalledVersion: "PKG-03-VER", - FixedVersion: "PKG-03-FIX-VER", - Severity: "MEDIUM", - Description: "CVE-0000-0003.DESC", - References: []string{ - "http://cve.com?id=CVE-0000-0003", - }, - Layer: &trivy.Layer{ - Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb12", + }, trivy.Report{ + Vulnerabilities: []trivy.Vulnerability{ + { + VulnerabilityID: "CVE-0000-0001", + PkgName: "PKG-01", + InstalledVersion: "PKG-01-VER", + FixedVersion: "PKG-01-FIX-VER", + Severity: "CRITICAL", + Description: "CVE-0000-0001.DESC", + References: []string{ + "http://cve.com?id=CVE-0000-0001", + "http://vendor.com?id=CVE-0000-0001", + }, + Layer: &trivy.Layer{ + Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb10", + }, + CVSS: map[string]trivy.CVSSInfo{ + "nvd": { + V2Vector: "AV:L/AC:M/Au:N/C:P/I:N/A:N", + V3Vector: "CVSS:3.1/AV:L/AC:H/PR:L/UI:N/S:U/C:H/I:N/A:N", + V2Score: float32Ptr(1.9), + V3Score: float32Ptr(4.7), + }, + }, + CweIDs: []string{ + "CWE-20", + "CWE-1289", + }, }, - PrimaryURL: "https://avd.aquasec.com/nvd/cve-0000-0003/", - }, - { - VulnerabilityID: "CVE-0000-0004", - PkgName: "PKG-04", - InstalledVersion: "PKG-04-VER", - FixedVersion: "PKG-04-FIX-VER", - Severity: "LOW", - Description: "CVE-0000-0004.DESC", - References: []string{ - "http://cve.com?id=CVE-0000-0004", + { + VulnerabilityID: "CVE-0000-0002", + PkgName: "PKG-02", + InstalledVersion: "PKG-02-VER", + FixedVersion: "", + Severity: "HIGH", + Description: "CVE-0000-0002.DESC", + References: []string{ + "http://cve.com?id=CVE-0000-0002", + }, + Layer: &trivy.Layer{ + Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb11", + }, }, - Layer: &trivy.Layer{ - Digest: "UNKNOWN", + { + VulnerabilityID: "CVE-0000-0003", + PkgName: "PKG-03", + InstalledVersion: "PKG-03-VER", + FixedVersion: "PKG-03-FIX-VER", + Severity: "MEDIUM", + Description: "CVE-0000-0003.DESC", + References: []string{ + "http://cve.com?id=CVE-0000-0003", + }, + Layer: &trivy.Layer{ + Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb12", + }, + PrimaryURL: "https://avd.aquasec.com/nvd/cve-0000-0003/", + }, + { + VulnerabilityID: "CVE-0000-0004", + PkgName: "PKG-04", + InstalledVersion: "PKG-04-VER", + FixedVersion: "PKG-04-FIX-VER", + Severity: "LOW", + Description: "CVE-0000-0004.DESC", + References: []string{ + "http://cve.com?id=CVE-0000-0004", + }, + Layer: &trivy.Layer{ + Digest: "UNKNOWN", + }, }, - }, - { - VulnerabilityID: "CVE-0000-0005", - PkgName: "PKG-05", - InstalledVersion: "PKG-05-VER", - Severity: "~~~UNKNOWN~~~", - Layer: &trivy.Layer{ - Digest: "", + { + VulnerabilityID: "CVE-0000-0005", + PkgName: "PKG-05", + InstalledVersion: "PKG-05-VER", + Severity: "~~~UNKNOWN~~~", + Layer: &trivy.Layer{ + Digest: "", + }, + }, + { + VulnerabilityID: "CVE-0000-0006", + PkgName: "PKG-06", + InstalledVersion: "PKG-06-VER", + Severity: "UNKNOWN", }, }, - { - VulnerabilityID: "CVE-0000-0006", - PkgName: "PKG-06", - InstalledVersion: "PKG-06-VER", - Severity: "UNKNOWN", - }, - }, - }) + }) assert.Equal(t, harbor.ScanReport{ GeneratedAt: fixedTime, Artifact: harbor.Artifact{ diff --git a/test/integration/persistence/redis/store_test.go b/test/integration/persistence/redis/store_test.go index 610ceec9..45546416 100644 --- a/test/integration/persistence/redis/store_test.go +++ b/test/integration/persistence/redis/store_test.go @@ -1,5 +1,4 @@ //go:build integration -// +build integration package redis @@ -55,28 +54,28 @@ func TestStore(t *testing.T) { store := redis.NewStore(config, pool) t.Run("CRUD", func(t *testing.T) { - scanJobID := "123" + scanJobKey := job.ScanJobKey{ID: "123"} err := store.Create(ctx, job.ScanJob{ - ID: scanJobID, + Key: scanJobKey, Status: job.Queued, }) require.NoError(t, err, "saving scan job should not fail") - j, err := store.Get(ctx, scanJobID) + j, err := store.Get(ctx, scanJobKey) require.NoError(t, err, "getting scan job should not fail") assert.Equal(t, &job.ScanJob{ - ID: scanJobID, + Key: scanJobKey, Status: job.Queued, }, j) - err = store.UpdateStatus(ctx, scanJobID, job.Pending) + err = store.UpdateStatus(ctx, scanJobKey, job.Pending) require.NoError(t, err, "updating scan job status should not fail") - j, err = store.Get(ctx, scanJobID) + j, err = store.Get(ctx, scanJobKey) require.NoError(t, err, "getting scan job should not fail") assert.Equal(t, &job.ScanJob{ - ID: scanJobID, + Key: scanJobKey, Status: job.Pending, }, j) @@ -89,20 +88,20 @@ func TestStore(t *testing.T) { }, } - err = store.UpdateReport(ctx, scanJobID, scanReport) + err = store.UpdateReport(ctx, scanJobKey, scanReport) require.NoError(t, err, "updating scan job reports should not fail") - j, err = store.Get(ctx, scanJobID) + j, err = store.Get(ctx, scanJobKey) require.NoError(t, err, "retrieving scan job should not fail") require.NotNil(t, j, "retrieved scan job must not be nil") assert.Equal(t, scanReport, j.Report) - err = store.UpdateStatus(ctx, scanJobID, job.Finished) + err = store.UpdateStatus(ctx, scanJobKey, job.Finished) require.NoError(t, err) time.Sleep(parseDuration(t, "12s")) - j, err = store.Get(ctx, scanJobID) + j, err = store.Get(ctx, scanJobKey) require.NoError(t, err, "retrieve scan job should not fail") require.Nil(t, j, "retrieved scan job should be nil, i.e. expired") }) From 2d702f4e56daa0f0f122ea9378a8f53f63b2f89a Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Thu, 11 Jan 2024 10:49:41 +0400 Subject: [PATCH 12/26] test: fix scan job key Signed-off-by: knqyf263 --- pkg/http/api/base_handler.go | 7 ++++++- test/integration/api/rest_api_test.go | 16 ++++++++++------ test/integration/persistence/redis/store_test.go | 6 +++++- 3 files changed, 21 insertions(+), 8 deletions(-) diff --git a/pkg/http/api/base_handler.go b/pkg/http/api/base_handler.go index ad7c31ac..00c316cc 100644 --- a/pkg/http/api/base_handler.go +++ b/pkg/http/api/base_handler.go @@ -6,6 +6,8 @@ import ( "log/slog" "net/http" "strings" + + "golang.org/x/xerrors" ) const ( @@ -82,6 +84,9 @@ func (mt *MIMEType) UnmarshalJSON(b []byte) error { } func (mt *MIMEType) String() string { + if mt.Type == "" || mt.Subtype == "" { + return "" + } s := fmt.Sprintf("%s/%s", mt.Type, mt.Subtype) if len(mt.Params) == 0 { return s @@ -106,7 +111,7 @@ func (mt *MIMEType) Parse(value string) error { mt.Params = MimeTypeSecuritySBOMReport.Params return nil } - return fmt.Errorf("unsupported mime type: %s", value) + return xerrors.Errorf("unsupported mime type: %s", value) } type BaseHandler struct { diff --git a/test/integration/api/rest_api_test.go b/test/integration/api/rest_api_test.go index 621dcff6..cc5f3f16 100644 --- a/test/integration/api/rest_api_test.go +++ b/test/integration/api/rest_api_test.go @@ -4,8 +4,8 @@ package api import ( "fmt" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/http/api" "io" - "io/ioutil" "net/http" "net/http/httptest" "strings" @@ -72,7 +72,7 @@ func TestRestApi(t *testing.T) { Repository: "library/oracle/nosql", Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", }, - }).Return(job.ScanJob{ID: "job:123"}, nil) + }).Return("job:123", nil) // when rs, err := ts.Client().Post(ts.URL+"/api/v1/scan", "application/json", strings.NewReader(`{ @@ -101,8 +101,12 @@ func TestRestApi(t *testing.T) { // given now := time.Now() - store.On("Get", mock.Anything, "job:123").Return(&job.ScanJob{ - ID: "job:123", + jobKey := job.ScanJobKey{ + ID: "job:123", + MIMEType: api.MimeTypeSecurityVulnerabilityReport, + } + store.On("Get", mock.Anything, jobKey).Return(&job.ScanJob{ + Key: jobKey, Status: job.Finished, Report: harbor.ScanReport{ GeneratedAt: now, @@ -143,7 +147,7 @@ func TestRestApi(t *testing.T) { assert.Equal(t, http.StatusOK, rs.StatusCode) assert.Equal(t, "application/vnd.security.vulnerability.report; version=1.1", rs.Header.Get("Content-Type")) - bodyBytes, err := ioutil.ReadAll(rs.Body) + bodyBytes, err := io.ReadAll(rs.Body) require.NoError(t, err) assert.JSONEq(t, fmt.Sprintf(`{ @@ -190,7 +194,7 @@ func TestRestApi(t *testing.T) { require.NoError(t, err) assert.Equal(t, http.StatusOK, rs.StatusCode) - bodyBytes, err := ioutil.ReadAll(rs.Body) + bodyBytes, err := io.ReadAll(rs.Body) require.NoError(t, err) assert.JSONEq(t, `{ diff --git a/test/integration/persistence/redis/store_test.go b/test/integration/persistence/redis/store_test.go index 45546416..cd935e24 100644 --- a/test/integration/persistence/redis/store_test.go +++ b/test/integration/persistence/redis/store_test.go @@ -5,6 +5,7 @@ package redis import ( "context" "fmt" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/http/api" "testing" "time" @@ -54,7 +55,10 @@ func TestStore(t *testing.T) { store := redis.NewStore(config, pool) t.Run("CRUD", func(t *testing.T) { - scanJobKey := job.ScanJobKey{ID: "123"} + scanJobKey := job.ScanJobKey{ + ID: "123", + MIMEType: api.MimeTypeSecurityVulnerabilityReport, + } err := store.Create(ctx, job.ScanJob{ Key: scanJobKey, From 08454d0024324c58b98481f167ee42e672461af2 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Thu, 11 Jan 2024 12:51:38 +0400 Subject: [PATCH 13/26] test: fix component tests Signed-off-by: knqyf263 --- Makefile | 8 ++++---- pkg/http/api/v1/handler.go | 3 +++ pkg/http/api/v1/handler_test.go | 3 +++ test/component/scanner/client.go | 1 - 4 files changed, 10 insertions(+), 5 deletions(-) diff --git a/Makefile b/Makefile index f237375c..d13bbd1a 100644 --- a/Makefile +++ b/Makefile @@ -8,17 +8,17 @@ IMAGE := aquasec/harbor-scanner-trivy:$(IMAGE_TAG) build: $(BINARY) test: build - GO111MODULE=on go test -v -short -race -coverprofile=coverage.txt -covermode=atomic ./... + go test -v -short -race -coverprofile=coverage.txt -covermode=atomic ./... test-integration: build - GO111MODULE=on go test -count=1 -v -tags=integration ./test/integration/... + go test -count=1 -v -tags=integration ./test/integration/... .PHONY: test-component test-component: docker-build - GO111MODULE=on go test -count=1 -v -tags=component ./test/component/... + go test -count=1 -v -tags=component ./test/component/... $(BINARY): $(SOURCES) - GOOS=linux GO111MODULE=on CGO_ENABLED=0 go build -o $(BINARY) cmd/scanner-trivy/main.go + GOOS=linux CGO_ENABLED=0 go build -o $(BINARY) cmd/scanner-trivy/main.go .PHONY: docker-build docker-build: build diff --git a/pkg/http/api/v1/handler.go b/pkg/http/api/v1/handler.go index 2446229d..227967e0 100644 --- a/pkg/http/api/v1/handler.go +++ b/pkg/http/api/v1/handler.go @@ -99,6 +99,9 @@ func (h *requestHandler) AcceptScanRequest(res http.ResponseWriter, req *http.Re if len(scanRequest.Capabilities) == 0 { scanRequest.Capabilities = append(scanRequest.Capabilities, harbor.Capability{ Type: harbor.CapabilityTypeVulnerability, + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecurityVulnerabilityReport, + }, }) } diff --git a/pkg/http/api/v1/handler_test.go b/pkg/http/api/v1/handler_test.go index 5801478f..4c61c544 100644 --- a/pkg/http/api/v1/handler_test.go +++ b/pkg/http/api/v1/handler_test.go @@ -107,6 +107,9 @@ func TestRequestHandler_AcceptScanRequest(t *testing.T) { Capabilities: []harbor.Capability{ { Type: harbor.CapabilityTypeVulnerability, + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecurityVulnerabilityReport, + }, }, }, Registry: harbor.Registry{ diff --git a/test/component/scanner/client.go b/test/component/scanner/client.go index 79eba0b8..d1aa8d92 100644 --- a/test/component/scanner/client.go +++ b/test/component/scanner/client.go @@ -1,5 +1,4 @@ //go:build component -// +build component package scanner From 219adbeba0328d88c2038f2249ea2cefd97df932 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Thu, 11 Jan 2024 13:11:31 +0400 Subject: [PATCH 14/26] test: fix integration tests Signed-off-by: knqyf263 --- test/integration/api/rest_api_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/test/integration/api/rest_api_test.go b/test/integration/api/rest_api_test.go index cc5f3f16..cfb8a9fc 100644 --- a/test/integration/api/rest_api_test.go +++ b/test/integration/api/rest_api_test.go @@ -62,6 +62,9 @@ func TestRestApi(t *testing.T) { Capabilities: []harbor.Capability{ { Type: harbor.CapabilityTypeVulnerability, + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecurityVulnerabilityReport, + }, }, }, Registry: harbor.Registry{ From e48bbfe92b1dfed4fbdfd25fb8178a5a5a352689 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Thu, 18 Jan 2024 11:13:57 +0400 Subject: [PATCH 15/26] feat: add additional_attributes Signed-off-by: knqyf263 --- pkg/harbor/model.go | 16 ++++----- pkg/http/api/v1/handler.go | 26 ++++++++++++--- pkg/http/api/v1/handler_test.go | 24 ++++++++++++-- pkg/queue/enqueuer.go | 48 ++++++++++++++++----------- pkg/scan/transformer.go | 2 +- test/integration/api/rest_api_test.go | 8 ++++- 6 files changed, 86 insertions(+), 38 deletions(-) diff --git a/pkg/harbor/model.go b/pkg/harbor/model.go index 75df610c..dfd3b019 100644 --- a/pkg/harbor/model.go +++ b/pkg/harbor/model.go @@ -125,8 +125,8 @@ type ScanReport struct { Severity Severity `json:"severity,omitempty"` // For SBOM - MediaType string `json:"media_type,omitempty"` - SBOM any `json:"sbom,omitempty"` + MediaType api.MediaType `json:"media_type,omitempty"` + SBOM any `json:"sbom,omitempty"` // For vulnerabilities Vulnerabilities []VulnerabilityItem `json:"vulnerabilities,omitempty"` @@ -172,14 +172,14 @@ type Scanner struct { } type Capability struct { - Type CapabilityType `json:"type"` - ConsumesMIMETypes []string `json:"consumes_mime_types"` - ProducesMIMETypes []api.MIMEType `json:"produces_mime_types"` - Parameters *CapabilityParameters `json:"parameters,omitempty"` + Type CapabilityType `json:"type"` + ConsumesMIMETypes []string `json:"consumes_mime_types"` + ProducesMIMETypes []api.MIMEType `json:"produces_mime_types"` + AdditionalAttributes *CapabilityAttributes `json:"additional_attributes,omitempty"` } -type CapabilityParameters struct { - MediaType api.MediaType `json:"accept_media_type,omitempty"` +type CapabilityAttributes struct { + SBOMMediaTypes []api.MediaType `json:"sbom_media_types,omitempty"` } func GetScannerMetadata() Scanner { diff --git a/pkg/http/api/v1/handler.go b/pkg/http/api/v1/handler.go index 227967e0..e6257ec4 100644 --- a/pkg/http/api/v1/handler.go +++ b/pkg/http/api/v1/handler.go @@ -165,11 +165,21 @@ func (h *requestHandler) validateCapabilities(capabilities []harbor.Capability) } } - if c.Type == harbor.CapabilityTypeSBOM && - !slices.Contains(harbor.SupportedSBOMMediaTypes, c.Parameters.MediaType) { - return &api.Error{ - HTTPCode: http.StatusUnprocessableEntity, - Message: "invalid SBOM media type", + if c.Type == harbor.CapabilityTypeSBOM { + if len(c.AdditionalAttributes.SBOMMediaTypes) == 0 { + return &api.Error{ + HTTPCode: http.StatusUnprocessableEntity, + Message: "missing SBOM media type", + } + } + + for _, mediaType := range c.AdditionalAttributes.SBOMMediaTypes { + if !slices.Contains(harbor.SupportedSBOMMediaTypes, mediaType) { + return &api.Error{ + HTTPCode: http.StatusUnprocessableEntity, + Message: fmt.Sprintf("unsupported SBOM media type: %s", mediaType), + } + } } } } @@ -313,6 +323,12 @@ func (h *requestHandler) GetMetadata(res http.ResponseWriter, _ *http.Request) { ProducesMIMETypes: []api.MIMEType{ api.MimeTypeSecuritySBOMReport, }, + AdditionalAttributes: &harbor.CapabilityAttributes{ + SBOMMediaTypes: []api.MediaType{ + api.MediaTypeSPDX, + api.MediaTypeCycloneDX, + }, + }, }, }, Properties: properties, diff --git a/pkg/http/api/v1/handler_test.go b/pkg/http/api/v1/handler_test.go index 4c61c544..3cb27d90 100644 --- a/pkg/http/api/v1/handler_test.go +++ b/pkg/http/api/v1/handler_test.go @@ -574,7 +574,13 @@ func TestRequestHandler_GetMetadata(t *testing.T) { ], "produces_mime_types":[ "application/vnd.security.sbom.report+json; version=1.0" - ] + ], + "additional_attributes": { + "sbom_media_types": [ + "application/spdx+json", + "application/vnd.cyclonedx+json" + ] + } } ], "properties":{ @@ -647,7 +653,13 @@ func TestRequestHandler_GetMetadata(t *testing.T) { ], "produces_mime_types":[ "application/vnd.security.sbom.report+json; version=1.0" - ] + ], + "additional_attributes": { + "sbom_media_types": [ + "application/spdx+json", + "application/vnd.cyclonedx+json" + ] + } } ], "properties":{ @@ -712,7 +724,13 @@ func TestRequestHandler_GetMetadata(t *testing.T) { ], "produces_mime_types":[ "application/vnd.security.sbom.report+json; version=1.0" - ] + ], + "additional_attributes": { + "sbom_media_types": [ + "application/spdx+json", + "application/vnd.cyclonedx+json" + ] + } } ], "properties":{ diff --git a/pkg/queue/enqueuer.go b/pkg/queue/enqueuer.go index 808b816a..6e805c5a 100644 --- a/pkg/queue/enqueuer.go +++ b/pkg/queue/enqueuer.go @@ -5,6 +5,7 @@ import ( "crypto/rand" "encoding/json" "fmt" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/http/api" "io" "log/slog" @@ -60,28 +61,35 @@ func (e *enqueuer) Enqueue(ctx context.Context, request harbor.ScanRequest) (str jobID := makeIdentifier() for _, c := range request.Capabilities { - mediaType := lo.FromPtr(c.Parameters).MediaType - for _, m := range c.ProducesMIMETypes { - jobKey := job.ScanJobKey{ - ID: jobID, - MIMEType: m, - MediaType: mediaType, - } - - j := Job{ - Name: scanArtifactJobName, - Key: jobKey, - Args: Args{ - ScanRequest: &request, - }, - } - scanJob := job.ScanJob{ - Key: jobKey, - Status: job.Queued, + if c.Type == harbor.CapabilityTypeVulnerability { + c.AdditionalAttributes = &harbor.CapabilityAttributes{ + SBOMMediaTypes: []api.MediaType{""}, } + } - if err := e.enqueue(ctx, j, scanJob); err != nil { - return "", xerrors.Errorf("enqueuing scan job: %v", err) + for _, mediaType := range lo.FromPtr(c.AdditionalAttributes).SBOMMediaTypes { + for _, m := range c.ProducesMIMETypes { + jobKey := job.ScanJobKey{ + ID: jobID, + MIMEType: m, + MediaType: mediaType, + } + + j := Job{ + Name: scanArtifactJobName, + Key: jobKey, + Args: Args{ + ScanRequest: &request, + }, + } + scanJob := job.ScanJob{ + Key: jobKey, + Status: job.Queued, + } + + if err := e.enqueue(ctx, j, scanJob); err != nil { + return "", xerrors.Errorf("enqueuing scan job: %v", err) + } } } } diff --git a/pkg/scan/transformer.go b/pkg/scan/transformer.go index 65d66426..fb944121 100644 --- a/pkg/scan/transformer.go +++ b/pkg/scan/transformer.go @@ -50,7 +50,7 @@ func (t *transformer) Transform(mediaType api.MediaType, req harbor.ScanRequest, switch mediaType { case api.MediaTypeSPDX, api.MediaTypeCycloneDX: - report.MediaType = string(req.Capabilities[0].Parameters.MediaType) + report.MediaType = mediaType report.SBOM = source.SBOM default: report.Vulnerabilities = t.transformVulnerabilities(source.Vulnerabilities) diff --git a/test/integration/api/rest_api_test.go b/test/integration/api/rest_api_test.go index cfb8a9fc..274d37b8 100644 --- a/test/integration/api/rest_api_test.go +++ b/test/integration/api/rest_api_test.go @@ -225,7 +225,13 @@ func TestRestApi(t *testing.T) { ], "produces_mime_types": [ "application/vnd.security.sbom.report+json; version=1.0" - ] + ], + "additional_attributes": { + "sbom_media_types": [ + "application/spdx+json", + "application/vnd.cyclonedx+json" + ] + } } ], "properties": { From 767979c3c6990e957a8203faaf13cab8238a9690 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Thu, 18 Jan 2024 14:32:47 +0400 Subject: [PATCH 16/26] feat: take "sbom_media_type" in the GET query parameter Signed-off-by: knqyf263 --- go.mod | 1 + go.sum | 2 ++ pkg/harbor/model.go | 16 ++++++++++++--- pkg/http/api/v1/handler.go | 40 +++++++++++++++++++++++++++----------- pkg/job/model.go | 5 ++++- pkg/queue/enqueuer.go | 4 ++-- 6 files changed, 51 insertions(+), 17 deletions(-) diff --git a/go.mod b/go.mod index a0a57862..1843fbba 100644 --- a/go.mod +++ b/go.mod @@ -8,6 +8,7 @@ require ( github.com/docker/go-connections v0.5.0 github.com/google/go-containerregistry v0.14.0 github.com/gorilla/mux v1.8.1 + github.com/gorilla/schema v1.2.1 github.com/opencontainers/go-digest v1.0.0 github.com/prometheus/client_golang v1.18.0 github.com/redis/go-redis/v9 v9.4.0 diff --git a/go.sum b/go.sum index c1fde1a4..222e1672 100644 --- a/go.sum +++ b/go.sum @@ -76,6 +76,8 @@ github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gorilla/mux v1.8.1 h1:TuBL49tXwgrFYWhqrNgrUNEY92u81SPhu7sTdzQEiWY= github.com/gorilla/mux v1.8.1/go.mod h1:AKf9I4AEqPTmMytcMc0KkNouC66V3BtZ4qD5fmWSiMQ= +github.com/gorilla/schema v1.2.1 h1:tjDxcmdb+siIqkTNoV+qRH2mjYdr2hHe5MKXbp61ziM= +github.com/gorilla/schema v1.2.1/go.mod h1:Dg5SSm5PV60mhF2NFaTV1xuYYj8tV8NOPRo4FggUMnM= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.16.5 h1:IFV2oUNUzZaz+XyusxpLzpzS8Pt5rh0Z16For/djlyI= diff --git a/pkg/harbor/model.go b/pkg/harbor/model.go index dfd3b019..7f42caa1 100644 --- a/pkg/harbor/model.go +++ b/pkg/harbor/model.go @@ -86,6 +86,11 @@ type Artifact struct { MimeType string `json:"mime_type,omitempty"` } +// ScanReportQuery is a struct for the query parameters at "/scan/{scan_request_id}/report". +type ScanReportQuery struct { + SBOMMediaType api.MediaType `schema:"sbom_media_type"` +} + type ScanRequest struct { Registry Registry `json:"registry"` Artifact Artifact `json:"artifact"` @@ -172,10 +177,15 @@ type Scanner struct { } type Capability struct { - Type CapabilityType `json:"type"` - ConsumesMIMETypes []string `json:"consumes_mime_types"` - ProducesMIMETypes []api.MIMEType `json:"produces_mime_types"` + Type CapabilityType `json:"type"` + ConsumesMIMETypes []string `json:"consumes_mime_types"` + ProducesMIMETypes []api.MIMEType `json:"produces_mime_types"` + + // For /metadata AdditionalAttributes *CapabilityAttributes `json:"additional_attributes,omitempty"` + + // For /scan + Parameters *CapabilityAttributes `json:"parameters,omitempty"` } type CapabilityAttributes struct { diff --git a/pkg/http/api/v1/handler.go b/pkg/http/api/v1/handler.go index e6257ec4..3a23d0c8 100644 --- a/pkg/http/api/v1/handler.go +++ b/pkg/http/api/v1/handler.go @@ -3,6 +3,7 @@ package v1 import ( "encoding/json" "fmt" + "github.com/samber/lo" "log/slog" "net/http" "net/url" @@ -18,6 +19,7 @@ import ( "github.com/aquasecurity/harbor-scanner-trivy/pkg/queue" "github.com/aquasecurity/harbor-scanner-trivy/pkg/trivy" "github.com/gorilla/mux" + "github.com/gorilla/schema" "github.com/prometheus/client_golang/prometheus/promhttp" ) @@ -31,6 +33,8 @@ const ( propertyJavaDBNextUpdateAt = "harbor.scanner-adapter/vulnerability-java-database-next-update-at" ) +var decoder = schema.NewDecoder() + type requestHandler struct { info etc.BuildInfo config etc.Config @@ -166,14 +170,15 @@ func (h *requestHandler) validateCapabilities(capabilities []harbor.Capability) } if c.Type == harbor.CapabilityTypeSBOM { - if len(c.AdditionalAttributes.SBOMMediaTypes) == 0 { + params := lo.FromPtr(c.Parameters) + if len(params.SBOMMediaTypes) == 0 { return &api.Error{ HTTPCode: http.StatusUnprocessableEntity, Message: "missing SBOM media type", } } - for _, mediaType := range c.AdditionalAttributes.SBOMMediaTypes { + for _, mediaType := range params.SBOMMediaTypes { if !slices.Contains(harbor.SupportedSBOMMediaTypes, mediaType) { return &api.Error{ HTTPCode: http.StatusUnprocessableEntity, @@ -187,33 +192,46 @@ func (h *requestHandler) validateCapabilities(capabilities []harbor.Capability) } func (h *requestHandler) GetScanReport(res http.ResponseWriter, req *http.Request) { - var reportMIMEType api.MIMEType + vars := mux.Vars(req) + scanJobID, ok := vars[pathVarScanRequestID] + if !ok { + slog.Error("Error while parsing `scan_request_id` path variable") + h.WriteJSONError(res, api.Error{ + HTTPCode: http.StatusBadRequest, + Message: "missing scan_request_id", + }) + return + } + reqLog := slog.With(slog.String("scan_job_id", scanJobID)) + var reportMIMEType api.MIMEType if err := reportMIMEType.Parse(req.Header.Get(api.HeaderAccept)); err != nil { + reqLog.Error("Error while parsing the Accept header", slog.String("err", err.Error())) h.WriteJSONError(res, api.Error{ HTTPCode: http.StatusUnsupportedMediaType, Message: fmt.Sprintf("unsupported media type %s", req.Header.Get(api.HeaderAccept)), }) return } + reqLog = reqLog.With(slog.String("mime_type", reportMIMEType.String())) - vars := mux.Vars(req) - scanJobID, ok := vars[pathVarScanRequestID] - if !ok { - slog.Error("Error while parsing `scan_request_id` path variable") + // Decode the query parameters into the struct + var query harbor.ScanReportQuery + if err := decoder.Decode(&query, req.URL.Query()); err != nil { + reqLog.Error("Error while parsing query parameters", slog.String("err", err.Error())) h.WriteJSONError(res, api.Error{ HTTPCode: http.StatusBadRequest, - Message: "missing scan_request_id", + Message: fmt.Sprintf("query parameter error: %s", err), }) return + } else if query.SBOMMediaType != "" { + reqLog = reqLog.With(slog.String("sbom_media_type", string(query.SBOMMediaType))) } - reqLog := slog.With(slog.String("scan_job_id", scanJobID), slog.String("mime_type", reportMIMEType.String())) - scanJob, err := h.store.Get(req.Context(), job.ScanJobKey{ ID: scanJobID, MIMEType: reportMIMEType, - MediaType: "", // TODO: Harbor should pass MediaType + MediaType: query.SBOMMediaType, }) if err != nil { reqLog.Error("Error while getting scan job") diff --git a/pkg/job/model.go b/pkg/job/model.go index 7f1e3c65..a4cf23d0 100644 --- a/pkg/job/model.go +++ b/pkg/job/model.go @@ -37,7 +37,10 @@ type ScanJobKey struct { } func (s *ScanJobKey) String() string { - return fmt.Sprintf("%s:%s", s.ID, s.MIMEType.String()) // TODO: add MediaType + if s.MediaType != "" { + return fmt.Sprintf("%s:%s:%s", s.ID, s.MIMEType.String(), s.MediaType) + } + return fmt.Sprintf("%s:%s", s.ID, s.MIMEType.String()) } type ScanJob struct { diff --git a/pkg/queue/enqueuer.go b/pkg/queue/enqueuer.go index 6e805c5a..a69d41b5 100644 --- a/pkg/queue/enqueuer.go +++ b/pkg/queue/enqueuer.go @@ -62,12 +62,12 @@ func (e *enqueuer) Enqueue(ctx context.Context, request harbor.ScanRequest) (str for _, c := range request.Capabilities { if c.Type == harbor.CapabilityTypeVulnerability { - c.AdditionalAttributes = &harbor.CapabilityAttributes{ + c.Parameters = &harbor.CapabilityAttributes{ SBOMMediaTypes: []api.MediaType{""}, } } - for _, mediaType := range lo.FromPtr(c.AdditionalAttributes).SBOMMediaTypes { + for _, mediaType := range lo.FromPtr(c.Parameters).SBOMMediaTypes { for _, m := range c.ProducesMIMETypes { jobKey := job.ScanJobKey{ ID: jobID, From 88edd7c380237882d0ff34aa2bff6fe8a9deb4d3 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Thu, 18 Jan 2024 15:47:18 +0400 Subject: [PATCH 17/26] test: add test cases for SBOM Signed-off-by: knqyf263 --- pkg/http/api/v1/handler.go | 4 +- pkg/http/api/v1/handler_test.go | 267 +++++++++++++++++++++++--------- 2 files changed, 198 insertions(+), 73 deletions(-) diff --git a/pkg/http/api/v1/handler.go b/pkg/http/api/v1/handler.go index 3a23d0c8..0485767f 100644 --- a/pkg/http/api/v1/handler.go +++ b/pkg/http/api/v1/handler.go @@ -182,7 +182,7 @@ func (h *requestHandler) validateCapabilities(capabilities []harbor.Capability) if !slices.Contains(harbor.SupportedSBOMMediaTypes, mediaType) { return &api.Error{ HTTPCode: http.StatusUnprocessableEntity, - Message: fmt.Sprintf("unsupported SBOM media type: %s", mediaType), + Message: fmt.Sprintf("unsupported SBOM media type: %q", mediaType), } } } @@ -209,7 +209,7 @@ func (h *requestHandler) GetScanReport(res http.ResponseWriter, req *http.Reques reqLog.Error("Error while parsing the Accept header", slog.String("err", err.Error())) h.WriteJSONError(res, api.Error{ HTTPCode: http.StatusUnsupportedMediaType, - Message: fmt.Sprintf("unsupported media type %s", req.Header.Get(api.HeaderAccept)), + Message: fmt.Sprintf("unsupported media type: %q", req.Header.Get(api.HeaderAccept)), }) return } diff --git a/pkg/http/api/v1/handler_test.go b/pkg/http/api/v1/handler_test.go index 3cb27d90..52378262 100644 --- a/pkg/http/api/v1/handler_test.go +++ b/pkg/http/api/v1/handler_test.go @@ -1,10 +1,12 @@ package v1 import ( + "encoding/json" "errors" "fmt" "net/http" "net/http/httptest" + "net/url" "strings" "testing" "time" @@ -91,6 +93,56 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { Message: "missing artifact.digest", }, }, + { + Name: "Should return error with unsupported scan type", + Request: harbor.ScanRequest{ + Capabilities: []harbor.Capability{ + { + Type: "unknown", + }, + }, + }, + ExpectedError: &api.Error{ + HTTPCode: http.StatusUnprocessableEntity, + Message: "invalid scan type", + }, + }, + { + Name: "Should return error when SBOM media type is missing", + Request: harbor.ScanRequest{ + Capabilities: []harbor.Capability{ + { + Type: harbor.CapabilityTypeSBOM, + Parameters: &harbor.CapabilityAttributes{ + SBOMMediaTypes: nil, + }, + }, + }, + }, + ExpectedError: &api.Error{ + HTTPCode: http.StatusUnprocessableEntity, + Message: "missing SBOM media type", + }, + }, + { + Name: "Should return error when unsupported SBOM media type is passed", + Request: harbor.ScanRequest{ + Capabilities: []harbor.Capability{ + { + Type: harbor.CapabilityTypeSBOM, + Parameters: &harbor.CapabilityAttributes{ + SBOMMediaTypes: []api.MediaType{ + "application/unsupported", + }, + }, + }, + }, + }, + ExpectedError: &api.Error{ + HTTPCode: http.StatusUnprocessableEntity, + Message: `unsupported SBOM media type: "application/unsupported"`, + }, + }, } for _, tc := range testCases { @@ -230,17 +282,28 @@ func TestRequestHandler_AcceptScanRequest(t *testing.T) { func TestRequestHandler_GetScanReport(t *testing.T) { now := time.Now() - scanJobKey := job.ScanJobKey{ + vulnScanJobKey := job.ScanJobKey{ ID: "job:123", MIMEType: api.MimeTypeSecurityVulnerabilityReport, } + sbomScanJobKey := job.ScanJobKey{ + ID: "job:123", + MIMEType: api.MimeTypeSecuritySBOMReport, + MediaType: api.MediaTypeSPDX, + } - testCases := []struct { + type apiError struct { + Err api.Error `json:"error"` + } + + tests := []struct { name string + acceptHeader string + query url.Values storeExpectation *mock.Expectation expectedStatus int expectedContentType string - expectedResponse string + expectedResponse any }{ { name: "Should respond with error 500 when retrieving scan job fails", @@ -248,7 +311,7 @@ func TestRequestHandler_GetScanReport(t *testing.T) { Method: "Get", Args: []interface{}{ mock.Anything, - scanJobKey, + vulnScanJobKey, }, ReturnArgs: []interface{}{ &job.ScanJob{}, @@ -257,11 +320,11 @@ func TestRequestHandler_GetScanReport(t *testing.T) { }, expectedStatus: http.StatusInternalServerError, expectedContentType: "application/vnd.scanner.adapter.error; version=1.0", - expectedResponse: `{ - "error": { - "message": "getting scan job: data store is down" - } -}`, + expectedResponse: apiError{ + Err: api.Error{ + Message: "getting scan job: data store is down", + }, + }, }, { name: "Should respond with error 404 when scan job cannot be found", @@ -269,7 +332,7 @@ func TestRequestHandler_GetScanReport(t *testing.T) { Method: "Get", Args: []interface{}{ mock.Anything, - scanJobKey, + vulnScanJobKey, }, ReturnArgs: []interface{}{ (*job.ScanJob)(nil), @@ -278,11 +341,11 @@ func TestRequestHandler_GetScanReport(t *testing.T) { }, expectedStatus: http.StatusNotFound, expectedContentType: "application/vnd.scanner.adapter.error; version=1.0", - expectedResponse: `{ - "error": { - "message": "cannot find scan job: job:123" - } -}`, + expectedResponse: apiError{ + Err: api.Error{ + Message: "cannot find scan job: job:123", + }, + }, }, { name: fmt.Sprintf("Should respond with found status 302 when scan job is %s", job.Queued), @@ -290,11 +353,11 @@ func TestRequestHandler_GetScanReport(t *testing.T) { Method: "Get", Args: []interface{}{ mock.Anything, - scanJobKey, + vulnScanJobKey, }, ReturnArgs: []interface{}{ &job.ScanJob{ - Key: scanJobKey, + Key: vulnScanJobKey, Status: job.Queued, }, nil, @@ -308,11 +371,11 @@ func TestRequestHandler_GetScanReport(t *testing.T) { Method: "Get", Args: []interface{}{ mock.Anything, - scanJobKey, + vulnScanJobKey, }, ReturnArgs: []interface{}{ &job.ScanJob{ - Key: scanJobKey, + Key: vulnScanJobKey, Status: job.Pending, }, nil, @@ -326,11 +389,11 @@ func TestRequestHandler_GetScanReport(t *testing.T) { Method: "Get", Args: []interface{}{ mock.Anything, - scanJobKey, + vulnScanJobKey, }, ReturnArgs: []interface{}{ &job.ScanJob{ - Key: scanJobKey, + Key: vulnScanJobKey, Status: job.Failed, Error: "queue worker failed", }, @@ -339,11 +402,11 @@ func TestRequestHandler_GetScanReport(t *testing.T) { }, expectedStatus: http.StatusInternalServerError, expectedContentType: "application/vnd.scanner.adapter.error; version=1.0", - expectedResponse: `{ - "error": { - "message": "queue worker failed" - } -}`, + expectedResponse: apiError{ + Err: api.Error{ + Message: "queue worker failed", + }, + }, }, { name: fmt.Sprintf("Should respond with error 500 when scan job is NOT %s", job.Finished), @@ -351,11 +414,11 @@ func TestRequestHandler_GetScanReport(t *testing.T) { Method: "Get", Args: []interface{}{ mock.Anything, - scanJobKey, + vulnScanJobKey, }, ReturnArgs: []interface{}{ &job.ScanJob{ - Key: scanJobKey, + Key: vulnScanJobKey, Status: 666, Error: "queue worker failed", }, @@ -364,11 +427,11 @@ func TestRequestHandler_GetScanReport(t *testing.T) { }, expectedStatus: http.StatusInternalServerError, expectedContentType: "application/vnd.scanner.adapter.error; version=1.0", - expectedResponse: `{ - "error": { - "message": "unexpected status Unknown of scan job job:123" - } -}`, + expectedResponse: apiError{ + Err: api.Error{ + Message: "unexpected status Unknown of scan job job:123", + }, + }, }, { name: "Should respond with vulnerabilities report", @@ -376,11 +439,11 @@ func TestRequestHandler_GetScanReport(t *testing.T) { Method: "Get", Args: []interface{}{ mock.Anything, - scanJobKey, + vulnScanJobKey, }, ReturnArgs: []interface{}{ &job.ScanJob{ - Key: scanJobKey, + Key: vulnScanJobKey, Status: job.Finished, Report: harbor.ScanReport{ GeneratedAt: now, @@ -417,55 +480,117 @@ func TestRequestHandler_GetScanReport(t *testing.T) { }, expectedStatus: http.StatusOK, expectedContentType: "application/vnd.security.vulnerability.report; version=1.1", - expectedResponse: fmt.Sprintf(`{ - "generated_at": "%s", - "artifact": { - "repository": "library/mongo", - "digest": "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b" - }, - "scanner": { - "name": "Trivy", - "vendor": "Aqua Security", - "version": "0.1.6" - }, - "severity": "Critical", - "vulnerabilities": [ - { - "id": "CVE-2019-1111", - "package": "openssl", - "version": "2.0-rc1", - "fix_version": "2.1", - "severity": "Critical", - "description": "You'd better upgrade your server", - "links": [ - "http://cve.com?id=CVE-2019-1111" - ], - "layer": { - "digest": "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb10" - } - } - ] -}`, now.Format(time.RFC3339Nano)), + expectedResponse: harbor.ScanReport{ + GeneratedAt: now, + Artifact: harbor.Artifact{ + Repository: "library/mongo", + Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", + }, + Scanner: harbor.Scanner{ + Name: "Trivy", + Vendor: "Aqua Security", + Version: "0.1.6", + }, + Severity: harbor.SevCritical, + Vulnerabilities: []harbor.VulnerabilityItem{ + { + ID: "CVE-2019-1111", + Pkg: "openssl", + Version: "2.0-rc1", + FixVersion: "2.1", + Severity: harbor.SevCritical, + Description: "You'd better upgrade your server", + Links: []string{ + "http://cve.com?id=CVE-2019-1111", + }, + Layer: &harbor.Layer{ + Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb10", + }, + }, + }, + }, + }, + { + name: "Should return error when unsupported mime type is passed", + acceptHeader: "unknown", + expectedStatus: http.StatusUnsupportedMediaType, + expectedContentType: "application/vnd.scanner.adapter.error; version=1.0", + expectedResponse: apiError{ + Err: api.Error{ + Message: `unsupported media type: "unknown"`, + }, + }, + }, + { + name: "Should respond with SBOM report", + acceptHeader: "application/vnd.security.sbom.report+json; version=1.0", + query: url.Values{ + "sbom_media_type": []string{"application/spdx+json"}, + }, + storeExpectation: &mock.Expectation{ + Method: "Get", + Args: []interface{}{ + mock.Anything, + sbomScanJobKey, + }, + ReturnArgs: []interface{}{ + &job.ScanJob{ + Key: sbomScanJobKey, + Status: job.Finished, + Report: harbor.ScanReport{ + GeneratedAt: now, + Artifact: harbor.Artifact{ + Repository: "library/mongo", + Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", + }, + MediaType: "application/spdx+json", + SBOM: "Generated SBOM here", + }, + }, + nil, + }, + }, + expectedStatus: http.StatusOK, + expectedContentType: "application/vnd.security.sbom.report+json; version=1.0", + expectedResponse: harbor.ScanReport{ + GeneratedAt: now, + Artifact: harbor.Artifact{ + Repository: "library/mongo", + Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", + }, + MediaType: "application/spdx+json", + SBOM: "Generated SBOM here", + }, }, } - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { enqueuer := mock.NewEnqueuer() store := mock.NewStore() - mock.ApplyExpectations(t, store, tc.storeExpectation) + mock.ApplyExpectations(t, store, tt.storeExpectation) rr := httptest.NewRecorder() r, err := http.NewRequest(http.MethodGet, "/api/v1/scan/job:123/report", nil) require.NoError(t, err) + if tt.acceptHeader != "" { + r.Header.Add("Accept", tt.acceptHeader) + } + + if tt.query != nil { + r.URL.RawQuery = tt.query.Encode() + } + NewAPIHandler(etc.BuildInfo{}, etc.Config{}, enqueuer, store, nil).ServeHTTP(rr, r) - assert.Equal(t, tc.expectedStatus, rr.Code) - assert.Equal(t, tc.expectedContentType, rr.Header().Get("Content-Type")) - if tc.expectedResponse != "" { - assert.JSONEq(t, tc.expectedResponse, rr.Body.String()) + assert.Equal(t, tt.expectedStatus, rr.Code) + assert.Equal(t, tt.expectedContentType, rr.Header().Get("Content-Type")) + if tt.expectedResponse != nil { + got, err := json.Marshal(tt.expectedResponse) + require.NoError(t, err) + assert.JSONEq(t, string(got), rr.Body.String()) } enqueuer.AssertExpectations(t) From 128d12853cdc5a2512e8c0e51769ff68ba37fe0f Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Thu, 18 Jan 2024 15:47:27 +0400 Subject: [PATCH 18/26] test: add cases for SBOM generation Signed-off-by: knqyf263 --- test/integration/api/rest_api_test.go | 132 +++++++++++++++++++++++++- 1 file changed, 128 insertions(+), 4 deletions(-) diff --git a/test/integration/api/rest_api_test.go b/test/integration/api/rest_api_test.go index 274d37b8..bfe5160f 100644 --- a/test/integration/api/rest_api_test.go +++ b/test/integration/api/rest_api_test.go @@ -8,6 +8,7 @@ import ( "io" "net/http" "net/http/httptest" + "net/url" "strings" "testing" "time" @@ -22,9 +23,9 @@ import ( "github.com/stretchr/testify/require" ) -// TestRestApi is an integration test for the REST API adapter. +// TestRestAPI is an integration test for the REST API adapter. // Tests only happy paths. All branches are covered in the corresponding unit tests. -func TestRestApi(t *testing.T) { +func TestRestAPI(t *testing.T) { if testing.Short() { t.Skip("An integration test") } @@ -56,7 +57,7 @@ func TestRestApi(t *testing.T) { ts := httptest.NewServer(app) defer ts.Close() - t.Run("POST /api/v1/scan", func(t *testing.T) { + t.Run("POST /api/v1/scan for vulnerabilities", func(t *testing.T) { // given enqueuer.On("Enqueue", mock.Anything, harbor.ScanRequest{ Capabilities: []harbor.Capability{ @@ -100,7 +101,67 @@ func TestRestApi(t *testing.T) { assert.JSONEq(t, `{"id": "job:123"}`, string(bodyBytes)) }) - t.Run("GET /api/v1/scan/{scan_request_id}/report", func(t *testing.T) { + t.Run("POST /api/v1/scan for SBOM", func(t *testing.T) { + // given + enqueuer.On("Enqueue", mock.Anything, harbor.ScanRequest{ + Capabilities: []harbor.Capability{ + { + Type: harbor.CapabilityTypeSBOM, + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecuritySBOMReport, + }, + Parameters: &harbor.CapabilityAttributes{ + SBOMMediaTypes: []api.MediaType{api.MediaTypeSPDX}, + }, + }, + }, + Registry: harbor.Registry{ + URL: "https://core.harbor.domain", + Authorization: "Bearer JWTTOKENGOESHERE", + }, + Artifact: harbor.Artifact{ + Repository: "library/oracle/nosql", + Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", + }, + }).Return("job:123", nil) + + // when + rs, err := ts.Client().Post(ts.URL+"/api/v1/scan", "application/json", strings.NewReader(`{ + "registry": { + "url": "https://core.harbor.domain", + "authorization": "Bearer JWTTOKENGOESHERE" + }, + "artifact": { + "repository": "library/oracle/nosql", + "digest": "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b" + }, + "enabled_capabilities": [ + { + "type": "sbom", + "produces_mime_types": [ + "application/vnd.security.sbom.report+json; version=1.0" + ], + "parameters": { + "sbom_media_types": [ + "application/spdx+json" + ] + } + } + ] +}`)) + + // then + require.NoError(t, err) + assert.Equal(t, http.StatusAccepted, rs.StatusCode) + assert.Equal(t, "application/vnd.scanner.adapter.scan.response+json; version=1.0", rs.Header.Get("Content-Type")) + + bodyBytes, err := io.ReadAll(rs.Body) + require.NoError(t, err) + + assert.JSONEq(t, `{"id": "job:123"}`, string(bodyBytes)) + }) + + t.Run("GET /api/v1/scan/{scan_request_id}/report for vulnerabilities", func(t *testing.T) { // given now := time.Now() @@ -184,6 +245,69 @@ func TestRestApi(t *testing.T) { }`, now.Format(time.RFC3339Nano)), string(bodyBytes)) }) + t.Run("GET /api/v1/scan/{scan_request_id}/report for SBOM", func(t *testing.T) { + // given + now := time.Now() + + jobKey := job.ScanJobKey{ + ID: "job:123", + MIMEType: api.MimeTypeSecuritySBOMReport, + MediaType: api.MediaTypeSPDX, + } + store.On("Get", mock.Anything, jobKey).Return(&job.ScanJob{ + Key: jobKey, + Status: job.Finished, + Report: harbor.ScanReport{ + GeneratedAt: now, + Artifact: harbor.Artifact{ + Repository: "library/mongo", + Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", + }, + Scanner: harbor.Scanner{ + Name: "Trivy", + Vendor: "Aqua Security", + Version: "Unknown", + }, + MediaType: api.MediaTypeSPDX, + SBOM: "SPDX Document", + }, + }, nil) + + // when + values := url.Values{} + values.Add("sbom_media_type", "application/spdx+json") + req, err := http.NewRequest("GET", ts.URL+"/api/v1/scan/job:123/report?"+values.Encode(), nil) + require.NoError(t, err) + req.Header.Add("Accept", "application/vnd.security.sbom.report+json; version=1.0") + rs, err := ts.Client().Do(req) + require.NoError(t, err) + + // then + require.NoError(t, err) + assert.Equal(t, http.StatusOK, rs.StatusCode) + assert.Equal(t, "application/vnd.security.sbom.report+json; version=1.0", rs.Header.Get("Content-Type")) + + bodyBytes, err := io.ReadAll(rs.Body) + require.NoError(t, err) + + expectedSBOMReport := fmt.Sprintf(`{ + "generated_at": "%s", + "artifact": { + "repository": "library/mongo", + "digest": "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b" + }, + "scanner": { + "name": "Trivy", + "vendor": "Aqua Security", + "version": "Unknown" + }, + "media_type": "application/spdx+json", + "sbom": "SPDX Document" + }`, now.Format(time.RFC3339Nano)) + + assert.JSONEq(t, expectedSBOMReport, string(bodyBytes)) + }) + t.Run("GET /api/v1/metadata", func(t *testing.T) { wrapper.On("GetVersion").Return(trivy.VersionInfo{ Version: "v0.5.2-17-g3c9af62", From af20c7fe583a34fd6c0451690b45bf06528f57d1 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Thu, 18 Jan 2024 16:11:55 +0400 Subject: [PATCH 19/26] test: add a component test for SBOM generation Signed-off-by: knqyf263 --- test/component/component_test.go | 113 +++++++++++++++++++------------ test/component/scanner/client.go | 20 ++++-- 2 files changed, 84 insertions(+), 49 deletions(-) diff --git a/test/component/component_test.go b/test/component/component_test.go index bd2da68d..57d53c2b 100644 --- a/test/component/component_test.go +++ b/test/component/component_test.go @@ -5,6 +5,7 @@ package component import ( "context" "fmt" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/http/api" "net/url" "path/filepath" "testing" @@ -147,52 +148,80 @@ func TestComponent(t *testing.T) { Password: registryPassword, } - testCases := []struct { - repository string - tag string - }{ - { - repository: "alpine", - tag: "3.14", - }, + const ( + repository = "alpine" + tag = "3.14" + ) + imageRef := fmt.Sprintf("%s:%s", repository, tag) + + // 0. Download a test image from DockerHub, tag it and push to the test registry. + artifactDigest, err := docker.ReplicateImage(imageRef, config) + require.NoError(t, err) + + artifact := harbor.Artifact{ + Repository: repository, + Digest: artifactDigest.String(), } - for _, cc := range testCases { - imageRef := fmt.Sprintf("%s:%s", cc.repository, cc.tag) - t.Run(fmt.Sprintf("Should scan %s", imageRef), func(t *testing.T) { - - // 1. Download a test image from DockerHub, tag it and push to the test registry. - artifactDigest, err := docker.ReplicateImage(imageRef, config) - require.NoError(t, err) - - artifact := harbor.Artifact{ - Repository: cc.repository, - Digest: artifactDigest.String(), - } - - c := scanner.NewClient(adapterURL) - // 2. Send ScanRequest to Scanner Adapter. - resp, err := c.RequestScan(harbor.ScanRequest{ - Registry: harbor.Registry{ - URL: registryInternalURL, - Authorization: config.GetBasicAuthorization(), + t.Run("scan image for vulnerabilities", func(t *testing.T) { + c := scanner.NewClient(adapterURL) + + // 1. Send ScanRequest to Scanner Adapter. + resp, err := c.RequestScan(harbor.ScanRequest{ + Registry: harbor.Registry{ + URL: registryInternalURL, + Authorization: config.GetBasicAuthorization(), + }, + Artifact: artifact, + }) + require.NoError(t, err) + + // 2. Poll Scanner Adapter for ScanReport. + report, err := c.GetScanReport(resp.ID, api.MimeTypeSecurityVulnerabilityReport.String(), "") + require.NoError(t, err) + + assert.Equal(t, artifact, report.Artifact) + assert.Equal(t, trivyScanner, report.Scanner) + // TODO Adding asserts on CVEs is tricky as we do not have any control over upstream vulnerabilities database used by Trivy. + for _, v := range report.Vulnerabilities { + t.Logf("ID %s, Package: %s, Version: %s, Severity: %s", v.ID, v.Pkg, v.Version, v.Severity) + } + }) + + t.Run("scan image for SBOM", func(t *testing.T) { + c := scanner.NewClient(adapterURL) + // 1. Send ScanRequest to Scanner Adapter. + resp, err := c.RequestScan(harbor.ScanRequest{ + Registry: harbor.Registry{ + URL: registryInternalURL, + Authorization: config.GetBasicAuthorization(), + }, + Artifact: artifact, + Capabilities: []harbor.Capability{ + { + Type: harbor.CapabilityTypeSBOM, + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecuritySBOMReport, + }, + Parameters: &harbor.CapabilityAttributes{ + SBOMMediaTypes: []api.MediaType{ + api.MediaTypeSPDX, + }, + }, }, - Artifact: artifact, - }) - require.NoError(t, err) - - // 3. Poll Scanner Adapter for ScanReport. - report, err := c.GetScanReport(resp.ID) - require.NoError(t, err) - - assert.Equal(t, artifact, report.Artifact) - assert.Equal(t, trivyScanner, report.Scanner) - // TODO Adding asserts on CVEs is tricky as we do not have any control over upstream vulnerabilities database used by Trivy. - for _, v := range report.Vulnerabilities { - t.Logf("ID %s, Package: %s, Version: %s, Severity: %s", v.ID, v.Pkg, v.Version, v.Severity) - } + }, }) - } + require.NoError(t, err) + + // 2. Poll Scanner Adapter for ScanReport. + report, err := c.GetScanReport(resp.ID, api.MimeTypeSecuritySBOMReport.String(), string(api.MediaTypeSPDX)) + require.NoError(t, err) + + assert.Equal(t, artifact, report.Artifact) + assert.Equal(t, trivyScanner, report.Scanner) + assert.Equal(t, api.MediaTypeSPDX, report.MediaType) + assert.NotEmpty(t, report.SBOM) + }) if t.Failed() { time.Sleep(15 * time.Second) diff --git a/test/component/scanner/client.go b/test/component/scanner/client.go index d1aa8d92..97e045a4 100644 --- a/test/component/scanner/client.go +++ b/test/component/scanner/client.go @@ -7,6 +7,7 @@ import ( "encoding/json" "fmt" "net/http" + "net/url" "strings" "time" @@ -57,11 +58,11 @@ func (c *Client) RequestScan(request harbor.ScanRequest) (scanResp harbor.ScanRe } // GetScanReport polls for ScanReport associated with the given ScanRequest ID. -func (c *Client) GetScanReport(scanRequestID string) (report harbor.ScanReport, err error) { - res, err := c.doGetScanReport(scanRequestID) +func (c *Client) GetScanReport(scanRequestID, mimeType, mediaType string) (report harbor.ScanReport, err error) { + res, err := c.doGetScanReport(scanRequestID, mimeType, mediaType) for err == nil && res.StatusCode == http.StatusFound { time.Sleep(10 * time.Second) - res, err = c.doGetScanReport(scanRequestID) + res, err = c.doGetScanReport(scanRequestID, mimeType, mediaType) } if err != nil { return @@ -76,13 +77,18 @@ func (c *Client) GetScanReport(scanRequestID string) (report harbor.ScanReport, return } -func (c *Client) doGetScanReport(scanRequestID string) (*http.Response, error) { - url := fmt.Sprintf("%s/api/v1/scan/%s/report", c.endpointURL, scanRequestID) - req, err := http.NewRequest(http.MethodGet, url, nil) +func (c *Client) doGetScanReport(scanRequestID, mimeType, mediaType string) (*http.Response, error) { + u := fmt.Sprintf("%s/api/v1/scan/%s/report", c.endpointURL, scanRequestID) + if mediaType != "" { + values := url.Values{} + values.Add("sbom_media_type", mediaType) + u += "?" + values.Encode() + } + req, err := http.NewRequest(http.MethodGet, u, nil) if err != nil { return nil, err } - req.Header.Set("Accept", "application/vnd.security.vulnerability.report; version=1.1") + req.Header.Set("Accept", mimeType) return http.DefaultTransport.RoundTrip(req) } From bcb93c3248ca18324f15254cd11c02032712554a Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Fri, 19 Jan 2024 14:12:14 +0400 Subject: [PATCH 20/26] refactor: remove ioutil Signed-off-by: knqyf263 --- test/component/docker/docker.go | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/test/component/docker/docker.go b/test/component/docker/docker.go index 7bd853ee..8020c74d 100644 --- a/test/component/docker/docker.go +++ b/test/component/docker/docker.go @@ -8,13 +8,11 @@ import ( "encoding/base64" "encoding/json" "fmt" - "io" - "io/ioutil" - "net/url" - "github.com/docker/docker/api/types" "github.com/docker/docker/client" "github.com/opencontainers/go-digest" + "io" + "net/url" ) type RegistryConfig struct { @@ -53,7 +51,7 @@ func ReplicateImage(imageRef string, dest RegistryConfig) (d digest.Digest, err _ = pullOut.Close() }() - _, err = io.Copy(ioutil.Discard, pullOut) + _, err = io.Copy(io.Discard, pullOut) if err != nil { return } @@ -76,7 +74,7 @@ func ReplicateImage(imageRef string, dest RegistryConfig) (d digest.Digest, err defer func() { _ = pushOut.Close() }() - _, err = io.Copy(ioutil.Discard, pushOut) + _, err = io.Copy(io.Discard, pushOut) inspect, err := cli.DistributionInspect(ctx, targetImageRef, auth) if err != nil { return From 6102cbf8bcfc93441b7b3b029d5efa86ffa2f302 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Mon, 22 Jan 2024 14:34:35 +0400 Subject: [PATCH 21/26] test(integration): remove mock Signed-off-by: knqyf263 --- go.mod | 9 + go.sum | 49 ++ pkg/etc/config.go | 2 +- pkg/etc/config_test.go | 38 +- pkg/harbor/model.go | 22 +- pkg/http/api/v1/handler.go | 2 +- pkg/http/api/v1/handler_test.go | 14 +- pkg/trivy/wrapper.go | 2 +- pkg/trivy/wrapper_test.go | 4 +- test/integration/api/rest_api_test.go | 717 +++++++++++------- test/integration/api/testdata/alpine.spdx | 54 ++ .../api/testdata/fixtures/alpine.yaml | 12 + .../api/testdata/fixtures/data-source.yaml | 7 + .../api/testdata/fixtures/vulnerability.yaml | 18 + 14 files changed, 646 insertions(+), 304 deletions(-) create mode 100644 test/integration/api/testdata/alpine.spdx create mode 100644 test/integration/api/testdata/fixtures/alpine.yaml create mode 100644 test/integration/api/testdata/fixtures/data-source.yaml create mode 100644 test/integration/api/testdata/fixtures/vulnerability.yaml diff --git a/go.mod b/go.mod index 1843fbba..b923e820 100644 --- a/go.mod +++ b/go.mod @@ -3,6 +3,8 @@ module github.com/aquasecurity/harbor-scanner-trivy go 1.21 require ( + github.com/alicebob/miniredis/v2 v2.31.1 + github.com/aquasecurity/bolt-fixtures v0.0.0-20200903104109-d34e7f983986 github.com/caarlos0/env/v6 v6.10.1 github.com/docker/docker v24.0.7+incompatible github.com/docker/go-connections v0.5.0 @@ -24,6 +26,7 @@ require ( github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 // indirect github.com/Microsoft/go-winio v0.6.1 // indirect github.com/Microsoft/hcsshim v0.11.4 // indirect + github.com/alicebob/gopher-json v0.0.0-20200520072559-a9ecdc9d1d3a // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect @@ -37,13 +40,17 @@ require ( github.com/docker/distribution v2.8.2+incompatible // indirect github.com/docker/docker-credential-helpers v0.7.0 // indirect github.com/docker/go-units v0.5.0 // indirect + github.com/fatih/color v1.9.0 // indirect github.com/go-ole/go-ole v1.2.6 // indirect + github.com/goccy/go-yaml v1.8.1 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/protobuf v1.5.3 // indirect github.com/google/uuid v1.3.1 // indirect github.com/klauspost/compress v1.16.5 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/magiconair/properties v1.8.7 // indirect + github.com/mattn/go-colorable v0.1.7 // indirect + github.com/mattn/go-isatty v0.0.12 // indirect github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 // indirect github.com/mitchellh/go-homedir v1.1.0 // indirect github.com/moby/patternmatcher v0.6.0 // indirect @@ -65,7 +72,9 @@ require ( github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/vbatts/tar-split v0.11.3 // indirect + github.com/yuin/gopher-lua v1.1.0 // indirect github.com/yusufpapurcu/wmi v1.2.3 // indirect + go.etcd.io/bbolt v1.3.7 // indirect golang.org/x/exp v0.0.0-20230510235704-dd950f8aeaea // indirect golang.org/x/mod v0.10.0 // indirect golang.org/x/sync v0.3.0 // indirect diff --git a/go.sum b/go.sum index 222e1672..ae48bc03 100644 --- a/go.sum +++ b/go.sum @@ -6,10 +6,17 @@ github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 h1:UQHMgLO+TxOEl github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1/go.mod h1:xomTg63KZ2rFqZQzSB4Vz2SUXa1BpHTVz9L5PTmPC4E= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/toml v1.2.1/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= +github.com/DmitriyVTitov/size v1.5.0/go.mod h1:le6rNI4CoLQV1b9gzp1+3d7hMAD/uu2QcJ+aYbNgiU0= github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= github.com/Microsoft/hcsshim v0.11.4 h1:68vKo2VN8DE9AdN4tnkWnmdhqdbpUFM8OF3Airm7fz8= github.com/Microsoft/hcsshim v0.11.4/go.mod h1:smjE4dvqPX9Zldna+t5FG3rnoHhaB7QYxPRqGcpAD9w= +github.com/alicebob/gopher-json v0.0.0-20200520072559-a9ecdc9d1d3a h1:HbKu58rmZpUGpz5+4FfNmIU+FmZg2P3Xaj2v2bfNWmk= +github.com/alicebob/gopher-json v0.0.0-20200520072559-a9ecdc9d1d3a/go.mod h1:SGnFV6hVsYE877CKEZ6tDNTjaSXYUk6QqoIK6PrAtcc= +github.com/alicebob/miniredis/v2 v2.31.1 h1:7XAt0uUg3DtwEKW5ZAGa+K7FZV2DdKQo5K/6TTnfX8Y= +github.com/alicebob/miniredis/v2 v2.31.1/go.mod h1:UB/T2Uztp7MlFSDakaX1sTXUv5CASoprx0wulRT6HBg= +github.com/aquasecurity/bolt-fixtures v0.0.0-20200903104109-d34e7f983986 h1:2a30xLN2sUZcMXl50hg+PJCIDdJgIvIbVcKqLJ/ZrtM= +github.com/aquasecurity/bolt-fixtures v0.0.0-20200903104109-d34e7f983986/go.mod h1:NT+jyeCzXk6vXR5MTkdn4z64TgGfE5HMLC8qfj5unl8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bsm/ginkgo/v2 v2.12.0 h1:Ny8MWAHyOepLGlLKYmXG4IEkioBysk6GpaRTLC8zwWs= @@ -23,6 +30,9 @@ github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyY github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/checkpoint-restore/go-criu/v5 v5.3.0/go.mod h1:E/eQpaFtUKGOOSEBZgmKAcn+zUUwWxqcaKZlF54wK8E= +github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= +github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= +github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/cilium/ebpf v0.7.0/go.mod h1:/oI2+1shJiTGAMgl6/RgJr36Eo1jzrRcAWbcXO2usCA= github.com/containerd/console v1.0.3/go.mod h1:7LqA/THxQ86k76b8c/EMSiaJ3h1eZkMkXar0TQ1gf3U= github.com/containerd/containerd v1.7.7 h1:QOC2K4A42RQpcrZyptP6z9EJZnlHfHJUfZrAAHe15q4= @@ -57,13 +67,23 @@ github.com/docker/go-connections v0.5.0/go.mod h1:ov60Kzw0kKElRwhNs9UlUHAE/F9Fe6 github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= github.com/docker/go-units v0.5.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= +github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= +github.com/fatih/color v1.9.0 h1:8xPHl4/q1VyqGIPif1F+1V3Y3lSmrq01EabUW3CoW5s= +github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU= github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= +github.com/go-playground/locales v0.13.0 h1:HyWk6mgj5qFqCT5fjGBuRArbVDfE4hi8+e8ceBS/t7Q= +github.com/go-playground/locales v0.13.0/go.mod h1:taPMhCMXrRLJO55olJkUXHZBHCxTMfnGwq/HNwmWNS8= +github.com/go-playground/universal-translator v0.17.0 h1:icxd5fm+REJzpZx7ZfpaD876Lmtgy7VtROAbHHXk8no= +github.com/go-playground/universal-translator v0.17.0/go.mod h1:UkSxE5sNxxRwHyU+Scu5vgOQjsIJAF8j9muTVoKLVtA= +github.com/goccy/go-yaml v1.8.1 h1:JuZRFlqLM5cWF6A+waL8AKVuCcqvKOuhJtUQI+L3ez0= +github.com/goccy/go-yaml v1.8.1/go.mod h1:wS4gNoLalDSJxo/SpngzPQ2BN4uuZVLCmbM4S3vd4+Y= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/godbus/dbus/v5 v5.0.6/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= +github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= @@ -90,10 +110,20 @@ github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3x github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/leodido/go-urn v1.2.0 h1:hpXL4XnriNwQ/ABnpepYM/1vCLWNDfUNts8dX3xTG6Y= +github.com/leodido/go-urn v1.2.0/go.mod h1:+8+nEpDfqqsY+g338gtMEUOtuK+4dEMhiQEgxpxOKII= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/magiconair/properties v1.8.7 h1:IeQXZAiQcpL9mgcAe1Nu6cX9LLw6ExEHKjN0VQdvPDY= github.com/magiconair/properties v1.8.7/go.mod h1:Dhd985XPs7jluiymwWYZ0G4Z61jb3vdS329zhj2hYo0= +github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= +github.com/mattn/go-colorable v0.1.7 h1:bQGKb3vps/j0E9GfJQ03JyhRuxsvdAanXlT9BTw3mdw= +github.com/mattn/go-colorable v0.1.7/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= +github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= +github.com/mattn/go-isatty v0.0.10/go.mod h1:qgIWMr58cqv1PHHyhnkY9lrL7etaEgOFcMEpPG5Rm84= +github.com/mattn/go-isatty v0.0.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOAqxQCu2WE= +github.com/mattn/go-isatty v0.0.12 h1:wuysRhFDzyxgEmMf5xjvJ2M9dZoWAXNNr5LSBS7uHXY= +github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 h1:jWpvCLoY8Z/e3VKvlsiIGKtc+UG6U5vzxaoagmhXfyg= github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0/go.mod h1:QUyp042oQthUoa9bqDv0ER0wrtXnBruoNd7aNjkbP+k= github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y= @@ -155,6 +185,8 @@ github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSS github.com/stretchr/objx v0.5.0 h1:1zr/of2m5FGMsad5YfcqgdqdWrIhu+EBEJRhR1U7z/c= github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= @@ -176,8 +208,13 @@ github.com/vishvananda/netlink v1.1.0/go.mod h1:cTgwzPIzzgDAYoQrMm0EdrjRUBkTqKYp github.com/vishvananda/netns v0.0.0-20191106174202-0a2b9b5464df/go.mod h1:JP3t17pCcGlemwknint6hfoeCVQrEMVwxRLRjXpq+BU= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/gopher-lua v1.1.0 h1:BojcDhfyDWgU2f2TOzYK/g5p2gxMrku8oupLDqlnSqE= +github.com/yuin/gopher-lua v1.1.0/go.mod h1:GBR0iDaNXjAgGg9zfCvksxSRnQx76gclCIb7kdAd1Pw= github.com/yusufpapurcu/wmi v1.2.3 h1:E1ctvB7uKFMOJw3fdOW32DwGE9I7t++CRUEMKvFoFiw= github.com/yusufpapurcu/wmi v1.2.3/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= +go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= +go.etcd.io/bbolt v1.3.7 h1:j+zJOnnEjF/kyHlDDgGnVL/AIqIJPq8UoB2GSNfkUfQ= +go.etcd.io/bbolt v1.3.7/go.mod h1:N9Mkw9X8x5fupy0IKsmuqVtoGDyxsaDlbk4Rd05IAQw= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= @@ -199,12 +236,20 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.3.0 h1:ftCYgMx6zT/asHUrPw8BLLscYtGznsLAnjq5RH9P66E= golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= +golang.org/x/sys v0.0.0-20190204203706-41f3e6584952/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190606203320-7fc4e5ec1444/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191008105621-543471e840be/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191010194322-b09406accb47/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191115151921-52ab43148777/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200831180312-196b9ba8737a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -222,6 +267,7 @@ golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= @@ -250,6 +296,9 @@ google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqw gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= +gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= +gopkg.in/go-playground/validator.v9 v9.30.0 h1:Wk0Z37oBmKj9/n+tPyBHZmeL19LaCoK3Qq48VwYENss= +gopkg.in/go-playground/validator.v9 v9.30.0/go.mod h1:+c9/zcJMFNgbLvly1L1V+PpxWdVbfP1avr/N00E2vyQ= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= diff --git a/pkg/etc/config.go b/pkg/etc/config.go index 3b106f77..ac301114 100644 --- a/pkg/etc/config.go +++ b/pkg/etc/config.go @@ -28,7 +28,7 @@ type Trivy struct { ReportsDir string `env:"SCANNER_TRIVY_REPORTS_DIR" envDefault:"/home/scanner/.cache/reports"` DebugMode bool `env:"SCANNER_TRIVY_DEBUG_MODE" envDefault:"false"` VulnType string `env:"SCANNER_TRIVY_VULN_TYPE" envDefault:"os,library"` - SecurityChecks string `env:"SCANNER_TRIVY_SECURITY_CHECKS" envDefault:"vuln"` + Scanners string `env:"SCANNER_TRIVY_SECURITY_CHECKS" envDefault:"vuln"` Severity string `env:"SCANNER_TRIVY_SEVERITY" envDefault:"UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL"` IgnoreUnfixed bool `env:"SCANNER_TRIVY_IGNORE_UNFIXED" envDefault:"false"` IgnorePolicy string `env:"SCANNER_TRIVY_IGNORE_POLICY"` diff --git a/pkg/etc/config_test.go b/pkg/etc/config_test.go index 1108e4a7..128aeb07 100644 --- a/pkg/etc/config_test.go +++ b/pkg/etc/config_test.go @@ -65,15 +65,15 @@ func TestGetConfig(t *testing.T) { IdleTimeout: parseDuration(t, "60s"), }, Trivy: Trivy{ - DebugMode: true, - CacheDir: "/home/scanner/.cache/trivy", - ReportsDir: "/home/scanner/.cache/reports", - VulnType: "os,library", - SecurityChecks: "vuln", - Severity: "UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL", - Insecure: false, - GitHubToken: "", - Timeout: parseDuration(t, "5m0s"), + DebugMode: true, + CacheDir: "/home/scanner/.cache/trivy", + ReportsDir: "/home/scanner/.cache/reports", + VulnType: "os,library", + Scanners: "vuln", + Severity: "UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL", + Insecure: false, + GitHubToken: "", + Timeout: parseDuration(t, "5m0s"), }, RedisPool: RedisPool{ URL: "redis://localhost:6379", @@ -104,15 +104,15 @@ func TestGetConfig(t *testing.T) { IdleTimeout: parseDuration(t, "60s"), }, Trivy: Trivy{ - DebugMode: false, - CacheDir: "/home/scanner/.cache/trivy", - ReportsDir: "/home/scanner/.cache/reports", - VulnType: "os,library", - SecurityChecks: "vuln", - Severity: "UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL", - Insecure: false, - GitHubToken: "", - Timeout: parseDuration(t, "5m0s"), + DebugMode: false, + CacheDir: "/home/scanner/.cache/trivy", + ReportsDir: "/home/scanner/.cache/reports", + VulnType: "os,library", + Scanners: "vuln", + Severity: "UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL", + Insecure: false, + GitHubToken: "", + Timeout: parseDuration(t, "5m0s"), }, RedisPool: RedisPool{ URL: "redis://localhost:6379", @@ -188,7 +188,7 @@ func TestGetConfig(t *testing.T) { ReportsDir: "/home/scanner/trivy-reports", DebugMode: true, VulnType: "os,library", - SecurityChecks: "vuln", + Scanners: "vuln", Severity: "CRITICAL", IgnoreUnfixed: true, SkipDBUpdate: true, diff --git a/pkg/harbor/model.go b/pkg/harbor/model.go index 7f42caa1..26138bca 100644 --- a/pkg/harbor/model.go +++ b/pkg/harbor/model.go @@ -151,17 +151,17 @@ type CVSSDetails struct { // VulnerabilityItem is an item in the vulnerability result returned by vulnerability details API. type VulnerabilityItem struct { - ID string `json:"id"` - Pkg string `json:"package"` - Version string `json:"version"` - FixVersion string `json:"fix_version,omitempty"` - Severity Severity `json:"severity"` - Description string `json:"description"` - Links []string `json:"links"` - Layer *Layer `json:"layer"` // Not defined by Scanners API - PreferredCVSS *CVSSDetails `json:"preferred_cvss,omitempty"` - CweIDs []string `json:"cwe_ids,omitempty"` - VendorAttributes map[string]interface{} `json:"vendor_attributes,omitempty"` + ID string `json:"id"` + Pkg string `json:"package"` + Version string `json:"version"` + FixVersion string `json:"fix_version,omitempty"` + Severity Severity `json:"severity"` + Description string `json:"description"` + Links []string `json:"links"` + Layer *Layer `json:"layer"` // Not defined by Scanners API + PreferredCVSS *CVSSDetails `json:"preferred_cvss,omitempty"` + CweIDs []string `json:"cwe_ids,omitempty"` + VendorAttributes map[string]any `json:"vendor_attributes,omitempty"` } type ScannerAdapterMetadata struct { diff --git a/pkg/http/api/v1/handler.go b/pkg/http/api/v1/handler.go index 0485767f..c6cc1e72 100644 --- a/pkg/http/api/v1/handler.go +++ b/pkg/http/api/v1/handler.go @@ -297,7 +297,7 @@ func (h *requestHandler) GetMetadata(res http.ResponseWriter, _ *http.Request) { "env.SCANNER_TRIVY_DEBUG_MODE": strconv.FormatBool(h.config.Trivy.DebugMode), "env.SCANNER_TRIVY_INSECURE": strconv.FormatBool(h.config.Trivy.Insecure), "env.SCANNER_TRIVY_VULN_TYPE": h.config.Trivy.VulnType, - "env.SCANNER_TRIVY_SECURITY_CHECKS": h.config.Trivy.SecurityChecks, + "env.SCANNER_TRIVY_SECURITY_CHECKS": h.config.Trivy.Scanners, "env.SCANNER_TRIVY_SEVERITY": h.config.Trivy.Severity, "env.SCANNER_TRIVY_TIMEOUT": h.config.Trivy.Timeout.String(), } diff --git a/pkg/http/api/v1/handler_test.go b/pkg/http/api/v1/handler_test.go index 52378262..e74a76b1 100644 --- a/pkg/http/api/v1/handler_test.go +++ b/pkg/http/api/v1/handler_test.go @@ -668,7 +668,7 @@ func TestRequestHandler_GetMetadata(t *testing.T) { DebugMode: true, Insecure: true, VulnType: "os,library", - SecurityChecks: "vuln", + Scanners: "vuln", Severity: "UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL", Timeout: 5 * time.Minute, }, @@ -747,7 +747,7 @@ func TestRequestHandler_GetMetadata(t *testing.T) { DebugMode: true, Insecure: true, VulnType: "os,library", - SecurityChecks: "vuln", + Scanners: "vuln", Severity: "UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL", Timeout: 5 * time.Minute, }, @@ -816,11 +816,11 @@ func TestRequestHandler_GetMetadata(t *testing.T) { }, config: etc.Config{ Trivy: etc.Trivy{ - VulnType: "os,library", - SecurityChecks: "vuln", - Severity: "UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL", - Timeout: 5 * time.Minute, - OfflineScan: true, + VulnType: "os,library", + Scanners: "vuln", + Severity: "UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL", + Timeout: 5 * time.Minute, + OfflineScan: true, }, }, expectedHTTPCode: http.StatusOK, diff --git a/pkg/trivy/wrapper.go b/pkg/trivy/wrapper.go index 81afbe3e..f5c4a1c4 100644 --- a/pkg/trivy/wrapper.go +++ b/pkg/trivy/wrapper.go @@ -180,7 +180,7 @@ func (w *wrapper) prepareScanCmd(target ScanTarget, outputFile string, opt ScanO } if target.kind == TargetImage { - args = append(args, "--scanners", w.config.SecurityChecks) + args = append(args, "--scanners", w.config.Scanners) } if w.config.IgnoreUnfixed { diff --git a/pkg/trivy/wrapper_test.go b/pkg/trivy/wrapper_test.go index c5dba510..89d87514 100644 --- a/pkg/trivy/wrapper_test.go +++ b/pkg/trivy/wrapper_test.go @@ -109,7 +109,7 @@ func TestWrapper_Scan(t *testing.T) { ReportsDir: reportsDir, DebugMode: true, VulnType: "os,library", - SecurityChecks: "vuln", + Scanners: "vuln", Severity: "CRITICAL,MEDIUM", IgnoreUnfixed: true, IgnorePolicy: "/home/scanner/opa/policy.rego", @@ -196,7 +196,7 @@ func TestWrapper_Scan(t *testing.T) { config := etc.Trivy{ CacheDir: cacheDir, ReportsDir: reportsDir, - SecurityChecks: "vuln", + Scanners: "vuln", VulnType: "library", Severity: "CRITICAL", SkipDBUpdate: true, diff --git a/test/integration/api/rest_api_test.go b/test/integration/api/rest_api_test.go index bfe5160f..339215fe 100644 --- a/test/integration/api/rest_api_test.go +++ b/test/integration/api/rest_api_test.go @@ -3,22 +3,42 @@ package api import ( + "bytes" + "context" + "encoding/json" "fmt" + fixtures "github.com/aquasecurity/bolt-fixtures" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/ext" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" "github.com/aquasecurity/harbor-scanner-trivy/pkg/http/api" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/persistence" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/queue" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/redisx" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/scan" + "github.com/google/go-containerregistry/pkg/crane" + "github.com/google/go-containerregistry/pkg/name" + "github.com/google/go-containerregistry/pkg/registry" + ggcr "github.com/google/go-containerregistry/pkg/v1" + "github.com/google/go-containerregistry/pkg/v1/empty" + "github.com/google/go-containerregistry/pkg/v1/mutate" + "github.com/google/go-containerregistry/pkg/v1/remote" + "github.com/google/go-containerregistry/pkg/v1/tarball" "io" "net/http" "net/http/httptest" "net/url" - "strings" + "os" + "path/filepath" + "runtime" "testing" "time" + "github.com/alicebob/miniredis/v2" "github.com/aquasecurity/harbor-scanner-trivy/pkg/etc" - "github.com/aquasecurity/harbor-scanner-trivy/pkg/harbor" - v1 "github.com/aquasecurity/harbor-scanner-trivy/pkg/http/api/v1" - "github.com/aquasecurity/harbor-scanner-trivy/pkg/job" - "github.com/aquasecurity/harbor-scanner-trivy/pkg/mock" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/http/api/v1" + "github.com/aquasecurity/harbor-scanner-trivy/pkg/persistence/redis" "github.com/aquasecurity/harbor-scanner-trivy/pkg/trivy" + goredis "github.com/redis/go-redis/v9" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -30,9 +50,26 @@ func TestRestAPI(t *testing.T) { t.Skip("An integration test") } - enqueuer := mock.NewEnqueuer() - store := mock.NewStore() - wrapper := trivy.NewMockWrapper() + ctx := context.Background() + now := time.Now() + + jobQueue := etc.JobQueue{ + Namespace: "test:job-queue", + WorkerConcurrency: 1, + } + + // Set up Redis + rdb, store := initRedis(t) + enqueuer := queue.NewEnqueuer(jobQueue, rdb, store) + + // Set up Trivy + wrapper, trivyConf := initTrivy(t, now) + + // Set up worker + initWorker(t, ctx, store, jobQueue, rdb, wrapper) + + // Set up registry + imageRef, sbomRef := initRegistry(t) app := v1.NewAPIHandler( etc.BuildInfo{ @@ -40,228 +77,181 @@ func TestRestAPI(t *testing.T) { Commit: "abc", Date: "2019-01-04T12:40", }, - etc.Config{ - Trivy: etc.Trivy{ - SkipDBUpdate: false, - SkipJavaDBUpdate: false, - IgnoreUnfixed: true, - DebugMode: true, - Insecure: true, - VulnType: "os,library", - Severity: "UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL", - SecurityChecks: "vuln", - Timeout: 5 * time.Minute, - }, - }, enqueuer, store, wrapper) + etc.Config{Trivy: trivyConf}, enqueuer, store, wrapper) ts := httptest.NewServer(app) - defer ts.Close() - - t.Run("POST /api/v1/scan for vulnerabilities", func(t *testing.T) { - // given - enqueuer.On("Enqueue", mock.Anything, harbor.ScanRequest{ - Capabilities: []harbor.Capability{ - { - Type: harbor.CapabilityTypeVulnerability, - ProducesMIMETypes: []api.MIMEType{ - api.MimeTypeSecurityVulnerabilityReport, - }, + t.Cleanup(ts.Close) + + t.Run("Scan image for vulnerabilities", func(t *testing.T) { + var scanJobID string + t.Run("POST /api/v1/scan", func(t *testing.T) { + // when + body := harbor.ScanRequest{ + Registry: harbor.Registry{ + URL: imageRef.Registry.Scheme() + "://" + imageRef.RegistryStr(), }, - }, - Registry: harbor.Registry{ - URL: "https://core.harbor.domain", - Authorization: "Bearer JWTTOKENGOESHERE", - }, - Artifact: harbor.Artifact{ - Repository: "library/oracle/nosql", - Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", - }, - }).Return("job:123", nil) - - // when - rs, err := ts.Client().Post(ts.URL+"/api/v1/scan", "application/json", strings.NewReader(`{ - "registry": { - "url": "https://core.harbor.domain", - "authorization": "Bearer JWTTOKENGOESHERE" - }, - "artifact": { - "repository": "library/oracle/nosql", - "digest": "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b" - } -}`)) - - // then - require.NoError(t, err) - assert.Equal(t, http.StatusAccepted, rs.StatusCode) - assert.Equal(t, "application/vnd.scanner.adapter.scan.response+json; version=1.0", rs.Header.Get("Content-Type")) - - bodyBytes, err := io.ReadAll(rs.Body) - require.NoError(t, err) - - assert.JSONEq(t, `{"id": "job:123"}`, string(bodyBytes)) - }) - - t.Run("POST /api/v1/scan for SBOM", func(t *testing.T) { - // given - enqueuer.On("Enqueue", mock.Anything, harbor.ScanRequest{ - Capabilities: []harbor.Capability{ - { - Type: harbor.CapabilityTypeSBOM, - ProducesMIMETypes: []api.MIMEType{ - api.MimeTypeSecuritySBOMReport, - }, - Parameters: &harbor.CapabilityAttributes{ - SBOMMediaTypes: []api.MediaType{api.MediaTypeSPDX}, - }, + Artifact: harbor.Artifact{ + Repository: imageRef.RepositoryStr(), + Digest: imageRef.DigestStr(), }, - }, - Registry: harbor.Registry{ - URL: "https://core.harbor.domain", - Authorization: "Bearer JWTTOKENGOESHERE", - }, - Artifact: harbor.Artifact{ - Repository: "library/oracle/nosql", - Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", - }, - }).Return("job:123", nil) - - // when - rs, err := ts.Client().Post(ts.URL+"/api/v1/scan", "application/json", strings.NewReader(`{ - "registry": { - "url": "https://core.harbor.domain", - "authorization": "Bearer JWTTOKENGOESHERE" - }, - "artifact": { - "repository": "library/oracle/nosql", - "digest": "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b" - }, - "enabled_capabilities": [ - { - "type": "sbom", - "produces_mime_types": [ - "application/vnd.security.sbom.report+json; version=1.0" - ], - "parameters": { - "sbom_media_types": [ - "application/spdx+json" - ] - } - } - ] -}`)) - - // then - require.NoError(t, err) - assert.Equal(t, http.StatusAccepted, rs.StatusCode) - assert.Equal(t, "application/vnd.scanner.adapter.scan.response+json; version=1.0", rs.Header.Get("Content-Type")) - - bodyBytes, err := io.ReadAll(rs.Body) - require.NoError(t, err) - - assert.JSONEq(t, `{"id": "job:123"}`, string(bodyBytes)) - }) - - t.Run("GET /api/v1/scan/{scan_request_id}/report for vulnerabilities", func(t *testing.T) { - // given - now := time.Now() - - jobKey := job.ScanJobKey{ - ID: "job:123", - MIMEType: api.MimeTypeSecurityVulnerabilityReport, - } - store.On("Get", mock.Anything, jobKey).Return(&job.ScanJob{ - Key: jobKey, - Status: job.Finished, - Report: harbor.ScanReport{ - GeneratedAt: now, + } + buf := &bytes.Buffer{} + err := json.NewEncoder(buf).Encode(body) + require.NoError(t, err) + rs, err := ts.Client().Post(ts.URL+"/api/v1/scan", "application/json", buf) + + // then + require.NoError(t, err) + assert.Equal(t, http.StatusAccepted, rs.StatusCode) + assert.Equal(t, "application/vnd.scanner.adapter.scan.response+json; version=1.0", rs.Header.Get("Content-Type")) + + var res harbor.ScanResponse + err = json.NewDecoder(rs.Body).Decode(&res) + require.NoError(t, err) + assert.NotEmpty(t, res.ID) + + scanJobID = res.ID + }) + + t.Run("GET /api/v1/scan/{scan_request_id}/report", func(t *testing.T) { + time.Sleep(3 * time.Second) + + rs, err := ts.Client().Get(fmt.Sprintf("%s/api/v1/scan/%s/report", ts.URL, scanJobID)) + require.NoError(t, err) + defer rs.Body.Close() + + // then + assert.Equal(t, http.StatusOK, rs.StatusCode) + assert.Equal(t, "application/vnd.security.vulnerability.report; version=1.1", rs.Header.Get("Content-Type")) + + // Parse response body + var got harbor.ScanReport + err = json.NewDecoder(rs.Body).Decode(&got) + require.NoError(t, err) + got.GeneratedAt = time.Time{} + + want := harbor.ScanReport{ Artifact: harbor.Artifact{ - Repository: "library/mongo", - Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", + Repository: imageRef.RepositoryStr(), + Digest: imageRef.DigestStr(), }, Scanner: harbor.Scanner{ Name: "Trivy", Vendor: "Aqua Security", Version: "Unknown", }, - Severity: harbor.SevCritical, + Severity: harbor.SevMedium, Vulnerabilities: []harbor.VulnerabilityItem{ { - ID: "CVE-2019-1111", - Pkg: "openssl", - Version: "2.0-rc1", - FixVersion: "2.1", - Severity: harbor.SevCritical, - Description: "You'd better upgrade your server", + ID: "CVE-2020-28928", + Pkg: "musl", + Version: "1.1.22-r4", + FixVersion: "1.1.22-r5", + Severity: harbor.SevMedium, + Description: "In musl libc through 1.2.1, wcsnrtombs mishandles particular combinations of destination buffer size and source character limit, as demonstrated by an invalid write access (buffer overflow).", + Links: []string{ + "https://avd.aquasec.com/nvd/cve-2020-28928", + }, + Layer: &harbor.Layer{ + Digest: "sha256:26d14edc4f17638cda363ea80b29c55e83058fc0dff1129b38ea3e8231217f7d", + DiffID: "sha256:e484d53633e7e5ebaabbe277838af1f26c388cbcebfa32e29fae72dd4086d54d", + }, + }, + { + ID: "CVE-2020-28928", + Pkg: "musl-utils", + Version: "1.1.22-r4", + FixVersion: "1.1.22-r5", + Severity: harbor.SevMedium, + Description: "In musl libc through 1.2.1, wcsnrtombs mishandles particular combinations of destination buffer size and source character limit, as demonstrated by an invalid write access (buffer overflow).", Links: []string{ - "http://cve.com?id=CVE-2019-1111", + "https://avd.aquasec.com/nvd/cve-2020-28928", }, Layer: &harbor.Layer{ - Digest: "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb10", + Digest: "sha256:26d14edc4f17638cda363ea80b29c55e83058fc0dff1129b38ea3e8231217f7d", + DiffID: "sha256:e484d53633e7e5ebaabbe277838af1f26c388cbcebfa32e29fae72dd4086d54d", }, }, }, - }, - }, nil) + } - // when - rs, err := ts.Client().Get(ts.URL + "/api/v1/scan/job:123/report") - require.NoError(t, err) - - // then - assert.Equal(t, http.StatusOK, rs.StatusCode) - assert.Equal(t, "application/vnd.security.vulnerability.report; version=1.1", rs.Header.Get("Content-Type")) - - bodyBytes, err := io.ReadAll(rs.Body) - require.NoError(t, err) - - assert.JSONEq(t, fmt.Sprintf(`{ - "generated_at": "%s", - "artifact": { - "repository": "library/mongo", - "digest": "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b" - }, - "scanner": { - "name": "Trivy", - "vendor": "Aqua Security", - "version": "Unknown" - }, - "severity": "Critical", - "vulnerabilities": [ - { - "id": "CVE-2019-1111", - "package": "openssl", - "version": "2.0-rc1", - "fix_version": "2.1", - "severity": "Critical", - "description": "You'd better upgrade your server", - "links": [ - "http://cve.com?id=CVE-2019-1111" - ], - "layer": { - "digest": "sha256:5216338b40a7b96416b8b9858974bbe4acc3096ee60acbc4dfb1ee02aecceb10" - } - } - ] -}`, now.Format(time.RFC3339Nano)), string(bodyBytes)) + assert.Equal(t, want, got) + }) }) - t.Run("GET /api/v1/scan/{scan_request_id}/report for SBOM", func(t *testing.T) { - // given - now := time.Now() - - jobKey := job.ScanJobKey{ - ID: "job:123", - MIMEType: api.MimeTypeSecuritySBOMReport, - MediaType: api.MediaTypeSPDX, - } - store.On("Get", mock.Anything, jobKey).Return(&job.ScanJob{ - Key: jobKey, - Status: job.Finished, - Report: harbor.ScanReport{ - GeneratedAt: now, + t.Run("Scan image for SBOM", func(t *testing.T) { + var scanJobID string + t.Run("POST /api/v1/scan", func(t *testing.T) { + // when + body := harbor.ScanRequest{ + Registry: harbor.Registry{ + URL: imageRef.Registry.Scheme() + "://" + imageRef.RegistryStr(), + }, + Artifact: harbor.Artifact{ + Repository: imageRef.RepositoryStr(), + Digest: imageRef.DigestStr(), + }, + Capabilities: []harbor.Capability{ + { + Type: harbor.CapabilityTypeSBOM, + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecuritySBOMReport, + }, + Parameters: &harbor.CapabilityAttributes{ + SBOMMediaTypes: []api.MediaType{ + api.MediaTypeSPDX, + }, + }, + }, + }, + } + buf := &bytes.Buffer{} + err := json.NewEncoder(buf).Encode(body) + require.NoError(t, err) + rs, err := ts.Client().Post(ts.URL+"/api/v1/scan", "application/json", buf) + + // then + require.NoError(t, err) + assert.Equal(t, http.StatusAccepted, rs.StatusCode) + assert.Equal(t, "application/vnd.scanner.adapter.scan.response+json; version=1.0", rs.Header.Get("Content-Type")) + + var res harbor.ScanResponse + err = json.NewDecoder(rs.Body).Decode(&res) + require.NoError(t, err) + assert.NotEmpty(t, res.ID) + + scanJobID = res.ID + }) + + t.Run("GET /api/v1/scan/{scan_request_id}/report", func(t *testing.T) { + time.Sleep(3 * time.Second) + + values := url.Values{} + values.Add("sbom_media_type", "application/spdx+json") + req, err := http.NewRequest("GET", + fmt.Sprintf("%s/api/v1/scan/%s/report?", ts.URL, scanJobID)+values.Encode(), nil) + require.NoError(t, err) + req.Header.Add("Accept", "application/vnd.security.sbom.report+json; version=1.0") + rs, err := ts.Client().Do(req) + require.NoError(t, err) + + // then + require.NoError(t, err) + assert.Equal(t, http.StatusOK, rs.StatusCode) + assert.Equal(t, "application/vnd.security.sbom.report+json; version=1.0", rs.Header.Get("Content-Type")) + + var got harbor.ScanReport + err = json.NewDecoder(rs.Body).Decode(&got) + require.NoError(t, err) + assert.NotEmpty(t, got.SBOM) + + got.GeneratedAt = time.Time{} + got.SBOM = nil + + want := harbor.ScanReport{ Artifact: harbor.Artifact{ - Repository: "library/mongo", - Digest: "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b", + Repository: imageRef.RepositoryStr(), + Digest: imageRef.DigestStr(), }, Scanner: harbor.Scanner{ Name: "Trivy", @@ -269,54 +259,105 @@ func TestRestAPI(t *testing.T) { Version: "Unknown", }, MediaType: api.MediaTypeSPDX, - SBOM: "SPDX Document", - }, - }, nil) - - // when - values := url.Values{} - values.Add("sbom_media_type", "application/spdx+json") - req, err := http.NewRequest("GET", ts.URL+"/api/v1/scan/job:123/report?"+values.Encode(), nil) - require.NoError(t, err) - req.Header.Add("Accept", "application/vnd.security.sbom.report+json; version=1.0") - rs, err := ts.Client().Do(req) - require.NoError(t, err) - - // then - require.NoError(t, err) - assert.Equal(t, http.StatusOK, rs.StatusCode) - assert.Equal(t, "application/vnd.security.sbom.report+json; version=1.0", rs.Header.Get("Content-Type")) - - bodyBytes, err := io.ReadAll(rs.Body) - require.NoError(t, err) - - expectedSBOMReport := fmt.Sprintf(`{ - "generated_at": "%s", - "artifact": { - "repository": "library/mongo", - "digest": "sha256:6c3c624b58dbbcd3c0dd82b4c53f04194d1247c6eebdaab7c610cf7d66709b3b" - }, - "scanner": { - "name": "Trivy", - "vendor": "Aqua Security", - "version": "Unknown" - }, - "media_type": "application/spdx+json", - "sbom": "SPDX Document" - }`, now.Format(time.RFC3339Nano)) + } + assert.Equal(t, want, got) + }) + }) - assert.JSONEq(t, expectedSBOMReport, string(bodyBytes)) + t.Run("Scan SBOM for vulnerabilities", func(t *testing.T) { + var scanJobID string + t.Run("POST /api/v1/scan", func(t *testing.T) { + // when + body := harbor.ScanRequest{ + Registry: harbor.Registry{ + URL: sbomRef.Registry.Scheme() + "://" + sbomRef.RegistryStr(), + }, + Artifact: harbor.Artifact{ + Repository: "testimage", + Digest: sbomRef.DigestStr(), + }, + Capabilities: []harbor.Capability{ + { + Type: harbor.CapabilityTypeVulnerability, + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecurityVulnerabilityReport, + }, + }, + }, + } + buf := &bytes.Buffer{} + err := json.NewEncoder(buf).Encode(body) + require.NoError(t, err) + + rs, err := ts.Client().Post(ts.URL+"/api/v1/scan", "application/json", buf) + + // then + require.NoError(t, err) + assert.Equal(t, http.StatusAccepted, rs.StatusCode) + assert.Equal(t, "application/vnd.scanner.adapter.scan.response+json; version=1.0", rs.Header.Get("Content-Type")) + + var res harbor.ScanResponse + err = json.NewDecoder(rs.Body).Decode(&res) + require.NoError(t, err) + assert.NotEmpty(t, res.ID) + + scanJobID = res.ID + }) + + t.Run("GET /api/v1/scan/{scan_request_id}/report", func(t *testing.T) { + time.Sleep(3 * time.Second) + rs, err := ts.Client().Get(fmt.Sprintf("%s/api/v1/scan/%s/report", ts.URL, scanJobID)) + require.NoError(t, err) + defer rs.Body.Close() + + // then + assert.Equal(t, http.StatusOK, rs.StatusCode) + assert.Equal(t, "application/vnd.security.vulnerability.report; version=1.1", rs.Header.Get("Content-Type")) + + var got harbor.ScanReport + err = json.NewDecoder(rs.Body).Decode(&got) + require.NoError(t, err) + got.GeneratedAt = time.Time{} // ignore generated_at + + want := harbor.ScanReport{ + Artifact: harbor.Artifact{ + Repository: "testimage", + Digest: sbomRef.DigestStr(), + }, + Scanner: harbor.Scanner{ + Name: "Trivy", + Vendor: "Aqua Security", + Version: "Unknown", + }, + Severity: harbor.SevMedium, + Vulnerabilities: []harbor.VulnerabilityItem{ + { + ID: "CVE-2019-1549", + Pkg: "libssl1.1", + Version: "1.1.1c-r0", + FixVersion: "1.1.1d-r0", + Severity: harbor.SevMedium, + Description: "OpenSSL 1.1.1 introduced a rewritten random number generator (RNG). This was intended to include protection in the event of a fork() system call in order to ensure that the parent and child processes did not share the same RNG state. However this protection was not being used in the default case. A partial mitigation for this issue is that the output from a high precision timer is mixed into the RNG state so the likelihood of a parent and child process sharing state is significantly reduced. If an application already calls OPENSSL_init_crypto() explicitly using OPENSSL_INIT_ATFORK then this problem does not occur at all. Fixed in OpenSSL 1.1.1d (Affected 1.1.1-1.1.1c).", + Links: []string{ + "https://avd.aquasec.com/nvd/cve-2019-1549", + }, + VendorAttributes: map[string]any{ + "CVSS": map[string]any{ + "nvd": map[string]any{ + "V3Score": 5.3, + "V3Vector": "CVSS:3.1/AV:N/AC:L/PR:N/UI:N/S:U/C:L/I:N/A:N", + }, + }, + }, + Layer: &harbor.Layer{}, + }, + }, + } + assert.Equal(t, want, got) + }) }) t.Run("GET /api/v1/metadata", func(t *testing.T) { - wrapper.On("GetVersion").Return(trivy.VersionInfo{ - Version: "v0.5.2-17-g3c9af62", - VulnerabilityDB: &trivy.Metadata{ - NextUpdate: time.Unix(1584507644, 0).UTC(), - UpdatedAt: time.Unix(1584517644, 0).UTC(), - }, - }, nil) - rs, err := ts.Client().Get(ts.URL + "/api/v1/metadata") require.NoError(t, err) assert.Equal(t, http.StatusOK, rs.StatusCode) @@ -324,7 +365,7 @@ func TestRestAPI(t *testing.T) { bodyBytes, err := io.ReadAll(rs.Body) require.NoError(t, err) - assert.JSONEq(t, `{ + assert.JSONEq(t, fmt.Sprintf(`{ "scanner": { "name": "Trivy", "vendor": "Aqua Security", @@ -358,26 +399,27 @@ func TestRestAPI(t *testing.T) { } } ], - "properties": { + "properties": { "harbor.scanner-adapter/scanner-type": "os-package-vulnerability", - "harbor.scanner-adapter/vulnerability-database-next-update-at": "2020-03-18T05:00:44Z", - "harbor.scanner-adapter/vulnerability-database-updated-at": "2020-03-18T07:47:24Z", + "harbor.scanner-adapter/vulnerability-database-updated-at": "%s", "org.label-schema.version": "1.0", "org.label-schema.build-date": "2019-01-04T12:40", "org.label-schema.vcs-ref": "abc", "org.label-schema.vcs": "https://github.com/aquasecurity/harbor-scanner-trivy", - "env.SCANNER_TRIVY_SKIP_UPDATE": "false", - "env.SCANNER_TRIVY_SKIP_JAVA_DB_UPDATE": "false", + "env.SCANNER_TRIVY_SKIP_UPDATE": "true", + "env.SCANNER_TRIVY_SKIP_JAVA_DB_UPDATE": "true", "env.SCANNER_TRIVY_OFFLINE_SCAN": "false", "env.SCANNER_TRIVY_IGNORE_UNFIXED": "true", "env.SCANNER_TRIVY_DEBUG_MODE": "true", "env.SCANNER_TRIVY_INSECURE": "true", - "env.SCANNER_TRIVY_VULN_TYPE": "os,library", - "env.SCANNER_TRIVY_SEVERITY": "UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL", + "env.SCANNER_TRIVY_VULN_TYPE": "os", + "env.SCANNER_TRIVY_SEVERITY": "LOW,MEDIUM,HIGH,CRITICAL", "env.SCANNER_TRIVY_SECURITY_CHECKS": "vuln", "env.SCANNER_TRIVY_TIMEOUT": "5m0s" } -}`, string(bodyBytes)) +}`, + now.UTC().Format(time.RFC3339)), + string(bodyBytes)) }) t.Run("GET /probe/healthy", func(t *testing.T) { @@ -391,7 +433,158 @@ func TestRestAPI(t *testing.T) { require.NoError(t, err) assert.Equal(t, http.StatusOK, rs.StatusCode) }) +} + +func initRedis(t *testing.T) (*goredis.Client, persistence.Store) { + mr, err := miniredis.Run() + require.NoError(t, err) + t.Cleanup(mr.Close) + + rdb, err := redisx.NewClient(etc.RedisPool{ + URL: "redis://" + mr.Addr(), + }) + require.NoError(t, err) + + store := redis.NewStore(etc.RedisStore{ + Namespace: "test:data-store", + ScanJobTTL: 5 * time.Minute, + }, rdb) + + return rdb, store +} + +func initTrivy(t *testing.T, now time.Time) (trivy.Wrapper, etc.Trivy) { + cacheDir := initVulnDB(t, now) + trivyConf := etc.Trivy{ + CacheDir: cacheDir, + SkipDBUpdate: true, + SkipJavaDBUpdate: true, + Insecure: true, + Timeout: 5 * time.Minute, + Severity: "LOW,MEDIUM,HIGH,CRITICAL", + VulnType: "os", + Scanners: "vuln", + IgnoreUnfixed: true, + DebugMode: true, + } + wrapper := trivy.NewWrapper(trivyConf, ext.DefaultAmbassador) + + return wrapper, trivyConf +} + +func initVulnDB(t *testing.T, now time.Time) string { + fixtureDir := filepath.Join("testdata", "fixtures") + entries, err := os.ReadDir(fixtureDir) + require.NoError(t, err) + + var fixtureFiles []string + for _, entry := range entries { + if entry.IsDir() { + continue + } + fixtureFiles = append(fixtureFiles, filepath.Join(fixtureDir, entry.Name())) + } + + dir := t.TempDir() + dbPath := filepath.Join(dir, "db", "trivy.db") + dbDir := filepath.Dir(dbPath) + err = os.MkdirAll(dbDir, 0700) + require.NoError(t, err) + + // Load testdata into BoltDB + loader, err := fixtures.New(dbPath, fixtureFiles) + require.NoError(t, err) + require.NoError(t, loader.Load()) + require.NoError(t, loader.Close()) + + // Generate metadata.json + metadataFile := filepath.Join(dbDir, "metadata.json") + f, err := os.Create(metadataFile) + require.NoError(t, err) + defer f.Close() + + metadata := struct { + Version int + NextUpdate time.Time + UpdatedAt time.Time + DownloadedAt time.Time + }{ + Version: 2, + NextUpdate: now.Add(24 * time.Hour), + UpdatedAt: now, + DownloadedAt: now, + } + err = json.NewEncoder(f).Encode(metadata) + require.NoError(t, err) + + return dir +} + +func initWorker(t *testing.T, ctx context.Context, store persistence.Store, jobQueue etc.JobQueue, + rdb *goredis.Client, wrapper trivy.Wrapper) { + controller := scan.NewController(store, wrapper, scan.NewTransformer(&scan.SystemClock{})) + worker := queue.NewWorker(jobQueue, rdb, controller) + t.Cleanup(worker.Stop) + + worker.Start(ctx) +} + +func initRegistry(t *testing.T) (name.Digest, name.Digest) { + reg := httptest.NewServer(registry.New()) + + regURL, err := url.Parse(reg.URL) + require.NoError(t, err) + + return setupTestImage(t, regURL), setupTestSBOM(t, regURL) +} + +func setupTestImage(t *testing.T, reg *url.URL) name.Digest { + const src = "alpine@sha256:451eee8bedcb2f029756dc3e9d73bab0e7943c1ac55cff3a4861c52a0fdd3e98" // alpine:3.10 + dst, err := name.NewTag(reg.Host + "/testimage:latest") + require.NoError(t, err) + err = crane.Copy(src, dst.String(), crane.WithPlatform(&ggcr.Platform{ + OS: "linux", + Architecture: runtime.GOARCH, + })) + require.NoError(t, err) + + img, err := remote.Image(dst) + require.NoError(t, err) + + d, err := img.Digest() + require.NoError(t, err) + + digest, err := name.NewDigest(dst.Repository.Name() + "@" + d.String()) + require.NoError(t, err) + + return digest +} + +func setupTestSBOM(t *testing.T, reg *url.URL) name.Digest { + repo := reg.Host + "/testimage" + ref, err := name.NewTag(repo + ":latest") + require.NoError(t, err) + + // Make an image + layer, err := tarball.LayerFromFile("testdata/alpine.spdx") + require.NoError(t, err) + + img, err := mutate.Append(empty.Image, mutate.Addendum{ + Layer: layer, + }) + require.NoError(t, err) + + img = mutate.ArtifactType(img, "application/vnd.goharbor.harbor.sbom.v1") + + // Push + err = remote.Write(ref, img) + require.NoError(t, err) + + digest, err := img.Digest() + require.NoError(t, err) + + d, err := name.NewDigest(repo + "@" + digest.String()) + require.NoError(t, err) - enqueuer.AssertExpectations(t) - store.AssertExpectations(t) + return d } diff --git a/test/integration/api/testdata/alpine.spdx b/test/integration/api/testdata/alpine.spdx new file mode 100644 index 00000000..0d869824 --- /dev/null +++ b/test/integration/api/testdata/alpine.spdx @@ -0,0 +1,54 @@ +SPDXVersion: SPDX-2.3 +DataLicense: CC0-1.0 +SPDXID: SPDXRef-DOCUMENT +DocumentName: alpine:3.10 +DocumentNamespace: http://aquasecurity.github.io/trivy/container_image/alpine:3.10-9ccc324d-c036-4006-903a-e54e9f4870df +Creator: Organization: aquasecurity +Creator: Tool: trivy-0.48.3 +Created: 2024-01-22T07:11:06Z + +##### Package: alpine:3.10 + +PackageName: alpine:3.10 +SPDXID: SPDXRef-ContainerImage-10b3db0c66f04c2a +PackageDownloadLocation: NONE +PrimaryPackagePurpose: CONTAINER +FilesAnalyzed: false +ExternalRef: PACKAGE-MANAGER purl pkg:oci/alpine@sha256%3A451eee8bedcb2f029756dc3e9d73bab0e7943c1ac55cff3a4861c52a0fdd3e98?arch=arm64&repository_url=index.docker.io%2Flibrary%2Falpine +PackageAttributionText: SchemaVersion: 2 +PackageAttributionText: ImageID: sha256:19a85ffa0ee14b7c64ca5104dc4e1c5021fa04bd6b4a692074691ce154f5d074 +PackageAttributionText: RepoDigest: alpine@sha256:451eee8bedcb2f029756dc3e9d73bab0e7943c1ac55cff3a4861c52a0fdd3e98 +PackageAttributionText: DiffID: sha256:e484d53633e7e5ebaabbe277838af1f26c388cbcebfa32e29fae72dd4086d54d +PackageAttributionText: RepoTag: alpine:3.10 + +##### Package: alpine + +PackageName: alpine +SPDXID: SPDXRef-OperatingSystem-db974625db6b7f00 +PackageVersion: 3.10.9 +PackageDownloadLocation: NONE +PrimaryPackagePurpose: OPERATING-SYSTEM +FilesAnalyzed: false + +##### Package: libssl1.1 + +PackageName: libssl1.1 +SPDXID: SPDXRef-Package-920f584e116c0a71 +PackageVersion: 1.1.1c-r0 +PackageSupplier: NOASSERTION +PackageDownloadLocation: NONE +PrimaryPackagePurpose: LIBRARY +FilesAnalyzed: false +PackageChecksum: SHA1: 1985c7fffc5a6ea3a2944670681c850707991d38 +PackageSourceInfo: built package from: openssl 1.1.1c-r0 +PackageLicenseConcluded: OpenSSL +PackageLicenseDeclared: OpenSSL +ExternalRef: PACKAGE-MANAGER purl pkg:apk/alpine/libssl1.1@1.1.1c-r0?arch=aarch64&distro=3.10.9 +PackageAttributionText: PkgID: libssl1.1@1.1.1c-r0 +PackageAttributionText: LayerDiffID: sha256:e484d53633e7e5ebaabbe277838af1f26c388cbcebfa32e29fae72dd4086d54d + +##### Relationships + +Relationship: SPDXRef-DOCUMENT DESCRIBES SPDXRef-ContainerImage-10b3db0c66f04c2a +Relationship: SPDXRef-ContainerImage-10b3db0c66f04c2a CONTAINS SPDXRef-OperatingSystem-db974625db6b7f00 +Relationship: SPDXRef-OperatingSystem-db974625db6b7f00 CONTAINS SPDXRef-Package-920f584e116c0a71 diff --git a/test/integration/api/testdata/fixtures/alpine.yaml b/test/integration/api/testdata/fixtures/alpine.yaml new file mode 100644 index 00000000..507a9aaa --- /dev/null +++ b/test/integration/api/testdata/fixtures/alpine.yaml @@ -0,0 +1,12 @@ +- bucket: alpine 3.10 + pairs: + - bucket: openssl + pairs: + - key: CVE-2019-1549 + value: + FixedVersion: 1.1.1d-r0 + - bucket: musl + pairs: + - key: CVE-2020-28928 + value: + FixedVersion: 1.1.22-r5 \ No newline at end of file diff --git a/test/integration/api/testdata/fixtures/data-source.yaml b/test/integration/api/testdata/fixtures/data-source.yaml new file mode 100644 index 00000000..ca781d2e --- /dev/null +++ b/test/integration/api/testdata/fixtures/data-source.yaml @@ -0,0 +1,7 @@ +- bucket: data-source + pairs: + - key: alpine 3.10 + value: + ID: "alpine" + Name: "Alpine Secdb" + URL: "https://secdb.alpinelinux.org/" \ No newline at end of file diff --git a/test/integration/api/testdata/fixtures/vulnerability.yaml b/test/integration/api/testdata/fixtures/vulnerability.yaml new file mode 100644 index 00000000..377858d8 --- /dev/null +++ b/test/integration/api/testdata/fixtures/vulnerability.yaml @@ -0,0 +1,18 @@ +- bucket: vulnerability + pairs: + - key: CVE-2019-1549 + value: + CVSS: + nvd: + V3Score: 5.3 + V3Vector: CVSS:3.1/AV:N/AC:L/PR:N/UI:N/S:U/C:L/I:N/A:N + Description: OpenSSL 1.1.1 introduced a rewritten random number generator (RNG). This was intended to include protection in the event of a fork() system call in order to ensure that the parent and child processes did not share the same RNG state. However this protection was not being used in the default case. A partial mitigation for this issue is that the output from a high precision timer is mixed into the RNG state so the likelihood of a parent and child process sharing state is significantly reduced. If an application already calls OPENSSL_init_crypto() explicitly using OPENSSL_INIT_ATFORK then this problem does not occur at all. Fixed in OpenSSL 1.1.1d (Affected 1.1.1-1.1.1c). + References: + - https://cve.mitre.org/cgi-bin/cvename.cgi?name=CVE-2019-1549 + Severity: MEDIUM + Title: "openssl: information disclosure in fork()" + - key: CVE-2020-28928 + value: + Description: In musl libc through 1.2.1, wcsnrtombs mishandles particular combinations of destination buffer size and source character limit, as demonstrated by an invalid write access (buffer overflow). + Severity: MEDIUM + Title: In musl libc through 1.2.1, wcsnrtombs mishandles particular combinati ... From e402b84d5d136fbfc4d78d82c5c5b20bbe279128 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Mon, 22 Jan 2024 16:22:16 +0400 Subject: [PATCH 22/26] fix: yaml lint issues Signed-off-by: knqyf263 --- test/integration/api/testdata/fixtures/alpine.yaml | 3 ++- test/integration/api/testdata/fixtures/data-source.yaml | 3 ++- test/integration/api/testdata/fixtures/vulnerability.yaml | 1 + 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/test/integration/api/testdata/fixtures/alpine.yaml b/test/integration/api/testdata/fixtures/alpine.yaml index 507a9aaa..7871a8c8 100644 --- a/test/integration/api/testdata/fixtures/alpine.yaml +++ b/test/integration/api/testdata/fixtures/alpine.yaml @@ -1,3 +1,4 @@ +--- - bucket: alpine 3.10 pairs: - bucket: openssl @@ -9,4 +10,4 @@ pairs: - key: CVE-2020-28928 value: - FixedVersion: 1.1.22-r5 \ No newline at end of file + FixedVersion: 1.1.22-r5 diff --git a/test/integration/api/testdata/fixtures/data-source.yaml b/test/integration/api/testdata/fixtures/data-source.yaml index ca781d2e..af45197a 100644 --- a/test/integration/api/testdata/fixtures/data-source.yaml +++ b/test/integration/api/testdata/fixtures/data-source.yaml @@ -1,7 +1,8 @@ +--- - bucket: data-source pairs: - key: alpine 3.10 value: ID: "alpine" Name: "Alpine Secdb" - URL: "https://secdb.alpinelinux.org/" \ No newline at end of file + URL: "https://secdb.alpinelinux.org/" diff --git a/test/integration/api/testdata/fixtures/vulnerability.yaml b/test/integration/api/testdata/fixtures/vulnerability.yaml index 377858d8..ccf4fc0e 100644 --- a/test/integration/api/testdata/fixtures/vulnerability.yaml +++ b/test/integration/api/testdata/fixtures/vulnerability.yaml @@ -1,3 +1,4 @@ +--- - bucket: vulnerability pairs: - key: CVE-2019-1549 From 6c68b729792565050fe4f7770c4d6736624e61ad Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Mon, 22 Jan 2024 17:35:16 +0400 Subject: [PATCH 23/26] chore: install Trivy Signed-off-by: knqyf263 --- Makefile | 1 + 1 file changed, 1 insertion(+) diff --git a/Makefile b/Makefile index d13bbd1a..8dfd9f33 100644 --- a/Makefile +++ b/Makefile @@ -29,6 +29,7 @@ lint: setup: curl -sfL https://raw.githubusercontent.com/golangci/golangci-lint/master/install.sh| sh -s v1.21.0 + curl -sfL https://raw.githubusercontent.com/aquasecurity/trivy/main/contrib/install.sh | sh -s -- -b /usr/local/bin v0.48.3 submodule: git submodule update --init --recursive From d5db690041fba1f30bbc91f6d06a5a4c8cb56d31 Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Mon, 22 Jan 2024 17:42:30 +0400 Subject: [PATCH 24/26] test: fix arch to amd64 Signed-off-by: knqyf263 --- test/integration/api/rest_api_test.go | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/test/integration/api/rest_api_test.go b/test/integration/api/rest_api_test.go index 339215fe..d8584dad 100644 --- a/test/integration/api/rest_api_test.go +++ b/test/integration/api/rest_api_test.go @@ -29,7 +29,6 @@ import ( "net/url" "os" "path/filepath" - "runtime" "testing" "time" @@ -153,8 +152,8 @@ func TestRestAPI(t *testing.T) { "https://avd.aquasec.com/nvd/cve-2020-28928", }, Layer: &harbor.Layer{ - Digest: "sha256:26d14edc4f17638cda363ea80b29c55e83058fc0dff1129b38ea3e8231217f7d", - DiffID: "sha256:e484d53633e7e5ebaabbe277838af1f26c388cbcebfa32e29fae72dd4086d54d", + Digest: "sha256:396c31837116ac290458afcb928f68b6cc1c7bdd6963fc72f52f365a2a89c1b5", + DiffID: "sha256:9fb3aa2f8b8023a4bebbf92aa567caf88e38e969ada9f0ac12643b2847391635", }, }, { @@ -168,8 +167,8 @@ func TestRestAPI(t *testing.T) { "https://avd.aquasec.com/nvd/cve-2020-28928", }, Layer: &harbor.Layer{ - Digest: "sha256:26d14edc4f17638cda363ea80b29c55e83058fc0dff1129b38ea3e8231217f7d", - DiffID: "sha256:e484d53633e7e5ebaabbe277838af1f26c388cbcebfa32e29fae72dd4086d54d", + Digest: "sha256:396c31837116ac290458afcb928f68b6cc1c7bdd6963fc72f52f365a2a89c1b5", + DiffID: "sha256:9fb3aa2f8b8023a4bebbf92aa567caf88e38e969ada9f0ac12643b2847391635", }, }, }, @@ -544,7 +543,7 @@ func setupTestImage(t *testing.T, reg *url.URL) name.Digest { require.NoError(t, err) err = crane.Copy(src, dst.String(), crane.WithPlatform(&ggcr.Platform{ OS: "linux", - Architecture: runtime.GOARCH, + Architecture: "amd64", })) require.NoError(t, err) From f9211e606746bbfed328f5994108cdf26285b29c Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Tue, 27 Feb 2024 10:51:30 +0400 Subject: [PATCH 25/26] fix: return 400 when missing sbom_media_type Signed-off-by: knqyf263 --- pkg/http/api/base_handler.go | 12 ++++++++++++ pkg/http/api/v1/handler.go | 9 ++++++++- pkg/http/api/v1/handler_test.go | 11 +++++++++++ 3 files changed, 31 insertions(+), 1 deletion(-) diff --git a/pkg/http/api/base_handler.go b/pkg/http/api/base_handler.go index 00c316cc..5fdd6b4c 100644 --- a/pkg/http/api/base_handler.go +++ b/pkg/http/api/base_handler.go @@ -114,6 +114,18 @@ func (mt *MIMEType) Parse(value string) error { return xerrors.Errorf("unsupported mime type: %s", value) } +func (mt *MIMEType) Equal(other MIMEType) bool { + if mt.Type != other.Type || mt.Subtype != other.Subtype || len(mt.Params) != len(other.Params) { + return false + } + for k, v := range mt.Params { + if other.Params[k] != v { + return false + } + } + return true +} + type BaseHandler struct { } diff --git a/pkg/http/api/v1/handler.go b/pkg/http/api/v1/handler.go index 20f1872d..80af3470 100644 --- a/pkg/http/api/v1/handler.go +++ b/pkg/http/api/v1/handler.go @@ -197,7 +197,7 @@ func (h *requestHandler) GetScanReport(res http.ResponseWriter, req *http.Reques vars := mux.Vars(req) scanJobID, ok := vars[pathVarScanRequestID] if !ok { - slog.Error("Error while parsing `scan_request_id` path variable") + slog.Error("`scan_request_id` is missing") h.WriteJSONError(res, api.Error{ HTTPCode: http.StatusBadRequest, Message: "missing scan_request_id", @@ -226,6 +226,13 @@ func (h *requestHandler) GetScanReport(res http.ResponseWriter, req *http.Reques Message: fmt.Sprintf("query parameter error: %s", err), }) return + } else if reportMIMEType.Equal(api.MimeTypeSecuritySBOMReport) && query.SBOMMediaType == "" { + slog.Error("`sbom_media_type` is missing") + h.WriteJSONError(res, api.Error{ + HTTPCode: http.StatusBadRequest, + Message: "missing sbom_media_type", + }) + return } else if query.SBOMMediaType != "" { reqLog = reqLog.With(slog.String("sbom_media_type", string(query.SBOMMediaType))) } diff --git a/pkg/http/api/v1/handler_test.go b/pkg/http/api/v1/handler_test.go index e74a76b1..21393f5d 100644 --- a/pkg/http/api/v1/handler_test.go +++ b/pkg/http/api/v1/handler_test.go @@ -521,6 +521,17 @@ func TestRequestHandler_GetScanReport(t *testing.T) { }, }, }, + { + name: "Should return error when sbom_media_type is missing", + acceptHeader: "application/vnd.security.sbom.report+json; version=1.0", + expectedStatus: http.StatusBadRequest, + expectedContentType: "application/vnd.scanner.adapter.error; version=1.0", + expectedResponse: apiError{ + Err: api.Error{ + Message: "missing sbom_media_type", + }, + }, + }, { name: "Should respond with SBOM report", acceptHeader: "application/vnd.security.sbom.report+json; version=1.0", From 3bc74554d3eec8a968b0c54138a2045def9d1ced Mon Sep 17 00:00:00 2001 From: knqyf263 Date: Tue, 27 Feb 2024 11:16:31 +0400 Subject: [PATCH 26/26] fix: return 400 when missing produces_mime_types Signed-off-by: knqyf263 --- pkg/http/api/v1/handler.go | 15 ++++++++---- pkg/http/api/v1/handler_test.go | 41 +++++++++++++++++++++++++++++++-- 2 files changed, 50 insertions(+), 6 deletions(-) diff --git a/pkg/http/api/v1/handler.go b/pkg/http/api/v1/handler.go index 80af3470..dd711bc6 100644 --- a/pkg/http/api/v1/handler.go +++ b/pkg/http/api/v1/handler.go @@ -164,6 +164,13 @@ func (h *requestHandler) ValidateScanRequest(req harbor.ScanRequest) *api.Error func (h *requestHandler) validateCapabilities(capabilities []harbor.Capability) *api.Error { for _, c := range capabilities { + if len(c.ProducesMIMETypes) == 0 { + return &api.Error{ + HTTPCode: http.StatusBadRequest, + Message: `"enabled_capabilities.produces_mime_types" is missing"`, + } + } + if c.Type != harbor.CapabilityTypeVulnerability && c.Type != harbor.CapabilityTypeSBOM { return &api.Error{ HTTPCode: http.StatusUnprocessableEntity, @@ -197,10 +204,10 @@ func (h *requestHandler) GetScanReport(res http.ResponseWriter, req *http.Reques vars := mux.Vars(req) scanJobID, ok := vars[pathVarScanRequestID] if !ok { - slog.Error("`scan_request_id` is missing") + slog.Error("scan request id is missing") h.WriteJSONError(res, api.Error{ HTTPCode: http.StatusBadRequest, - Message: "missing scan_request_id", + Message: "missing scan request id", }) return } @@ -227,10 +234,10 @@ func (h *requestHandler) GetScanReport(res http.ResponseWriter, req *http.Reques }) return } else if reportMIMEType.Equal(api.MimeTypeSecuritySBOMReport) && query.SBOMMediaType == "" { - slog.Error("`sbom_media_type` is missing") + slog.Error("SBOM media type is missing") h.WriteJSONError(res, api.Error{ HTTPCode: http.StatusBadRequest, - Message: "missing sbom_media_type", + Message: "missing SBOM media type", }) return } else if query.SBOMMediaType != "" { diff --git a/pkg/http/api/v1/handler_test.go b/pkg/http/api/v1/handler_test.go index 21393f5d..73debdc6 100644 --- a/pkg/http/api/v1/handler_test.go +++ b/pkg/http/api/v1/handler_test.go @@ -31,7 +31,12 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { Name: "Should return error when Registry URL is blank", Request: harbor.ScanRequest{ Capabilities: []harbor.Capability{ - {Type: harbor.CapabilityTypeVulnerability}, + { + Type: harbor.CapabilityTypeVulnerability, + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecurityVulnerabilityReport, + }, + }, }, }, ExpectedError: &api.Error{ @@ -45,6 +50,9 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { Capabilities: []harbor.Capability{ { Type: harbor.CapabilityTypeVulnerability, + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecurityVulnerabilityReport, + }, }, }, Registry: harbor.Registry{ @@ -62,6 +70,9 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { Capabilities: []harbor.Capability{ { Type: harbor.CapabilityTypeVulnerability, + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecurityVulnerabilityReport, + }, }, }, Registry: harbor.Registry{ @@ -79,6 +90,9 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { Capabilities: []harbor.Capability{ { Type: harbor.CapabilityTypeVulnerability, + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecurityVulnerabilityReport, + }, }, }, Registry: harbor.Registry{ @@ -93,12 +107,29 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { Message: "missing artifact.digest", }, }, + { + Name: "Should return error without produces MIME type ", + Request: harbor.ScanRequest{ + Capabilities: []harbor.Capability{ + { + Type: harbor.CapabilityTypeVulnerability, + }, + }, + }, + ExpectedError: &api.Error{ + HTTPCode: http.StatusBadRequest, + Message: `"enabled_capabilities.produces_mime_types" is missing"`, + }, + }, { Name: "Should return error with unsupported scan type", Request: harbor.ScanRequest{ Capabilities: []harbor.Capability{ { Type: "unknown", + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecurityVulnerabilityReport, + }, }, }, }, @@ -113,6 +144,9 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { Capabilities: []harbor.Capability{ { Type: harbor.CapabilityTypeSBOM, + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecurityVulnerabilityReport, + }, Parameters: &harbor.CapabilityAttributes{ SBOMMediaTypes: nil, }, @@ -130,6 +164,9 @@ func TestRequestHandler_ValidateScanRequest(t *testing.T) { Capabilities: []harbor.Capability{ { Type: harbor.CapabilityTypeSBOM, + ProducesMIMETypes: []api.MIMEType{ + api.MimeTypeSecurityVulnerabilityReport, + }, Parameters: &harbor.CapabilityAttributes{ SBOMMediaTypes: []api.MediaType{ "application/unsupported", @@ -528,7 +565,7 @@ func TestRequestHandler_GetScanReport(t *testing.T) { expectedContentType: "application/vnd.scanner.adapter.error; version=1.0", expectedResponse: apiError{ Err: api.Error{ - Message: "missing sbom_media_type", + Message: "missing SBOM media type", }, }, },