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/11/16 09:35:15 UTC

[GitHub] [incubator-devlake] abeizn opened a new pull request, #3747: feat: project deploy commit diff

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

   ### ⚠️ Pre Checklist
   
   > Please complete _ALL_ items in this checklist, and remove before submitting
   
   - [ ] I have read through the [Contributing Documentation](https://devlake.apache.org/community/).
   - [ ] I have added relevant tests.
   - [ ] I have added relevant documentation.
   - [ ] I will add labels to the PR, such as `pr-type/bug-fix`, `pr-type/feature-development`, etc.
   
   ### Summary
   enhance refdiff to support project. 
   1. supprot new commits_diffs table.
   2. adapt project params
   
   ### Does this close any open issues?
   related to #3685 
   
   ### Screenshots
   Include any relevant screenshots here.
   
   ### 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] abeizn merged pull request #3747: feat: project deploy commit diff

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


-- 
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] likyh commented on a diff in pull request #3747: feat: project deploy commit diff

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


##########
plugins/refdiff/tasks/project_deployment_commit_diff_calculator.go:
##########
@@ -0,0 +1,190 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"reflect"
+
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/models/domainlayer/code"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/refdiff/utils"
+)
+
+func CalculateProjectDeploymentCommitsDiff(taskCtx core.SubTaskContext) errors.Error {
+	data := taskCtx.GetData().(*RefdiffTaskData)
+	db := taskCtx.GetDal()
+	ctx := taskCtx.GetContext()
+	logger := taskCtx.GetLogger()
+
+	projectName := data.Options.ProjectName
+	if projectName == "" {
+		return nil
+	}
+
+	cursorScope, err := db.Cursor(
+		dal.Select("row_id"),
+		dal.From("project_mapping"),
+		dal.Where("project_name = ?", projectName),
+	)
+	if err != nil {
+		return err
+	}
+	defer cursorScope.Close()
+
+	var ExistFinishedCommitDiff []code.FinishedCommitsDiffs
+	err = db.All(&ExistFinishedCommitDiff,
+		dal.Select("*"),
+		dal.From("finished_commits_diffs"),
+	)
+	if err != nil {
+		return err
+	}
+
+	for cursorScope.Next() {
+		var scopeId string
+		err = errors.Convert(cursorScope.Scan(&scopeId))
+		if err != nil {
+			return err
+		}
+
+		var commitShaList []string

Review Comment:
   name it as `pipelineCommitShaList`?



##########
plugins/refdiff/tasks/project_deployment_commit_diff_calculator.go:
##########
@@ -0,0 +1,190 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"reflect"
+
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/models/domainlayer/code"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/refdiff/utils"
+)
+
+func CalculateProjectDeploymentCommitsDiff(taskCtx core.SubTaskContext) errors.Error {
+	data := taskCtx.GetData().(*RefdiffTaskData)
+	db := taskCtx.GetDal()
+	ctx := taskCtx.GetContext()
+	logger := taskCtx.GetLogger()
+
+	projectName := data.Options.ProjectName
+	if projectName == "" {
+		return nil
+	}
+
+	cursorScope, err := db.Cursor(
+		dal.Select("row_id"),
+		dal.From("project_mapping"),
+		dal.Where("project_name = ?", projectName),
+	)
+	if err != nil {
+		return err
+	}
+	defer cursorScope.Close()
+
+	var ExistFinishedCommitDiff []code.FinishedCommitsDiffs
+	err = db.All(&ExistFinishedCommitDiff,
+		dal.Select("*"),
+		dal.From("finished_commits_diffs"),
+	)
+	if err != nil {
+		return err
+	}
+
+	for cursorScope.Next() {
+		var scopeId string
+		err = errors.Convert(cursorScope.Scan(&scopeId))
+		if err != nil {
+			return err
+		}
+
+		var commitShaList []string
+		err := db.All(&commitShaList,
+			dal.Select("commit_sha"),
+			dal.From("cicd_tasks ct"),
+			dal.Join("left join cicd_pipelines cp on cp.id = ct.pipeline_id"),
+			dal.Join("left join cicd_pipeline_commits cpc on cpc.pipeline_id = cp.id"),
+			dal.Where("ct.type = ? and commit_sha != ? and repo_id=? ", "DEPLOYMENT", "", scopeId),
+			dal.Orderby("ct.started_date"),
+		)
+		if err != nil {
+			return err
+		}
+
+		var commitPairs []code.CommitsDiff
+		var finishedCommitDiffs []code.FinishedCommitsDiffs
+
+		for i := 0; i < len(commitShaList)-1; i++ {
+			for _, item := range ExistFinishedCommitDiff {
+				if commitShaList[i+1] == item.NewCommitSha && commitShaList[i] == item.OldCommitSha {
+					i++

Review Comment:
   break?



##########
plugins/refdiff/tasks/project_deployment_commit_diff_calculator.go:
##########
@@ -0,0 +1,190 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"reflect"
+
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/models/domainlayer/code"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/refdiff/utils"
+)
+
+func CalculateProjectDeploymentCommitsDiff(taskCtx core.SubTaskContext) errors.Error {
+	data := taskCtx.GetData().(*RefdiffTaskData)
+	db := taskCtx.GetDal()
+	ctx := taskCtx.GetContext()
+	logger := taskCtx.GetLogger()
+
+	projectName := data.Options.ProjectName
+	if projectName == "" {
+		return nil
+	}
+
+	cursorScope, err := db.Cursor(
+		dal.Select("row_id"),
+		dal.From("project_mapping"),
+		dal.Where("project_name = ?", projectName),
+	)
+	if err != nil {
+		return err
+	}
+	defer cursorScope.Close()
+
+	var ExistFinishedCommitDiff []code.FinishedCommitsDiffs

Review Comment:
   use camelcase `existFinishedCommitDiff`?



##########
plugins/refdiff/tasks/project_deployment_commit_diff_calculator.go:
##########
@@ -0,0 +1,190 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"reflect"
+
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/models/domainlayer/code"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/refdiff/utils"
+)
+
+func CalculateProjectDeploymentCommitsDiff(taskCtx core.SubTaskContext) errors.Error {
+	data := taskCtx.GetData().(*RefdiffTaskData)
+	db := taskCtx.GetDal()
+	ctx := taskCtx.GetContext()
+	logger := taskCtx.GetLogger()
+
+	projectName := data.Options.ProjectName
+	if projectName == "" {
+		return nil
+	}
+
+	cursorScope, err := db.Cursor(
+		dal.Select("row_id"),
+		dal.From("project_mapping"),
+		dal.Where("project_name = ?", projectName),
+	)
+	if err != nil {
+		return err
+	}
+	defer cursorScope.Close()
+
+	var ExistFinishedCommitDiff []code.FinishedCommitsDiffs
+	err = db.All(&ExistFinishedCommitDiff,
+		dal.Select("*"),
+		dal.From("finished_commits_diffs"),
+	)
+	if err != nil {
+		return err
+	}
+
+	for cursorScope.Next() {
+		var scopeId string
+		err = errors.Convert(cursorScope.Scan(&scopeId))
+		if err != nil {
+			return err
+		}
+
+		var commitShaList []string
+		err := db.All(&commitShaList,
+			dal.Select("commit_sha"),
+			dal.From("cicd_tasks ct"),
+			dal.Join("left join cicd_pipelines cp on cp.id = ct.pipeline_id"),
+			dal.Join("left join cicd_pipeline_commits cpc on cpc.pipeline_id = cp.id"),
+			dal.Where("ct.type = ? and commit_sha != ? and repo_id=? ", "DEPLOYMENT", "", scopeId),
+			dal.Orderby("ct.started_date"),
+		)
+		if err != nil {
+			return err
+		}
+
+		var commitPairs []code.CommitsDiff
+		var finishedCommitDiffs []code.FinishedCommitsDiffs
+
+		for i := 0; i < len(commitShaList)-1; i++ {
+			for _, item := range ExistFinishedCommitDiff {
+				if commitShaList[i+1] == item.NewCommitSha && commitShaList[i] == item.OldCommitSha {
+					i++
+				}
+			}
+			commitPairs = append(commitPairs, code.CommitsDiff{NewCommitSha: commitShaList[i+1], OldCommitSha: commitShaList[i]})
+			finishedCommitDiffs = append(finishedCommitDiffs, code.FinishedCommitsDiffs{NewCommitSha: commitShaList[i+1], OldCommitSha: commitShaList[i]})
+		}
+
+		insertCountLimitOfDeployCommitsDiff := int(65535 / reflect.ValueOf(code.CommitsDiff{}).NumField())
+		commitNodeGraph := utils.NewCommitNodeGraph()
+
+		var CommitParentList []code.CommitParent
+		err = db.All(&CommitParentList,
+			dal.Select("cp.*"),
+			dal.Join("LEFT JOIN repo_commits rc ON (rc.commit_sha = cp.commit_sha)"),
+			dal.From("commit_parents cp"),
+			dal.Where("rc.repo_id = ?", scopeId),
+		)
+		if err != nil {
+			return err
+		}
+
+		for i := 0; i < len(CommitParentList); i++ {
+			commitNodeGraph.AddParent(CommitParentList[i].CommitSha, CommitParentList[i].ParentCommitSha)
+		}
+		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.CommitsDiff{}
+		lenCommitPairs := len(commitPairs)
+		taskCtx.SetProgress(0, lenCommitPairs)
+
+		for _, pair := range commitPairs {
+			select {
+			case <-ctx.Done():
+				return errors.Convert(ctx.Err())
+			default:
+			}
+
+			commitsDiff.NewCommitSha = pair.NewCommitSha
+			commitsDiff.OldCommitSha = pair.OldCommitSha
+
+			if commitsDiff.NewCommitSha == commitsDiff.OldCommitSha {
+				// different deploy might point to a same commit, it is ok
+				logger.Info(
+					"skipping ref pair due to they are the same %s",
+					commitsDiff.NewCommitSha,
+				)
+				continue
+			}
+
+			lostSha, oldCount, newCount := commitNodeGraph.CalculateLostSha(commitsDiff.OldCommitSha, commitsDiff.NewCommitSha)
+
+			commitsDiffs := []code.CommitsDiff{}
+			commitsDiff.SortingIndex = 1
+			for _, sha := range lostSha {
+				commitsDiff.CommitSha = sha
+				commitsDiffs = append(commitsDiffs, *commitsDiff)
+
+				// sql limit placeholders count only 65535
+				if commitsDiff.SortingIndex%insertCountLimitOfDeployCommitsDiff == 0 {
+					logger.Info("commitsDiffs count in limited[%d] index[%d]--exec and clean", len(commitsDiffs), commitsDiff.SortingIndex)
+					err = db.CreateIfNotExist(commitsDiffs)
+					if err != nil {
+						return err
+					}
+					commitsDiffs = []code.CommitsDiff{}
+				}
+
+				commitsDiff.SortingIndex++
+			}
+
+			if len(commitsDiffs) > 0 {
+				logger.Info("insert data count [%d]", len(commitsDiffs))
+				err = db.CreateIfNotExist(commitsDiffs)
+				if err != nil {
+					return err
+				}
+				err = db.CreateIfNotExist(finishedCommitDiffs)

Review Comment:
   if `commitsDiffs` save and clean in Line#145-154 `commitsDiff.SortingIndex%insertCountLimitOfDeployCommitsDiff == 0`, `finishedCommitDiffs` will not be saved?



-- 
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 #3747: feat: project deploy commit diff

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


##########
plugins/refdiff/e2e/raw_tables/cicd_pipeline_commits.csv:
##########
@@ -0,0 +1,6 @@
+pipeline_id,created_at,updated_at,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark,commit_sha,branch,repo_id,repo_url
+github:GithubRun:1:1,2022-11-17 06:35:35.758,2022-11-17 06:35:35.758,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake-website""}",_raw_github_api_runs,1496,"",commit_sha1,main,github:GithubRepo:1:484251804,""
+github:GithubRun:1:2,2022-11-17 06:35:35.758,2022-11-17 06:35:35.758,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake-website""}",_raw_github_api_runs,1495,"",commit_sha2,main,github:GithubRepo:1:484251804,""
+github:GithubRun:1:3,2022-11-17 06:35:35.758,2022-11-17 06:35:35.758,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake-website""}",_raw_github_api_runs,1494,"",commit_sha3,fix-edit-url,github:GithubRepo:1:484251804,""
+github:GithubRun:1:4,2022-11-17 06:35:35.758,2022-11-17 06:35:35.758,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake-website""}",_raw_github_api_runs,1493,"",commit_sha4,main,github:GithubRepo:1:484251804,""
+github:GithubRun:1:5,2022-11-17 06:35:35.758,2022-11-17 06:35:35.758,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake-website""}",_raw_github_api_runs,1492,"",commit_sha5,main,github:GithubRepo:1:484251804,""

Review Comment:
   And please ignore irrelevant fields like raw_*



-- 
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 #3747: feat: project deploy commit diff

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


##########
plugins/refdiff/e2e/raw_tables/cicd_pipeline_commits.csv:
##########
@@ -0,0 +1,6 @@
+pipeline_id,created_at,updated_at,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark,commit_sha,branch,repo_id,repo_url
+github:GithubRun:1:1,2022-11-17 06:35:35.758,2022-11-17 06:35:35.758,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake-website""}",_raw_github_api_runs,1496,"",commit_sha1,main,github:GithubRepo:1:484251804,""
+github:GithubRun:1:2,2022-11-17 06:35:35.758,2022-11-17 06:35:35.758,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake-website""}",_raw_github_api_runs,1495,"",commit_sha2,main,github:GithubRepo:1:484251804,""
+github:GithubRun:1:3,2022-11-17 06:35:35.758,2022-11-17 06:35:35.758,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake-website""}",_raw_github_api_runs,1494,"",commit_sha3,fix-edit-url,github:GithubRepo:1:484251804,""
+github:GithubRun:1:4,2022-11-17 06:35:35.758,2022-11-17 06:35:35.758,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake-website""}",_raw_github_api_runs,1493,"",commit_sha4,main,github:GithubRepo:1:484251804,""
+github:GithubRun:1:5,2022-11-17 06:35:35.758,2022-11-17 06:35:35.758,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake-website""}",_raw_github_api_runs,1492,"",commit_sha5,main,github:GithubRepo:1:484251804,""

Review Comment:
   Please add a blank line



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