ソースを参照

Change the way --dry-run works & add --dump-plan

Signed-off-by: Vadim Markovtsev <vadim@sourced.tech>
Vadim Markovtsev 6 年 前
コミット
1d1a2b8935

+ 4 - 4
cmd/hercules/root.go

@@ -226,20 +226,20 @@ targets can be added using the --plugin system.`,
 			log.Fatalf("failed to list the commits: %v", err)
 			log.Fatalf("failed to list the commits: %v", err)
 		}
 		}
 		cmdlineFacts[hercules.ConfigPipelineCommits] = commits
 		cmdlineFacts[hercules.ConfigPipelineCommits] = commits
+		dryRun, _ := cmdlineFacts[hercules.ConfigPipelineDryRun].(bool)
 		var deployed []hercules.LeafPipelineItem
 		var deployed []hercules.LeafPipelineItem
 		for name, valPtr := range cmdlineDeployed {
 		for name, valPtr := range cmdlineDeployed {
 			if *valPtr {
 			if *valPtr {
 				item := pipeline.DeployItem(hercules.Registry.Summon(name)[0])
 				item := pipeline.DeployItem(hercules.Registry.Summon(name)[0])
-				deployed = append(deployed, item.(hercules.LeafPipelineItem))
+				if !dryRun {
+					deployed = append(deployed, item.(hercules.LeafPipelineItem))
+				}
 			}
 			}
 		}
 		}
 		err = pipeline.Initialize(cmdlineFacts)
 		err = pipeline.Initialize(cmdlineFacts)
 		if err != nil {
 		if err != nil {
 			log.Fatal(err)
 			log.Fatal(err)
 		}
 		}
-		if dryRun, _ := cmdlineFacts[hercules.ConfigPipelineDryRun].(bool); dryRun {
-			return
-		}
 		results, err := pipeline.Run(commits)
 		results, err := pipeline.Run(commits)
 		if err != nil {
 		if err != nil {
 			log.Fatalf("failed to run the pipeline: %v", err)
 			log.Fatalf("failed to run the pipeline: %v", err)

+ 6 - 3
core.go

@@ -61,13 +61,16 @@ func MetadataToCommonAnalysisResult(meta *core.Metadata) *CommonAnalysisResult {
 type Pipeline = core.Pipeline
 type Pipeline = core.Pipeline
 
 
 const (
 const (
-	// ConfigPipelineDumpPath is the name of the Pipeline configuration option (Pipeline.Initialize())
+	// ConfigPipelineDAGPath is the name of the Pipeline configuration option (Pipeline.Initialize())
 	// which enables saving the items DAG to the specified file.
 	// which enables saving the items DAG to the specified file.
-	ConfigPipelineDumpPath = core.ConfigPipelineDumpPath
+	ConfigPipelineDAGPath = core.ConfigPipelineDAGPath
+	// ConfigPipelineDumpPlan is the name of the Pipeline configuration option (Pipeline.Initialize())
+	// which outputs the execution plan to stderr.
+	ConfigPipelineDumpPlan = core.ConfigPipelineDumpPlan
 	// ConfigPipelineDryRun is the name of the Pipeline configuration option (Pipeline.Initialize())
 	// ConfigPipelineDryRun is the name of the Pipeline configuration option (Pipeline.Initialize())
 	// which disables Configure() and Initialize() invocation on each PipelineItem during the
 	// which disables Configure() and Initialize() invocation on each PipelineItem during the
 	// Pipeline initialization.
 	// Pipeline initialization.
-	// Subsequent Run() calls are going to fail. Useful with ConfigPipelineDumpPath=true.
+	// Subsequent Run() calls are going to fail. Useful with ConfigPipelineDAGPath=true.
 	ConfigPipelineDryRun = core.ConfigPipelineDryRun
 	ConfigPipelineDryRun = core.ConfigPipelineDryRun
 	// ConfigPipelineCommits is the name of the Pipeline configuration option (Pipeline.Initialize())
 	// ConfigPipelineCommits is the name of the Pipeline configuration option (Pipeline.Initialize())
 	// which allows to specify the custom commit sequence. By default, Pipeline.Commits() is used.
 	// which allows to specify the custom commit sequence. By default, Pipeline.Commits() is used.

+ 24 - 11
internal/core/forks.go

@@ -1,7 +1,9 @@
 package core
 package core
 
 
 import (
 import (
+	"fmt"
 	"log"
 	"log"
+	"os"
 	"reflect"
 	"reflect"
 	"sort"
 	"sort"
 
 
@@ -81,6 +83,11 @@ const (
 	rootBranchIndex = 1
 	rootBranchIndex = 1
 )
 )
 
 
+// planPrintFunc is used to print the execution plan in prepareRunPlan().
+var planPrintFunc = func(args... interface{}) {
+	fmt.Fprintln(os.Stderr, args...)
+}
+
 type runAction struct {
 type runAction struct {
 	Action int
 	Action int
 	Commit *object.Commit
 	Commit *object.Commit
@@ -127,7 +134,7 @@ func getMasterBranch(branches map[int][]PipelineItem) []PipelineItem {
 }
 }
 
 
 // prepareRunPlan schedules the actions for Pipeline.Run().
 // prepareRunPlan schedules the actions for Pipeline.Run().
-func prepareRunPlan(commits []*object.Commit) []runAction {
+func prepareRunPlan(commits []*object.Commit, printResult bool) []runAction {
 	hashes, dag := buildDag(commits)
 	hashes, dag := buildDag(commits)
 	leaveRootComponent(hashes, dag)
 	leaveRootComponent(hashes, dag)
 	mergedDag, mergedSeq := mergeDag(hashes, dag)
 	mergedDag, mergedSeq := mergeDag(hashes, dag)
@@ -144,17 +151,23 @@ func prepareRunPlan(commits []*object.Commit) []runAction {
 	fmt.Printf("}\n")*/
 	fmt.Printf("}\n")*/
 	plan := generatePlan(orderNodes, hashes, mergedDag, dag, mergedSeq)
 	plan := generatePlan(orderNodes, hashes, mergedDag, dag, mergedSeq)
 	plan = collectGarbage(plan)
 	plan = collectGarbage(plan)
-	/*for _, p := range plan {
-		firstItem := p.Items[0]
-		switch p.Action {
-		case runActionCommit:
-			fmt.Fprintln(os.Stderr, "C", firstItem, p.Commit.Hash.String())
-		case runActionFork:
-			fmt.Fprintln(os.Stderr, "F", p.Items)
-		case runActionMerge:
-			fmt.Fprintln(os.Stderr, "M", p.Items)
+	if printResult {
+		for _, p := range plan {
+			firstItem := p.Items[0]
+			switch p.Action {
+			case runActionCommit:
+				planPrintFunc("C", firstItem, p.Commit.Hash.String())
+			case runActionFork:
+				planPrintFunc("F", p.Items)
+			case runActionMerge:
+				planPrintFunc("M", p.Items)
+			case runActionEmerge:
+				planPrintFunc("E", p.Items)
+			case runActionDelete:
+				planPrintFunc("D", p.Items)
+			}
 		}
 		}
-	}*/
+	}
 	return plan
 	return plan
 }
 }
 
 

+ 31 - 11
internal/core/pipeline.go

@@ -223,6 +223,12 @@ type Pipeline struct {
 	// second is the total number of steps.
 	// second is the total number of steps.
 	OnProgress func(int, int)
 	OnProgress func(int, int)
 
 
+	// DryRun indicates whether the items are not executed.
+	DryRun bool
+
+	// DumpPlan indicates whether to print the execution plan to stderr.
+	DumpPlan bool
+
 	// Repository points to the analysed Git repository struct from go-git.
 	// Repository points to the analysed Git repository struct from go-git.
 	repository *git.Repository
 	repository *git.Repository
 
 
@@ -238,17 +244,20 @@ type Pipeline struct {
 }
 }
 
 
 const (
 const (
-	// ConfigPipelineDumpPath is the name of the Pipeline configuration option (Pipeline.Initialize())
+	// ConfigPipelineDAGPath is the name of the Pipeline configuration option (Pipeline.Initialize())
 	// which enables saving the items DAG to the specified file.
 	// which enables saving the items DAG to the specified file.
-	ConfigPipelineDumpPath = "Pipeline.DumpPath"
+	ConfigPipelineDAGPath = "Pipeline.DAGPath"
 	// ConfigPipelineDryRun is the name of the Pipeline configuration option (Pipeline.Initialize())
 	// ConfigPipelineDryRun is the name of the Pipeline configuration option (Pipeline.Initialize())
 	// which disables Configure() and Initialize() invocation on each PipelineItem during the
 	// which disables Configure() and Initialize() invocation on each PipelineItem during the
 	// Pipeline initialization.
 	// Pipeline initialization.
-	// Subsequent Run() calls are going to fail. Useful with ConfigPipelineDumpPath=true.
+	// Subsequent Run() calls are going to fail. Useful with ConfigPipelineDAGPath=true.
 	ConfigPipelineDryRun = "Pipeline.DryRun"
 	ConfigPipelineDryRun = "Pipeline.DryRun"
 	// ConfigPipelineCommits is the name of the Pipeline configuration option (Pipeline.Initialize())
 	// ConfigPipelineCommits is the name of the Pipeline configuration option (Pipeline.Initialize())
 	// which allows to specify the custom commit sequence. By default, Pipeline.Commits() is used.
 	// which allows to specify the custom commit sequence. By default, Pipeline.Commits() is used.
-	ConfigPipelineCommits = "commits"
+	ConfigPipelineCommits = "Pipeline.Commits"
+	// ConfigPipelineDumpPlan is the name of the Pipeline configuration option (Pipeline.Initialize())
+	// which outputs the execution plan to stderr.
+	ConfigPipelineDumpPlan = "Pipeline.DumpPlan"
 	// DependencyCommit is the name of one of the three items in `deps` supplied to PipelineItem.Consume()
 	// DependencyCommit is the name of one of the three items in `deps` supplied to PipelineItem.Consume()
 	// which always exists. It corresponds to the currently analyzed commit.
 	// which always exists. It corresponds to the currently analyzed commit.
 	DependencyCommit = "commit"
 	DependencyCommit = "commit"
@@ -588,10 +597,16 @@ func (pipeline *Pipeline) Initialize(facts map[string]interface{}) error {
 			log.Panicf("failed to list the commits: %v", err)
 			log.Panicf("failed to list the commits: %v", err)
 		}
 		}
 	}
 	}
-	dumpPath, _ := facts[ConfigPipelineDumpPath].(string)
+	dumpPath, _ := facts[ConfigPipelineDAGPath].(string)
 	pipeline.resolve(dumpPath)
 	pipeline.resolve(dumpPath)
-	if dryRun, _ := facts[ConfigPipelineDryRun].(bool); dryRun {
-		return nil
+	if dumpPlan, exists := facts[ConfigPipelineDumpPlan].(bool); exists {
+		pipeline.DumpPlan = dumpPlan
+	}
+	if dryRun, exists := facts[ConfigPipelineDryRun].(bool); exists {
+		pipeline.DryRun = dryRun
+		if dryRun {
+			return nil
+		}
 	}
 	}
 	for _, item := range pipeline.items {
 	for _, item := range pipeline.items {
 		err := item.Configure(facts)
 		err := item.Configure(facts)
@@ -620,7 +635,7 @@ func (pipeline *Pipeline) Run(commits []*object.Commit) (map[LeafPipelineItem]in
 	if onProgress == nil {
 	if onProgress == nil {
 		onProgress = func(int, int) {}
 		onProgress = func(int, int) {}
 	}
 	}
-	plan := prepareRunPlan(commits)
+	plan := prepareRunPlan(commits, pipeline.DumpPlan)
 	progressSteps := len(plan) + 2
 	progressSteps := len(plan) + 2
 	branches := map[int][]PipelineItem{}
 	branches := map[int][]PipelineItem{}
 	// we will need rootClone if there is more than one root branch
 	// we will need rootClone if there is more than one root branch
@@ -631,6 +646,9 @@ func (pipeline *Pipeline) Run(commits []*object.Commit) (map[LeafPipelineItem]in
 	commitIndex := 0
 	commitIndex := 0
 	for index, step := range plan {
 	for index, step := range plan {
 		onProgress(index+1, progressSteps)
 		onProgress(index+1, progressSteps)
+		if pipeline.DryRun {
+			continue
+		}
 		firstItem := step.Items[0]
 		firstItem := step.Items[0]
 		switch step.Action {
 		switch step.Action {
 		case runActionCommit:
 		case runActionCommit:
@@ -688,9 +706,11 @@ func (pipeline *Pipeline) Run(commits []*object.Commit) (map[LeafPipelineItem]in
 	}
 	}
 	onProgress(len(plan)+1, progressSteps)
 	onProgress(len(plan)+1, progressSteps)
 	result := map[LeafPipelineItem]interface{}{}
 	result := map[LeafPipelineItem]interface{}{}
-	for index, item := range getMasterBranch(branches) {
-		if casted, ok := item.(LeafPipelineItem); ok {
-			result[pipeline.items[index].(LeafPipelineItem)] = casted.Finalize()
+	if !pipeline.DryRun {
+		for index, item := range getMasterBranch(branches) {
+			if casted, ok := item.(LeafPipelineItem); ok {
+				result[pipeline.items[index].(LeafPipelineItem)] = casted.Finalize()
+			}
 		}
 		}
 	}
 	}
 	onProgress(progressSteps, progressSteps)
 	onProgress(progressSteps, progressSteps)

+ 75 - 3
internal/core/pipeline_test.go

@@ -1,6 +1,7 @@
 package core
 package core
 
 
 import (
 import (
+	"bytes"
 	"errors"
 	"errors"
 	"fmt"
 	"fmt"
 	"io"
 	"io"
@@ -446,6 +447,77 @@ func TestPipelineError(t *testing.T) {
 	assert.NotNil(t, err)
 	assert.NotNil(t, err)
 }
 }
 
 
+func TestPipelineDryRun(t *testing.T) {
+	pipeline := NewPipeline(test.Repository)
+	item := &testPipelineItem{}
+	item.TestError = true
+	pipeline.AddItem(item)
+	pipeline.DryRun = true
+	pipeline.Initialize(map[string]interface{}{})
+	assert.True(t, pipeline.DryRun)
+	pipeline.DryRun = false
+	pipeline.Initialize(map[string]interface{}{ConfigPipelineDryRun: true})
+	assert.True(t, pipeline.DryRun)
+	commits := make([]*object.Commit, 1)
+	commits[0], _ = test.Repository.CommitObject(plumbing.NewHash(
+		"af9ddc0db70f09f3f27b4b98e415592a7485171c"))
+	result, err := pipeline.Run(commits)
+	assert.NotNil(t, result)
+	assert.Len(t, result, 1)
+	assert.Contains(t, result, nil)
+	assert.Nil(t, err)
+}
+
+func TestPipelineDryRunFalse(t *testing.T) {
+	pipeline := NewPipeline(test.Repository)
+	item := &testPipelineItem{}
+	pipeline.AddItem(item)
+	pipeline.Initialize(map[string]interface{}{ConfigPipelineDryRun: false})
+	commits := make([]*object.Commit, 1)
+	commits[0], _ = test.Repository.CommitObject(plumbing.NewHash(
+		"af9ddc0db70f09f3f27b4b98e415592a7485171c"))
+	result, err := pipeline.Run(commits)
+	assert.NotNil(t, result)
+	assert.Len(t, result, 2)
+	assert.Contains(t, result, nil)
+	assert.Contains(t, result, item)
+	assert.Nil(t, err)
+	assert.True(t, item.DepsConsumed)
+	assert.True(t, item.CommitMatches)
+	assert.True(t, item.IndexMatches)
+	assert.Equal(t, 1, *item.MergeState)
+	assert.True(t, item.Forked)
+	assert.False(t, *item.Merged)
+}
+
+func TestPipelineDumpPlanConfigure(t *testing.T) {
+	pipeline := NewPipeline(test.Repository)
+	item := &testPipelineItem{}
+	pipeline.AddItem(item)
+	pipeline.DumpPlan = true
+	pipeline.DryRun = true
+	pipeline.Initialize(map[string]interface{}{})
+	assert.True(t, pipeline.DumpPlan)
+	pipeline.DumpPlan = false
+	pipeline.Initialize(map[string]interface{}{ConfigPipelineDumpPlan: true})
+	assert.True(t, pipeline.DumpPlan)
+	stream := &bytes.Buffer{}
+	planPrintFunc = func(args... interface{}) {
+		fmt.Fprintln(stream, args...)
+	}
+	commits := make([]*object.Commit, 1)
+	commits[0], _ = test.Repository.CommitObject(plumbing.NewHash(
+		"af9ddc0db70f09f3f27b4b98e415592a7485171c"))
+	result, err := pipeline.Run(commits)
+	assert.NotNil(t, result)
+	assert.Len(t, result, 1)
+	assert.Contains(t, result, nil)
+	assert.Nil(t, err)
+	assert.Equal(t, `E [1]
+C 1 af9ddc0db70f09f3f27b4b98e415592a7485171c
+`, stream.String())
+}
+
 func TestCommonAnalysisResultMerge(t *testing.T) {
 func TestCommonAnalysisResultMerge(t *testing.T) {
 	c1 := CommonAnalysisResult{
 	c1 := CommonAnalysisResult{
 		BeginTime: 1513620635, EndTime: 1513720635, CommitsNumber: 1, RunTime: 100,
 		BeginTime: 1513620635, EndTime: 1513720635, CommitsNumber: 1, RunTime: 100,
@@ -508,7 +580,7 @@ func TestPrepareRunPlanTiny(t *testing.T) {
 	if err != nil {
 	if err != nil {
 		t.Fatal(err)
 		t.Fatal(err)
 	}
 	}
-	plan := prepareRunPlan([]*object.Commit{rootCommit})
+	plan := prepareRunPlan([]*object.Commit{rootCommit}, false)
 	assert.Len(t, plan, 2)
 	assert.Len(t, plan, 2)
 	assert.Equal(t, runActionEmerge, plan[0].Action)
 	assert.Equal(t, runActionEmerge, plan[0].Action)
 	assert.Equal(t, rootBranchIndex, plan[0].Items[0])
 	assert.Equal(t, rootBranchIndex, plan[0].Items[0])
@@ -535,7 +607,7 @@ func TestPrepareRunPlanSmall(t *testing.T) {
 		}
 		}
 		return nil
 		return nil
 	})
 	})
-	plan := prepareRunPlan(commits)
+	plan := prepareRunPlan(commits, false)
 	/*for _, p := range plan {
 	/*for _, p := range plan {
 		if p.Commit != nil {
 		if p.Commit != nil {
 			fmt.Println(p.Action, p.Commit.Hash.String(), p.Items)
 			fmt.Println(p.Action, p.Commit.Hash.String(), p.Items)
@@ -663,7 +735,7 @@ func TestPrepareRunPlanBig(t *testing.T) {
 				}
 				}
 				return nil
 				return nil
 			})
 			})
-			plan := prepareRunPlan(commits)
+			plan := prepareRunPlan(commits, false)
 			/*for _, p := range plan {
 			/*for _, p := range plan {
 				if p.Commit != nil {
 				if p.Commit != nil {
 					fmt.Println(p.Action, p.Commit.Hash.String(), p.Items)
 					fmt.Println(p.Action, p.Commit.Hash.String(), p.Items)

+ 6 - 2
internal/core/registry.go

@@ -202,12 +202,16 @@ func (registry *PipelineItemRegistry) AddFlags(flagSet *pflag.FlagSet) (
 		iface := interface{}("")
 		iface := interface{}("")
 		ptr1 := (**string)(unsafe.Pointer(uintptr(unsafe.Pointer(&iface)) + unsafe.Sizeof(&iface)))
 		ptr1 := (**string)(unsafe.Pointer(uintptr(unsafe.Pointer(&iface)) + unsafe.Sizeof(&iface)))
 		*ptr1 = flagSet.String("dump-dag", "", "Write the pipeline DAG to a Graphviz file.")
 		*ptr1 = flagSet.String("dump-dag", "", "Write the pipeline DAG to a Graphviz file.")
-		flags[ConfigPipelineDumpPath] = iface
+		flags[ConfigPipelineDAGPath] = iface
 		iface = interface{}(true)
 		iface = interface{}(true)
 		ptr2 := (**bool)(unsafe.Pointer(uintptr(unsafe.Pointer(&iface)) + unsafe.Sizeof(&iface)))
 		ptr2 := (**bool)(unsafe.Pointer(uintptr(unsafe.Pointer(&iface)) + unsafe.Sizeof(&iface)))
 		*ptr2 = flagSet.Bool("dry-run", false, "Do not run any analyses - only resolve the DAG. "+
 		*ptr2 = flagSet.Bool("dry-run", false, "Do not run any analyses - only resolve the DAG. "+
-			"Useful for --dump-dag.")
+			"Useful for --dump-dag or --dump-plan.")
 		flags[ConfigPipelineDryRun] = iface
 		flags[ConfigPipelineDryRun] = iface
+		iface = interface{}(true)
+		ptr3 := (**bool)(unsafe.Pointer(uintptr(unsafe.Pointer(&iface)) + unsafe.Sizeof(&iface)))
+		*ptr3 = flagSet.Bool("dump-plan", false, "Print the pipeline execution plan to stderr.")
+		flags[ConfigPipelineDumpPlan] = iface
 	}
 	}
 	var features []string
 	var features []string
 	for f := range registry.featureFlags.Choices {
 	for f := range registry.featureFlags.Choices {

+ 4 - 2
internal/core/registry_test.go

@@ -135,16 +135,18 @@ func TestRegistryAddFlags(t *testing.T) {
 		Run:   func(cmd *cobra.Command, args []string) {},
 		Run:   func(cmd *cobra.Command, args []string) {},
 	}
 	}
 	facts, deployed := reg.AddFlags(testCmd.Flags())
 	facts, deployed := reg.AddFlags(testCmd.Flags())
-	assert.Len(t, facts, 4)
+	assert.Len(t, facts, 5)
 	assert.IsType(t, 0, facts[(&testPipelineItem{}).ListConfigurationOptions()[0].Name])
 	assert.IsType(t, 0, facts[(&testPipelineItem{}).ListConfigurationOptions()[0].Name])
 	assert.IsType(t, true, facts[(&dummyPipelineItem{}).ListConfigurationOptions()[0].Name])
 	assert.IsType(t, true, facts[(&dummyPipelineItem{}).ListConfigurationOptions()[0].Name])
 	assert.Contains(t, facts, ConfigPipelineDryRun)
 	assert.Contains(t, facts, ConfigPipelineDryRun)
-	assert.Contains(t, facts, ConfigPipelineDumpPath)
+	assert.Contains(t, facts, ConfigPipelineDAGPath)
+	assert.Contains(t, facts, ConfigPipelineDumpPlan)
 	assert.Len(t, deployed, 1)
 	assert.Len(t, deployed, 1)
 	assert.Contains(t, deployed, (&testPipelineItem{}).Name())
 	assert.Contains(t, deployed, (&testPipelineItem{}).Name())
 	assert.NotNil(t, testCmd.Flags().Lookup((&testPipelineItem{}).Flag()))
 	assert.NotNil(t, testCmd.Flags().Lookup((&testPipelineItem{}).Flag()))
 	assert.NotNil(t, testCmd.Flags().Lookup("feature"))
 	assert.NotNil(t, testCmd.Flags().Lookup("feature"))
 	assert.NotNil(t, testCmd.Flags().Lookup("dump-dag"))
 	assert.NotNil(t, testCmd.Flags().Lookup("dump-dag"))
+	assert.NotNil(t, testCmd.Flags().Lookup("dump-plan"))
 	assert.NotNil(t, testCmd.Flags().Lookup("dry-run"))
 	assert.NotNil(t, testCmd.Flags().Lookup("dry-run"))
 	assert.NotNil(t, testCmd.Flags().Lookup(
 	assert.NotNil(t, testCmd.Flags().Lookup(
 		(&testPipelineItem{}).ListConfigurationOptions()[0].Flag))
 		(&testPipelineItem{}).ListConfigurationOptions()[0].Flag))

+ 4 - 4
internal/global_test.go

@@ -18,11 +18,11 @@ func TestPipelineSerialize(t *testing.T) {
 	pipeline.SetFeature(uast_items.FeatureUast)
 	pipeline.SetFeature(uast_items.FeatureUast)
 	pipeline.DeployItem(&leaves.BurndownAnalysis{})
 	pipeline.DeployItem(&leaves.BurndownAnalysis{})
 	facts := map[string]interface{}{}
 	facts := map[string]interface{}{}
-	facts["Pipeline.DryRun"] = true
+	facts[core.ConfigPipelineDryRun] = true
 	tmpdir, _ := ioutil.TempDir("", "hercules-")
 	tmpdir, _ := ioutil.TempDir("", "hercules-")
 	defer os.RemoveAll(tmpdir)
 	defer os.RemoveAll(tmpdir)
 	dotpath := path.Join(tmpdir, "graph.dot")
 	dotpath := path.Join(tmpdir, "graph.dot")
-	facts["Pipeline.DumpPath"] = dotpath
+	facts[core.ConfigPipelineDAGPath] = dotpath
 	pipeline.Initialize(facts)
 	pipeline.Initialize(facts)
 	bdot, _ := ioutil.ReadFile(dotpath)
 	bdot, _ := ioutil.ReadFile(dotpath)
 	dot := string(bdot)
 	dot := string(bdot)
@@ -58,11 +58,11 @@ func TestPipelineSerializeNoUast(t *testing.T) {
 	// pipeline.SetFeature(FeatureUast)
 	// pipeline.SetFeature(FeatureUast)
 	pipeline.DeployItem(&leaves.BurndownAnalysis{})
 	pipeline.DeployItem(&leaves.BurndownAnalysis{})
 	facts := map[string]interface{}{}
 	facts := map[string]interface{}{}
-	facts["Pipeline.DryRun"] = true
+	facts[core.ConfigPipelineDryRun] = true
 	tmpdir, _ := ioutil.TempDir("", "hercules-")
 	tmpdir, _ := ioutil.TempDir("", "hercules-")
 	defer os.RemoveAll(tmpdir)
 	defer os.RemoveAll(tmpdir)
 	dotpath := path.Join(tmpdir, "graph.dot")
 	dotpath := path.Join(tmpdir, "graph.dot")
-	facts["Pipeline.DumpPath"] = dotpath
+	facts[core.ConfigPipelineDAGPath] = dotpath
 	pipeline.Initialize(facts)
 	pipeline.Initialize(facts)
 	bdot, _ := ioutil.ReadFile(dotpath)
 	bdot, _ := ioutil.ReadFile(dotpath)
 	dot := string(bdot)
 	dot := string(bdot)

+ 1 - 1
internal/plumbing/identity/identity_test.go

@@ -104,7 +104,7 @@ Vadim|vadim@sourced.tech`)
 		}
 		}
 		commits = append(commits, commit)
 		commits = append(commits, commit)
 	}
 	}
-	facts["commits"] = commits
+	facts[core.ConfigPipelineCommits] = commits
 	id = fixtureIdentityDetector()
 	id = fixtureIdentityDetector()
 	id.PeopleDict = nil
 	id.PeopleDict = nil
 	id.ReversedPeopleDict = nil
 	id.ReversedPeopleDict = nil