You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@devlake.apache.org by wa...@apache.org on 2022/06/10 03:43:01 UTC

[incubator-devlake] branch main updated (3b377822 -> d7565fc1)

This is an automated email from the ASF dual-hosted git repository.

warren pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-devlake.git


    from 3b377822 New contributors guide to backend development (#2125)
     new a59ea07a feat: refdiff support pattern
     new 2d19b39b fix: fix an error lost before directrun
     new 2c9023cc refactor: add type commitpair and pairlist
     new 2b15cefc feat: add order type semver
     new dd00165a refactor: set tagslimit args to int from string
     new d7565fc1 refactor: caculate all pairs on prepare

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 plugins/refdiff/README.md                          |  28 ++-
 plugins/refdiff/refdiff.go                         |  46 ++++-
 .../refdiff/tasks/ref_commit_diff_calculator.go    |  57 ++----
 plugins/refdiff/tasks/ref_issue_diff_calculator.go |  22 ++-
 plugins/refdiff/tasks/refdiff_task_data.go         | 199 +++++++++++++++++++++
 5 files changed, 290 insertions(+), 62 deletions(-)


[incubator-devlake] 03/06: refactor: add type commitpair and pairlist

Posted by wa...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

warren pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-devlake.git

commit 2c9023cc9f900941559b24a8ec8644f47d21f3d6
Author: Nddtfjiang <jz...@qq.com>
AuthorDate: Wed Jun 8 02:12:52 2022 +0000

    refactor: add type commitpair and pairlist
    
    Add type RefCommitPair
    Add type RefPairList
    Add type RefCommitPairs
    Add type RefPairLists
    
    Nddtfjiang <zh...@merico.dev>
---
 plugins/refdiff/tasks/ref_commit_diff_calculator.go | 12 ++++++------
 plugins/refdiff/tasks/ref_issue_diff_calculator.go  | 10 +++++-----
 plugins/refdiff/tasks/refdiff_task_data.go          |  4 ++++
 3 files changed, 15 insertions(+), 11 deletions(-)

diff --git a/plugins/refdiff/tasks/ref_commit_diff_calculator.go b/plugins/refdiff/tasks/ref_commit_diff_calculator.go
index 420ed963..606d1462 100644
--- a/plugins/refdiff/tasks/ref_commit_diff_calculator.go
+++ b/plugins/refdiff/tasks/ref_commit_diff_calculator.go
@@ -28,7 +28,7 @@ import (
 )
 
 // Calculate the commits pairs both from Options.Pairs and TagPattern
-func CalculateCommitsPairs(taskCtx core.SubTaskContext) ([][4]string, error) {
+func CalculateCommitsPairs(taskCtx core.SubTaskContext) (RefCommitPairs, error) {
 	data := taskCtx.GetData().(*RefdiffTaskData)
 	repoId := data.Options.RepoId
 	pairs := data.Options.Pairs
@@ -37,13 +37,13 @@ func CalculateCommitsPairs(taskCtx core.SubTaskContext) ([][4]string, error) {
 
 	rs, err := CaculateTagPattern(taskCtx)
 	if err != nil {
-		return [][4]string{}, err
+		return RefCommitPairs{}, err
 	}
 	if tagsLimit > rs.Len() {
 		tagsLimit = rs.Len()
 	}
 
-	commitPairs := make([][4]string, 0, tagsLimit+len(pairs))
+	commitPairs := make(RefCommitPairs, 0, tagsLimit+len(pairs))
 	for i := 1; i < tagsLimit; i++ {
 		commitPairs = append(commitPairs, [4]string{rs[i-1].CommitSha, rs[i].CommitSha, rs[i-1].Name, rs[i].Name})
 	}
@@ -67,14 +67,14 @@ func CalculateCommitsPairs(taskCtx core.SubTaskContext) ([][4]string, error) {
 		// get new ref's commit sha
 		newCommit, err := ref2sha(refPair.NewRef)
 		if err != nil {
-			return [][4]string{}, fmt.Errorf("failed to load commit sha for NewRef on pair #%d: %w", i, err)
+			return RefCommitPairs{}, fmt.Errorf("failed to load commit sha for NewRef on pair #%d: %w", i, err)
 		}
 		// get old ref's commit sha
 		oldCommit, err := ref2sha(refPair.OldRef)
 		if err != nil {
-			return [][4]string{}, fmt.Errorf("failed to load commit sha for OleRef on pair #%d: %w", i, err)
+			return RefCommitPairs{}, fmt.Errorf("failed to load commit sha for OleRef on pair #%d: %w", i, err)
 		}
-		commitPairs = append(commitPairs, [4]string{newCommit, oldCommit, refPair.NewRef, refPair.OldRef})
+		commitPairs = append(commitPairs, RefCommitPair{newCommit, oldCommit, refPair.NewRef, refPair.OldRef})
 	}
 
 	return commitPairs, nil
diff --git a/plugins/refdiff/tasks/ref_issue_diff_calculator.go b/plugins/refdiff/tasks/ref_issue_diff_calculator.go
index 5b9d92a7..aed0f4e8 100644
--- a/plugins/refdiff/tasks/ref_issue_diff_calculator.go
+++ b/plugins/refdiff/tasks/ref_issue_diff_calculator.go
@@ -27,7 +27,7 @@ import (
 )
 
 // Calculate the pair list both from Options.Pairs and TagPattern
-func CaculatePairList(taskCtx core.SubTaskContext) ([][2]string, error) {
+func CaculatePairList(taskCtx core.SubTaskContext) (RefPairLists, error) {
 	data := taskCtx.GetData().(*RefdiffTaskData)
 	repoId := data.Options.RepoId
 	pairs := data.Options.Pairs
@@ -35,19 +35,19 @@ func CaculatePairList(taskCtx core.SubTaskContext) ([][2]string, error) {
 
 	rs, err := CaculateTagPattern(taskCtx)
 	if err != nil {
-		return [][2]string{}, err
+		return RefPairLists{}, err
 	}
 	if tagsLimit > rs.Len() {
 		tagsLimit = rs.Len()
 	}
 
-	pairList := make([][2]string, 0, tagsLimit+len(pairs))
+	pairList := make(RefPairLists, 0, tagsLimit+len(pairs))
 	for i := 1; i < tagsLimit; i++ {
-		pairList = append(pairList, [2]string{fmt.Sprintf("%s:%s", repoId, rs[i-1].Id), fmt.Sprintf("%s:%s", repoId, rs[i].Id)})
+		pairList = append(pairList, RefPairList{fmt.Sprintf("%s:%s", repoId, rs[i-1].Id), fmt.Sprintf("%s:%s", repoId, rs[i].Id)})
 	}
 
 	for _, pair := range pairs {
-		pairList = append(pairList, [2]string{fmt.Sprintf("%s:%s", repoId, pair.NewRef), fmt.Sprintf("%s:%s", repoId, pair.OldRef)})
+		pairList = append(pairList, RefPairList{fmt.Sprintf("%s:%s", repoId, pair.NewRef), fmt.Sprintf("%s:%s", repoId, pair.OldRef)})
 	}
 
 	return pairList, nil
diff --git a/plugins/refdiff/tasks/refdiff_task_data.go b/plugins/refdiff/tasks/refdiff_task_data.go
index 648f671e..ad36b2c1 100644
--- a/plugins/refdiff/tasks/refdiff_task_data.go
+++ b/plugins/refdiff/tasks/refdiff_task_data.go
@@ -46,6 +46,10 @@ type RefPair struct {
 	NewRef string
 	OldRef string
 }
+type RefCommitPair [4]string
+type RefPairList [2]string
+type RefCommitPairs []RefCommitPair
+type RefPairLists []RefPairList
 
 type Refs []code.Ref
 type RefsAlphabetically Refs


[incubator-devlake] 06/06: refactor: caculate all pairs on prepare

Posted by wa...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

warren pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-devlake.git

commit d7565fc150f2c22b69c54d23f7d36be2f85ab3f4
Author: Nddtfjiang <jz...@qq.com>
AuthorDate: Thu Jun 9 12:31:59 2022 +0000

    refactor: caculate all pairs on prepare
    
    Calculate AllPairs on PrepareTaskData
    Changed db to dal on CaculateTagPattern and CalculateCommitPairs.
    
    Nddtfjiang <zh...@merico.dev>
---
 plugins/refdiff/refdiff.go                         |  8 ++-
 .../refdiff/tasks/ref_commit_diff_calculator.go    | 50 +--------------
 plugins/refdiff/tasks/ref_issue_diff_calculator.go | 12 ++--
 plugins/refdiff/tasks/refdiff_task_data.go         | 73 +++++++++++++++++++---
 4 files changed, 75 insertions(+), 68 deletions(-)

diff --git a/plugins/refdiff/refdiff.go b/plugins/refdiff/refdiff.go
index cec4370c..52dce6c8 100644
--- a/plugins/refdiff/refdiff.go
+++ b/plugins/refdiff/refdiff.go
@@ -62,12 +62,16 @@ func (plugin RefDiff) PrepareTaskData(taskCtx core.TaskContext, options map[stri
 		return nil, err
 	}
 
-	db := taskCtx.GetDb()
+	db := taskCtx.GetDal()
 	tagsPattern := op.TagsPattern
 	tagsLimit := op.TagsLimit
 	tagsOrder := op.TagsOrder
 
-	op.TagsRefs, err = tasks.CaculateTagPattern(db, tagsPattern, tagsLimit, tagsOrder)
+	rs, err := tasks.CaculateTagPattern(db, tagsPattern, tagsLimit, tagsOrder)
+	if err != nil {
+		return nil, err
+	}
+	op.AllPairs, err = tasks.CalculateCommitPairs(db, op.RepoId, op.Pairs, rs)
 	if err != nil {
 		return nil, err
 	}
diff --git a/plugins/refdiff/tasks/ref_commit_diff_calculator.go b/plugins/refdiff/tasks/ref_commit_diff_calculator.go
index 7e2b3504..af71c905 100644
--- a/plugins/refdiff/tasks/ref_commit_diff_calculator.go
+++ b/plugins/refdiff/tasks/ref_commit_diff_calculator.go
@@ -27,51 +27,6 @@ import (
 	"gorm.io/gorm/clause"
 )
 
-// Calculate the commits pairs both from Options.Pairs and TagPattern
-func CalculateCommitsPairs(taskCtx core.SubTaskContext) (RefCommitPairs, error) {
-	data := taskCtx.GetData().(*RefdiffTaskData)
-	repoId := data.Options.RepoId
-	pairs := data.Options.Pairs
-	rs := data.Options.TagsRefs
-	db := taskCtx.GetDb()
-
-	commitPairs := make(RefCommitPairs, 0, len(rs)+len(pairs))
-	for i := 1; i < len(rs); i++ {
-		commitPairs = append(commitPairs, [4]string{rs[i-1].CommitSha, rs[i].CommitSha, rs[i-1].Name, rs[i].Name})
-	}
-
-	// caculate pairs part
-	// convert ref pairs into commit pairs
-	ref2sha := func(refName string) (string, error) {
-		ref := &code.Ref{}
-		if refName == "" {
-			return "", fmt.Errorf("ref name is empty")
-		}
-		ref.Id = fmt.Sprintf("%s:%s", repoId, refName)
-		err := db.First(ref).Error
-		if err != nil {
-			return "", fmt.Errorf("faild to load Ref info for repoId:%s, refName:%s", repoId, refName)
-		}
-		return ref.CommitSha, nil
-	}
-
-	for i, refPair := range pairs {
-		// get new ref's commit sha
-		newCommit, err := ref2sha(refPair.NewRef)
-		if err != nil {
-			return RefCommitPairs{}, fmt.Errorf("failed to load commit sha for NewRef on pair #%d: %w", i, err)
-		}
-		// get old ref's commit sha
-		oldCommit, err := ref2sha(refPair.OldRef)
-		if err != nil {
-			return RefCommitPairs{}, fmt.Errorf("failed to load commit sha for OleRef on pair #%d: %w", i, err)
-		}
-		commitPairs = append(commitPairs, RefCommitPair{newCommit, oldCommit, refPair.NewRef, refPair.OldRef})
-	}
-
-	return commitPairs, nil
-}
-
 func CalculateCommitsDiff(taskCtx core.SubTaskContext) error {
 	data := taskCtx.GetData().(*RefdiffTaskData)
 	repoId := data.Options.RepoId
@@ -80,10 +35,7 @@ func CalculateCommitsDiff(taskCtx core.SubTaskContext) error {
 	logger := taskCtx.GetLogger()
 	insertCountLimitOfRefsCommitsDiff := int(65535 / reflect.ValueOf(code.RefsCommitsDiff{}).NumField())
 
-	commitPairs, err := CalculateCommitsPairs(taskCtx)
-	if err != nil {
-		return err
-	}
+	commitPairs := data.Options.AllPairs
 
 	commitNodeGraph := utils.NewCommitNodeGraph()
 
diff --git a/plugins/refdiff/tasks/ref_issue_diff_calculator.go b/plugins/refdiff/tasks/ref_issue_diff_calculator.go
index 3f6809f5..43e9c853 100644
--- a/plugins/refdiff/tasks/ref_issue_diff_calculator.go
+++ b/plugins/refdiff/tasks/ref_issue_diff_calculator.go
@@ -26,20 +26,16 @@ import (
 	"github.com/apache/incubator-devlake/plugins/helper"
 )
 
-// Calculate the pair list both from Options.Pairs and TagPattern
+// CaculatePairList Calculate the pair list both from Options.Pairs and TagPattern
 func CaculatePairList(taskCtx core.SubTaskContext) (RefPairLists, error) {
 	data := taskCtx.GetData().(*RefdiffTaskData)
 	repoId := data.Options.RepoId
-	pairs := data.Options.Pairs
-	rs := data.Options.TagsRefs
+	pairs := data.Options.AllPairs
 
-	pairList := make(RefPairLists, 0, len(rs)+len(pairs))
-	for i := 1; i < len(rs); i++ {
-		pairList = append(pairList, RefPairList{fmt.Sprintf("%s:%s", repoId, rs[i-1].Id), fmt.Sprintf("%s:%s", repoId, rs[i].Id)})
-	}
+	pairList := make(RefPairLists, 0, len(pairs))
 
 	for _, pair := range pairs {
-		pairList = append(pairList, RefPairList{fmt.Sprintf("%s:%s", repoId, pair.NewRef), fmt.Sprintf("%s:%s", repoId, pair.OldRef)})
+		pairList = append(pairList, RefPairList{fmt.Sprintf("%s:%s", repoId, pair[2]), fmt.Sprintf("%s:%s", repoId, pair[3])})
 	}
 
 	return pairList, nil
diff --git a/plugins/refdiff/tasks/refdiff_task_data.go b/plugins/refdiff/tasks/refdiff_task_data.go
index 37781308..8e0c9fec 100644
--- a/plugins/refdiff/tasks/refdiff_task_data.go
+++ b/plugins/refdiff/tasks/refdiff_task_data.go
@@ -25,7 +25,7 @@ import (
 	"time"
 
 	"github.com/apache/incubator-devlake/models/domainlayer/code"
-	"gorm.io/gorm"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
 )
 
 type RefdiffOptions struct {
@@ -33,10 +33,11 @@ type RefdiffOptions struct {
 	Tasks  []string `json:"tasks,omitempty"`
 	Pairs  []RefPair
 
-	TagsPattern string     // The Pattern to match from all tags
-	TagsLimit   int        // How many tags be matched should be used.
-	TagsOrder   string     // The Rule to Order the tag list
-	TagsRefs    []code.Ref // Caculate out by TagsPattern TagsLimit TagsOrder from db
+	TagsPattern string // The Pattern to match from all tags
+	TagsLimit   int    // How many tags be matched should be used.
+	TagsOrder   string // The Rule to Order the tag list
+
+	AllPairs RefCommitPairs // Pairs and TagsPattern Pairs
 }
 
 type RefdiffTaskData struct {
@@ -133,15 +134,20 @@ func (rs RefsReverseSemver) Swap(i, j int) {
 	rs[i], rs[j] = rs[j], rs[i]
 }
 
-// Calculate the TagPattern order by tagsOrder and return the Refs
-func CaculateTagPattern(db *gorm.DB, tagsPattern string, tagsLimit int, tagsOrder string) (Refs, error) {
+// CaculateTagPattern Calculate the TagPattern order by tagsOrder and return the Refs
+func CaculateTagPattern(db dal.Dal, tagsPattern string, tagsLimit int, tagsOrder string) (Refs, error) {
 	rs := Refs{}
 
 	// caculate Pattern part
 	if tagsPattern == "" || tagsLimit <= 1 {
 		return rs, nil
 	}
-	rows, err := db.Model(&code.Ref{}).Order("created_date desc").Rows()
+	rows, err := db.Cursor(
+		dal.From("refs"),
+		dal.Where(""),
+		dal.Orderby("created_date desc"),
+	)
+
 	if err != nil {
 		return rs, err
 	}
@@ -152,7 +158,7 @@ func CaculateTagPattern(db *gorm.DB, tagsPattern string, tagsLimit int, tagsOrde
 	}
 	for rows.Next() {
 		var ref code.Ref
-		err = db.ScanRows(rows, &ref)
+		err = db.Fetch(rows, &ref)
 		if err != nil {
 			return rs, err
 		}
@@ -180,3 +186,52 @@ func CaculateTagPattern(db *gorm.DB, tagsPattern string, tagsLimit int, tagsOrde
 
 	return rs, nil
 }
+
+// CalculateCommitPairs Calculate the commits pairs both from Options.Pairs and TagPattern
+func CalculateCommitPairs(db dal.Dal, repoId string, pairs []RefPair, rs Refs) (RefCommitPairs, error) {
+	commitPairs := make(RefCommitPairs, 0, len(rs)+len(pairs))
+	for i := 1; i < len(rs); i++ {
+		commitPairs = append(commitPairs, [4]string{rs[i-1].CommitSha, rs[i].CommitSha, rs[i-1].Name, rs[i].Name})
+	}
+
+	// caculate pairs part
+	// convert ref pairs into commit pairs
+	ref2sha := func(refName string) (string, error) {
+		ref := &code.Ref{}
+		if refName == "" {
+			return "", fmt.Errorf("ref name is empty")
+		}
+		ref.Id = fmt.Sprintf("%s:%s", repoId, refName)
+		err := db.First(ref)
+		if err != nil {
+			return "", fmt.Errorf("faild to load Ref info for repoId:%s, refName:%s", repoId, refName)
+		}
+		return ref.CommitSha, nil
+	}
+
+	for i, refPair := range pairs {
+		// get new ref's commit sha
+		newCommit, err := ref2sha(refPair.NewRef)
+		if err != nil {
+			return RefCommitPairs{}, fmt.Errorf("failed to load commit sha for NewRef on pair #%d: %w", i, err)
+		}
+		// get old ref's commit sha
+		oldCommit, err := ref2sha(refPair.OldRef)
+		if err != nil {
+			return RefCommitPairs{}, fmt.Errorf("failed to load commit sha for OleRef on pair #%d: %w", i, err)
+		}
+
+		have := false
+		for _, cp := range commitPairs {
+			if cp[0] == newCommit && cp[1] == oldCommit {
+				have = true
+				break
+			}
+		}
+		if !have {
+			commitPairs = append(commitPairs, RefCommitPair{newCommit, oldCommit, refPair.NewRef, refPair.OldRef})
+		}
+	}
+
+	return commitPairs, nil
+}


[incubator-devlake] 05/06: refactor: set tagslimit args to int from string

Posted by wa...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

warren pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-devlake.git

commit dd00165a3fb7690a678f5c4257512530c2b42665
Author: Nddtfjiang <jz...@qq.com>
AuthorDate: Thu Jun 9 06:51:01 2022 +0000

    refactor: set tagslimit args to int from string
    
    Change tagsLimit to IntP from StringP
    
    Nddtfjiang <zh...@merico.dev>
---
 plugins/refdiff/refdiff.go | 10 ++--------
 1 file changed, 2 insertions(+), 8 deletions(-)

diff --git a/plugins/refdiff/refdiff.go b/plugins/refdiff/refdiff.go
index 8fb502e1..cec4370c 100644
--- a/plugins/refdiff/refdiff.go
+++ b/plugins/refdiff/refdiff.go
@@ -18,8 +18,6 @@ limitations under the License.
 package main
 
 import (
-	"strconv"
-
 	"github.com/apache/incubator-devlake/plugins/refdiff/tasks"
 	"github.com/apache/incubator-devlake/runner"
 	"github.com/mitchellh/mapstructure"
@@ -96,7 +94,7 @@ func main() {
 	oldRef := refdiffCmd.Flags().StringP("old-ref", "o", "", "old ref")
 
 	tagsPattern := refdiffCmd.Flags().StringP("tags-pattern", "p", "", "tags pattern")
-	tagsLimit := refdiffCmd.Flags().StringP("tags-limit", "l", "", "tags limit")
+	tagsLimit := refdiffCmd.Flags().IntP("tags-limit", "l", 2, "tags limit")
 	tagsOrder := refdiffCmd.Flags().StringP("tags-order", "d", "", "tags order")
 
 	_ = refdiffCmd.MarkFlagRequired("repo-id")
@@ -104,10 +102,6 @@ func main() {
 	//_ = refdiffCmd.MarkFlagRequired("old-ref")
 
 	refdiffCmd.Run = func(cmd *cobra.Command, args []string) {
-		tl, err := strconv.Atoi(*tagsLimit)
-		if err != nil {
-			panic(err)
-		}
 		pairs := make([]map[string]string, 0, 1)
 		if *newRef == "" && *oldRef == "" {
 			if *tagsPattern == "" {
@@ -124,7 +118,7 @@ func main() {
 			"repoId":      repoId,
 			"pairs":       pairs,
 			"tagsPattern": *tagsPattern,
-			"tagsLimit":   tl,
+			"tagsLimit":   *tagsLimit,
 			"tagsOrder":   *tagsOrder,
 		})
 	}


[incubator-devlake] 02/06: fix: fix an error lost before directrun

Posted by wa...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

warren pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-devlake.git

commit 2d19b39b8ded3f7bf4df190153aa4403aef23cd9
Author: Nddtfjiang <jz...@qq.com>
AuthorDate: Tue Jun 7 09:43:57 2022 +0000

    fix: fix an error lost before directrun
    
    Add tagsPattern document to readme
    Remove an unuse link in readme.
    Add some comment
    fix an error lost before DirectRun on refdiff.
    
    Nddtfjiang <zh...@merico.dev>
---
 plugins/refdiff/README.md                          | 28 +++++++++++++++++-----
 plugins/refdiff/refdiff.go                         |  5 +++-
 .../refdiff/tasks/ref_commit_diff_calculator.go    |  1 +
 plugins/refdiff/tasks/ref_issue_diff_calculator.go |  1 +
 plugins/refdiff/tasks/refdiff_task_data.go         |  7 +++---
 5 files changed, 32 insertions(+), 10 deletions(-)

diff --git a/plugins/refdiff/README.md b/plugins/refdiff/README.md
index 01f649e3..3c065aea 100644
--- a/plugins/refdiff/README.md
+++ b/plugins/refdiff/README.md
@@ -1,10 +1,4 @@
 # RefDiff
-
-
-| [English](README.md) | [中文](README-zh-CN.md) |
-| --- | --- |
-
-
 ## Summary
 
 For development workload analysis, we often need to know how many commits have been created between 2 releases. This plugin offers the ability to calculate the commits of difference between 2 Ref(branch/tag), and the result will be stored back into database for further analysis.
@@ -90,6 +84,28 @@ and if you want to perform certain subtasks.
   ]
 ]
 ```
+Or you can use tagsPattern to match the tags you want
+And you can use tagOrder (support `alphabetically` and `reverse alphabetically`) to set the order rule with tagLimit to limit the count of matching.
+This is support to calculateCommitsDiff and calculateIssuesDiff
+```json
+[
+  [
+    {
+      "plugin": "refdiff",
+      "subtasks": [
+        "calculateCommitsDiff",
+        "calculateIssuesDiff",
+      ],
+      "options": {
+        "repoId": "github:GithubRepo:384111310",
+        "tagsPattern":".*\\.11\\..*",
+        "tagLimit":3,
+        "tagOrder":"reverse alphabetically",
+      }
+    }
+  ]
+]
+```
 
 2. Curl Mode:
 ```
diff --git a/plugins/refdiff/refdiff.go b/plugins/refdiff/refdiff.go
index 9cfaa1f8..c9c3b9d0 100644
--- a/plugins/refdiff/refdiff.go
+++ b/plugins/refdiff/refdiff.go
@@ -94,7 +94,10 @@ func main() {
 	//_ = refdiffCmd.MarkFlagRequired("old-ref")
 
 	refdiffCmd.Run = func(cmd *cobra.Command, args []string) {
-		tl, _ := strconv.Atoi(*tagsLimit)
+		tl, err := strconv.Atoi(*tagsLimit)
+		if err != nil {
+			panic(err)
+		}
 		pairs := make([]map[string]string, 0, 1)
 		if *newRef == "" && *oldRef == "" {
 			if *tagsPattern == "" {
diff --git a/plugins/refdiff/tasks/ref_commit_diff_calculator.go b/plugins/refdiff/tasks/ref_commit_diff_calculator.go
index 69adb46f..420ed963 100644
--- a/plugins/refdiff/tasks/ref_commit_diff_calculator.go
+++ b/plugins/refdiff/tasks/ref_commit_diff_calculator.go
@@ -27,6 +27,7 @@ import (
 	"gorm.io/gorm/clause"
 )
 
+// Calculate the commits pairs both from Options.Pairs and TagPattern
 func CalculateCommitsPairs(taskCtx core.SubTaskContext) ([][4]string, error) {
 	data := taskCtx.GetData().(*RefdiffTaskData)
 	repoId := data.Options.RepoId
diff --git a/plugins/refdiff/tasks/ref_issue_diff_calculator.go b/plugins/refdiff/tasks/ref_issue_diff_calculator.go
index 3a80482d..5b9d92a7 100644
--- a/plugins/refdiff/tasks/ref_issue_diff_calculator.go
+++ b/plugins/refdiff/tasks/ref_issue_diff_calculator.go
@@ -26,6 +26,7 @@ import (
 	"github.com/apache/incubator-devlake/plugins/helper"
 )
 
+// Calculate the pair list both from Options.Pairs and TagPattern
 func CaculatePairList(taskCtx core.SubTaskContext) ([][2]string, error) {
 	data := taskCtx.GetData().(*RefdiffTaskData)
 	repoId := data.Options.RepoId
diff --git a/plugins/refdiff/tasks/refdiff_task_data.go b/plugins/refdiff/tasks/refdiff_task_data.go
index c2b364fc..648f671e 100644
--- a/plugins/refdiff/tasks/refdiff_task_data.go
+++ b/plugins/refdiff/tasks/refdiff_task_data.go
@@ -32,9 +32,9 @@ type RefdiffOptions struct {
 	Tasks  []string `json:"tasks,omitempty"`
 	Pairs  []RefPair
 
-	TagsPattern string
-	TagsLimit   int
-	TagsOrder   string
+	TagsPattern string // The Pattern to match from all tags
+	TagsLimit   int    // How many tags be matched should be used.
+	TagsOrder   string // The Rule to Order the tag list
 }
 
 type RefdiffTaskData struct {
@@ -79,6 +79,7 @@ func (rs RefsReverseAlphabetically) Swap(i, j int) {
 	rs[i], rs[j] = rs[j], rs[i]
 }
 
+// Calculate the TagPattern order by tagsOrder and return the Refs
 func CaculateTagPattern(taskCtx core.SubTaskContext) (Refs, error) {
 	rs := Refs{}
 	data := taskCtx.GetData().(*RefdiffTaskData)


[incubator-devlake] 04/06: feat: add order type semver

Posted by wa...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

warren pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-devlake.git

commit 2b15cefc6b94212cf8ea3337eb1b467a2a1b0286
Author: Nddtfjiang <jz...@qq.com>
AuthorDate: Wed Jun 8 13:52:04 2022 +0000

    feat: add order type semver
    
    Add tagOrder rule semver
    Add tagOrder rule reverse semver
    CaculateTagPattern now only run once in PrepareTaskData.
    
    Nddtfjiang <zh...@merico.dev>
---
 plugins/refdiff/README.md                          |  4 +-
 plugins/refdiff/refdiff.go                         | 10 +++
 .../refdiff/tasks/ref_commit_diff_calculator.go    | 14 +---
 plugins/refdiff/tasks/ref_issue_diff_calculator.go | 14 +---
 plugins/refdiff/tasks/refdiff_task_data.go         | 77 ++++++++++++++++++----
 5 files changed, 84 insertions(+), 35 deletions(-)

diff --git a/plugins/refdiff/README.md b/plugins/refdiff/README.md
index 3c065aea..1db6a4d2 100644
--- a/plugins/refdiff/README.md
+++ b/plugins/refdiff/README.md
@@ -85,7 +85,7 @@ and if you want to perform certain subtasks.
 ]
 ```
 Or you can use tagsPattern to match the tags you want
-And you can use tagOrder (support `alphabetically` and `reverse alphabetically`) to set the order rule with tagLimit to limit the count of matching.
+And you can use tagOrder (support `alphabetically`,`reverse alphabetically`,`semver`,`reverse semver`) to set the order rule with tagLimit to limit the count of matching.
 This is support to calculateCommitsDiff and calculateIssuesDiff
 ```json
 [
@@ -100,7 +100,7 @@ This is support to calculateCommitsDiff and calculateIssuesDiff
         "repoId": "github:GithubRepo:384111310",
         "tagsPattern":".*\\.11\\..*",
         "tagLimit":3,
-        "tagOrder":"reverse alphabetically",
+        "tagOrder":"reverse semver",
       }
     }
   ]
diff --git a/plugins/refdiff/refdiff.go b/plugins/refdiff/refdiff.go
index c9c3b9d0..8fb502e1 100644
--- a/plugins/refdiff/refdiff.go
+++ b/plugins/refdiff/refdiff.go
@@ -64,6 +64,16 @@ func (plugin RefDiff) PrepareTaskData(taskCtx core.TaskContext, options map[stri
 		return nil, err
 	}
 
+	db := taskCtx.GetDb()
+	tagsPattern := op.TagsPattern
+	tagsLimit := op.TagsLimit
+	tagsOrder := op.TagsOrder
+
+	op.TagsRefs, err = tasks.CaculateTagPattern(db, tagsPattern, tagsLimit, tagsOrder)
+	if err != nil {
+		return nil, err
+	}
+
 	return &tasks.RefdiffTaskData{
 		Options: &op,
 	}, nil
diff --git a/plugins/refdiff/tasks/ref_commit_diff_calculator.go b/plugins/refdiff/tasks/ref_commit_diff_calculator.go
index 606d1462..7e2b3504 100644
--- a/plugins/refdiff/tasks/ref_commit_diff_calculator.go
+++ b/plugins/refdiff/tasks/ref_commit_diff_calculator.go
@@ -32,19 +32,11 @@ func CalculateCommitsPairs(taskCtx core.SubTaskContext) (RefCommitPairs, error)
 	data := taskCtx.GetData().(*RefdiffTaskData)
 	repoId := data.Options.RepoId
 	pairs := data.Options.Pairs
-	tagsLimit := data.Options.TagsLimit
+	rs := data.Options.TagsRefs
 	db := taskCtx.GetDb()
 
-	rs, err := CaculateTagPattern(taskCtx)
-	if err != nil {
-		return RefCommitPairs{}, err
-	}
-	if tagsLimit > rs.Len() {
-		tagsLimit = rs.Len()
-	}
-
-	commitPairs := make(RefCommitPairs, 0, tagsLimit+len(pairs))
-	for i := 1; i < tagsLimit; i++ {
+	commitPairs := make(RefCommitPairs, 0, len(rs)+len(pairs))
+	for i := 1; i < len(rs); i++ {
 		commitPairs = append(commitPairs, [4]string{rs[i-1].CommitSha, rs[i].CommitSha, rs[i-1].Name, rs[i].Name})
 	}
 
diff --git a/plugins/refdiff/tasks/ref_issue_diff_calculator.go b/plugins/refdiff/tasks/ref_issue_diff_calculator.go
index aed0f4e8..3f6809f5 100644
--- a/plugins/refdiff/tasks/ref_issue_diff_calculator.go
+++ b/plugins/refdiff/tasks/ref_issue_diff_calculator.go
@@ -31,18 +31,10 @@ func CaculatePairList(taskCtx core.SubTaskContext) (RefPairLists, error) {
 	data := taskCtx.GetData().(*RefdiffTaskData)
 	repoId := data.Options.RepoId
 	pairs := data.Options.Pairs
-	tagsLimit := data.Options.TagsLimit
+	rs := data.Options.TagsRefs
 
-	rs, err := CaculateTagPattern(taskCtx)
-	if err != nil {
-		return RefPairLists{}, err
-	}
-	if tagsLimit > rs.Len() {
-		tagsLimit = rs.Len()
-	}
-
-	pairList := make(RefPairLists, 0, tagsLimit+len(pairs))
-	for i := 1; i < tagsLimit; i++ {
+	pairList := make(RefPairLists, 0, len(rs)+len(pairs))
+	for i := 1; i < len(rs); i++ {
 		pairList = append(pairList, RefPairList{fmt.Sprintf("%s:%s", repoId, rs[i-1].Id), fmt.Sprintf("%s:%s", repoId, rs[i].Id)})
 	}
 
diff --git a/plugins/refdiff/tasks/refdiff_task_data.go b/plugins/refdiff/tasks/refdiff_task_data.go
index ad36b2c1..37781308 100644
--- a/plugins/refdiff/tasks/refdiff_task_data.go
+++ b/plugins/refdiff/tasks/refdiff_task_data.go
@@ -21,10 +21,11 @@ import (
 	"fmt"
 	"regexp"
 	"sort"
+	"strings"
 	"time"
 
 	"github.com/apache/incubator-devlake/models/domainlayer/code"
-	"github.com/apache/incubator-devlake/plugins/core"
+	"gorm.io/gorm"
 )
 
 type RefdiffOptions struct {
@@ -32,9 +33,10 @@ type RefdiffOptions struct {
 	Tasks  []string `json:"tasks,omitempty"`
 	Pairs  []RefPair
 
-	TagsPattern string // The Pattern to match from all tags
-	TagsLimit   int    // How many tags be matched should be used.
-	TagsOrder   string // The Rule to Order the tag list
+	TagsPattern string     // The Pattern to match from all tags
+	TagsLimit   int        // How many tags be matched should be used.
+	TagsOrder   string     // The Rule to Order the tag list
+	TagsRefs    []code.Ref // Caculate out by TagsPattern TagsLimit TagsOrder from db
 }
 
 type RefdiffTaskData struct {
@@ -54,6 +56,8 @@ type RefPairLists []RefPairList
 type Refs []code.Ref
 type RefsAlphabetically Refs
 type RefsReverseAlphabetically Refs
+type RefsSemver Refs
+type RefsReverseSemver Refs
 
 func (rs Refs) Len() int {
 	return len(rs)
@@ -83,16 +87,58 @@ func (rs RefsReverseAlphabetically) Swap(i, j int) {
 	rs[i], rs[j] = rs[j], rs[i]
 }
 
+func (rs RefsSemver) Len() int {
+	return len(rs)
+}
+
+func (rs RefsSemver) Less(i, j int) bool {
+	parti := strings.Split(rs[i].Name, ".")
+	partj := strings.Split(rs[j].Name, ".")
+
+	for k := 0; k < len(partj); k++ {
+		if k >= len(parti) {
+			return true
+		}
+
+		if len(parti[k]) < len(partj[k]) {
+			return true
+		}
+		if len(parti[k]) > len(partj[k]) {
+			return false
+		}
+
+		if parti[k] < partj[k] {
+			return true
+		}
+		if parti[k] > partj[k] {
+			return false
+		}
+	}
+	return false
+}
+
+func (rs RefsSemver) Swap(i, j int) {
+	rs[i], rs[j] = rs[j], rs[i]
+}
+
+func (rs RefsReverseSemver) Len() int {
+	return len(rs)
+}
+
+func (rs RefsReverseSemver) Less(i, j int) bool {
+	return RefsSemver(rs).Less(j, i)
+}
+
+func (rs RefsReverseSemver) Swap(i, j int) {
+	rs[i], rs[j] = rs[j], rs[i]
+}
+
 // Calculate the TagPattern order by tagsOrder and return the Refs
-func CaculateTagPattern(taskCtx core.SubTaskContext) (Refs, error) {
+func CaculateTagPattern(db *gorm.DB, tagsPattern string, tagsLimit int, tagsOrder string) (Refs, error) {
 	rs := Refs{}
-	data := taskCtx.GetData().(*RefdiffTaskData)
-	tagsPattern := data.Options.TagsPattern
-	tagsOrder := data.Options.TagsOrder
-	db := taskCtx.GetDb()
 
 	// caculate Pattern part
-	if data.Options.TagsPattern == "" || data.Options.TagsLimit <= 1 {
+	if tagsPattern == "" || tagsLimit <= 1 {
 		return rs, nil
 	}
 	rows, err := db.Model(&code.Ref{}).Order("created_date desc").Rows()
@@ -111,7 +157,7 @@ func CaculateTagPattern(taskCtx core.SubTaskContext) (Refs, error) {
 			return rs, err
 		}
 
-		if ok := r.Match([]byte(ref.Id)); ok {
+		if ok := r.Match([]byte(ref.Name)); ok {
 			rs = append(rs, ref)
 		}
 	}
@@ -120,8 +166,17 @@ func CaculateTagPattern(taskCtx core.SubTaskContext) (Refs, error) {
 		sort.Sort(RefsAlphabetically(rs))
 	case "reverse alphabetically":
 		sort.Sort(RefsReverseAlphabetically(rs))
+	case "semver":
+		sort.Sort(RefsSemver(rs))
+	case "reverse semver":
+		sort.Sort(RefsReverseSemver(rs))
 	default:
 		break
 	}
+
+	if tagsLimit < rs.Len() {
+		rs = rs[:tagsLimit]
+	}
+
 	return rs, nil
 }


[incubator-devlake] 01/06: feat: refdiff support pattern

Posted by wa...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

warren pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-devlake.git

commit a59ea07aee90f819487d33b29576cd36edde80af
Author: Nddtfjiang <jz...@qq.com>
AuthorDate: Mon Jun 6 15:06:46 2022 +0000

    feat: refdiff support pattern
    
    Add CaculateTagPattern
    Add CaculatePairList for CalculateIssuesDiff
    Add CalculateCommitsPairs for CalculateCommitsDiff
    Support tagsPattern tagsLimit and tagsOrder
    
    Nddtfjiang <zh...@merico.dev>
---
 plugins/refdiff/refdiff.go                         | 35 ++++++---
 .../refdiff/tasks/ref_commit_diff_calculator.go    | 70 ++++++++++++------
 plugins/refdiff/tasks/ref_issue_diff_calculator.go | 33 +++++++--
 plugins/refdiff/tasks/refdiff_task_data.go         | 84 ++++++++++++++++++++++
 4 files changed, 187 insertions(+), 35 deletions(-)

diff --git a/plugins/refdiff/refdiff.go b/plugins/refdiff/refdiff.go
index 881feab1..9cfaa1f8 100644
--- a/plugins/refdiff/refdiff.go
+++ b/plugins/refdiff/refdiff.go
@@ -18,6 +18,8 @@ limitations under the License.
 package main
 
 import (
+	"strconv"
+
 	"github.com/apache/incubator-devlake/plugins/refdiff/tasks"
 	"github.com/apache/incubator-devlake/runner"
 	"github.com/mitchellh/mapstructure"
@@ -83,19 +85,34 @@ func main() {
 	newRef := refdiffCmd.Flags().StringP("new-ref", "n", "", "new ref")
 	oldRef := refdiffCmd.Flags().StringP("old-ref", "o", "", "old ref")
 
+	tagsPattern := refdiffCmd.Flags().StringP("tags-pattern", "p", "", "tags pattern")
+	tagsLimit := refdiffCmd.Flags().StringP("tags-limit", "l", "", "tags limit")
+	tagsOrder := refdiffCmd.Flags().StringP("tags-order", "d", "", "tags order")
+
 	_ = refdiffCmd.MarkFlagRequired("repo-id")
-	_ = refdiffCmd.MarkFlagRequired("new-ref")
-	_ = refdiffCmd.MarkFlagRequired("old-ref")
+	//_ = refdiffCmd.MarkFlagRequired("new-ref")
+	//_ = refdiffCmd.MarkFlagRequired("old-ref")
 
 	refdiffCmd.Run = func(cmd *cobra.Command, args []string) {
+		tl, _ := strconv.Atoi(*tagsLimit)
+		pairs := make([]map[string]string, 0, 1)
+		if *newRef == "" && *oldRef == "" {
+			if *tagsPattern == "" {
+				panic("You must set at least one part of '-p' or '-n -o' for tagsPattern or newRef,oldRef")
+			}
+		} else {
+			pairs = append(pairs, map[string]string{
+				"NewRef": *newRef,
+				"OldRef": *oldRef,
+			})
+		}
+
 		runner.DirectRun(cmd, args, PluginEntry, map[string]interface{}{
-			"repoId": repoId,
-			"pairs": []map[string]string{
-				{
-					"NewRef": *newRef,
-					"OldRef": *oldRef,
-				},
-			},
+			"repoId":      repoId,
+			"pairs":       pairs,
+			"tagsPattern": *tagsPattern,
+			"tagsLimit":   tl,
+			"tagsOrder":   *tagsOrder,
 		})
 	}
 	runner.RunCmd(refdiffCmd)
diff --git a/plugins/refdiff/tasks/ref_commit_diff_calculator.go b/plugins/refdiff/tasks/ref_commit_diff_calculator.go
index 17aa2bfd..69adb46f 100644
--- a/plugins/refdiff/tasks/ref_commit_diff_calculator.go
+++ b/plugins/refdiff/tasks/ref_commit_diff_calculator.go
@@ -27,14 +27,27 @@ import (
 	"gorm.io/gorm/clause"
 )
 
-func CalculateCommitsDiff(taskCtx core.SubTaskContext) error {
+func CalculateCommitsPairs(taskCtx core.SubTaskContext) ([][4]string, error) {
 	data := taskCtx.GetData().(*RefdiffTaskData)
 	repoId := data.Options.RepoId
 	pairs := data.Options.Pairs
+	tagsLimit := data.Options.TagsLimit
 	db := taskCtx.GetDb()
-	ctx := taskCtx.GetContext()
-	logger := taskCtx.GetLogger()
-	insertCountLimitOfRefsCommitsDiff := int(65535 / reflect.ValueOf(code.RefsCommitsDiff{}).NumField())
+
+	rs, err := CaculateTagPattern(taskCtx)
+	if err != nil {
+		return [][4]string{}, err
+	}
+	if tagsLimit > rs.Len() {
+		tagsLimit = rs.Len()
+	}
+
+	commitPairs := make([][4]string, 0, tagsLimit+len(pairs))
+	for i := 1; i < tagsLimit; i++ {
+		commitPairs = append(commitPairs, [4]string{rs[i-1].CommitSha, rs[i].CommitSha, rs[i-1].Name, rs[i].Name})
+	}
+
+	// caculate pairs part
 	// convert ref pairs into commit pairs
 	ref2sha := func(refName string) (string, error) {
 		ref := &code.Ref{}
@@ -48,21 +61,37 @@ func CalculateCommitsDiff(taskCtx core.SubTaskContext) error {
 		}
 		return ref.CommitSha, nil
 	}
-	commitPairs := make([][4]string, 0, len(pairs))
+
 	for i, refPair := range pairs {
 		// get new ref's commit sha
 		newCommit, err := ref2sha(refPair.NewRef)
 		if err != nil {
-			return fmt.Errorf("failed to load commit sha for NewRef on pair #%d: %w", i, err)
+			return [][4]string{}, fmt.Errorf("failed to load commit sha for NewRef on pair #%d: %w", i, err)
 		}
 		// get old ref's commit sha
 		oldCommit, err := ref2sha(refPair.OldRef)
 		if err != nil {
-			return fmt.Errorf("failed to load commit sha for OleRef on pair #%d: %w", i, err)
+			return [][4]string{}, fmt.Errorf("failed to load commit sha for OleRef on pair #%d: %w", i, err)
 		}
 		commitPairs = append(commitPairs, [4]string{newCommit, oldCommit, refPair.NewRef, refPair.OldRef})
 	}
 
+	return commitPairs, nil
+}
+
+func CalculateCommitsDiff(taskCtx core.SubTaskContext) error {
+	data := taskCtx.GetData().(*RefdiffTaskData)
+	repoId := data.Options.RepoId
+	db := taskCtx.GetDb()
+	ctx := taskCtx.GetContext()
+	logger := taskCtx.GetLogger()
+	insertCountLimitOfRefsCommitsDiff := int(65535 / reflect.ValueOf(code.RefsCommitsDiff{}).NumField())
+
+	commitPairs, err := CalculateCommitsPairs(taskCtx)
+	if err != nil {
+		return err
+	}
+
 	commitNodeGraph := utils.NewCommitNodeGraph()
 
 	// load commits from db
@@ -90,7 +119,7 @@ func CalculateCommitsDiff(taskCtx core.SubTaskContext) error {
 		commitNodeGraph.AddParent(commitParent.CommitSha, commitParent.ParentCommitSha)
 	}
 
-	logger.Info("refdiff", fmt.Sprintf("Create a commit node graph with node count[%d]", commitNodeGraph.Size()))
+	logger.Info("Create a commit node graph with node count[%d]", commitNodeGraph.Size())
 
 	// calculate diffs for commits pairs and store them into database
 	commitsDiff := &code.RefsCommitsDiff{}
@@ -118,13 +147,10 @@ func CalculateCommitsDiff(taskCtx core.SubTaskContext) error {
 		if commitsDiff.NewRefCommitSha == commitsDiff.OldRefCommitSha {
 			// different refs might point to a same commit, it is ok
 			logger.Info(
-				"refdiff",
-				fmt.Sprintf(
-					"skipping ref pair due to they are the same %s %s => %s",
-					commitsDiff.NewRefId,
-					commitsDiff.OldRefId,
-					commitsDiff.NewRefCommitSha,
-				),
+				"skipping ref pair due to they are the same %s %s => %s",
+				commitsDiff.NewRefId,
+				commitsDiff.OldRefId,
+				commitsDiff.NewRefCommitSha,
 			)
 			continue
 		}
@@ -140,7 +166,7 @@ func CalculateCommitsDiff(taskCtx core.SubTaskContext) error {
 
 			// sql limit placeholders count only 65535
 			if commitsDiff.SortingIndex%insertCountLimitOfRefsCommitsDiff == 0 {
-				logger.Info("refdiff", fmt.Sprintf("commitsDiffs count in limited[%d] index[%d]--exec and clean", len(commitsDiffs), commitsDiff.SortingIndex))
+				logger.Info("commitsDiffs count in limited[%d] index[%d]--exec and clean", len(commitsDiffs), commitsDiff.SortingIndex)
 				err = db.Clauses(clause.OnConflict{DoNothing: true}).Create(commitsDiffs).Error
 				if err != nil {
 					return err
@@ -152,20 +178,20 @@ func CalculateCommitsDiff(taskCtx core.SubTaskContext) error {
 		}
 
 		if len(commitsDiffs) > 0 {
-			logger.Info("refdiff", fmt.Sprintf("insert data count [%d]", len(commitsDiffs)))
+			logger.Info("insert data count [%d]", len(commitsDiffs))
 			err = db.Clauses(clause.OnConflict{DoNothing: true}).Create(commitsDiffs).Error
 			if err != nil {
 				return err
 			}
 		}
 
-		logger.Info("refdiff", fmt.Sprintf(
-			"total %d commits of difference found between %s and %s(total:%d)",
+		logger.Info(
+			"total %d commits of difference found between [new][%s] and [old][%s(total:%d)]",
 			newCount,
-			commitsDiff.NewRefCommitSha,
-			commitsDiff.OldRefCommitSha,
+			commitsDiff.NewRefId,
+			commitsDiff.OldRefId,
 			oldCount,
-		))
+		)
 		taskCtx.IncProgress(1)
 	}
 	return nil
diff --git a/plugins/refdiff/tasks/ref_issue_diff_calculator.go b/plugins/refdiff/tasks/ref_issue_diff_calculator.go
index a164055b..3a80482d 100644
--- a/plugins/refdiff/tasks/ref_issue_diff_calculator.go
+++ b/plugins/refdiff/tasks/ref_issue_diff_calculator.go
@@ -26,15 +26,40 @@ import (
 	"github.com/apache/incubator-devlake/plugins/helper"
 )
 
-func CalculateIssuesDiff(taskCtx core.SubTaskContext) error {
+func CaculatePairList(taskCtx core.SubTaskContext) ([][2]string, error) {
 	data := taskCtx.GetData().(*RefdiffTaskData)
 	repoId := data.Options.RepoId
 	pairs := data.Options.Pairs
+	tagsLimit := data.Options.TagsLimit
+
+	rs, err := CaculateTagPattern(taskCtx)
+	if err != nil {
+		return [][2]string{}, err
+	}
+	if tagsLimit > rs.Len() {
+		tagsLimit = rs.Len()
+	}
+
+	pairList := make([][2]string, 0, tagsLimit+len(pairs))
+	for i := 1; i < tagsLimit; i++ {
+		pairList = append(pairList, [2]string{fmt.Sprintf("%s:%s", repoId, rs[i-1].Id), fmt.Sprintf("%s:%s", repoId, rs[i].Id)})
+	}
+
+	for _, pair := range pairs {
+		pairList = append(pairList, [2]string{fmt.Sprintf("%s:%s", repoId, pair.NewRef), fmt.Sprintf("%s:%s", repoId, pair.OldRef)})
+	}
+
+	return pairList, nil
+}
+
+func CalculateIssuesDiff(taskCtx core.SubTaskContext) error {
+	data := taskCtx.GetData().(*RefdiffTaskData)
+	repoId := data.Options.RepoId
 	db := taskCtx.GetDb()
 	// use to calculate progress
-	pairList := make([][2]string, len(pairs))
-	for i, pair := range pairs {
-		pairList[i] = [2]string{fmt.Sprintf("%s:%s", repoId, pair.NewRef), fmt.Sprintf("%s:%s", repoId, pair.OldRef)}
+	pairList, err := CaculatePairList(taskCtx)
+	if err != nil {
+		return err
 	}
 	cursor, err := db.Table("refs_commits_diffs").
 		Joins(
diff --git a/plugins/refdiff/tasks/refdiff_task_data.go b/plugins/refdiff/tasks/refdiff_task_data.go
index 9b53c88a..c2b364fc 100644
--- a/plugins/refdiff/tasks/refdiff_task_data.go
+++ b/plugins/refdiff/tasks/refdiff_task_data.go
@@ -18,13 +18,23 @@ limitations under the License.
 package tasks
 
 import (
+	"fmt"
+	"regexp"
+	"sort"
 	"time"
+
+	"github.com/apache/incubator-devlake/models/domainlayer/code"
+	"github.com/apache/incubator-devlake/plugins/core"
 )
 
 type RefdiffOptions struct {
 	RepoId string
 	Tasks  []string `json:"tasks,omitempty"`
 	Pairs  []RefPair
+
+	TagsPattern string
+	TagsLimit   int
+	TagsOrder   string
 }
 
 type RefdiffTaskData struct {
@@ -36,3 +46,77 @@ type RefPair struct {
 	NewRef string
 	OldRef string
 }
+
+type Refs []code.Ref
+type RefsAlphabetically Refs
+type RefsReverseAlphabetically Refs
+
+func (rs Refs) Len() int {
+	return len(rs)
+}
+
+func (rs RefsAlphabetically) Len() int {
+	return len(rs)
+}
+
+func (rs RefsAlphabetically) Less(i, j int) bool {
+	return rs[i].Id < rs[j].Id
+}
+
+func (rs RefsAlphabetically) Swap(i, j int) {
+	rs[i], rs[j] = rs[j], rs[i]
+}
+
+func (rs RefsReverseAlphabetically) Len() int {
+	return len(rs)
+}
+
+func (rs RefsReverseAlphabetically) Less(i, j int) bool {
+	return rs[i].Id > rs[j].Id
+}
+
+func (rs RefsReverseAlphabetically) Swap(i, j int) {
+	rs[i], rs[j] = rs[j], rs[i]
+}
+
+func CaculateTagPattern(taskCtx core.SubTaskContext) (Refs, error) {
+	rs := Refs{}
+	data := taskCtx.GetData().(*RefdiffTaskData)
+	tagsPattern := data.Options.TagsPattern
+	tagsOrder := data.Options.TagsOrder
+	db := taskCtx.GetDb()
+
+	// caculate Pattern part
+	if data.Options.TagsPattern == "" || data.Options.TagsLimit <= 1 {
+		return rs, nil
+	}
+	rows, err := db.Model(&code.Ref{}).Order("created_date desc").Rows()
+	if err != nil {
+		return rs, err
+	}
+	defer rows.Next()
+	r, err := regexp.Compile(tagsPattern)
+	if err != nil {
+		return rs, fmt.Errorf("unable to parse: %s\r\n%s", tagsPattern, err.Error())
+	}
+	for rows.Next() {
+		var ref code.Ref
+		err = db.ScanRows(rows, &ref)
+		if err != nil {
+			return rs, err
+		}
+
+		if ok := r.Match([]byte(ref.Id)); ok {
+			rs = append(rs, ref)
+		}
+	}
+	switch tagsOrder {
+	case "alphabetically":
+		sort.Sort(RefsAlphabetically(rs))
+	case "reverse alphabetically":
+		sort.Sort(RefsReverseAlphabetically(rs))
+	default:
+		break
+	}
+	return rs, nil
+}