浏览代码

Merge pull request #86 from vmarkovtsev/master

Survive git/git
Vadim Markovtsev 6 年之前
父节点
当前提交
30e1d40436

+ 2 - 1
cmd/hercules/root.go

@@ -183,6 +183,7 @@ targets can be added using the --plugin system.`,
 		if commitsFile == "" {
 			// list of commits belonging to the default branch, from oldest to newest
 			// rev-list --first-parent
+			fmt.Fprint(os.Stderr, "git log...\r")
 			commits = pipeline.Commits()
 		} else {
 			var err error
@@ -192,7 +193,7 @@ targets can be added using the --plugin system.`,
 			}
 		}
 		cmdlineFacts["commits"] = commits
-		deployed := []hercules.LeafPipelineItem{}
+		var deployed []hercules.LeafPipelineItem
 		for name, valPtr := range cmdlineDeployed {
 			if *valPtr {
 				item := pipeline.DeployItem(hercules.Registry.Summon(name)[0])

+ 3 - 0
internal/burndown/file.go

@@ -240,6 +240,9 @@ func (file *File) Update(time int, pos int, insLength int, delLength int) {
 func (file *File) Merge(day int, others... *File) bool {
 	dirty := false
 	for _, other := range others {
+		if other == nil {
+			panic("merging File with nil")
+		}
 		if file.Hash != other.Hash {
 			dirty = true
 			break

+ 7 - 0
internal/burndown/file_test.go

@@ -642,4 +642,11 @@ func TestFileMergeNoop(t *testing.T) {
 	dirty = file1.Merge(7, file2)
 	// because the hashes are still the same
 	assert.False(t, dirty)
+}
+
+func TestFileMergeNil(t *testing.T) {
+	file, _ := fixtureFile()
+	assert.Panics(t, func() {
+		file.Merge(1, nil)
+	})
 }

+ 45 - 46
internal/core/forks.go

@@ -8,7 +8,7 @@ import (
 	"gopkg.in/src-d/go-git.v4/plumbing/object"
 	"gopkg.in/src-d/go-git.v4/plumbing"
 	"gopkg.in/src-d/hercules.v4/internal/toposort"
-		)
+)
 
 // OneShotMergeProcessor provides the convenience method to consume merges only once.
 type OneShotMergeProcessor struct {
@@ -77,6 +77,8 @@ const (
 	runActionFork = iota
 	// runActionMerge merges several branches together
 	runActionMerge = iota
+	// runActionEmerge starts a root branch
+	runActionEmerge = iota
 	// runActionDelete removes the branch as it is no longer needed
 	runActionDelete = iota
 )
@@ -130,7 +132,6 @@ func getMasterBranch(branches map[int][]PipelineItem) []PipelineItem {
 func prepareRunPlan(commits []*object.Commit) []runAction {
 	hashes, dag := buildDag(commits)
 	leaveRootComponent(hashes, dag)
-	numParents := bindNumParents(hashes, dag)
 	mergedDag, mergedSeq := mergeDag(hashes, dag)
 	orderNodes := bindOrderNodes(mergedDag)
 	collapseFastForwards(orderNodes, hashes, mergedDag, dag, mergedSeq)
@@ -143,7 +144,7 @@ func prepareRunPlan(commits []*object.Commit) []runAction {
 		}
 	}
 	fmt.Printf("}\n")*/
-	plan := generatePlan(orderNodes, numParents, hashes, mergedDag, dag, mergedSeq)
+	plan := generatePlan(orderNodes, hashes, mergedDag, dag, mergedSeq)
 	plan = optimizePlan(plan)
 	/*for _, p := range plan {
 		firstItem := p.Items[0]
@@ -186,26 +187,6 @@ func buildDag(commits []*object.Commit) (
 	return hashes, dag
 }
 
-// bindNumParents returns curried "numParents" function.
-func bindNumParents(
-	hashes map[string]*object.Commit,
-	dag map[plumbing.Hash][]*object.Commit) func(c *object.Commit) int {
-	return func(c *object.Commit) int {
-		r := 0
-		for _, parent := range c.ParentHashes {
-			if p, exists := hashes[parent.String()]; exists {
-				for _, pc := range dag[p.Hash] {
-					if pc.Hash == c.Hash {
-						r++
-						break
-					}
-				}
-			}
-		}
-		return r
-	}
-}
-
 // leaveRootComponent runs connected components analysis and throws away everything
 // but the part which grows from the root.
 func leaveRootComponent(
@@ -306,18 +287,24 @@ func bindOrderNodes(mergedDag map[plumbing.Hash][]*object.Commit) orderer {
 	}
 }
 
-// mergeDag turns sequences of consecutive commits into single nodes.
-func mergeDag(
-	hashes map[string]*object.Commit,
-	dag map[plumbing.Hash][]*object.Commit) (
-	mergedDag, mergedSeq map[plumbing.Hash][]*object.Commit) {
-
+// inverts `dag`
+func buildParents(dag map[plumbing.Hash][]*object.Commit) map[plumbing.Hash][]plumbing.Hash {
 	parents := map[plumbing.Hash][]plumbing.Hash{}
 	for key, vals := range dag {
 		for _, val := range vals {
 			parents[val.Hash] = append(parents[val.Hash], key)
 		}
 	}
+	return parents
+}
+
+// mergeDag turns sequences of consecutive commits into single nodes.
+func mergeDag(
+	hashes map[string]*object.Commit,
+	dag map[plumbing.Hash][]*object.Commit) (
+	mergedDag, mergedSeq map[plumbing.Hash][]*object.Commit) {
+
+	parents := buildParents(dag)
 	mergedDag = map[plumbing.Hash][]*object.Commit{}
 	mergedSeq = map[plumbing.Hash][]*object.Commit{}
 	visited := map[plumbing.Hash]bool{}
@@ -357,12 +344,7 @@ func collapseFastForwards(
 	orderNodes orderer, hashes map[string]*object.Commit,
 	mergedDag, dag, mergedSeq map[plumbing.Hash][]*object.Commit)  {
 
-	parents := map[plumbing.Hash][]plumbing.Hash{}
-	for key, vals := range mergedDag {
-		for _, val := range vals {
-			parents[val.Hash] = append(parents[val.Hash], key)
-		}
-	}
+	parents := buildParents(mergedDag)
 	processed := map[plumbing.Hash]bool{}
 	for _, strkey := range orderNodes(false, true) {
 		key := hashes[strkey].Hash
@@ -473,18 +455,24 @@ func collapseFastForwards(
 
 // generatePlan creates the list of actions from the commit DAG.
 func generatePlan(
-	orderNodes orderer, numParents func(c *object.Commit) int,
-	hashes map[string]*object.Commit,
+	orderNodes orderer, hashes map[string]*object.Commit,
 	mergedDag, dag, mergedSeq map[plumbing.Hash][]*object.Commit) []runAction {
 
+	parents := buildParents(dag)
 	var plan []runAction
 	branches := map[plumbing.Hash]int{}
 	branchers := map[plumbing.Hash]map[plumbing.Hash]int{}
-	counter := 1
-	for seqIndex, name := range orderNodes(false, true) {
+	counter := 0
+	for _, name := range orderNodes(false, true) {
 		commit := hashes[name]
-		if seqIndex == 0 {
-			branches[commit.Hash] = 0
+		if len(parents[commit.Hash]) == 0 {
+			branches[commit.Hash] = counter
+			plan = append(plan, runAction{
+				Action: runActionEmerge,
+				Commit: commit,
+				Items: []int{counter},
+			})
+			counter++
 		}
 		var branch int
 		{
@@ -504,16 +492,13 @@ func generatePlan(
 
 		}
 		appendMergeIfNeeded := func() {
-			if numParents(commit) < 2 {
+			if len(parents[commit.Hash]) < 2 {
 				return
 			}
 			// merge after the merge commit (the first in the sequence)
 			var items []int
 			minBranch := 1 << 31
-			for _, parent := range commit.ParentHashes {
-				if _, exists := hashes[parent.String()]; !exists {
-					continue
-				}
+			for _, parent := range parents[commit.Hash] {
 				parentBranch := -1
 				if parents, exists := branchers[commit.Hash]; exists {
 					if inheritedBranch, exists := parents[parent]; exists {
@@ -522,6 +507,10 @@ func generatePlan(
 				}
 				if parentBranch == -1 {
 					parentBranch = branches[parent]
+					if parentBranch == -1 {
+						log.Panicf("parent %s > %s does not have a branch assigned",
+							parent.String(), commit.Hash.String())
+					}
 				}
 				if len(dag[parent]) == 1 && minBranch > parentBranch {
 					minBranch = parentBranch
@@ -531,6 +520,7 @@ func generatePlan(
 					appendCommit(commit, parentBranch)
 				}
 			}
+			// there should be no duplicates in items
 			if minBranch < 1 << 31 {
 				branch = minBranch
 				branches[commit.Hash] = minBranch
@@ -606,12 +596,19 @@ func optimizePlan(plan []runAction) []runAction {
 		case runActionCommit:
 			lives[firstItem]++
 			lastMentioned[firstItem] = i
+			if firstItem == -1 {
+				log.Panicf("commit %s does not have an assigned branch",
+					p.Commit.Hash.String())
+			}
 		case runActionFork:
 			lastMentioned[firstItem] = i
 		case runActionMerge:
 			for _, item := range p.Items {
 				lastMentioned[item] = i
 			}
+		case runActionEmerge:
+			lives[firstItem]++
+			lastMentioned[firstItem] = i
 		}
 	}
 	branchesToDelete := map[int]bool{}
@@ -673,6 +670,8 @@ func optimizePlan(plan []runAction) []runAction {
 						Items:  newBranches,
 					})
 				}
+			case runActionEmerge:
+				optimizedPlan = append(optimizedPlan, p)
 			}
 		}
 		if pair[1] >= 0 {

+ 16 - 6
internal/core/pipeline.go

@@ -553,9 +553,12 @@ func (pipeline *Pipeline) Run(commits []*object.Commit) (map[LeafPipelineItem]in
 	}
 	plan := prepareRunPlan(commits)
 	progressSteps := len(plan) + 2
-	branches := map[int][]PipelineItem{0: pipeline.items}
+	branches := map[int][]PipelineItem{}
+	// we will need rootClone if there is more than one root branch
+	rootClone := cloneItems(pipeline.items, 1)[0]
 	var newestTime int64
 
+	commitIndex := 0
 	for index, step := range plan {
 		onProgress(index + 1, progressSteps)
 		firstItem := step.Items[0]
@@ -563,13 +566,13 @@ func (pipeline *Pipeline) Run(commits []*object.Commit) (map[LeafPipelineItem]in
 		case runActionCommit:
 			state := map[string]interface{}{
 				DependencyCommit: step.Commit,
-				DependencyIndex: index,
+				DependencyIndex: commitIndex,
 			}
 			for _, item := range branches[firstItem] {
 				update, err := item.Consume(state)
 				if err != nil {
-					log.Printf("%s failed on commit #%d %s\n",
-						item.Name(), index + 1, step.Commit.Hash.String())
+					log.Printf("%s failed on commit #%d (%d) %s\n",
+						item.Name(), commitIndex + 1, index + 1, step.Commit.Hash.String())
 					return nil, err
 				}
 				for _, key := range item.Provides() {
@@ -580,10 +583,11 @@ func (pipeline *Pipeline) Run(commits []*object.Commit) (map[LeafPipelineItem]in
 					state[key] = val
 				}
 			}
-			commitTime := step.Commit.Author.When.Unix()
+			commitTime := step.Commit.Committer.When.Unix()
 			if commitTime > newestTime {
 				newestTime = commitTime
 			}
+			commitIndex++
 		case runActionFork:
 			for i, clone := range cloneItems(branches[firstItem], len(step.Items)-1) {
 				branches[step.Items[i+1]] = clone
@@ -594,6 +598,12 @@ func (pipeline *Pipeline) Run(commits []*object.Commit) (map[LeafPipelineItem]in
 				merged[i] = branches[b]
 			}
 			mergeItems(merged)
+		case runActionEmerge:
+			if firstItem == 0 {
+				branches[firstItem] = pipeline.items
+			} else {
+				branches[firstItem] = cloneItems(rootClone, 1)[0]
+			}
 		case runActionDelete:
 			delete(branches, firstItem)
 		}
@@ -607,7 +617,7 @@ func (pipeline *Pipeline) Run(commits []*object.Commit) (map[LeafPipelineItem]in
 	}
 	onProgress(progressSteps, progressSteps)
 	result[nil] = &CommonAnalysisResult{
-		BeginTime:     plan[0].Commit.Author.When.Unix(),
+		BeginTime:     plan[0].Commit.Committer.When.Unix(),
 		EndTime:       newestTime,
 		CommitsNumber: len(commits),
 		RunTime:       time.Since(startRunTime),

+ 30 - 17
internal/core/pipeline_test.go

@@ -156,7 +156,7 @@ func (item *dependingTestPipelineItem) Consume(deps map[string]interface{}) (map
 }
 
 func (item *dependingTestPipelineItem) Fork(n int) []PipelineItem {
-	return nil
+	return make([]PipelineItem, n)
 }
 
 func (item *dependingTestPipelineItem) Merge(branches []PipelineItem) {
@@ -210,14 +210,14 @@ func TestPipelineRun(t *testing.T) {
 	assert.Equal(t, 2, len(result))
 	assert.Equal(t, item, result[item].(*testPipelineItem))
 	common := result[nil].(*CommonAnalysisResult)
-	assert.Equal(t, common.BeginTime, int64(1481719092))
-	assert.Equal(t, common.EndTime, int64(1481719092))
+	assert.Equal(t, common.BeginTime, int64(1481719198))
+	assert.Equal(t, common.EndTime, int64(1481719198))
 	assert.Equal(t, common.CommitsNumber, 1)
 	assert.True(t, common.RunTime.Nanoseconds()/1e6 < 100)
 	assert.True(t, item.DepsConsumed)
 	assert.True(t, item.CommitMatches)
 	assert.True(t, item.IndexMatches)
-	assert.False(t, item.Forked)
+	assert.True(t, item.Forked)
 	assert.False(t, *item.Merged)
 	pipeline.RemoveItem(item)
 	result, err = pipeline.Run(commits)
@@ -261,13 +261,16 @@ func TestPipelineOnProgress(t *testing.T) {
 	progressOk := 0
 
 	onProgress := func(step int, total int) {
-		if step == 1 && total == 3 {
+		if step == 1 && total == 4 {
+			progressOk++
+		}
+		if step == 2 && total == 4 {
 			progressOk++
 		}
-		if step == 2 && total == 3 {
+		if step == 3 && total == 4 {
 			progressOk++
 		}
-		if step == 3 && total == 3 {
+		if step == 4 && total == 4 {
 			progressOk++
 		}
 	}
@@ -279,7 +282,7 @@ func TestPipelineOnProgress(t *testing.T) {
 	result, err := pipeline.Run(commits)
 	assert.Nil(t, err)
 	assert.Equal(t, 1, len(result))
-	assert.Equal(t, 3, progressOk)
+	assert.Equal(t, 4, progressOk)
 }
 
 func TestPipelineCommits(t *testing.T) {
@@ -427,10 +430,13 @@ func TestPrepareRunPlanTiny(t *testing.T) {
 		t.Fatal(err)
 	}
 	plan := prepareRunPlan([]*object.Commit{rootCommit})
-	assert.Len(t, plan, 1)
-	assert.Equal(t, runActionCommit, plan[0].Action)
+	assert.Len(t, plan, 2)
+	assert.Equal(t, runActionEmerge, plan[0].Action)
 	assert.Equal(t, 0, plan[0].Items[0])
 	assert.Equal(t, "cce947b98a050c6d356bc6ba95030254914027b1", plan[0].Commit.Hash.String())
+	assert.Equal(t, runActionCommit, plan[1].Action)
+	assert.Equal(t, 0, plan[1].Items[0])
+	assert.Equal(t, "cce947b98a050c6d356bc6ba95030254914027b1", plan[1].Commit.Hash.String())
 }
 
 func TestPrepareRunPlanSmall(t *testing.T) {
@@ -459,16 +465,19 @@ func TestPrepareRunPlanSmall(t *testing.T) {
 		}
 	}*/
 	// fork, merge and one artificial commit per branch
-	assert.Len(t, plan, len(commits))
-	assert.Equal(t, runActionCommit, plan[0].Action)
-	assert.Equal(t, 0, plan[0].Items[0])
+	assert.Len(t, plan, len(commits) + 1)
+	assert.Equal(t, runActionEmerge, plan[0].Action)
 	assert.Equal(t, "cce947b98a050c6d356bc6ba95030254914027b1", plan[0].Commit.Hash.String())
+	assert.Equal(t, 0, plan[0].Items[0])
 	assert.Equal(t, runActionCommit, plan[1].Action)
 	assert.Equal(t, 0, plan[1].Items[0])
-	assert.Equal(t, "a3ee37f91f0d705ec9c41ae88426f0ae44b2fbc3", plan[1].Commit.Hash.String())
-	assert.Equal(t, runActionCommit, plan[9].Action)
-	assert.Equal(t, 0, plan[9].Items[0])
-	assert.Equal(t, "a28e9064c70618dc9d68e1401b889975e0680d11", plan[9].Commit.Hash.String())
+	assert.Equal(t, "cce947b98a050c6d356bc6ba95030254914027b1", plan[1].Commit.Hash.String())
+	assert.Equal(t, runActionCommit, plan[2].Action)
+	assert.Equal(t, 0, plan[2].Items[0])
+	assert.Equal(t, "a3ee37f91f0d705ec9c41ae88426f0ae44b2fbc3", plan[2].Commit.Hash.String())
+	assert.Equal(t, runActionCommit, plan[10].Action)
+	assert.Equal(t, 0, plan[10].Items[0])
+	assert.Equal(t, "a28e9064c70618dc9d68e1401b889975e0680d11", plan[10].Commit.Hash.String())
 }
 
 func TestMergeDag(t *testing.T) {
@@ -587,6 +596,7 @@ func TestPrepareRunPlanBig(t *testing.T) {
 			numForks := 0
 			numMerges := 0
 			numDeletes := 0
+			numEmerges := 0
 			processed := map[plumbing.Hash]map[int]int{}
 			for _, p := range plan {
 				switch p.Action {
@@ -614,6 +624,8 @@ func TestPrepareRunPlanBig(t *testing.T) {
 					numMerges++
 				case runActionDelete:
 					numDeletes++
+				case runActionEmerge:
+					numEmerges++
 				}
 			}
 			for c, branches := range processed {
@@ -625,6 +637,7 @@ func TestPrepareRunPlanBig(t *testing.T) {
 			assert.Equal(t, numForks, testCase[4], fmt.Sprintf("forks %v", testCase))
 			assert.Equal(t, numMerges, testCase[5], fmt.Sprintf("merges %v", testCase))
 			assert.Equal(t, numDeletes, testCase[6], fmt.Sprintf("deletes %v", testCase))
+			assert.Equal(t, numEmerges, 1, fmt.Sprintf("emerges %v", testCase))
 		}()
 	}
 }

+ 5 - 5
internal/plumbing/day.go

@@ -13,7 +13,7 @@ import (
 // It is a PipelineItem.
 type DaysSinceStart struct {
 	core.NoopMerger
-	day0        time.Time
+	day0        *time.Time
 	previousDay int
 	commits     map[int][]plumbing.Hash
 }
@@ -63,7 +63,7 @@ func (days *DaysSinceStart) Configure(facts map[string]interface{}) {
 // Initialize resets the temporary caches and prepares this PipelineItem for a series of Consume()
 // calls. The repository which is going to be analysed is supplied as an argument.
 func (days *DaysSinceStart) Initialize(repository *git.Repository) {
-	days.day0 = time.Time{}
+	days.day0 = &time.Time{}
 	days.previousDay = 0
 	if len(days.commits) > 0 {
 		keys := make([]int, len(days.commits))
@@ -86,11 +86,11 @@ func (days *DaysSinceStart) Consume(deps map[string]interface{}) (map[string]int
 	index := deps[core.DependencyIndex].(int)
 	if index == 0 {
 		// first iteration - initialize the file objects from the tree
-		days.day0 = commit.Author.When
+		*days.day0 = commit.Committer.When
 		// our precision is 1 day
-		days.day0 = days.day0.Truncate(24 * time.Hour)
+		*days.day0 = days.day0.Truncate(24 * time.Hour)
 	}
-	day := int(commit.Author.When.Sub(days.day0).Hours() / 24)
+	day := int(commit.Committer.When.Sub(*days.day0).Hours() / 24)
 	if day < days.previousDay {
 		// rebase works miracles, but we need the monotonous time
 		day = days.previousDay

+ 17 - 3
internal/plumbing/diff.go

@@ -147,19 +147,33 @@ func CountLines(file *object.Blob) (int, error) {
 	var scanner *bufio.Scanner
 	buffer := make([]byte, bufio.MaxScanTokenSize)
 	counter := 0
+	utf8Errors := 0
 	for scanner == nil || scanner.Err() == bufio.ErrTooLong {
-		if scanner != nil && !utf8.Valid(scanner.Bytes()) {
-			return -1, errors.New("binary")
+		if scanner != nil {
+			chunk := scanner.Bytes()
+			if !utf8.Valid(chunk) {
+				utf8Errors++
+			}
+			if bytes.IndexByte(chunk, 0) >= 0 {
+				return -1, errors.New("binary")
+			}
 		}
 		scanner = bufio.NewScanner(reader)
 		scanner.Buffer(buffer, 0)
 		for scanner.Scan() {
-			if !utf8.Valid(scanner.Bytes()) {
+			chunk := scanner.Bytes()
+			if !utf8.Valid(chunk) {
+				utf8Errors++
+			}
+			if bytes.IndexByte(chunk, 0) >= 0 {
 				return -1, errors.New("binary")
 			}
 			counter++
 		}
 	}
+	if float32(utf8Errors) / float32(counter) >= 0.01 {
+		return -1, errors.New("binary")
+	}
 	return counter, nil
 }
 

+ 8 - 4
leaves/burndown.go

@@ -21,7 +21,7 @@ import (
 	items "gopkg.in/src-d/hercules.v4/internal/plumbing"
 	"gopkg.in/src-d/hercules.v4/internal/plumbing/identity"
 	"gopkg.in/src-d/hercules.v4/yaml"
-	)
+)
 
 // BurndownAnalysis allows to gather the line burndown statistics for a Git repository.
 // It is a LeafPipelineItem.
@@ -303,9 +303,13 @@ func (analyser *BurndownAnalysis) Fork(n int) []core.PipelineItem {
 // Merge combines several items together. We apply the special file merging logic here.
 func (analyser *BurndownAnalysis) Merge(branches []core.PipelineItem) {
 	for key, file := range analyser.files {
-		others := make([]*burndown.File, len(branches))
-		for i, branch := range branches {
-			others[i] = branch.(*BurndownAnalysis).files[key]
+		others := make([]*burndown.File, 0, len(branches))
+		for _, branch := range branches {
+			file := branch.(*BurndownAnalysis).files[key]
+			if file != nil {
+				// file can be nil if it is considered binary in the other branch
+				others = append(others, file)
+			}
 		}
 		// don't worry, we compare the hashes first before heavy-lifting
 		if file.Merge(analyser.day, others...) {