You've already forked woodpecker
							
							
				mirror of
				https://github.com/woodpecker-ci/woodpecker.git
				synced 2025-10-30 23:27:39 +02:00 
			
		
		
		
	Fifo queue with dependencies
This commit is contained in:
		| @@ -7,6 +7,8 @@ import ( | |||||||
| 	"runtime" | 	"runtime" | ||||||
| 	"sync" | 	"sync" | ||||||
| 	"time" | 	"time" | ||||||
|  |  | ||||||
|  | 	"github.com/Sirupsen/logrus" | ||||||
| ) | ) | ||||||
|  |  | ||||||
| type entry struct { | type entry struct { | ||||||
| @@ -50,6 +52,19 @@ func (q *fifo) Push(c context.Context, task *Task) error { | |||||||
| 	return nil | 	return nil | ||||||
| } | } | ||||||
|  |  | ||||||
|  | // Push pushes an item to the tail of this queue. | ||||||
|  | func (q *fifo) PushAtOnce(c context.Context, tasks []*Task) error { | ||||||
|  | 	q.Lock() | ||||||
|  | 	for _, task := range tasks { | ||||||
|  | 		q.pending.PushBack(task) | ||||||
|  | 	} | ||||||
|  | 	q.Unlock() | ||||||
|  | 	for range tasks { | ||||||
|  | 		go q.process() | ||||||
|  | 	} | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  |  | ||||||
| // Poll retrieves and removes the head of this queue. | // Poll retrieves and removes the head of this queue. | ||||||
| func (q *fifo) Poll(c context.Context, f Filter) (*Task, error) { | func (q *fifo) Poll(c context.Context, f Filter) (*Task, error) { | ||||||
| 	q.Lock() | 	q.Lock() | ||||||
| @@ -187,21 +202,48 @@ func (q *fifo) process() { | |||||||
| loop: | loop: | ||||||
| 	for e := q.pending.Front(); e != nil; e = next { | 	for e := q.pending.Front(); e != nil; e = next { | ||||||
| 		next = e.Next() | 		next = e.Next() | ||||||
| 		item := e.Value.(*Task) | 		task := e.Value.(*Task) | ||||||
|  | 		logrus.Debugf("queue: trying to assign task: %v with deps %v", task.ID, task.Dependencies) | ||||||
|  | 		if q.depsInQueue(task) { | ||||||
|  | 			continue | ||||||
|  | 		} | ||||||
| 		for w := range q.workers { | 		for w := range q.workers { | ||||||
| 			if w.filter(item) { | 			if w.filter(task) { | ||||||
| 				delete(q.workers, w) | 				delete(q.workers, w) | ||||||
| 				q.pending.Remove(e) | 				q.pending.Remove(e) | ||||||
|  |  | ||||||
| 				q.running[item.ID] = &entry{ | 				q.running[task.ID] = &entry{ | ||||||
| 					item:     item, | 					item:     task, | ||||||
| 					done:     make(chan bool), | 					done:     make(chan bool), | ||||||
| 					deadline: time.Now().Add(q.extension), | 					deadline: time.Now().Add(q.extension), | ||||||
| 				} | 				} | ||||||
|  |  | ||||||
| 				w.channel <- item | 				logrus.Debugf("queue: assigned task: %v with deps %v", task.ID, task.Dependencies) | ||||||
|  | 				w.channel <- task | ||||||
| 				break loop | 				break loop | ||||||
| 			} | 			} | ||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
| } | } | ||||||
|  |  | ||||||
|  | func (q *fifo) depsInQueue(task *Task) bool { | ||||||
|  | 	var next *list.Element | ||||||
|  | 	for e := q.pending.Front(); e != nil; e = next { | ||||||
|  | 		next = e.Next() | ||||||
|  | 		possibleDep, ok := e.Value.(*Task) | ||||||
|  | 		logrus.Debugf("queue: in queue right now: %v", possibleDep.ID) | ||||||
|  | 		for _, dep := range task.Dependencies { | ||||||
|  | 			if ok && possibleDep.ID == dep { | ||||||
|  | 				return true | ||||||
|  | 			} | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	for possibleDepID := range q.running { | ||||||
|  | 		for _, dep := range task.Dependencies { | ||||||
|  | 			if possibleDepID == dep { | ||||||
|  | 				return true | ||||||
|  | 			} | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	return false | ||||||
|  | } | ||||||
|   | |||||||
| @@ -117,3 +117,30 @@ func TestFifoEvict(t *testing.T) { | |||||||
| 		t.Errorf("expect not found error when evicting item not in queue, got %s", err) | 		t.Errorf("expect not found error when evicting item not in queue, got %s", err) | ||||||
| 	} | 	} | ||||||
| } | } | ||||||
|  |  | ||||||
|  | func TestFifoDependencies(t *testing.T) { | ||||||
|  | 	task1 := &Task{ | ||||||
|  | 		ID: "1", | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	task2 := &Task{ | ||||||
|  | 		ID:           "2", | ||||||
|  | 		Dependencies: []string{"1"}, | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	q := New().(*fifo) | ||||||
|  | 	q.Push(noContext, task2) | ||||||
|  | 	q.Push(noContext, task1) | ||||||
|  |  | ||||||
|  | 	got, _ := q.Poll(noContext, func(*Task) bool { return true }) | ||||||
|  | 	if got != task1 { | ||||||
|  | 		t.Errorf("expect task1 returned from queue as task2 depends on it") | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	got, _ = q.Poll(noContext, func(*Task) bool { return true }) | ||||||
|  | 	if got != task2 { | ||||||
|  | 		t.Errorf("expect task2 returned from queue") | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  | } | ||||||
|   | |||||||
| @@ -23,6 +23,9 @@ type Task struct { | |||||||
|  |  | ||||||
| 	// Labels represents the key-value pairs the entry is lebeled with. | 	// Labels represents the key-value pairs the entry is lebeled with. | ||||||
| 	Labels map[string]string `json:"labels,omitempty"` | 	Labels map[string]string `json:"labels,omitempty"` | ||||||
|  |  | ||||||
|  | 	// Task IDs this task depend on | ||||||
|  | 	Dependencies []string | ||||||
| } | } | ||||||
|  |  | ||||||
| // InfoT provides runtime information. | // InfoT provides runtime information. | ||||||
| @@ -44,9 +47,12 @@ type Filter func(*Task) bool | |||||||
| // Queue defines a task queue for scheduling tasks among | // Queue defines a task queue for scheduling tasks among | ||||||
| // a pool of workers. | // a pool of workers. | ||||||
| type Queue interface { | type Queue interface { | ||||||
| 	// Push pushes an task to the tail of this queue. | 	// Push pushes a task to the tail of this queue. | ||||||
| 	Push(c context.Context, task *Task) error | 	Push(c context.Context, task *Task) error | ||||||
|  |  | ||||||
|  | 	// Push pushes a task to the tail of this queue. | ||||||
|  | 	PushAtOnce(c context.Context, tasks []*Task) error | ||||||
|  |  | ||||||
| 	// Poll retrieves and removes a task head of this queue. | 	// Poll retrieves and removes a task head of this queue. | ||||||
| 	Poll(c context.Context, f Filter) (*Task, error) | 	Poll(c context.Context, f Filter) (*Task, error) | ||||||
|  |  | ||||||
|   | |||||||
| @@ -54,7 +54,7 @@ type persistentQueue struct { | |||||||
| 	store TaskStore | 	store TaskStore | ||||||
| } | } | ||||||
|  |  | ||||||
| // Push pushes an task to the tail of this queue. | // Push pushes a task to the tail of this queue. | ||||||
| func (q *persistentQueue) Push(c context.Context, task *queue.Task) error { | func (q *persistentQueue) Push(c context.Context, task *queue.Task) error { | ||||||
| 	q.store.TaskInsert(&Task{ | 	q.store.TaskInsert(&Task{ | ||||||
| 		ID:     task.ID, | 		ID:     task.ID, | ||||||
| @@ -68,6 +68,24 @@ func (q *persistentQueue) Push(c context.Context, task *queue.Task) error { | |||||||
| 	return err | 	return err | ||||||
| } | } | ||||||
|  |  | ||||||
|  | // Push pushes multiple tasks to the tail of this queue. | ||||||
|  | func (q *persistentQueue) PushAtOnce(c context.Context, tasks []*queue.Task) error { | ||||||
|  | 	for _, task := range tasks { | ||||||
|  | 		q.store.TaskInsert(&Task{ | ||||||
|  | 			ID:     task.ID, | ||||||
|  | 			Data:   task.Data, | ||||||
|  | 			Labels: task.Labels, | ||||||
|  | 		}) | ||||||
|  | 	} | ||||||
|  | 	err := q.Queue.PushAtOnce(c, tasks) | ||||||
|  | 	if err != nil { | ||||||
|  | 		for _, task := range tasks { | ||||||
|  | 			q.store.TaskDelete(task.ID) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	return err | ||||||
|  | } | ||||||
|  |  | ||||||
| // Poll retrieves and removes a task head of this queue. | // Poll retrieves and removes a task head of this queue. | ||||||
| func (q *persistentQueue) Poll(c context.Context, f queue.Filter) (*queue.Task, error) { | func (q *persistentQueue) Poll(c context.Context, f queue.Filter) (*queue.Task, error) { | ||||||
| 	task, err := q.Queue.Poll(c, f) | 	task, err := q.Queue.Poll(c, f) | ||||||
|   | |||||||
| @@ -315,9 +315,9 @@ func PostApproval(c *gin.Context) { | |||||||
| 		} | 		} | ||||||
| 	}() | 	}() | ||||||
|  |  | ||||||
| 	var yamls []string | 	var yamls []*remote.FileMeta | ||||||
| 	for _, y := range configs { | 	for _, y := range configs { | ||||||
| 		yamls = append(yamls, string(y.Data)) | 		yamls = append(yamls, &remote.FileMeta{Data: []byte(y.Data), Name: y.Name}) | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| 	b := procBuilder{ | 	b := procBuilder{ | ||||||
| @@ -478,6 +478,13 @@ func PostBuild(c *gin.Context) { | |||||||
| 		return | 		return | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
|  | 	err = persistBuildConfigs(configs, build.ID) | ||||||
|  | 	if err != nil { | ||||||
|  | 		logrus.Errorf("failure to persist build config for %s. %s", repo.FullName, err) | ||||||
|  | 		c.AbortWithError(500, err) | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  |  | ||||||
| 	// Read query string parameters into buildParams, exclude reserved params | 	// Read query string parameters into buildParams, exclude reserved params | ||||||
| 	var buildParams = map[string]string{} | 	var buildParams = map[string]string{} | ||||||
| 	for key, val := range c.Request.URL.Query() { | 	for key, val := range c.Request.URL.Query() { | ||||||
| @@ -508,9 +515,9 @@ func PostBuild(c *gin.Context) { | |||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| 	var yamls []string | 	var yamls []*remote.FileMeta | ||||||
| 	for _, y := range configs { | 	for _, y := range configs { | ||||||
| 		yamls = append(yamls, string(y.Data)) | 		yamls = append(yamls, &remote.FileMeta{Data: []byte(y.Data), Name: y.Name}) | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| 	b := procBuilder{ | 	b := procBuilder{ | ||||||
| @@ -589,6 +596,20 @@ func DeleteBuildLogs(c *gin.Context) { | |||||||
| 	c.String(204, "") | 	c.String(204, "") | ||||||
| } | } | ||||||
|  |  | ||||||
|  | func persistBuildConfigs(configs []*model.Config, buildID int64) error { | ||||||
|  | 	for _, conf := range configs { | ||||||
|  | 		buildConfig := &model.BuildConfig{ | ||||||
|  | 			ConfigID: conf.ID, | ||||||
|  | 			BuildID:  buildID, | ||||||
|  | 		} | ||||||
|  | 		err := Config.Storage.Config.BuildConfigCreate(buildConfig) | ||||||
|  | 		if err != nil { | ||||||
|  | 			return err | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  |  | ||||||
| var deleteStr = `[ | var deleteStr = `[ | ||||||
| 	{ | 	{ | ||||||
| 	  "proc": %q, | 	  "proc": %q, | ||||||
|   | |||||||
| @@ -180,8 +180,7 @@ func PostHook(c *gin.Context) { | |||||||
|  |  | ||||||
| 	// persist the build config for historical correctness, restarts, etc | 	// persist the build config for historical correctness, restarts, etc | ||||||
| 	for _, remoteYamlConfig := range remoteYamlConfigs { | 	for _, remoteYamlConfig := range remoteYamlConfigs { | ||||||
| 		conf, err := findOrPersistPipelineConfig(build, remoteYamlConfig.Data) | 		_, err := findOrPersistPipelineConfig(build, remoteYamlConfig) | ||||||
| 		fmt.Println(conf) |  | ||||||
| 		if err != nil { | 		if err != nil { | ||||||
| 			logrus.Errorf("failure to find or persist build config for %s. %s", repo.FullName, err) | 			logrus.Errorf("failure to find or persist build config for %s. %s", repo.FullName, err) | ||||||
| 			c.AbortWithError(500, err) | 			c.AbortWithError(500, err) | ||||||
| @@ -230,11 +229,6 @@ func PostHook(c *gin.Context) { | |||||||
| 		} | 		} | ||||||
| 	}() | 	}() | ||||||
|  |  | ||||||
| 	var yamls []string |  | ||||||
| 	for _, y := range remoteYamlConfigs { |  | ||||||
| 		yamls = append(yamls, string(y.Data)) |  | ||||||
| 	} |  | ||||||
|  |  | ||||||
| 	b := procBuilder{ | 	b := procBuilder{ | ||||||
| 		Repo:  repo, | 		Repo:  repo, | ||||||
| 		Curr:  build, | 		Curr:  build, | ||||||
| @@ -244,7 +238,7 @@ func PostHook(c *gin.Context) { | |||||||
| 		Regs:  regs, | 		Regs:  regs, | ||||||
| 		Envs:  envs, | 		Envs:  envs, | ||||||
| 		Link:  httputil.GetURL(c.Request), | 		Link:  httputil.GetURL(c.Request), | ||||||
| 		Yamls: yamls, | 		Yamls: remoteYamlConfigs, | ||||||
| 	} | 	} | ||||||
| 	buildItems, err := b.Build() | 	buildItems, err := b.Build() | ||||||
| 	if err != nil { | 	if err != nil { | ||||||
| @@ -265,14 +259,15 @@ func PostHook(c *gin.Context) { | |||||||
| 	queueBuild(build, repo, buildItems) | 	queueBuild(build, repo, buildItems) | ||||||
| } | } | ||||||
|  |  | ||||||
| func findOrPersistPipelineConfig(build *model.Build, remoteYamlConfig []byte) (*model.Config, error) { | func findOrPersistPipelineConfig(build *model.Build, remoteYamlConfig *remote.FileMeta) (*model.Config, error) { | ||||||
| 	sha := shasum(remoteYamlConfig) | 	sha := shasum(remoteYamlConfig.Data) | ||||||
| 	conf, err := Config.Storage.Config.ConfigFindIdentical(build.RepoID, sha) | 	conf, err := Config.Storage.Config.ConfigFindIdentical(build.RepoID, sha) | ||||||
| 	if err != nil { | 	if err != nil { | ||||||
| 		conf = &model.Config{ | 		conf = &model.Config{ | ||||||
| 			RepoID: build.RepoID, | 			RepoID: build.RepoID, | ||||||
| 			Data:   string(remoteYamlConfig), | 			Data:   string(remoteYamlConfig.Data), | ||||||
| 			Hash:   sha, | 			Hash:   sha, | ||||||
|  | 			Name:   sanitizePath(remoteYamlConfig.Name), | ||||||
| 		} | 		} | ||||||
| 		err = Config.Storage.Config.ConfigCreate(conf) | 		err = Config.Storage.Config.ConfigCreate(conf) | ||||||
| 		if err != nil { | 		if err != nil { | ||||||
| @@ -296,6 +291,7 @@ func findOrPersistPipelineConfig(build *model.Build, remoteYamlConfig []byte) (* | |||||||
| 	return conf, nil | 	return conf, nil | ||||||
| } | } | ||||||
|  |  | ||||||
|  | // publishes message to UI clients | ||||||
| func publishToTopic(c *gin.Context, build *model.Build, repo *model.Repo) { | func publishToTopic(c *gin.Context, build *model.Build, repo *model.Repo) { | ||||||
| 	message := pubsub.Message{ | 	message := pubsub.Message{ | ||||||
| 		Labels: map[string]string{ | 		Labels: map[string]string{ | ||||||
| @@ -314,6 +310,7 @@ func publishToTopic(c *gin.Context, build *model.Build, repo *model.Repo) { | |||||||
| } | } | ||||||
|  |  | ||||||
| func queueBuild(build *model.Build, repo *model.Repo, buildItems []*buildItem) { | func queueBuild(build *model.Build, repo *model.Repo, buildItems []*buildItem) { | ||||||
|  | 	var tasks []*queue.Task | ||||||
| 	for _, item := range buildItems { | 	for _, item := range buildItems { | ||||||
| 		task := new(queue.Task) | 		task := new(queue.Task) | ||||||
| 		task.ID = fmt.Sprint(item.Proc.ID) | 		task.ID = fmt.Sprint(item.Proc.ID) | ||||||
| @@ -323,6 +320,7 @@ func queueBuild(build *model.Build, repo *model.Repo, buildItems []*buildItem) { | |||||||
| 		} | 		} | ||||||
| 		task.Labels["platform"] = item.Platform | 		task.Labels["platform"] = item.Platform | ||||||
| 		task.Labels["repo"] = repo.FullName | 		task.Labels["repo"] = repo.FullName | ||||||
|  | 		task.Dependencies = taskIds(item.DependsOn, buildItems) | ||||||
|  |  | ||||||
| 		task.Data, _ = json.Marshal(rpc.Pipeline{ | 		task.Data, _ = json.Marshal(rpc.Pipeline{ | ||||||
| 			ID:      fmt.Sprint(item.Proc.ID), | 			ID:      fmt.Sprint(item.Proc.ID), | ||||||
| @@ -331,8 +329,21 @@ func queueBuild(build *model.Build, repo *model.Repo, buildItems []*buildItem) { | |||||||
| 		}) | 		}) | ||||||
|  |  | ||||||
| 		Config.Services.Logs.Open(context.Background(), task.ID) | 		Config.Services.Logs.Open(context.Background(), task.ID) | ||||||
| 		Config.Services.Queue.Push(context.Background(), task) | 		tasks = append(tasks, task) | ||||||
| 	} | 	} | ||||||
|  | 	Config.Services.Queue.PushAtOnce(context.Background(), tasks) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func taskIds(dependsOn []string, buildItems []*buildItem) []string { | ||||||
|  | 	taskIds := []string{} | ||||||
|  | 	for _, dep := range dependsOn { | ||||||
|  | 		for _, buildItem := range buildItems { | ||||||
|  | 			if buildItem.Proc.Name == dep { | ||||||
|  | 				taskIds = append(taskIds, fmt.Sprint(buildItem.Proc.ID)) | ||||||
|  | 			} | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	return taskIds | ||||||
| } | } | ||||||
|  |  | ||||||
| func shasum(raw []byte) string { | func shasum(raw []byte) string { | ||||||
|   | |||||||
| @@ -28,6 +28,7 @@ import ( | |||||||
| 	"github.com/laszlocph/drone-oss-08/cncd/pipeline/pipeline/frontend/yaml/linter" | 	"github.com/laszlocph/drone-oss-08/cncd/pipeline/pipeline/frontend/yaml/linter" | ||||||
| 	"github.com/laszlocph/drone-oss-08/cncd/pipeline/pipeline/frontend/yaml/matrix" | 	"github.com/laszlocph/drone-oss-08/cncd/pipeline/pipeline/frontend/yaml/matrix" | ||||||
| 	"github.com/laszlocph/drone-oss-08/model" | 	"github.com/laszlocph/drone-oss-08/model" | ||||||
|  | 	"github.com/laszlocph/drone-oss-08/remote" | ||||||
| ) | ) | ||||||
|  |  | ||||||
| // Takes the hook data and the yaml and returns in internal data model | // Takes the hook data and the yaml and returns in internal data model | ||||||
| @@ -39,15 +40,16 @@ type procBuilder struct { | |||||||
| 	Secs  []*model.Secret | 	Secs  []*model.Secret | ||||||
| 	Regs  []*model.Registry | 	Regs  []*model.Registry | ||||||
| 	Link  string | 	Link  string | ||||||
| 	Yamls []string | 	Yamls []*remote.FileMeta | ||||||
| 	Envs  map[string]string | 	Envs  map[string]string | ||||||
| } | } | ||||||
|  |  | ||||||
| type buildItem struct { | type buildItem struct { | ||||||
| 	Proc     *model.Proc | 	Proc      *model.Proc | ||||||
| 	Platform string | 	Platform  string | ||||||
| 	Labels   map[string]string | 	Labels    map[string]string | ||||||
| 	Config   *backend.Config | 	DependsOn []string | ||||||
|  | 	Config    *backend.Config | ||||||
| } | } | ||||||
|  |  | ||||||
| func (b *procBuilder) Build() ([]*buildItem, error) { | func (b *procBuilder) Build() ([]*buildItem, error) { | ||||||
| @@ -55,7 +57,7 @@ func (b *procBuilder) Build() ([]*buildItem, error) { | |||||||
|  |  | ||||||
| 	for j, y := range b.Yamls { | 	for j, y := range b.Yamls { | ||||||
| 		// matrix axes | 		// matrix axes | ||||||
| 		axes, err := matrix.ParseString(y) | 		axes, err := matrix.ParseString(string(y.Data)) | ||||||
| 		if err != nil { | 		if err != nil { | ||||||
| 			return nil, err | 			return nil, err | ||||||
| 		} | 		} | ||||||
| @@ -70,6 +72,7 @@ func (b *procBuilder) Build() ([]*buildItem, error) { | |||||||
| 				PGID:    j + i + 1, | 				PGID:    j + i + 1, | ||||||
| 				State:   model.StatusPending, | 				State:   model.StatusPending, | ||||||
| 				Environ: axis, | 				Environ: axis, | ||||||
|  | 				Name:    sanitizePath(y.Name), | ||||||
| 			} | 			} | ||||||
| 			b.Curr.Procs = append(b.Curr.Procs, proc) | 			b.Curr.Procs = append(b.Curr.Procs, proc) | ||||||
|  |  | ||||||
| @@ -77,13 +80,13 @@ func (b *procBuilder) Build() ([]*buildItem, error) { | |||||||
| 			environ := b.environmentVariables(metadata, axis) | 			environ := b.environmentVariables(metadata, axis) | ||||||
|  |  | ||||||
| 			// substitute vars | 			// substitute vars | ||||||
| 			y, err := b.envsubst_(y, environ) | 			substituted, err := b.envsubst_(string(y.Data), environ) | ||||||
| 			if err != nil { | 			if err != nil { | ||||||
| 				return nil, err | 				return nil, err | ||||||
| 			} | 			} | ||||||
|  |  | ||||||
| 			// parse yaml pipeline | 			// parse yaml pipeline | ||||||
| 			parsed, err := yaml.ParseString(y) | 			parsed, err := yaml.ParseString(substituted) | ||||||
| 			if err != nil { | 			if err != nil { | ||||||
| 				return nil, err | 				return nil, err | ||||||
| 			} | 			} | ||||||
| @@ -101,10 +104,11 @@ func (b *procBuilder) Build() ([]*buildItem, error) { | |||||||
| 			ir := b.toInternalRepresentation(parsed, environ, metadata, proc.ID) | 			ir := b.toInternalRepresentation(parsed, environ, metadata, proc.ID) | ||||||
|  |  | ||||||
| 			item := &buildItem{ | 			item := &buildItem{ | ||||||
| 				Proc:     proc, | 				Proc:      proc, | ||||||
| 				Config:   ir, | 				Config:    ir, | ||||||
| 				Labels:   parsed.Labels, | 				Labels:    parsed.Labels, | ||||||
| 				Platform: metadata.Sys.Arch, | 				DependsOn: parsed.DependsOn, | ||||||
|  | 				Platform:  metadata.Sys.Arch, | ||||||
| 			} | 			} | ||||||
| 			if item.Labels == nil { | 			if item.Labels == nil { | ||||||
| 				item.Labels = map[string]string{} | 				item.Labels = map[string]string{} | ||||||
| @@ -289,3 +293,10 @@ func metadataFromStruct(repo *model.Repo, build, last *model.Build, proc *model. | |||||||
| 		}, | 		}, | ||||||
| 	} | 	} | ||||||
| } | } | ||||||
|  |  | ||||||
|  | func sanitizePath(path string) string { | ||||||
|  | 	path = strings.TrimSuffix(path, ".yml") | ||||||
|  | 	path = strings.TrimPrefix(path, ".drone/") | ||||||
|  | 	path = strings.TrimPrefix(path, ".") | ||||||
|  | 	return path | ||||||
|  | } | ||||||
|   | |||||||
| @@ -19,6 +19,7 @@ import ( | |||||||
| 	"testing" | 	"testing" | ||||||
|  |  | ||||||
| 	"github.com/laszlocph/drone-oss-08/model" | 	"github.com/laszlocph/drone-oss-08/model" | ||||||
|  | 	"github.com/laszlocph/drone-oss-08/remote" | ||||||
| ) | ) | ||||||
|  |  | ||||||
| func TestMultilineEnvsubst(t *testing.T) { | func TestMultilineEnvsubst(t *testing.T) { | ||||||
| @@ -33,17 +34,20 @@ bbb`, | |||||||
| 		Secs:  []*model.Secret{}, | 		Secs:  []*model.Secret{}, | ||||||
| 		Regs:  []*model.Registry{}, | 		Regs:  []*model.Registry{}, | ||||||
| 		Link:  "", | 		Link:  "", | ||||||
| 		Yamls: []string{`pipeline: | 		Yamls: []*remote.FileMeta{ | ||||||
|  | 			&remote.FileMeta{Data: []byte(` | ||||||
|  | pipeline: | ||||||
|   xxx: |   xxx: | ||||||
|     image: scratch |     image: scratch | ||||||
|     yyy: ${DRONE_COMMIT_MESSAGE} |     yyy: ${DRONE_COMMIT_MESSAGE} | ||||||
| `, `pipeline: | `)}, | ||||||
|  | 			&remote.FileMeta{Data: []byte(` | ||||||
|  | pipeline: | ||||||
|   build: |   build: | ||||||
|     image: scratch |     image: scratch | ||||||
|     yyy: ${DRONE_COMMIT_MESSAGE} |     yyy: ${DRONE_COMMIT_MESSAGE} | ||||||
| `, | `)}, | ||||||
| 		}, | 		}} | ||||||
| 	} |  | ||||||
|  |  | ||||||
| 	if buildItems, err := b.Build(); err != nil { | 	if buildItems, err := b.Build(); err != nil { | ||||||
| 		t.Fatal(err) | 		t.Fatal(err) | ||||||
| @@ -61,15 +65,19 @@ func TestMultiPipeline(t *testing.T) { | |||||||
| 		Secs:  []*model.Secret{}, | 		Secs:  []*model.Secret{}, | ||||||
| 		Regs:  []*model.Registry{}, | 		Regs:  []*model.Registry{}, | ||||||
| 		Link:  "", | 		Link:  "", | ||||||
| 		Yamls: []string{`pipeline: | 		Yamls: []*remote.FileMeta{ | ||||||
|   lint: | 			&remote.FileMeta{Data: []byte(` | ||||||
|  | pipeline: | ||||||
|  |   xxx: | ||||||
|     image: scratch |     image: scratch | ||||||
|     yyy: ${DRONE_COMMIT_MESSAGE} |     yyy: ${DRONE_COMMIT_MESSAGE} | ||||||
| `, `pipeline: | `)}, | ||||||
|   test: | 			&remote.FileMeta{Data: []byte(` | ||||||
|  | pipeline: | ||||||
|  |   build: | ||||||
|     image: scratch |     image: scratch | ||||||
|     yyy: ${DRONE_COMMIT_MESSAGE} |     yyy: ${DRONE_COMMIT_MESSAGE} | ||||||
| `, | `)}, | ||||||
| 		}, | 		}, | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| @@ -81,3 +89,38 @@ func TestMultiPipeline(t *testing.T) { | |||||||
| 		t.Fatal("Should have generated 2 buildItems") | 		t.Fatal("Should have generated 2 buildItems") | ||||||
| 	} | 	} | ||||||
| } | } | ||||||
|  |  | ||||||
|  | func TestDependsOn(t *testing.T) { | ||||||
|  | 	b := procBuilder{ | ||||||
|  | 		Repo:  &model.Repo{}, | ||||||
|  | 		Curr:  &model.Build{}, | ||||||
|  | 		Last:  &model.Build{}, | ||||||
|  | 		Netrc: &model.Netrc{}, | ||||||
|  | 		Secs:  []*model.Secret{}, | ||||||
|  | 		Regs:  []*model.Registry{}, | ||||||
|  | 		Link:  "", | ||||||
|  | 		Yamls: []*remote.FileMeta{ | ||||||
|  | 			&remote.FileMeta{Data: []byte(` | ||||||
|  | pipeline: | ||||||
|  |   deploy: | ||||||
|  |     image: scratch | ||||||
|  |  | ||||||
|  | depends_on: | ||||||
|  |   - lint | ||||||
|  |   - test | ||||||
|  |   - build | ||||||
|  | `)}, | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	buildItems, err := b.Build() | ||||||
|  | 	if err != nil { | ||||||
|  | 		t.Fatal(err) | ||||||
|  | 	} | ||||||
|  | 	if len(buildItems[0].DependsOn) != 3 { | ||||||
|  | 		t.Fatal("Should have 3 dependencies") | ||||||
|  | 	} | ||||||
|  | 	if buildItems[0].DependsOn[1] != "test" { | ||||||
|  | 		t.Fatal("Should depend on test") | ||||||
|  | 	} | ||||||
|  | } | ||||||
|   | |||||||
		Reference in New Issue
	
	Block a user