diff --git a/docs/content/en/docs-dev/user-guide/configuration-reference.md b/docs/content/en/docs-dev/user-guide/configuration-reference.md index 47c461a249..94404d656d 100644 --- a/docs/content/en/docs-dev/user-guide/configuration-reference.md +++ b/docs/content/en/docs-dev/user-guide/configuration-reference.md @@ -598,6 +598,13 @@ Note: The available values are identical to those found in the aws-sdk-go-v2 Typ | Field | Type | Description | Required | |-|-|-|-| +## SkipOptions + +| Field | Type | Description | Required | +|-|-|-|-| +| commitMessagePrefixes | []string | List of commit message's prefixes. The stage will be skipped when the prefix of the commit's message matches any of them. Empty means the stage will not be skipped by this condition. | No | +| paths | []string | List of paths to directories or files. When all commit changes match them, the stage will be skipped. Empty means the stage will not be skipped by this condition. Regular expression can be used. | No | + ## StageOptions ### KubernetesPrimaryRolloutStageOptions @@ -705,12 +712,14 @@ Note: By default, the sum of traffic is rounded to 100. If both `primary` and `c |-|-|-|-| | duration | duration | Maximum time to perform the analysis. | Yes | | metrics | [][AnalysisMetrics](#analysismetrics) | Configuration for analysis by metrics. | No | +| skipOn | [SkipOptions](#skipoptions) | When to skip this stage. | No | ### WaitStageOptions | Field | Type | Description | Required | |-|-|-|-| | duration | duration | Time to wait. | Yes | +| skipOn | [SkipOptions](#skipoptions) | When to skip this stage. | No | ### WaitApprovalStageOptions @@ -719,6 +728,7 @@ Note: By default, the sum of traffic is rounded to 100. If both `primary` and `c | timeout | duration | The maximum length of time to wait before giving up. Default is 6h. | No | | approvers | []string | List of username who has permission to approve. | Yes | | minApproverNum | int | Number of minimum needed approvals to make this stage complete. Default is 1. | No | +| skipOn | [SkipOptions](#skipoptions) | When to skip this stage. | No | ### CustomSyncStageOptions (deprecated) | Field | Type | Description | Required | @@ -733,6 +743,7 @@ Note: By default, the sum of traffic is rounded to 100. If both `primary` and `c | run | string | Script run on this stage. | Yes | | env | map[string]string | Environment variables used with scripts. | No | | timeout | duration | The maximum time the stage can be taken to run. Default is `6h`| No | +| skipOn | [SkipOptions](#skipoptions) | When to skip this stage. | No | ## PostSync diff --git a/pkg/app/piped/controller/scheduler.go b/pkg/app/piped/controller/scheduler.go index 61daf1f12c..59716990ff 100644 --- a/pkg/app/piped/controller/scheduler.go +++ b/pkg/app/piped/controller/scheduler.go @@ -528,6 +528,24 @@ func (s *scheduler) executeStage(sig executor.StopSignal, ps model.PipelineStage Notifier: s.notifier, } + // Skip the stage if needed based on the skip config. + skip, err := s.shouldSkipStage(sig.Context(), input) + if err != nil { + lp.Errorf("failed to check whether skipping the stage: %w", err.Error()) + if err := s.reportStageStatus(ctx, ps.Id, model.StageStatus_STAGE_FAILURE, ps.Requires); err != nil { + s.logger.Error("failed to report stage status", zap.Error(err)) + } + return model.StageStatus_STAGE_FAILURE + } + if skip { + if err := s.reportStageStatus(ctx, ps.Id, model.StageStatus_STAGE_SKIPPED, ps.Requires); err != nil { + s.logger.Error("failed to report stage status", zap.Error(err)) + return model.StageStatus_STAGE_FAILURE + } + lp.Info("The stage was successfully skipped due to the skip configuration of the stage.") + return model.StageStatus_STAGE_SKIPPED + } + // Find the executor for this stage. ex, ok := executorFactory(input) if !ok { diff --git a/pkg/app/piped/controller/skipstage.go b/pkg/app/piped/controller/skipstage.go new file mode 100644 index 0000000000..ccbd5633e1 --- /dev/null +++ b/pkg/app/piped/controller/skipstage.go @@ -0,0 +1,113 @@ +// Copyright 2024 The PipeCD 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 controller + +import ( + "context" + "strings" + + "github.com/pipe-cd/pipecd/pkg/app/piped/executor" + "github.com/pipe-cd/pipecd/pkg/config" + "github.com/pipe-cd/pipecd/pkg/filematcher" + "github.com/pipe-cd/pipecd/pkg/git" + "github.com/pipe-cd/pipecd/pkg/model" +) + +// checkSkipStage checks whether the stage should be skipped or not. +func (s *scheduler) shouldSkipStage(ctx context.Context, in executor.Input) (skip bool, err error) { + stageConfig := in.StageConfig + var skipOptions config.SkipOptions + switch stageConfig.Name { + case model.StageAnalysis: + skipOptions = stageConfig.AnalysisStageOptions.SkipOn + case model.StageWait: + skipOptions = stageConfig.WaitStageOptions.SkipOn + case model.StageWaitApproval: + skipOptions = stageConfig.WaitApprovalStageOptions.SkipOn + case model.StageScriptRun: + skipOptions = stageConfig.ScriptRunStageOptions.SkipOn + default: + return false, nil + } + + if len(skipOptions.Paths) == 0 && len(skipOptions.CommitMessagePrefixes) == 0 { + // When no condition is specified. + return false, nil + } + + repoCfg := in.Application.GitPath.Repo + repo, err := in.GitClient.Clone(ctx, repoCfg.Id, repoCfg.Remote, repoCfg.Branch, "") + if err != nil { + return false, err + } + + // Check by path pattern + skip, err = skipByPathPattern(ctx, skipOptions, repo, in.RunningDSP.Revision(), in.TargetDSP.Revision()) + if err != nil { + return false, err + } + if skip { + return true, nil + } + + // Check by prefix of commit message + skip, err = skipByCommitMessagePrefixes(ctx, skipOptions, repo, in.TargetDSP.Revision()) + return skip, err +} + +// skipByCommitMessagePrefixes returns true if the commit message has ANY one of the specified prefixes. +func skipByCommitMessagePrefixes(ctx context.Context, opt config.SkipOptions, repo git.Repo, targetRev string) (skip bool, err error) { + if len(opt.CommitMessagePrefixes) == 0 { + return false, nil + } + + commit, err := repo.GetCommitForRev(ctx, targetRev) + if err != nil { + return false, err + } + + for _, prefix := range opt.CommitMessagePrefixes { + if strings.HasPrefix(commit.Message, prefix) { + return true, nil + } + } + return false, nil +} + +// skipByPathPattern returns true if and only if ALL changed files are included in `opt.Paths`. +// If ANY changed file does not match all `skipPatterns`, it returns false. +func skipByPathPattern(ctx context.Context, opt config.SkipOptions, repo git.Repo, runningRev, targetRev string) (skip bool, err error) { + if len(opt.Paths) == 0 { + return false, nil + } + + changedFiles, err := repo.ChangedFiles(ctx, runningRev, targetRev) + if err != nil { + return false, err + } + + matcher, err := filematcher.NewPatternMatcher(opt.Paths) + if err != nil { + return false, err + } + + for _, changedFile := range changedFiles { + if !matcher.Matches(changedFile) { + return false, nil + } + } + + return true, nil +} diff --git a/pkg/app/piped/controller/skipstage_test.go b/pkg/app/piped/controller/skipstage_test.go new file mode 100644 index 0000000000..074ad9fed1 --- /dev/null +++ b/pkg/app/piped/controller/skipstage_test.go @@ -0,0 +1,157 @@ +// Copyright 2024 The PipeCD 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 controller + +import ( + "context" + "testing" + + "github.com/golang/mock/gomock" + "github.com/pipe-cd/pipecd/pkg/config" + "github.com/pipe-cd/pipecd/pkg/git" + "github.com/pipe-cd/pipecd/pkg/git/gittest" + "github.com/stretchr/testify/assert" +) + +func TestSkipByCommitMessagePrefixes(t *testing.T) { + t.Parallel() + testcases := []struct { + name string + commitMessage string + prefixes []string + skip bool + }{ + { + name: "no prefixes", + commitMessage: "test message", + prefixes: []string{}, + skip: false, + }, + { + name: "no commit message", + commitMessage: "", + prefixes: []string{"to-skip"}, + skip: false, + }, + { + name: "prefix matches", + commitMessage: "to-skip: test message", + prefixes: []string{"to-skip"}, + skip: true, + }, + } + + for _, tc := range testcases { + tc := tc + t.Run(tc.name, func(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + repoMock := gittest.NewMockRepo(ctrl) + repoMock.EXPECT().GetCommitForRev(gomock.Any(), gomock.Any()).Return(git.Commit{ + Message: tc.commitMessage, + }, nil).AnyTimes() + + opt := config.SkipOptions{ + CommitMessagePrefixes: tc.prefixes, + } + skip, err := skipByCommitMessagePrefixes(context.Background(), opt, repoMock, "test-rev") + assert.Equal(t, tc.skip, skip) + assert.NoError(t, err) + }) + } +} + +func TestSkipByPathPattern(t *testing.T) { + t.Parallel() + testcases := []struct { + name string + skipPatterns []string + changedFiles []string + skip bool + }{ + { + name: "no skip patterns", + skipPatterns: nil, + changedFiles: []string{"file1"}, + skip: false, + }, + { + name: "no changed files", + skipPatterns: []string{"file1"}, + changedFiles: nil, + skip: true, + }, + { + name: "no skip patterns and no changed files", + skipPatterns: nil, + changedFiles: nil, + skip: false, + }, + { + name: "skip pattern matches all changed files", + skipPatterns: []string{"file1", "file2"}, + changedFiles: []string{"file1", "file2"}, + skip: true, + }, + { + name: "skip pattern does not match changed files", + skipPatterns: []string{"file1", "file2"}, + changedFiles: []string{"file1", "file3"}, + skip: false, + }, + { + name: "skip files of a directory", + skipPatterns: []string{"dir1/*"}, + changedFiles: []string{"dir1/file1", "dir1/file2"}, + skip: true, + }, + { + name: "skip files recursively", + skipPatterns: []string{"dir1/**"}, + changedFiles: []string{"dir1/file1", "dir1/sub/file2"}, + skip: true, + }, + { + name: "skip files with the extension recursively", + skipPatterns: []string{"dir1/**/*.yaml"}, + changedFiles: []string{"dir1/file1.yaml", "dir1/sub1/file2.yaml", "dir1/sub1/sub2/file3.yaml"}, + skip: true, + }, + { + name: "skip files not recursively", + skipPatterns: []string{"*.yaml"}, + changedFiles: []string{"dir1/file1.yaml"}, + skip: false, + }, + } + + for _, tc := range testcases { + tc := tc + t.Run(tc.name, func(t *testing.T) { + // We do not use t.Parallel() here due to https://pkg.go.dev/github.com/pipe-cd/pipecd/pkg/filematcher#PatternMatcher.Matches. + ctrl := gomock.NewController(t) + defer ctrl.Finish() + repoMock := gittest.NewMockRepo(ctrl) + repoMock.EXPECT().ChangedFiles(gomock.Any(), gomock.Any(), gomock.Any()).Return(tc.changedFiles, nil).AnyTimes() + + opt := config.SkipOptions{ + Paths: tc.skipPatterns, + } + actual, err := skipByPathPattern(context.Background(), opt, repoMock, "running-rev", "target-rev") + assert.NoError(t, err) + assert.Equal(t, tc.skip, actual) + }) + } +} diff --git a/pkg/app/piped/executor/analysis/analysis.go b/pkg/app/piped/executor/analysis/analysis.go index 9dedaa34ef..775e4fb606 100644 --- a/pkg/app/piped/executor/analysis/analysis.go +++ b/pkg/app/piped/executor/analysis/analysis.go @@ -112,7 +112,7 @@ func (e *Executor) Execute(sig executor.StopSignal) model.StageStatus { for { select { case <-ticker.C: - if !e.checkSkipped(ctx) { + if !e.checkSkippedByCmd(ctx) { continue } status = model.StageStatus_STAGE_SKIPPED @@ -339,7 +339,7 @@ func (e *Executor) buildAppArgs(customArgs map[string]string) argsTemplate { return args } -func (e *Executor) checkSkipped(ctx context.Context) bool { +func (e *Executor) checkSkippedByCmd(ctx context.Context) bool { var skipCmd *model.ReportableCommand commands := e.CommandLister.ListCommands() diff --git a/pkg/config/application.go b/pkg/config/application.go index de27d26ed8..d34599da83 100644 --- a/pkg/config/application.go +++ b/pkg/config/application.go @@ -404,18 +404,26 @@ func (s *PipelineStage) UnmarshalJSON(data []byte) error { return err } +// SkipOptions contains all configurable values for skipping a stage. +type SkipOptions struct { + CommitMessagePrefixes []string `json:"commitMessagePrefixes,omitempty"` + Paths []string `json:"paths,omitempty"` +} + // WaitStageOptions contains all configurable values for a WAIT stage. type WaitStageOptions struct { - Duration Duration `json:"duration"` + Duration Duration `json:"duration"` + SkipOn SkipOptions `json:"skipOn,omitempty"` } // WaitStageOptions contains all configurable values for a WAIT_APPROVAL stage. type WaitApprovalStageOptions struct { // The maximum length of time to wait before giving up. // Defaults to 6h. - Timeout Duration `json:"timeout" default:"6h"` - Approvers []string `json:"approvers"` - MinApproverNum int `json:"minApproverNum" default:"1"` + Timeout Duration `json:"timeout" default:"6h"` + Approvers []string `json:"approvers"` + MinApproverNum int `json:"minApproverNum" default:"1"` + SkipOn SkipOptions `json:"skipOn,omitempty"` } func (w *WaitApprovalStageOptions) Validate() error { @@ -448,6 +456,7 @@ type AnalysisStageOptions struct { Metrics []TemplatableAnalysisMetrics `json:"metrics,omitempty"` Logs []TemplatableAnalysisLog `json:"logs,omitempty"` HTTPS []TemplatableAnalysisHTTP `json:"https,omitempty"` + SkipOn SkipOptions `json:"skipOn,omitempty"` } func (a *AnalysisStageOptions) Validate() error { @@ -498,6 +507,7 @@ type ScriptRunStageOptions struct { Run string `json:"run"` Timeout Duration `json:"timeout" default:"6h"` OnRollback string `json:"onRollback"` + SkipOn SkipOptions `json:"skipOn,omitempty"` } // Validate checks the required fields of ScriptRunStageOptions. diff --git a/pkg/git/gittest/git.mock.go b/pkg/git/gittest/git.mock.go index 62e5cc58be..fbb260bdc3 100644 --- a/pkg/git/gittest/git.mock.go +++ b/pkg/git/gittest/git.mock.go @@ -135,19 +135,19 @@ func (mr *MockRepoMockRecorder) GetClonedBranch() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetClonedBranch", reflect.TypeOf((*MockRepo)(nil).GetClonedBranch)) } -// GetCommitHashForRev mocks base method. -func (m *MockRepo) GetCommitHashForRev(arg0 context.Context, arg1 string) (string, error) { +// GetCommitForRev mocks base method. +func (m *MockRepo) GetCommitForRev(arg0 context.Context, arg1 string) (git.Commit, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "GetCommitHashForRev", arg0, arg1) - ret0, _ := ret[0].(string) + ret := m.ctrl.Call(m, "GetCommitForRev", arg0, arg1) + ret0, _ := ret[0].(git.Commit) ret1, _ := ret[1].(error) return ret0, ret1 } -// GetCommitHashForRev indicates an expected call of GetCommitHashForRev. -func (mr *MockRepoMockRecorder) GetCommitHashForRev(arg0, arg1 interface{}) *gomock.Call { +// GetCommitForRev indicates an expected call of GetCommitForRev. +func (mr *MockRepoMockRecorder) GetCommitForRev(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetCommitHashForRev", reflect.TypeOf((*MockRepo)(nil).GetCommitHashForRev), arg0, arg1) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetCommitForRev", reflect.TypeOf((*MockRepo)(nil).GetCommitForRev), arg0, arg1) } // GetLatestCommit mocks base method. diff --git a/pkg/git/repo.go b/pkg/git/repo.go index 4072bc8779..115f84e0be 100644 --- a/pkg/git/repo.go +++ b/pkg/git/repo.go @@ -38,7 +38,7 @@ type Repo interface { ListCommits(ctx context.Context, visionRange string) ([]Commit, error) GetLatestCommit(ctx context.Context) (Commit, error) - GetCommitHashForRev(ctx context.Context, rev string) (string, error) + GetCommitForRev(ctx context.Context, rev string) (Commit, error) ChangedFiles(ctx context.Context, from, to string) ([]string, error) Checkout(ctx context.Context, commitish string) error CheckoutPullRequest(ctx context.Context, number int, branch string) error @@ -130,14 +130,21 @@ func (r *repo) GetLatestCommit(ctx context.Context) (Commit, error) { return commits[0], nil } -// GetCommitHashForRev returns the hash value of the commit for a given rev. -func (r *repo) GetCommitHashForRev(ctx context.Context, rev string) (string, error) { - out, err := r.runGitCommand(ctx, "rev-parse", rev) +// GetCommitFromRev returns the commit for the given rev. +func (r *repo) GetCommitForRev(ctx context.Context, rev string) (Commit, error) { + args := []string{ + "show", + "--quiet", // suppress diff output + "--no-decorate", + fmt.Sprintf("--pretty=format:%s", commitLogFormat), + rev, + } + out, err := r.runGitCommand(ctx, args...) if err != nil { - return "", formatCommandError(err, out) + return Commit{}, formatCommandError(err, out) } - return strings.TrimSpace(string(out)), nil + return parseCommit(string(out)) } // ChangedFiles returns a list of files those were touched between two commits. diff --git a/pkg/git/repo_test.go b/pkg/git/repo_test.go index 0d32f7b6ef..576f565a3d 100644 --- a/pkg/git/repo_test.go +++ b/pkg/git/repo_test.go @@ -28,33 +28,6 @@ import ( "github.com/stretchr/testify/require" ) -func TestGetCommitHashForRev(t *testing.T) { - faker, err := newFaker() - require.NoError(t, err) - defer faker.clean() - - var ( - org = "test-repo-org" - repoName = "repo-get-commit-hash-for-rev" - ctx = context.Background() - ) - - err = faker.makeRepo(org, repoName) - require.NoError(t, err) - r := &repo{ - dir: faker.repoDir(org, repoName), - gitPath: faker.gitPath, - } - - commits, err := r.ListCommits(ctx, "") - require.NoError(t, err) - assert.Equal(t, 1, len(commits)) - - latestCommitHash, err := r.GetCommitHashForRev(ctx, "HEAD") - require.NoError(t, err) - assert.Equal(t, commits[0].Hash, latestCommitHash) -} - func TestChangedFiles(t *testing.T) { faker, err := newFaker() require.NoError(t, err) @@ -73,9 +46,9 @@ func TestChangedFiles(t *testing.T) { gitPath: faker.gitPath, } - previousCommitHash, err := r.GetCommitHashForRev(ctx, "HEAD") + previousCommit, err := r.GetCommitForRev(ctx, "HEAD") require.NoError(t, err) - require.NotEqual(t, "", previousCommitHash) + require.NotEqual(t, "", previousCommit.Hash) err = os.MkdirAll(filepath.Join(r.dir, "new-dir"), os.ModePerm) require.NoError(t, err) @@ -90,11 +63,11 @@ func TestChangedFiles(t *testing.T) { err = r.addCommit(ctx, "Added new file") require.NoError(t, err) - headCommitHash, err := r.GetCommitHashForRev(ctx, "HEAD") + headCommit, err := r.GetCommitForRev(ctx, "HEAD") require.NoError(t, err) - require.NotEqual(t, "", headCommitHash) + require.NotEqual(t, "", headCommit.Hash) - changedFiles, err := r.ChangedFiles(ctx, previousCommitHash, headCommitHash) + changedFiles, err := r.ChangedFiles(ctx, previousCommit.Hash, headCommit.Hash) sort.Strings(changedFiles) expectedChangedFiles := []string{ "new-dir/new-file.txt", @@ -277,3 +250,30 @@ func TestCopy(t *testing.T) { assert.Equal(t, commits, newRepoCommits) } + +func TestGetCommitForRev(t *testing.T) { + faker, err := newFaker() + require.NoError(t, err) + defer faker.clean() + + var ( + org = "test-repo-org" + repoName = "repo-get-commit-from-rev" + ctx = context.Background() + ) + + err = faker.makeRepo(org, repoName) + require.NoError(t, err) + r := &repo{ + dir: faker.repoDir(org, repoName), + gitPath: faker.gitPath, + } + + commits, err := r.ListCommits(ctx, "") + require.NoError(t, err) + assert.Equal(t, 1, len(commits)) + + commit, err := r.GetCommitForRev(ctx, "HEAD") + require.NoError(t, err) + assert.Equal(t, commits[0].Hash, commit.Hash) +}