mirror of
https://github.com/goharbor/harbor.git
synced 2024-12-24 01:27:49 +01:00
define the base interfaces for the tag retention feature
Signed-off-by: Steven Zou <szou@vmware.com>
This commit is contained in:
parent
63e2ce7606
commit
24ee32d7d1
57
src/pkg/retention/client.go
Normal file
57
src/pkg/retention/client.go
Normal file
@ -0,0 +1,57 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package retention
|
||||
|
||||
import "github.com/goharbor/harbor/src/pkg/retention/res"
|
||||
|
||||
// Client is designed to access core service to get required infos
|
||||
type Client interface {
|
||||
// Get the tag candidates under the repository
|
||||
//
|
||||
// Arguments:
|
||||
// repo string : name of the repository with namespace
|
||||
//
|
||||
// Returns:
|
||||
// []*res.Candidate : candidates returned
|
||||
// error : common error if any errors occurred
|
||||
GetCandidates(repo string) ([]*res.Candidate, error)
|
||||
|
||||
// Delete the specified candidate
|
||||
//
|
||||
// Arguments:
|
||||
// candidate *res.Candidate : the deleting candidate
|
||||
//
|
||||
// Returns:
|
||||
// error : common error if any errors occurred
|
||||
Delete(candidate *res.Candidate) error
|
||||
}
|
||||
|
||||
// New basic client
|
||||
func New() Client {
|
||||
return &basicClient{}
|
||||
}
|
||||
|
||||
// basicClient is a default
|
||||
type basicClient struct{}
|
||||
|
||||
// GetCandidates gets the tag candidates under the repository
|
||||
func (bc *basicClient) GetCandidates(repo string) ([]*res.Candidate, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Deletes the specified candidate
|
||||
func (bc *basicClient) Delete(candidate *res.Candidate) error {
|
||||
return nil
|
||||
}
|
30
src/pkg/retention/controller.go
Normal file
30
src/pkg/retention/controller.go
Normal file
@ -0,0 +1,30 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package retention
|
||||
|
||||
import "github.com/goharbor/harbor/src/jobservice/job"
|
||||
|
||||
// APIController to handle the requests related with retention
|
||||
type APIController interface {
|
||||
// Handle the related hooks from the job service and launch the corresponding actions if needed
|
||||
//
|
||||
// Arguments:
|
||||
// policyID string : uuid of the retention policy
|
||||
// event *job.StatusChange : event object sent by job service
|
||||
//
|
||||
// Returns:
|
||||
// common error object if any errors occurred
|
||||
HandleHook(policyID string, event *job.StatusChange) error
|
||||
}
|
42
src/pkg/retention/job.go
Normal file
42
src/pkg/retention/job.go
Normal file
@ -0,0 +1,42 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package retention
|
||||
|
||||
import (
|
||||
"github.com/goharbor/harbor/src/jobservice/job"
|
||||
)
|
||||
|
||||
// Job of running retention process
|
||||
type Job struct{}
|
||||
|
||||
// MaxFails of the job
|
||||
func (pj *Job) MaxFails() uint {
|
||||
return 3
|
||||
}
|
||||
|
||||
// ShouldRetry indicates job can be retried if failed
|
||||
func (pj *Job) ShouldRetry() bool {
|
||||
return true
|
||||
}
|
||||
|
||||
// Validate the parameters
|
||||
func (pj *Job) Validate(params Parameters) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Run the job
|
||||
func (pj *Job) Run(ctx job.Context, params Parameters) error {
|
||||
return nil
|
||||
}
|
24
src/pkg/retention/launcher.go
Normal file
24
src/pkg/retention/launcher.go
Normal file
@ -0,0 +1,24 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package retention
|
||||
|
||||
import "github.com/goharbor/harbor/src/pkg/retention/policy"
|
||||
|
||||
// Launcher provides function to launch the async jobs to run retentions based on the provided policy.
|
||||
type Launcher interface {
|
||||
// Launch async jobs for the retention policy
|
||||
// A separate job will be launched for each repository
|
||||
Launch(policy *policy.Metadata) (string, []string, error)
|
||||
}
|
46
src/pkg/retention/manager.go
Normal file
46
src/pkg/retention/manager.go
Normal file
@ -0,0 +1,46 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package retention
|
||||
|
||||
import (
|
||||
"github.com/goharbor/harbor/src/pkg/retention/policy"
|
||||
"github.com/goharbor/harbor/src/pkg/retention/q"
|
||||
)
|
||||
|
||||
// Manager defines operations of managing policy
|
||||
type Manager interface {
|
||||
// Create new policy and return uuid
|
||||
CreatePolicy(p *policy.Metadata) (string, error)
|
||||
// Update the existing policy
|
||||
// Full update
|
||||
UpdatePolicy(p *policy.Metadata) error
|
||||
// Delete the specified policy
|
||||
// No actual use so far
|
||||
DeletePolicy(ID string) error
|
||||
// Get the specified policy
|
||||
GetPolicy(ID string) (*policy.Metadata, error)
|
||||
// Create a new retention execution
|
||||
CreateExecution(execution *Execution) (string, error)
|
||||
// Update the specified execution
|
||||
UpdateExecution(execution *Execution) error
|
||||
// Get the specified execution
|
||||
GetExecution(eid string) (*Execution, error)
|
||||
// List execution histories
|
||||
ListExecutions(query *q.Query) ([]*Execution, error)
|
||||
// Add new history
|
||||
AppendHistory(history *History) error
|
||||
// List all the histories marked by the specified execution
|
||||
ListHistories(executionID string, query *q.Query) ([]*History, error)
|
||||
}
|
38
src/pkg/retention/models.go
Normal file
38
src/pkg/retention/models.go
Normal file
@ -0,0 +1,38 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package retention
|
||||
|
||||
import "time"
|
||||
|
||||
// Retention execution
|
||||
type Execution struct {
|
||||
ID string `json:"id"`
|
||||
PolicyID string `json:"policy_id"`
|
||||
StartTime time.Time `json:"start_time"`
|
||||
EndTime time.Time `json:"end_time"`
|
||||
Status string `json:"status"`
|
||||
}
|
||||
|
||||
// Retention history
|
||||
type History struct {
|
||||
ExecutionID string `json:"execution_id"`
|
||||
Rule struct {
|
||||
ID string `json:"id"`
|
||||
DisplayText string `json:"display_text"`
|
||||
} `json:"rule_id"`
|
||||
// full path: :ns/:repo:tag
|
||||
Artifact string `json:"tag"`
|
||||
Timestamp time.Time `json:"timestamp"`
|
||||
}
|
44
src/pkg/retention/periodic_job.go
Normal file
44
src/pkg/retention/periodic_job.go
Normal file
@ -0,0 +1,44 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package retention
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/goharbor/harbor/src/jobservice/job"
|
||||
"time"
|
||||
)
|
||||
|
||||
// PeriodicJob is designed to generate hook event periodically
|
||||
type PeriodicJob struct{}
|
||||
|
||||
// MaxFails of the job
|
||||
func (pj *PeriodicJob) MaxFails() uint {
|
||||
return 3
|
||||
}
|
||||
|
||||
// ShouldRetry indicates job can be retried if failed
|
||||
func (pj *PeriodicJob) ShouldRetry() bool {
|
||||
return true
|
||||
}
|
||||
|
||||
// Validate the parameters
|
||||
func (pj *PeriodicJob) Validate(params Parameters) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Run the job
|
||||
func (pj *PeriodicJob) Run(ctx job.Context, params Parameters) error {
|
||||
return ctx.Checkin(fmt.Sprintf("pong=%d", time.Now().Unix()))
|
||||
}
|
52
src/pkg/retention/policy/action/index.go
Normal file
52
src/pkg/retention/policy/action/index.go
Normal file
@ -0,0 +1,52 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package action
|
||||
|
||||
import (
|
||||
"github.com/pkg/errors"
|
||||
"sync"
|
||||
)
|
||||
|
||||
// index for keeping the mapping action and its performer
|
||||
var index sync.Map
|
||||
|
||||
// Register the performer with the corresponding action
|
||||
func Register(action string, factory PerformerFactory) {
|
||||
if len(action) == 0 || factory == nil {
|
||||
// do nothing
|
||||
return
|
||||
}
|
||||
|
||||
index.Store(action, factory)
|
||||
}
|
||||
|
||||
// Get performer with the provided action
|
||||
func Get(action string) (Performer, error) {
|
||||
if len(action) == 0 {
|
||||
return nil, errors.New("empty action")
|
||||
}
|
||||
|
||||
v, ok := index.Load(action)
|
||||
if !ok {
|
||||
return nil, errors.Errorf("action %s is not registered", action)
|
||||
}
|
||||
|
||||
factory, ok := v.(PerformerFactory)
|
||||
if !ok {
|
||||
return nil, errors.Errorf("invalid action performer registered for action %s", action)
|
||||
}
|
||||
|
||||
return factory(), nil
|
||||
}
|
46
src/pkg/retention/policy/action/performer.go
Normal file
46
src/pkg/retention/policy/action/performer.go
Normal file
@ -0,0 +1,46 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package action
|
||||
|
||||
import "github.com/goharbor/harbor/src/pkg/retention/res"
|
||||
|
||||
// Performer performs the related actions targeting the candidates
|
||||
type Performer interface {
|
||||
// Perform the action
|
||||
//
|
||||
// Arguments:
|
||||
// candidates []*res.Candidate : the targets to perform
|
||||
//
|
||||
// Returns:
|
||||
// []*res.Result : result infos
|
||||
// error : common error if any errors occurred
|
||||
Perform(candidates []*res.Candidate) ([]*res.Result, error)
|
||||
}
|
||||
|
||||
// PerformerFactory is factory method for creating Performer
|
||||
type PerformerFactory func() Performer
|
||||
|
||||
// retainAction make sure all the candidates will be retained and others will be cleared
|
||||
type retainAction struct{}
|
||||
|
||||
// Perform the action
|
||||
func (ra *retainAction) Perform(candidates []*res.Candidate) ([]*res.Result, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// NewRetainAction is factory method for RetainAction
|
||||
func NewRetainAction() Performer {
|
||||
return &retainAction{}
|
||||
}
|
140
src/pkg/retention/policy/alg/or/processor.go
Normal file
140
src/pkg/retention/policy/alg/or/processor.go
Normal file
@ -0,0 +1,140 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package or
|
||||
|
||||
import (
|
||||
"github.com/goharbor/harbor/src/common/utils/log"
|
||||
"github.com/goharbor/harbor/src/pkg/retention/policy/action"
|
||||
"github.com/goharbor/harbor/src/pkg/retention/policy/alg"
|
||||
"github.com/goharbor/harbor/src/pkg/retention/policy/rule"
|
||||
"github.com/goharbor/harbor/src/pkg/retention/res"
|
||||
"github.com/pkg/errors"
|
||||
"sync"
|
||||
)
|
||||
|
||||
// processor to handle the rules with OR mapping ways
|
||||
type processor struct {
|
||||
performer action.Performer
|
||||
// keep evaluator and its related selector if existing
|
||||
// attentions here, the selector can be nil, that means match all "**"
|
||||
evaluators map[*rule.Evaluator]res.Selector
|
||||
}
|
||||
|
||||
// New processor
|
||||
func New() alg.Processor {
|
||||
return &processor{}
|
||||
}
|
||||
|
||||
// Process the candidates with the rules
|
||||
func (p *processor) Process(artifacts []*res.Candidate) ([]*res.Result, error) {
|
||||
if len(artifacts) == 0 {
|
||||
log.Debug("no artifacts to retention")
|
||||
return make([]*res.Result, 0), nil
|
||||
}
|
||||
|
||||
var (
|
||||
// collect errors by wrapping
|
||||
err error
|
||||
// collect results
|
||||
retained = make([]*res.Candidate, 0)
|
||||
)
|
||||
|
||||
// for sync
|
||||
resChan := make(chan []*res.Candidate, 1)
|
||||
// handle error
|
||||
errChan := make(chan error, 1)
|
||||
// control chan
|
||||
done := make(chan bool, 1)
|
||||
|
||||
defer func() {
|
||||
// signal the result listener loop exit
|
||||
done <- true
|
||||
}()
|
||||
|
||||
// go routine for receiving results/error
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
case retainedOnes := <-resChan:
|
||||
retained = append(retained, retainedOnes...)
|
||||
case e := <-errChan:
|
||||
if err == nil {
|
||||
err = errors.Wrap(e, "artifact processing error")
|
||||
} else {
|
||||
err = errors.Wrap(e, err.Error())
|
||||
}
|
||||
case <-done:
|
||||
// exit
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
wg := new(sync.WaitGroup)
|
||||
wg.Add(len(p.evaluators))
|
||||
|
||||
for eva, selector := range p.evaluators {
|
||||
var evaluator rule.Evaluator = *eva
|
||||
|
||||
go func(evaluator rule.Evaluator, selector res.Selector) {
|
||||
var (
|
||||
processed []*res.Candidate
|
||||
err error
|
||||
)
|
||||
|
||||
defer func() {
|
||||
wg.Done()
|
||||
}()
|
||||
|
||||
if selector != nil {
|
||||
if processed, err = selector.Select(artifacts); err != nil {
|
||||
errChan <- err
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
if processed, err = evaluator.Process(processed); err != nil {
|
||||
errChan <- err
|
||||
return
|
||||
}
|
||||
|
||||
if len(processed) > 0 {
|
||||
// Pass to the outside
|
||||
resChan <- processed
|
||||
}
|
||||
}(evaluator, selector)
|
||||
}
|
||||
|
||||
// waiting for all the rules are evaluated
|
||||
wg.Wait()
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return p.performer.Perform(retained)
|
||||
}
|
||||
|
||||
// AddEvaluator appends a rule evaluator for processing
|
||||
func (p *processor) AddEvaluator(evaluator rule.Evaluator, selector res.Selector) {
|
||||
if evaluator != nil {
|
||||
p.evaluators[&evaluator] = selector
|
||||
}
|
||||
}
|
||||
|
||||
// SetPerformer sets a action performer to the processor
|
||||
func (p *processor) SetPerformer(performer action.Performer) {
|
||||
p.performer = performer
|
||||
}
|
49
src/pkg/retention/policy/alg/processor.go
Normal file
49
src/pkg/retention/policy/alg/processor.go
Normal file
@ -0,0 +1,49 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package alg
|
||||
|
||||
import (
|
||||
"github.com/goharbor/harbor/src/pkg/retention/policy/action"
|
||||
"github.com/goharbor/harbor/src/pkg/retention/policy/rule"
|
||||
"github.com/goharbor/harbor/src/pkg/retention/res"
|
||||
)
|
||||
|
||||
// Processor processing the whole policy targeting a repository.
|
||||
// Methods are defined to reflect the standard structure of the policy:
|
||||
// list of rules with corresponding selectors plus an action performer.
|
||||
type Processor interface {
|
||||
// Process the artifact candidates
|
||||
//
|
||||
// Arguments:
|
||||
// artifacts []*res.Candidate : process the retention candidates
|
||||
//
|
||||
// Returns:
|
||||
// []*res.Result : the processed results
|
||||
// error : common error object if any errors occurred
|
||||
Process(artifacts []*res.Candidate) ([]*res.Result, error)
|
||||
|
||||
// Add a rule evaluator for the processor
|
||||
//
|
||||
// Arguments:
|
||||
// evaluator rule.Evaluator : a rule evaluator
|
||||
// selector res.Selector : selector to narrow down the scope, optional
|
||||
AddEvaluator(evaluator rule.Evaluator, selector res.Selector)
|
||||
|
||||
// Set performer for the processor
|
||||
//
|
||||
// Arguments:
|
||||
// performer action.Performer : a performer implementation
|
||||
SetPerformer(performer action.Performer)
|
||||
}
|
57
src/pkg/retention/policy/builder.go
Normal file
57
src/pkg/retention/policy/builder.go
Normal file
@ -0,0 +1,57 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package policy
|
||||
|
||||
import (
|
||||
"github.com/goharbor/harbor/src/pkg/retention/policy/alg"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
// Builder builds the runnable processor from the raw policy
|
||||
type Builder interface {
|
||||
// Builds runnable processor
|
||||
//
|
||||
// Arguments:
|
||||
// rawPolicy string : the simple retention policy with JSON format
|
||||
//
|
||||
// Returns:
|
||||
// Processor : a processor implementation to process the candidates
|
||||
// error : common error object if any errors occurred
|
||||
Build(rawPolicy string) (alg.Processor, error)
|
||||
}
|
||||
|
||||
// basicBuilder is default implementation of Builder interface
|
||||
type basicBuilder struct{}
|
||||
|
||||
// Build policy processor from the raw policy
|
||||
func (bb *basicBuilder) Build(rawPolicy string) (alg.Processor, error) {
|
||||
if len(rawPolicy) == 0 {
|
||||
return nil, errors.New("empty raw policy to build processor")
|
||||
}
|
||||
|
||||
// Decode metadata
|
||||
liteMeta := &LiteMeta{}
|
||||
if err := liteMeta.Decode(rawPolicy); err != nil {
|
||||
return nil, errors.Wrap(err, "build policy processor")
|
||||
}
|
||||
|
||||
switch liteMeta.Algorithm {
|
||||
case AlgorithmOR:
|
||||
default:
|
||||
return nil, errors.Errorf("algorithm %s is not supported", liteMeta.Algorithm)
|
||||
}
|
||||
|
||||
return nil, errors.New("not implemented")
|
||||
}
|
102
src/pkg/retention/policy/models.go
Normal file
102
src/pkg/retention/policy/models.go
Normal file
@ -0,0 +1,102 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package policy
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"github.com/goharbor/harbor/src/pkg/retention/policy/rule"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
const (
|
||||
// AlgorithmOR for OR algorithm
|
||||
AlgorithmOR = "or"
|
||||
)
|
||||
|
||||
// Metadata of policy
|
||||
type Metadata struct {
|
||||
// UUID of the policy
|
||||
ID string `json:"id"`
|
||||
|
||||
// Algorithm applied to the rules
|
||||
// "OR" / "AND"
|
||||
Algorithm string `json:"algorithm"`
|
||||
|
||||
// Rule collection
|
||||
Rules []rule.Metadata `json:"rules"`
|
||||
|
||||
// Trigger about how to launch the policy
|
||||
Trigger *Trigger `json:"trigger"`
|
||||
|
||||
// Which scope the policy will be applied to
|
||||
Scope *Scope `json:"scope"`
|
||||
|
||||
// The max number of rules in a policy
|
||||
Capacity int `json:"cap"`
|
||||
}
|
||||
|
||||
// Trigger of the policy
|
||||
type Trigger struct {
|
||||
// Const string to declare the trigger type
|
||||
// 'Schedule'
|
||||
Kind string `json:"kind"`
|
||||
|
||||
// Settings for the specified trigger
|
||||
// '[cron]="* 22 11 * * *"' for the 'Schedule'
|
||||
Settings map[string]interface{} `json:"settings"`
|
||||
|
||||
// References of the trigger
|
||||
// e.g: schedule job ID
|
||||
References map[string]interface{} `json:"references"`
|
||||
}
|
||||
|
||||
// Scope definition
|
||||
type Scope struct {
|
||||
// Scope level declaration
|
||||
// 'system', 'project' and 'repository'
|
||||
Level string `json:"level"`
|
||||
|
||||
// The reference identity for the specified level
|
||||
// '' for 'system', project ID for 'project' and repo ID for 'repository'
|
||||
Reference string `json:"ref"`
|
||||
}
|
||||
|
||||
// LiteMeta contains partial metadata of policy
|
||||
type LiteMeta struct {
|
||||
// Algorithm applied to the rules
|
||||
// "OR" / "AND"
|
||||
Algorithm string `json:"algorithm"`
|
||||
|
||||
// Rule collection
|
||||
Rules []rule.Metadata `json:"rules"`
|
||||
}
|
||||
|
||||
// Encode the lit meta by compressed json string
|
||||
func (lm *LiteMeta) Encode() (string, error) {
|
||||
data, err := json.Marshal(lm)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
return string(data), nil
|
||||
}
|
||||
|
||||
// Decode the lite meta from the data string
|
||||
func (lm *LiteMeta) Decode(data string) error {
|
||||
if len(data) == 0 {
|
||||
return errors.New("no data for decoding")
|
||||
}
|
||||
|
||||
return json.Unmarshal([]byte(data), lm)
|
||||
}
|
33
src/pkg/retention/policy/rule/evaluator.go
Normal file
33
src/pkg/retention/policy/rule/evaluator.go
Normal file
@ -0,0 +1,33 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package rule
|
||||
|
||||
import "github.com/goharbor/harbor/src/pkg/retention/res"
|
||||
|
||||
// Evaluator defines method of executing rule
|
||||
type Evaluator interface {
|
||||
// Filter the inputs and return the filtered outputs
|
||||
//
|
||||
// Arguments:
|
||||
// artifacts []*res.Candidate : candidates for processing
|
||||
//
|
||||
// Returns:
|
||||
// []*res.Candidate : matched candidates for next stage
|
||||
// error : common error object if any errors occurred
|
||||
Process(artifacts []*res.Candidate) ([]*res.Candidate, error)
|
||||
}
|
||||
|
||||
// RuleFactory defines a factory method for creating rule evaluator
|
||||
type RuleFactory func(parameters Parameters) Evaluator
|
116
src/pkg/retention/policy/rule/index.go
Normal file
116
src/pkg/retention/policy/rule/index.go
Normal file
@ -0,0 +1,116 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package rule
|
||||
|
||||
import (
|
||||
"github.com/pkg/errors"
|
||||
"sync"
|
||||
)
|
||||
|
||||
// index for keeping the mapping between template ID and evaluator
|
||||
var index sync.Map
|
||||
|
||||
// IndexMeta defines metadata for rule registration
|
||||
type IndexMeta struct {
|
||||
TemplateID string `json:"rule_template"`
|
||||
|
||||
// Action of the rule performs
|
||||
// "retain"
|
||||
Action string `json:"action"`
|
||||
|
||||
Parameters []*IndexedParam `json:"params"`
|
||||
}
|
||||
|
||||
// IndexedParam declares the param info
|
||||
type IndexedParam struct {
|
||||
Name string `json:"name"`
|
||||
|
||||
//Type of the param
|
||||
// "int", "string" or "[]string"
|
||||
Type string `json:"type"`
|
||||
|
||||
Unit string `json:"unit"`
|
||||
|
||||
Required bool `json:"required"`
|
||||
}
|
||||
|
||||
// indexedItem is the item saved in the sync map
|
||||
type indexedItem struct {
|
||||
Meta *IndexMeta
|
||||
|
||||
Factory RuleFactory
|
||||
}
|
||||
|
||||
// Register the rule evaluator with the corresponding rule template
|
||||
func Register(meta *IndexMeta, factory RuleFactory) {
|
||||
if meta == nil || factory == nil || len(meta.TemplateID) == 0 {
|
||||
// do nothing
|
||||
return
|
||||
}
|
||||
|
||||
index.Store(meta.TemplateID, &indexedItem{
|
||||
Meta: meta,
|
||||
Factory: factory,
|
||||
})
|
||||
}
|
||||
|
||||
// Get rule evaluator with the provided template ID
|
||||
func Get(templateID string, parameters Parameters) (Evaluator, error) {
|
||||
if len(templateID) == 0 {
|
||||
return nil, errors.New("empty rule template ID")
|
||||
}
|
||||
|
||||
v, ok := index.Load(templateID)
|
||||
if !ok {
|
||||
return nil, errors.Errorf("rule evaluator %s is not registered", templateID)
|
||||
}
|
||||
|
||||
item := v.(*indexedItem)
|
||||
|
||||
// We can check more things if we want to do in the future
|
||||
if len(item.Meta.Parameters) > 0 {
|
||||
for _, p := range item.Meta.Parameters {
|
||||
if p.Required {
|
||||
exists := parameters != nil
|
||||
if exists {
|
||||
_, exists = parameters[p.Name]
|
||||
}
|
||||
|
||||
if !exists {
|
||||
return nil, errors.Errorf("missing required parameter %s for rule %s", p.Name, templateID)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
factory := item.Factory
|
||||
|
||||
return factory(parameters), nil
|
||||
}
|
||||
|
||||
// Index returns all the metadata of the registered rules
|
||||
func Index() []*IndexMeta {
|
||||
res := make([]*IndexMeta, 0)
|
||||
|
||||
index.Range(func(k, v interface{}) bool {
|
||||
if item, ok := v.(*indexedItem); ok {
|
||||
res = append(res, item.Meta)
|
||||
return true
|
||||
}
|
||||
|
||||
return false
|
||||
})
|
||||
|
||||
return res
|
||||
}
|
28
src/pkg/retention/policy/rule/lastx/evaluator.go
Normal file
28
src/pkg/retention/policy/rule/lastx/evaluator.go
Normal file
@ -0,0 +1,28 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package lastx
|
||||
|
||||
import "github.com/goharbor/harbor/src/pkg/retention/res"
|
||||
|
||||
// Evaluator for evaluating last x days
|
||||
type Evaluator struct {
|
||||
// last x days
|
||||
x int
|
||||
}
|
||||
|
||||
// Process the candidates based on the rule definition
|
||||
func (e *Evaluator) Process(artifacts []*res.Candidate) ([]*res.Candidate, error) {
|
||||
return nil, nil
|
||||
}
|
28
src/pkg/retention/policy/rule/latestk/evaluator.go
Normal file
28
src/pkg/retention/policy/rule/latestk/evaluator.go
Normal file
@ -0,0 +1,28 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package latestk
|
||||
|
||||
import "github.com/goharbor/harbor/src/pkg/retention/res"
|
||||
|
||||
// Evaluator for evaluating latest x tags
|
||||
type Evaluator struct {
|
||||
// latest x
|
||||
k int
|
||||
}
|
||||
|
||||
// Process the candidates based on the rule definition
|
||||
func (e *Evaluator) Process(artifacts []*res.Candidate) ([]*res.Candidate, error) {
|
||||
return nil, nil
|
||||
}
|
62
src/pkg/retention/policy/rule/models.go
Normal file
62
src/pkg/retention/policy/rule/models.go
Normal file
@ -0,0 +1,62 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package rule
|
||||
|
||||
// Metadata of the retention rule
|
||||
type Metadata struct {
|
||||
// UUID of rule
|
||||
ID string `json:"id"`
|
||||
|
||||
// Priority of rule when doing calculating
|
||||
Priority int `json:"priority"`
|
||||
|
||||
// Action of the rule performs
|
||||
// "retain"
|
||||
Action string `json:"action"`
|
||||
|
||||
// Template ID
|
||||
Template string `json:"template"`
|
||||
|
||||
// The parameters of this rule
|
||||
Parameters Parameters `json:"params"`
|
||||
|
||||
// Selector attached to the rule for filtering tags
|
||||
TagSelector *Selector `json:"tag_selector"`
|
||||
|
||||
// Selector attached to the rule for filtering scope (e.g: repositories or namespaces)
|
||||
ScopeSelectors []*Selector `json:"scope_selectors"`
|
||||
}
|
||||
|
||||
// Selector to narrow down the list
|
||||
type Selector struct {
|
||||
// Kind of the selector
|
||||
// "regularExpression", "label" or "list"
|
||||
Kind string `json:"kind"`
|
||||
|
||||
// Decorated the selector
|
||||
// for "regularExpression" : "matches" and "excludes"
|
||||
// for "label" : "with" and "without"
|
||||
// for "list" : "in" and "not in"
|
||||
decoration string `json:"decoration"`
|
||||
|
||||
// Param for the selector
|
||||
Value Parameter `json:"param"`
|
||||
}
|
||||
|
||||
// Parameters of rule, indexed by the key
|
||||
type Parameters map[string]Parameter
|
||||
|
||||
// Parameter of rule
|
||||
type Parameter interface{}
|
21
src/pkg/retention/q/query.go
Normal file
21
src/pkg/retention/q/query.go
Normal file
@ -0,0 +1,21 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package q
|
||||
|
||||
// Query parameters
|
||||
type Query struct {
|
||||
PageNumber int
|
||||
PageSize int
|
||||
}
|
39
src/pkg/retention/res/candidate.go
Normal file
39
src/pkg/retention/res/candidate.go
Normal file
@ -0,0 +1,39 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package res
|
||||
|
||||
const (
|
||||
// Image kind
|
||||
Image = "image"
|
||||
// Chart kind
|
||||
Chart = "chart"
|
||||
)
|
||||
|
||||
// Candidate for retention processor to match
|
||||
type Candidate struct {
|
||||
// Namespace
|
||||
Namespace string
|
||||
// Repository name
|
||||
Repository string
|
||||
// Kind of the candidate
|
||||
// "image" or "chart"
|
||||
Kind string
|
||||
// Tag info
|
||||
Tag string
|
||||
// Pushed time in seconds
|
||||
PushedTime int64
|
||||
// Labels attached with the candidate
|
||||
Labels []string
|
||||
}
|
44
src/pkg/retention/res/label/selector.go
Normal file
44
src/pkg/retention/res/label/selector.go
Normal file
@ -0,0 +1,44 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package label
|
||||
|
||||
import "github.com/goharbor/harbor/src/pkg/retention/res"
|
||||
|
||||
const (
|
||||
With = "with labels"
|
||||
Without = "without labels"
|
||||
)
|
||||
|
||||
// selector is for label selector
|
||||
type selector struct {
|
||||
// Pre defined pattern decorations
|
||||
// "with" or "without"
|
||||
decoration string
|
||||
// Label list
|
||||
labels []string
|
||||
}
|
||||
|
||||
// Select candidates by regular expressions
|
||||
func (s *selector) Select(artifacts []*res.Candidate) ([]*res.Candidate, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// New is factory method for list selector
|
||||
func New(decoration string, pattern interface{}) res.Selector {
|
||||
return &selector{
|
||||
decoration: decoration,
|
||||
labels: pattern.([]string),
|
||||
}
|
||||
}
|
50
src/pkg/retention/res/list/selector.go
Normal file
50
src/pkg/retention/res/list/selector.go
Normal file
@ -0,0 +1,50 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package list
|
||||
|
||||
import "github.com/goharbor/harbor/src/pkg/retention/res"
|
||||
|
||||
const (
|
||||
// InRepos for in [repositories]
|
||||
InRepos = "in repositories"
|
||||
// NotInRepos for not in [repositories]
|
||||
NotInRepos = "not in repositories"
|
||||
// InTags for in [tags]
|
||||
InTags = "in tags"
|
||||
// NotInTags for not in [tags]
|
||||
NotInTags = "not in tags"
|
||||
)
|
||||
|
||||
// selector for regular expression
|
||||
type selector struct {
|
||||
// Pre defined pattern declarator
|
||||
// "InRepo", "NotInRepo", "InTag" and "NotInTags"
|
||||
decoration string
|
||||
// The item list
|
||||
values []string
|
||||
}
|
||||
|
||||
// Select candidates by regular expressions
|
||||
func (s *selector) Select(artifacts []*res.Candidate) ([]*res.Candidate, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// New is factory method for list selector
|
||||
func New(decoration string, pattern interface{}) res.Selector {
|
||||
return &selector{
|
||||
decoration: decoration,
|
||||
values: pattern.([]string),
|
||||
}
|
||||
}
|
46
src/pkg/retention/res/regexp/selector.go
Normal file
46
src/pkg/retention/res/regexp/selector.go
Normal file
@ -0,0 +1,46 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package regexp
|
||||
|
||||
import "github.com/goharbor/harbor/src/pkg/retention/res"
|
||||
|
||||
const (
|
||||
// Matches [pattern]
|
||||
Matches = "matches"
|
||||
// Excludes [pattern]
|
||||
Excludes = "excludes"
|
||||
)
|
||||
|
||||
// selector for regular expression
|
||||
type selector struct {
|
||||
// Pre defined pattern declarator
|
||||
// "matches" and "excludes"
|
||||
decoration string
|
||||
// The pattern expression
|
||||
pattern string
|
||||
}
|
||||
|
||||
// Select candidates by regular expressions
|
||||
func (s *selector) Select(artifacts []*res.Candidate) ([]*res.Candidate, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// New is factory method for regexp selector
|
||||
func New(decoration string, pattern interface{}) res.Selector {
|
||||
return &selector{
|
||||
decoration: decoration,
|
||||
pattern: pattern.(string),
|
||||
}
|
||||
}
|
22
src/pkg/retention/res/result.go
Normal file
22
src/pkg/retention/res/result.go
Normal file
@ -0,0 +1,22 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package res
|
||||
|
||||
// Result keeps the action result
|
||||
type Result struct {
|
||||
Target *Candidate
|
||||
// nil error means success
|
||||
Error error
|
||||
}
|
30
src/pkg/retention/res/selector.go
Normal file
30
src/pkg/retention/res/selector.go
Normal file
@ -0,0 +1,30 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package res
|
||||
|
||||
// Selector is used to filter the inputting list
|
||||
type Selector interface {
|
||||
// Select the matched ones
|
||||
//
|
||||
// Arguments:
|
||||
// artifacts []*Candidate : candidates for matching
|
||||
//
|
||||
// Returns:
|
||||
// []*Candidate : matched candidates
|
||||
Select(artifacts []*Candidate) ([]*Candidate, error)
|
||||
}
|
||||
|
||||
// SelectorFactory is factory method to return a selector implementation
|
||||
type SelectorFactory func(decoration string, pattern interface{}) Selector
|
55
src/pkg/retention/res/selector/index.go
Normal file
55
src/pkg/retention/res/selector/index.go
Normal file
@ -0,0 +1,55 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package selector
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/goharbor/harbor/src/pkg/retention/res"
|
||||
"github.com/pkg/errors"
|
||||
"sync"
|
||||
)
|
||||
|
||||
// index for keeping the mapping between selector meta and its implementation
|
||||
var index sync.Map
|
||||
|
||||
// Register the selector with the corresponding selector kind and decoration
|
||||
func Register(kind, decoration string, factory res.SelectorFactory) {
|
||||
id := fmt.Sprintf("%s:%s", kind, decoration)
|
||||
if len(id) == 0 || factory == nil {
|
||||
// do nothing
|
||||
return
|
||||
}
|
||||
|
||||
index.Store(id, factory)
|
||||
}
|
||||
|
||||
// Get selector with the provided kind and decoration
|
||||
func Get(kind, decoration string, pattern interface{}) (res.Selector, error) {
|
||||
if len(templateID) == 0 {
|
||||
return nil, errors.New("empty rule template ID")
|
||||
}
|
||||
|
||||
v, ok := index.Load(templateID)
|
||||
if !ok {
|
||||
return nil, errors.Errorf("rule evaluator %s is not registered", templateID)
|
||||
}
|
||||
|
||||
factory, ok := v.(RuleFactory)
|
||||
if !ok {
|
||||
return nil, errors.Errorf("invalid rule evaluator registered for %s", templateID)
|
||||
}
|
||||
|
||||
return factory(parameters), nil
|
||||
}
|
37
src/pkg/retention/scheduler.go
Normal file
37
src/pkg/retention/scheduler.go
Normal file
@ -0,0 +1,37 @@
|
||||
// Copyright Project Harbor Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package retention
|
||||
|
||||
// Scheduler of launching retention jobs
|
||||
type Scheduler interface {
|
||||
// Schedule the job to periodically run the retentions
|
||||
//
|
||||
// Arguments:
|
||||
// policyID string : uuid of the retention policy
|
||||
// cron string : cron pattern like `0-59/5 12 * * * *`
|
||||
// Returns:
|
||||
// the returned job ID
|
||||
// common error object if any errors occurred
|
||||
Schedule(policyID string, cron string) (string, error)
|
||||
|
||||
// Unschedule the specified retention policy
|
||||
//
|
||||
// Arguments:
|
||||
// policyID string : uuid of the retention policy
|
||||
//
|
||||
// Returns:
|
||||
// common error object if any errors occurred
|
||||
UnSchedule(policyID string) error
|
||||
}
|
Loading…
Reference in New Issue
Block a user