You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@devlake.apache.org by GitBox <gi...@apache.org> on 2022/06/06 15:37:34 UTC

[GitHub] [incubator-devlake] mappjzc opened a new pull request, #2096: Add TagsPattern for use Pattern in refdiff

mappjzc opened a new pull request, #2096:
URL: https://github.com/apache/incubator-devlake/pull/2096

   # Summary
   Add TagsPattern to set Pattern for refdiff
   Add TagsLimit to limit the count of TagsPattern matching.
   Add TagsOrder to set the order rule for  list of TagsPattern  matching.
   Add surpport of DirctRun for new param. 
   Optimize partial logs
   The TagsPattern support CalculateIssuesDiff and CalculateCommitsDiff.
   
   <!--
   Thanks for submitting a pull request!
   
   We appreciate you spending the time to work on these changes.
   Please fill out as many sections below as possible.
   -->
   
   ### Does this close any open issues?
   Please mention the issues here.
   
   ### Screenshots
   ![image](https://user-images.githubusercontent.com/2921251/172193821-afae87a3-cf89-49d4-b67e-d930a3bf127c.png)
   ![image](https://user-images.githubusercontent.com/2921251/172193936-68db23e8-56ab-4297-83eb-414df4f8acf0.png)
   
   ![image](https://user-images.githubusercontent.com/2921251/172194173-bf5ccf03-322d-418f-b8c7-2356a7b6e2d0.png)
   ![image](https://user-images.githubusercontent.com/2921251/172194234-ec4dc412-0125-41c8-a19b-4d2f45661eec.png)
   
   ### Other Information
   Any other information that is important to this PR.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] warren830 commented on a diff in pull request #2096: Add TagsPattern for use Pattern in refdiff

Posted by GitBox <gi...@apache.org>.
warren830 commented on code in PR #2096:
URL: https://github.com/apache/incubator-devlake/pull/2096#discussion_r890931105


##########
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) {

Review Comment:
   and what is the [][4]string? Or define a CommitsPairs to replace [4]string



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] warren830 commented on a diff in pull request #2096: Add TagsPattern for use Pattern in refdiff

Posted by GitBox <gi...@apache.org>.
warren830 commented on code in PR #2096:
URL: https://github.com/apache/incubator-devlake/pull/2096#discussion_r890930640


##########
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)

Review Comment:
   same as above



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] klesh commented on a diff in pull request #2096: Add TagsPattern for use Pattern in refdiff

Posted by GitBox <gi...@apache.org>.
klesh commented on code in PR #2096:
URL: https://github.com/apache/incubator-devlake/pull/2096#discussion_r892024754


##########
plugins/refdiff/refdiff.go:
##########
@@ -83,19 +85,37 @@ 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")

Review Comment:
   why not make it `IntP` or `UnitP`? then you won't have to convert it youself.



##########
plugins/refdiff/tasks/ref_commit_diff_calculator.go:
##########
@@ -27,14 +27,28 @@ import (
 	"gorm.io/gorm/clause"
 )
 
-func CalculateCommitsDiff(taskCtx core.SubTaskContext) error {
+// 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
+	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)

Review Comment:
   I think we should calculate `pairs` inside `PrepareTaskData` method once and for all instead of calculation inside every subtasks



##########
plugins/refdiff/README.md:
##########
@@ -90,6 +84,28 @@ and if you want to perform certain subtasks.
   ]
 ]
 ```
+Or you can use tagsPattern to match the tags you want

Review Comment:
   I think alphabelcially order is no enough.
   we should consider `semver` sorting, sth in that nature



##########
plugins/refdiff/refdiff.go:
##########
@@ -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")

Review Comment:
   @warren830  there is a misunderstanding here,  errors occur when the name is invalid or missing. 
   it is safe to ignore the error here.
   and, we did this everywhere in plugin.main



##########
plugins/refdiff/refdiff.go:
##########
@@ -83,19 +85,37 @@ 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")

Review Comment:
   should have assigned a default value for it, like 10



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] warren830 commented on a diff in pull request #2096: Add TagsPattern for use Pattern in refdiff

Posted by GitBox <gi...@apache.org>.
warren830 commented on code in PR #2096:
URL: https://github.com/apache/incubator-devlake/pull/2096#discussion_r890931105


##########
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) {

Review Comment:
   and what is the [][4]string? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] warren830 commented on a diff in pull request #2096: Add TagsPattern for use Pattern in refdiff

Posted by GitBox <gi...@apache.org>.
warren830 commented on code in PR #2096:
URL: https://github.com/apache/incubator-devlake/pull/2096#discussion_r890928444


##########
plugins/refdiff/refdiff.go:
##########
@@ -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")

Review Comment:
   can you write some comments for these three variables?



##########
plugins/refdiff/refdiff.go:
##########
@@ -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)

Review Comment:
   why swallow the err?



##########
plugins/refdiff/tasks/refdiff_task_data.go:
##########
@@ -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":

Review Comment:
   I think `alphabetically` is too long for an arg



##########
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)

Review Comment:
   same as above



##########
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) {

Review Comment:
   please write comment



##########
plugins/refdiff/refdiff.go:
##########
@@ -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")

Review Comment:
   if we don't process the returned err, why do we call `MarkFlagRequired`
   



##########
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) {

Review Comment:
   and what is the return?



##########
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) {

Review Comment:
   please write comment to describe what does this func do



##########
plugins/refdiff/tasks/refdiff_task_data.go:
##########
@@ -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) {

Review Comment:
   please write comment



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] warren830 merged pull request #2096: Add TagsPattern for use Pattern in refdiff

Posted by GitBox <gi...@apache.org>.
warren830 merged PR #2096:
URL: https://github.com/apache/incubator-devlake/pull/2096


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] warren830 commented on a diff in pull request #2096: Add TagsPattern for use Pattern in refdiff

Posted by GitBox <gi...@apache.org>.
warren830 commented on code in PR #2096:
URL: https://github.com/apache/incubator-devlake/pull/2096#discussion_r891834056


##########
plugins/refdiff/README.md:
##########
@@ -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

Review Comment:
   The following json supports both calculateCommitsDiff and calculateIssuesDiff
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] klesh commented on a diff in pull request #2096: Add TagsPattern for use Pattern in refdiff

Posted by GitBox <gi...@apache.org>.
klesh commented on code in PR #2096:
URL: https://github.com/apache/incubator-devlake/pull/2096#discussion_r893124268


##########
plugins/refdiff/refdiff.go:
##########
@@ -62,6 +64,16 @@ func (plugin RefDiff) PrepareTaskData(taskCtx core.TaskContext, options map[stri
 		return nil, err
 	}
 
+	db := taskCtx.GetDb()

Review Comment:
   Use `GetDal` instead



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] warren830 commented on a diff in pull request #2096: Add TagsPattern for use Pattern in refdiff

Posted by GitBox <gi...@apache.org>.
warren830 commented on code in PR #2096:
URL: https://github.com/apache/incubator-devlake/pull/2096#discussion_r891838603


##########
plugins/refdiff/README.md:
##########
@@ -90,6 +84,28 @@ and if you want to perform certain subtasks.
   ]
 ]
 ```
+Or you can use tagsPattern to match the tags you want

Review Comment:
   You can also use the following args:
   tagsPattern: use regex pattern to match the tags you want
   tagOrder: (either `alphabetically` or `reverse alphabetically`) set how you want to order the tags
   tagLimit: limit the count of matching



##########
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) {

Review Comment:
   and what is the [][2]string? Or define a Pairs to replace [2]string



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org