runservice: implement task dependencies conditions

Handle the task dependencies conditions:
* on_success (default if no conditions are specified)
* on_failure
* on_skipped

Not the runservice won't stop run but continue executing tasks that depends on a
parent also if this is failed
This commit is contained in:
Simone Gotti 2019-04-12 16:46:04 +02:00
parent 5165984030
commit 68e95ad3be
8 changed files with 567 additions and 55 deletions

View File

@ -109,6 +109,7 @@ type DependCondition string
const ( const (
DependConditionOnSuccess DependCondition = "on_success" DependConditionOnSuccess DependCondition = "on_success"
DependConditionOnFailure DependCondition = "on_failure" DependConditionOnFailure DependCondition = "on_failure"
DependConditionOnSkipped DependCondition = "on_skipped"
) )
type Depend struct { type Depend struct {

View File

@ -364,6 +364,13 @@ func GetAllParents(rcts map[string]*rstypes.RunConfigTask, task *rstypes.RunConf
return parents return parents
} }
func GetParentDependConditions(t, pt *rstypes.RunConfigTask) []rstypes.RunConfigTaskDependCondition {
if dt, ok := t.Depends[pt.ID]; ok {
return dt.Conditions
}
return nil
}
func genEnv(cenv map[string]config.Value, variables map[string]string) map[string]string { func genEnv(cenv map[string]config.Value, variables map[string]string) map[string]string {
env := map[string]string{} env := map[string]string{}
for envName, envVar := range cenv { for envName, envVar := range cenv {

View File

@ -16,10 +16,10 @@ package command
import ( import (
"context" "context"
"fmt"
"path" "path"
"time" "time"
uuid "github.com/satori/go.uuid"
"github.com/sorintlab/agola/internal/db" "github.com/sorintlab/agola/internal/db"
"github.com/sorintlab/agola/internal/etcd" "github.com/sorintlab/agola/internal/etcd"
"github.com/sorintlab/agola/internal/objectstorage" "github.com/sorintlab/agola/internal/objectstorage"
@ -199,7 +199,7 @@ func (s *CommandHandler) newRun(ctx context.Context, req *RunCreateRequest) (*ty
Annotations: req.Annotations, Annotations: req.Annotations,
} }
run := s.genRun(ctx, rc) run := genRun(rc)
s.log.Debugf("created run: %s", util.Dump(run)) s.log.Debugf("created run: %s", util.Dump(run))
return &types.RunBundle{ return &types.RunBundle{
@ -222,10 +222,6 @@ func (s *CommandHandler) recreateRun(ctx context.Context, req *RunCreateRequest)
if err != nil { if err != nil {
return nil, util.NewErrBadRequest(errors.Wrapf(err, "runconfig %q doens't exist", req.RunID)) return nil, util.NewErrBadRequest(errors.Wrapf(err, "runconfig %q doens't exist", req.RunID))
} }
// update the run config ID
rc.ID = id
// update the run config Environment
rc.Environment = req.Environment
run, err := store.GetRunEtcdOrLTS(ctx, s.e, s.wal, req.RunID) run, err := store.GetRunEtcdOrLTS(ctx, s.e, s.wal, req.RunID)
if err != nil { if err != nil {
@ -235,6 +231,9 @@ func (s *CommandHandler) recreateRun(ctx context.Context, req *RunCreateRequest)
return nil, util.NewErrBadRequest(errors.Wrapf(err, "run %q doens't exist", req.RunID)) return nil, util.NewErrBadRequest(errors.Wrapf(err, "run %q doens't exist", req.RunID))
} }
s.log.Infof("rc: %s", util.Dump(rc))
s.log.Infof("run: %s", util.Dump(run))
if req.FromStart { if req.FromStart {
if canRestart, reason := run.CanRestartFromScratch(); !canRestart { if canRestart, reason := run.CanRestartFromScratch(); !canRestart {
return nil, util.NewErrBadRequest(errors.Errorf("run cannot be restarted: %s", reason)) return nil, util.NewErrBadRequest(errors.Errorf("run cannot be restarted: %s", reason))
@ -245,8 +244,22 @@ func (s *CommandHandler) recreateRun(ctx context.Context, req *RunCreateRequest)
} }
} }
rb := recreateRun(util.DefaultUUIDGenerator{}, run, rc, id, req)
s.log.Infof("created rc from existing rc: %s", util.Dump(rb.Rc))
s.log.Infof("created run from existing run: %s", util.Dump(rb.Run))
return rb, nil
}
func recreateRun(uuid util.UUIDGenerator, run *types.Run, rc *types.RunConfig, newID string, req *RunCreateRequest) *types.RunBundle {
// update the run config ID
rc.ID = newID
// update the run config Environment
rc.Environment = req.Environment
// update the run ID // update the run ID
run.ID = id run.ID = newID
// reset run revision // reset run revision
run.Revision = 0 run.Revision = 0
// reset phase/result/archived/stop // reset phase/result/archived/stop
@ -254,48 +267,102 @@ func (s *CommandHandler) recreateRun(ctx context.Context, req *RunCreateRequest)
run.Result = types.RunResultUnknown run.Result = types.RunResultUnknown
run.Archived = false run.Archived = false
run.Stop = false run.Stop = false
run.EnqueueTime = nil
run.StartTime = nil
run.EndTime = nil
// TODO(sgotti) handle reset tasks // TODO(sgotti) handle reset tasks
// currently we only restart a run resetting al failed tasks // currently we only restart a run resetting al failed tasks
tasksToAdd := []*types.RunTask{} recreatedRCTasks := map[string]struct{}{}
tasksToDelete := []string{}
for _, rt := range run.RunTasks { for _, rt := range run.RunTasks {
if req.FromStart || rt.Status != types.RunTaskStatusSuccess { if req.FromStart || rt.Status != types.RunTaskStatusSuccess {
rct := rc.Tasks[rt.ID] rct, ok := rc.Tasks[rt.ID]
if !ok {
panic(fmt.Errorf("no runconfig task %q", rt.ID))
}
// change rct id // change rct id
rct.ID = uuid.NewV4().String() rct.ID = uuid.New(rct.Name).String()
// update runconfig // update runconfig with new tasks
delete(rc.Tasks, rt.ID) delete(rc.Tasks, rt.ID)
rc.Tasks[rct.ID] = rct rc.Tasks[rct.ID] = rct
// update other tasks depends to new task id
// update other runconfig tasks depends to new task id
for _, t := range rc.Tasks { for _, t := range rc.Tasks {
for _, d := range t.Depends { if d, ok := t.Depends[rt.ID]; ok {
if d.TaskID == rt.ID { delete(t.Depends, rt.ID)
d.TaskID = rct.ID nd := &types.RunConfigTaskDepend{
TaskID: rct.ID,
Conditions: d.Conditions,
} }
t.Depends[rct.ID] = nd
} }
} }
nrt := s.genRunTask(ctx, rct) recreatedRCTasks[rct.ID] = struct{}{}
tasksToAdd = append(tasksToAdd, nrt)
tasksToDelete = append(tasksToDelete, rt.ID)
} }
} }
for _, rt := range tasksToAdd {
run.RunTasks[rt.ID] = rt // also recreate all runconfig tasks that are childs of a previously recreated
// runconfig task
rcTasksToRecreate := map[string]struct{}{}
for _, rct := range rc.Tasks {
parents := runconfig.GetAllParents(rc.Tasks, rct)
for _, parent := range parents {
if _, ok := recreatedRCTasks[parent.ID]; ok {
rcTasksToRecreate[rct.ID] = struct{}{}
break
}
}
}
for rcTaskToRecreate := range rcTasksToRecreate {
rct := rc.Tasks[rcTaskToRecreate]
// change rct id
rct.ID = uuid.New(rct.Name).String()
// update runconfig with new tasks
delete(rc.Tasks, rcTaskToRecreate)
rc.Tasks[rct.ID] = rct
// update other runconfig tasks depends to new task id
for _, t := range rc.Tasks {
if d, ok := t.Depends[rcTaskToRecreate]; ok {
delete(t.Depends, rcTaskToRecreate)
nd := &types.RunConfigTaskDepend{
TaskID: rct.ID,
Conditions: d.Conditions,
}
t.Depends[rct.ID] = nd
}
}
}
// update run
// remove deleted tasks from run config
tasksToDelete := []string{}
for _, rt := range run.RunTasks {
if _, ok := rc.Tasks[rt.ID]; !ok {
tasksToDelete = append(tasksToDelete, rt.ID)
}
} }
for _, rtID := range tasksToDelete { for _, rtID := range tasksToDelete {
delete(run.RunTasks, rtID) delete(run.RunTasks, rtID)
} }
// create new tasks from runconfig
s.log.Debugf("created run from existing run: %s", util.Dump(run)) for _, rct := range rc.Tasks {
if _, ok := run.RunTasks[rct.ID]; !ok {
nrt := genRunTask(rct)
run.RunTasks[nrt.ID] = nrt
}
}
return &types.RunBundle{ return &types.RunBundle{
Run: run, Run: run,
Rc: rc, Rc: rc,
}, nil }
} }
func (s *CommandHandler) saveRun(ctx context.Context, rb *types.RunBundle, runcgt *types.ChangeGroupsUpdateToken) error { func (s *CommandHandler) saveRun(ctx context.Context, rb *types.RunBundle, runcgt *types.ChangeGroupsUpdateToken) error {
@ -310,6 +377,8 @@ func (s *CommandHandler) saveRun(ctx context.Context, rb *types.RunBundle, runcg
c++ c++
run.Counter = c run.Counter = c
run.EnqueueTime = util.TimePtr(time.Now())
actions := []*wal.Action{} actions := []*wal.Action{}
// persist group counter // persist group counter
@ -340,7 +409,7 @@ func (s *CommandHandler) saveRun(ctx context.Context, rb *types.RunBundle, runcg
return nil return nil
} }
func (s *CommandHandler) genRunTask(ctx context.Context, rct *types.RunConfigTask) *types.RunTask { func genRunTask(rct *types.RunConfigTask) *types.RunTask {
rt := &types.RunTask{ rt := &types.RunTask{
ID: rct.ID, ID: rct.ID,
Status: types.RunTaskStatusNotStarted, Status: types.RunTaskStatusNotStarted,
@ -376,7 +445,7 @@ func (s *CommandHandler) genRunTask(ctx context.Context, rct *types.RunConfigTas
return rt return rt
} }
func (s *CommandHandler) genRun(ctx context.Context, rc *types.RunConfig) *types.Run { func genRun(rc *types.RunConfig) *types.Run {
r := &types.Run{ r := &types.Run{
ID: rc.ID, ID: rc.ID,
Name: rc.Name, Name: rc.Name,
@ -385,7 +454,6 @@ func (s *CommandHandler) genRun(ctx context.Context, rc *types.RunConfig) *types
Phase: types.RunPhaseQueued, Phase: types.RunPhaseQueued,
Result: types.RunResultUnknown, Result: types.RunResultUnknown,
RunTasks: make(map[string]*types.RunTask), RunTasks: make(map[string]*types.RunTask),
EnqueueTime: util.TimePtr(time.Now()),
} }
if len(rc.SetupErrors) > 0 { if len(rc.SetupErrors) > 0 {
@ -394,7 +462,7 @@ func (s *CommandHandler) genRun(ctx context.Context, rc *types.RunConfig) *types
} }
for _, rct := range rc.Tasks { for _, rct := range rc.Tasks {
rt := s.genRunTask(ctx, rct) rt := genRunTask(rct)
r.RunTasks[rt.ID] = rt r.RunTasks[rt.ID] = rt
} }

View File

@ -0,0 +1,280 @@
// Copyright 2019 Sorint.lab
//
// 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 command
import (
"testing"
"github.com/google/go-cmp/cmp"
"github.com/sorintlab/agola/internal/services/runservice/types"
"github.com/sorintlab/agola/internal/util"
)
func TestRecreateRun(t *testing.T) {
inuuid := func(s string) string {
u := &util.TestPrefixUUIDGenerator{Prefix: "in"}
return u.New(s).String()
}
outuuid := func(s string) string {
u := &util.TestPrefixUUIDGenerator{Prefix: "out"}
return u.New(s).String()
}
// a global run config for all tests
rc := &types.RunConfig{
ID: inuuid("old"),
Tasks: map[string]*types.RunConfigTask{
inuuid("task01"): &types.RunConfigTask{
ID: inuuid("task01"),
Name: "task01",
Depends: map[string]*types.RunConfigTaskDepend{},
Runtime: &types.Runtime{Type: types.RuntimeType("pod"),
Containers: []*types.Container{{Image: "image01"}},
},
Environment: map[string]string{},
Steps: []interface{}{},
Skip: false,
},
inuuid("task02"): &types.RunConfigTask{
ID: inuuid("task02"),
Name: "task02",
Depends: map[string]*types.RunConfigTaskDepend{
inuuid("task01"): &types.RunConfigTaskDepend{TaskID: inuuid("task01"), Conditions: []types.RunConfigTaskDependCondition{types.RunConfigTaskDependConditionOnSuccess}},
},
Runtime: &types.Runtime{Type: types.RuntimeType("pod"),
Containers: []*types.Container{{Image: "image01"}},
},
Environment: map[string]string{},
Steps: []interface{}{},
Skip: false,
},
inuuid("task03"): &types.RunConfigTask{
ID: inuuid("task03"),
Name: "task03",
Depends: map[string]*types.RunConfigTaskDepend{},
Runtime: &types.Runtime{Type: types.RuntimeType("pod"),
Containers: []*types.Container{{Image: "image01"}},
},
Environment: map[string]string{},
Steps: []interface{}{},
Skip: false,
},
inuuid("task04"): &types.RunConfigTask{
ID: inuuid("task04"),
Name: "task04",
Runtime: &types.Runtime{Type: types.RuntimeType("pod"),
Containers: []*types.Container{{Image: "image01"}},
},
Environment: map[string]string{},
Steps: []interface{}{},
Skip: false,
},
inuuid("task05"): &types.RunConfigTask{
ID: inuuid("task05"),
Name: "task05",
Depends: map[string]*types.RunConfigTaskDepend{
inuuid("task03"): &types.RunConfigTaskDepend{TaskID: inuuid("task03"), Conditions: []types.RunConfigTaskDependCondition{types.RunConfigTaskDependConditionOnSuccess}},
inuuid("task04"): &types.RunConfigTaskDepend{TaskID: inuuid("task04"), Conditions: []types.RunConfigTaskDependCondition{types.RunConfigTaskDependConditionOnSuccess}},
},
Runtime: &types.Runtime{Type: types.RuntimeType("pod"),
Containers: []*types.Container{{Image: "image01"}},
},
Environment: map[string]string{},
Steps: []interface{}{},
Skip: false,
},
},
}
outrc := &types.RunConfig{
ID: outuuid("new"),
Tasks: map[string]*types.RunConfigTask{
outuuid("task01"): &types.RunConfigTask{
ID: outuuid("task01"),
Name: "task01",
Depends: map[string]*types.RunConfigTaskDepend{},
Runtime: &types.Runtime{Type: types.RuntimeType("pod"),
Containers: []*types.Container{{Image: "image01"}},
},
Environment: map[string]string{},
Steps: []interface{}{},
Skip: false,
},
outuuid("task02"): &types.RunConfigTask{
ID: outuuid("task02"),
Name: "task02",
Depends: map[string]*types.RunConfigTaskDepend{
outuuid("task01"): &types.RunConfigTaskDepend{TaskID: outuuid("task01"), Conditions: []types.RunConfigTaskDependCondition{types.RunConfigTaskDependConditionOnSuccess}},
},
Runtime: &types.Runtime{Type: types.RuntimeType("pod"),
Containers: []*types.Container{{Image: "image01"}},
},
Environment: map[string]string{},
Steps: []interface{}{},
Skip: false,
},
outuuid("task03"): &types.RunConfigTask{
ID: outuuid("task03"),
Name: "task03",
Depends: map[string]*types.RunConfigTaskDepend{},
Runtime: &types.Runtime{Type: types.RuntimeType("pod"),
Containers: []*types.Container{{Image: "image01"}},
},
Environment: map[string]string{},
Steps: []interface{}{},
Skip: false,
},
outuuid("task04"): &types.RunConfigTask{
ID: outuuid("task04"),
Name: "task04",
Runtime: &types.Runtime{Type: types.RuntimeType("pod"),
Containers: []*types.Container{{Image: "image01"}},
},
Environment: map[string]string{},
Steps: []interface{}{},
Skip: false,
},
outuuid("task05"): &types.RunConfigTask{
ID: outuuid("task05"),
Name: "task05",
Depends: map[string]*types.RunConfigTaskDepend{
outuuid("task03"): &types.RunConfigTaskDepend{TaskID: outuuid("task03"), Conditions: []types.RunConfigTaskDependCondition{types.RunConfigTaskDependConditionOnSuccess}},
outuuid("task04"): &types.RunConfigTaskDepend{TaskID: outuuid("task04"), Conditions: []types.RunConfigTaskDependCondition{types.RunConfigTaskDependConditionOnSuccess}},
},
Runtime: &types.Runtime{Type: types.RuntimeType("pod"),
Containers: []*types.Container{{Image: "image01"}},
},
Environment: map[string]string{},
Steps: []interface{}{},
Skip: false,
},
},
}
// initial run that matched the runconfig, all tasks are not started or skipped
// (if the runconfig task as Skip == true). This must match the status
// generated by command.genRun()
run := genRun(rc)
outrun := genRun(outrc)
tests := []struct {
name string
rc *types.RunConfig
r *types.Run
req *RunCreateRequest
outrc *types.RunConfig
outr *types.Run
err error
}{
{
name: "test recreate run from start with all not start tasks",
rc: rc.DeepCopy(),
r: run.DeepCopy(),
outrc: outrc.DeepCopy(),
outr: outrun.DeepCopy(),
req: &RunCreateRequest{FromStart: true},
},
{
name: "test recreate run from failed tasks with all not start tasks",
rc: rc.DeepCopy(),
r: run.DeepCopy(),
outrc: outrc.DeepCopy(),
outr: outrun.DeepCopy(),
req: &RunCreateRequest{FromStart: false},
},
{
name: "test recreate run from start tasks with task01 failed and child task02 successful (should recreate all tasks)",
rc: rc.DeepCopy(),
r: func() *types.Run {
run := run.DeepCopy()
run.RunTasks[inuuid("task01")].Status = types.RunTaskStatusFailed
run.RunTasks[inuuid("task02")].Status = types.RunTaskStatusSuccess
run.RunTasks[inuuid("task03")].Status = types.RunTaskStatusSuccess
run.RunTasks[inuuid("task04")].Status = types.RunTaskStatusSuccess
run.RunTasks[inuuid("task05")].Status = types.RunTaskStatusSuccess
return run
}(),
outrc: outrc.DeepCopy(),
outr: outrun.DeepCopy(),
req: &RunCreateRequest{FromStart: true},
},
{
name: "test recreate run from failed tasks with task01 failed and child task02 successful (should recreate task01 and task02)",
rc: rc.DeepCopy(),
r: func() *types.Run {
run := run.DeepCopy()
run.RunTasks[inuuid("task01")].Status = types.RunTaskStatusFailed
run.RunTasks[inuuid("task02")].Status = types.RunTaskStatusSuccess
run.RunTasks[inuuid("task03")].Status = types.RunTaskStatusSuccess
run.RunTasks[inuuid("task04")].Status = types.RunTaskStatusSuccess
run.RunTasks[inuuid("task05")].Status = types.RunTaskStatusSuccess
return run
}(),
// task01 and task02 recreated
outrc: func() *types.RunConfig {
rc := rc.DeepCopy()
outrc := outrc.DeepCopy()
nrc := rc.DeepCopy()
nrc.ID = outuuid("new")
nrc.Tasks = map[string]*types.RunConfigTask{
outuuid("task01"): outrc.Tasks[outuuid("task01")],
outuuid("task02"): outrc.Tasks[outuuid("task02")],
inuuid("task03"): rc.Tasks[inuuid("task03")],
inuuid("task04"): rc.Tasks[inuuid("task04")],
inuuid("task05"): rc.Tasks[inuuid("task05")],
}
return nrc
}(),
// task01 and task02 recreated and status reset to NotStarted
outr: func() *types.Run {
run := run.DeepCopy()
outrun := outrun.DeepCopy()
nrun := run.DeepCopy()
nrun.ID = outuuid("new")
nrun.RunTasks = map[string]*types.RunTask{
outuuid("task01"): outrun.RunTasks[outuuid("task01")],
outuuid("task02"): outrun.RunTasks[outuuid("task02")],
inuuid("task03"): run.RunTasks[inuuid("task03")],
inuuid("task04"): run.RunTasks[inuuid("task04")],
inuuid("task05"): run.RunTasks[inuuid("task05")],
}
nrun.RunTasks[inuuid("task03")].Status = types.RunTaskStatusSuccess
nrun.RunTasks[inuuid("task04")].Status = types.RunTaskStatusSuccess
nrun.RunTasks[inuuid("task05")].Status = types.RunTaskStatusSuccess
return nrun
}(),
req: &RunCreateRequest{FromStart: false},
},
}
u := &util.TestPrefixUUIDGenerator{Prefix: "out"}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
newID := outuuid("new")
rb := recreateRun(u, tt.r, tt.rc, newID, tt.req)
if diff := cmp.Diff(tt.outrc, rb.Rc); diff != "" {
t.Error(diff)
}
if diff := cmp.Diff(tt.outr, rb.Run); diff != "" {
t.Error(diff)
}
})
}
}

View File

@ -105,23 +105,51 @@ func advanceRunTasks(ctx context.Context, r *types.Run, rc *types.RunConfig) err
rct := rc.Tasks[rt.ID] rct := rc.Tasks[rt.ID]
parents := runconfig.GetParents(rc.Tasks, rct) parents := runconfig.GetParents(rc.Tasks, rct)
canRun := true finishedParents := 0
allParentsSkipped := false
for _, p := range parents { for _, p := range parents {
allParentsSkipped = true
rp := r.RunTasks[p.ID] rp := r.RunTasks[p.ID]
canRun = rp.Status.IsFinished() && rp.ArchivesFetchFinished() if rp.Status.IsFinished() && rp.ArchivesFetchFinished() {
// skip only if all parents are skipped finishedParents++
if rp.Status != types.RunTaskStatusSkipped {
allParentsSkipped = false
} }
} }
if allParentsSkipped { allParentsFinished := finishedParents == len(parents)
rt.Status = types.RunTaskStatusSkipped
} // if all parents are finished check if the task could be executed or be skipped
matchedNum := 0
if allParentsFinished {
for _, p := range parents {
matched := false
rp := r.RunTasks[p.ID]
conds := runconfig.GetParentDependConditions(rct, p)
for _, cond := range conds {
switch cond {
case types.RunConfigTaskDependConditionOnSuccess:
if rp.Status == types.RunTaskStatusSuccess {
matched = true
}
case types.RunConfigTaskDependConditionOnFailure:
if rp.Status == types.RunTaskStatusFailed {
matched = true
}
case types.RunConfigTaskDependConditionOnSkipped:
if rp.Status == types.RunTaskStatusSkipped {
matched = true
}
}
}
if matched {
matchedNum++
}
}
// if all parents are matched then we can start it, otherwise we mark the step to be skipped
skip := len(parents) != matchedNum
if skip {
rt.Status = types.RunTaskStatusSkipped
continue
}
if canRun {
// now that the task can run set it to waiting approval if needed // now that the task can run set it to waiting approval if needed
if rct.NeedsApproval && !rt.WaitingApproval && !rt.Approved { if rct.NeedsApproval && !rt.WaitingApproval && !rt.Approved {
rt.WaitingApproval = true rt.WaitingApproval = true
@ -148,13 +176,17 @@ func getTasksToRun(ctx context.Context, r *types.Run, rc *types.RunConfig) ([]*t
rct := rc.Tasks[rt.ID] rct := rc.Tasks[rt.ID]
parents := runconfig.GetParents(rc.Tasks, rct) parents := runconfig.GetParents(rc.Tasks, rct)
canRun := true finishedParents := 0
for _, p := range parents { for _, p := range parents {
rp := r.RunTasks[p.ID] rp := r.RunTasks[p.ID]
canRun = rp.Status.IsFinished() && rp.ArchivesFetchFinished() if rp.Status.IsFinished() && rp.ArchivesFetchFinished() {
finishedParents++
}
} }
if canRun { allParentsFinished := finishedParents == len(parents)
if allParentsFinished {
// Run only if approved (when needs approval) // Run only if approved (when needs approval)
if !rct.NeedsApproval || (rct.NeedsApproval && rt.Approved) { if !rct.NeedsApproval || (rct.NeedsApproval && rt.Approved) {
tasksToRun = append(tasksToRun, rt) tasksToRun = append(tasksToRun, rt)
@ -411,7 +443,8 @@ func (s *Scheduler) scheduleRun(ctx context.Context, r *types.Run, rc *types.Run
} }
} }
if !r.Result.IsSet() && r.Phase == types.RunPhaseRunning { // advance tasks
if r.Phase == types.RunPhaseRunning {
if err := advanceRunTasks(ctx, r, rc); err != nil { if err := advanceRunTasks(ctx, r, rc); err != nil {
return err return err
} }
@ -454,7 +487,7 @@ func advanceRun(ctx context.Context, r *types.Run, rc *types.RunConfig, hasActiv
} }
} }
// see if run could me marked as success // see if run could be marked as success
if !r.Result.IsSet() && r.Phase == types.RunPhaseRunning { if !r.Result.IsSet() && r.Phase == types.RunPhaseRunning {
finished := true finished := true
for _, rt := range r.RunTasks { for _, rt := range r.RunTasks {
@ -475,11 +508,17 @@ func advanceRun(ctx context.Context, r *types.Run, rc *types.RunConfig, hasActiv
} }
} }
// if the run has a result defined then we can stop current tasks // if the run has a result defined AND all tasks are finished AND there're no executor tasks scheduled we can mark
// the run phase as finished
if r.Result.IsSet() { if r.Result.IsSet() {
if !r.Phase.IsFinished() { finished := true
// if the run has a result defined AND there're no executor tasks scheduled we can mark for _, rt := range r.RunTasks {
// the run phase as finished if !rt.Status.IsFinished() {
finished = false
}
}
if finished && !r.Phase.IsFinished() {
if !hasActiveTasks { if !hasActiveTasks {
r.ChangePhase(types.RunPhaseFinished) r.ChangePhase(types.RunPhaseFinished)
} }

View File

@ -42,7 +42,7 @@ func TestAdvanceRunTasks(t *testing.T) {
ID: "task02", ID: "task02",
Name: "task02", Name: "task02",
Depends: map[string]*types.RunConfigTaskDepend{ Depends: map[string]*types.RunConfigTaskDepend{
"task01": &types.RunConfigTaskDepend{TaskID: "task01"}, "task01": &types.RunConfigTaskDepend{TaskID: "task01", Conditions: []types.RunConfigTaskDependCondition{types.RunConfigTaskDependConditionOnSuccess}},
}, },
Runtime: &types.Runtime{Type: types.RuntimeType("pod"), Runtime: &types.Runtime{Type: types.RuntimeType("pod"),
Containers: []*types.Container{{Image: "image01"}}, Containers: []*types.Container{{Image: "image01"}},
@ -76,8 +76,8 @@ func TestAdvanceRunTasks(t *testing.T) {
ID: "task05", ID: "task05",
Name: "task05", Name: "task05",
Depends: map[string]*types.RunConfigTaskDepend{ Depends: map[string]*types.RunConfigTaskDepend{
"task03": &types.RunConfigTaskDepend{TaskID: "task03"}, "task03": &types.RunConfigTaskDepend{TaskID: "task03", Conditions: []types.RunConfigTaskDependCondition{types.RunConfigTaskDependConditionOnSuccess}},
"task04": &types.RunConfigTaskDepend{TaskID: "task04"}, "task04": &types.RunConfigTaskDepend{TaskID: "task04", Conditions: []types.RunConfigTaskDependCondition{types.RunConfigTaskDependConditionOnSuccess}},
}, },
Runtime: &types.Runtime{Type: types.RuntimeType("pod"), Runtime: &types.Runtime{Type: types.RuntimeType("pod"),
Containers: []*types.Container{{Image: "image01"}}, Containers: []*types.Container{{Image: "image01"}},
@ -172,7 +172,7 @@ func TestAdvanceRunTasks(t *testing.T) {
}(), }(),
}, },
{ {
name: "test task status not set to skipped when not all parent status is skipped", name: "test task set to skipped when only some parents status is skipped",
rc: func() *types.RunConfig { rc: func() *types.RunConfig {
rc := rc.DeepCopy() rc := rc.DeepCopy()
rc.Tasks["task03"].Skip = true rc.Tasks["task03"].Skip = true
@ -188,6 +188,116 @@ func TestAdvanceRunTasks(t *testing.T) {
run := run.DeepCopy() run := run.DeepCopy()
run.RunTasks["task03"].Status = types.RunTaskStatusSkipped run.RunTasks["task03"].Status = types.RunTaskStatusSkipped
run.RunTasks["task04"].Status = types.RunTaskStatusSuccess run.RunTasks["task04"].Status = types.RunTaskStatusSuccess
run.RunTasks["task05"].Status = types.RunTaskStatusSkipped
return run
}(),
},
{
name: "test task set to skipped when one of the parents doesn't match default conditions (on_success)",
rc: func() *types.RunConfig {
rc := rc.DeepCopy()
rc.Tasks["task03"].Skip = true
return rc
}(),
r: func() *types.Run {
run := run.DeepCopy()
run.RunTasks["task03"].Status = types.RunTaskStatusSkipped
run.RunTasks["task04"].Status = types.RunTaskStatusSuccess
return run
}(),
out: func() *types.Run {
run := run.DeepCopy()
run.RunTasks["task03"].Status = types.RunTaskStatusSkipped
run.RunTasks["task04"].Status = types.RunTaskStatusSuccess
run.RunTasks["task05"].Status = types.RunTaskStatusSkipped
return run
}(),
},
{
name: "test task set to skipped when one of the parents doesn't match custom conditions",
rc: func() *types.RunConfig {
rc := rc.DeepCopy()
rc.Tasks["task03"].Skip = true
rc.Tasks["task05"].Depends["task03"].Conditions = []types.RunConfigTaskDependCondition{types.RunConfigTaskDependConditionOnFailure}
return rc
}(),
r: func() *types.Run {
run := run.DeepCopy()
run.RunTasks["task03"].Status = types.RunTaskStatusSkipped
run.RunTasks["task04"].Status = types.RunTaskStatusSuccess
return run
}(),
out: func() *types.Run {
run := run.DeepCopy()
run.RunTasks["task03"].Status = types.RunTaskStatusSkipped
run.RunTasks["task04"].Status = types.RunTaskStatusSuccess
run.RunTasks["task05"].Status = types.RunTaskStatusSkipped
return run
}(),
},
{
name: "test task set to not skipped when one of the parent is skipped and task condition is on_skipped",
rc: func() *types.RunConfig {
rc := rc.DeepCopy()
rc.Tasks["task03"].Skip = true
rc.Tasks["task05"].Depends["task03"].Conditions = []types.RunConfigTaskDependCondition{types.RunConfigTaskDependConditionOnSkipped}
return rc
}(),
r: func() *types.Run {
run := run.DeepCopy()
run.RunTasks["task03"].Status = types.RunTaskStatusSkipped
run.RunTasks["task04"].Status = types.RunTaskStatusSuccess
return run
}(),
out: func() *types.Run {
run := run.DeepCopy()
run.RunTasks["task03"].Status = types.RunTaskStatusSkipped
run.RunTasks["task04"].Status = types.RunTaskStatusSuccess
return run
}(),
},
{
name: "test task not set to waiting approval when task is skipped",
rc: func() *types.RunConfig {
rc := rc.DeepCopy()
rc.Tasks["task03"].Skip = true
rc.Tasks["task05"].NeedsApproval = true
return rc
}(),
r: func() *types.Run {
run := run.DeepCopy()
run.RunTasks["task03"].Status = types.RunTaskStatusSkipped
run.RunTasks["task04"].Status = types.RunTaskStatusSuccess
return run
}(),
out: func() *types.Run {
run := run.DeepCopy()
run.RunTasks["task03"].Status = types.RunTaskStatusSkipped
run.RunTasks["task04"].Status = types.RunTaskStatusSuccess
run.RunTasks["task05"].Status = types.RunTaskStatusSkipped
return run
}(),
},
{
name: "test task set to waiting approval when all the parents are finished and task is not skipped",
rc: func() *types.RunConfig {
rc := rc.DeepCopy()
rc.Tasks["task03"].Skip = true
rc.Tasks["task05"].NeedsApproval = true
rc.Tasks["task05"].Depends["task03"].Conditions = []types.RunConfigTaskDependCondition{types.RunConfigTaskDependConditionOnSkipped}
return rc
}(),
r: func() *types.Run {
run := run.DeepCopy()
run.RunTasks["task03"].Status = types.RunTaskStatusSkipped
run.RunTasks["task04"].Status = types.RunTaskStatusSuccess
return run
}(),
out: func() *types.Run {
run := run.DeepCopy()
run.RunTasks["task03"].Status = types.RunTaskStatusSkipped
run.RunTasks["task04"].Status = types.RunTaskStatusSuccess
run.RunTasks["task05"].WaitingApproval = true
return run return run
}(), }(),
}, },
@ -225,7 +335,7 @@ func TestGetTasksToRun(t *testing.T) {
ID: "task02", ID: "task02",
Name: "task02", Name: "task02",
Depends: map[string]*types.RunConfigTaskDepend{ Depends: map[string]*types.RunConfigTaskDepend{
"task01": &types.RunConfigTaskDepend{TaskID: "task01"}, "task01": &types.RunConfigTaskDepend{TaskID: "task01", Conditions: []types.RunConfigTaskDependCondition{types.RunConfigTaskDependConditionOnSuccess}},
}, },
Runtime: &types.Runtime{Type: types.RuntimeType("pod"), Runtime: &types.Runtime{Type: types.RuntimeType("pod"),
Containers: []*types.Container{{Image: "image01"}}, Containers: []*types.Container{{Image: "image01"}},
@ -259,8 +369,8 @@ func TestGetTasksToRun(t *testing.T) {
ID: "task05", ID: "task05",
Name: "task05", Name: "task05",
Depends: map[string]*types.RunConfigTaskDepend{ Depends: map[string]*types.RunConfigTaskDepend{
"task03": &types.RunConfigTaskDepend{TaskID: "task03"}, "task03": &types.RunConfigTaskDepend{TaskID: "task03", Conditions: []types.RunConfigTaskDependCondition{types.RunConfigTaskDependConditionOnSuccess}},
"task04": &types.RunConfigTaskDepend{TaskID: "task04"}, "task04": &types.RunConfigTaskDepend{TaskID: "task04", Conditions: []types.RunConfigTaskDependCondition{types.RunConfigTaskDependConditionOnSuccess}},
}, },
Runtime: &types.Runtime{Type: types.RuntimeType("pod"), Runtime: &types.Runtime{Type: types.RuntimeType("pod"),
Containers: []*types.Container{{Image: "image01"}}, Containers: []*types.Container{{Image: "image01"}},

View File

@ -336,6 +336,7 @@ type RunConfigTaskDependCondition string
const ( const (
RunConfigTaskDependConditionOnSuccess RunConfigTaskDependCondition = "on_success" RunConfigTaskDependConditionOnSuccess RunConfigTaskDependCondition = "on_success"
RunConfigTaskDependConditionOnFailure RunConfigTaskDependCondition = "on_failure" RunConfigTaskDependConditionOnFailure RunConfigTaskDependCondition = "on_failure"
RunConfigTaskDependConditionOnSkipped RunConfigTaskDependCondition = "on_skipped"
) )
type RunConfigTaskDepend struct { type RunConfigTaskDepend struct {

View File

@ -33,3 +33,9 @@ type TestUUIDGenerator struct{}
func (u TestUUIDGenerator) New(s string) uuid.UUID { func (u TestUUIDGenerator) New(s string) uuid.UUID {
return uuid.NewV5(uuid.NamespaceDNS, s) return uuid.NewV5(uuid.NamespaceDNS, s)
} }
type TestPrefixUUIDGenerator struct{ Prefix string }
func (u TestPrefixUUIDGenerator) New(s string) uuid.UUID {
return uuid.NewV5(uuid.NamespaceDNS, u.Prefix+s)
}