You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@devlake.apache.org by zh...@apache.org on 2022/11/21 02:43:26 UTC

[incubator-devlake] branch main updated: feat(dora): update logic for change lead time (#3742)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 8eb889106 feat(dora): update logic for change lead time (#3742)
8eb889106 is described below

commit 8eb88910643d07f3e18c3e683a714d46a8d0985b
Author: Warren Chen <yi...@merico.dev>
AuthorDate: Mon Nov 21 10:43:21 2022 +0800

    feat(dora): update logic for change lead time (#3742)
    
    * feat(dora): update logic for change lead time
    
    closes #3516
    
    * feat(dora): update e2e according to review
---
 models/domainlayer/code/pull_request.go            |  35 ++--
 .../domainlayer/crossdomain/project_pr_metric.go   |  39 +++++
 .../20221111_add_project_pr_metric.go              |  63 +++++++
 .../archived/project_pr_metrics.go                 |  35 ++++
 models/migrationscripts/register.go                |   1 +
 .../dora/e2e/calculate_change_lead_time_test.go    |  61 ++-----
 .../dora/e2e/raw_tables/cicd_pipeline_commits.csv  |  17 ++
 plugins/dora/e2e/raw_tables/cicd_scopes.csv        |   4 +
 .../e2e/raw_tables/cicd_tasks_changeleadtime.csv   |  19 +++
 plugins/dora/e2e/raw_tables/commits_diffs.csv      |  13 ++
 plugins/dora/e2e/raw_tables/project_mapping.csv    |   6 +
 plugins/dora/e2e/raw_tables/pull_requests.csv      |  22 +--
 plugins/dora/e2e/raw_tables/repos.csv              |   7 +-
 .../e2e/snapshot_tables/project_pr_metrics.csv     |   7 +
 plugins/dora/e2e/snapshot_tables/pull_requests.csv |  10 --
 plugins/dora/tasks/change_lead_time_calculator.go  | 183 ++++++++++++---------
 plugins/dora/tasks/task_data.go                    |   2 +
 17 files changed, 359 insertions(+), 165 deletions(-)

diff --git a/models/domainlayer/code/pull_request.go b/models/domainlayer/code/pull_request.go
index 126884a80..e17e6970d 100644
--- a/models/domainlayer/code/pull_request.go
+++ b/models/domainlayer/code/pull_request.go
@@ -33,28 +33,19 @@ type PullRequest struct {
 	Url         string `gorm:"type:varchar(255)"`
 	AuthorName  string `gorm:"type:varchar(100)"`
 	//User		   domainUser.User `gorm:"foreignKey:AuthorId"`
-	AuthorId           string `gorm:"type:varchar(100)"`
-	ParentPrId         string `gorm:"index;type:varchar(100)"`
-	PullRequestKey     int
-	CreatedDate        time.Time
-	MergedDate         *time.Time
-	ClosedDate         *time.Time
-	Type               string `gorm:"type:varchar(100)"`
-	Component          string `gorm:"type:varchar(100)"`
-	MergeCommitSha     string `gorm:"type:varchar(40)"`
-	HeadRef            string `gorm:"type:varchar(255)"`
-	BaseRef            string `gorm:"type:varchar(255)"`
-	BaseCommitSha      string `gorm:"type:varchar(40)"`
-	HeadCommitSha      string `gorm:"type:varchar(40)"`
-	CodingTimespan     *int64
-	ReviewLag          *int64
-	ReviewTimespan     *int64
-	DeployTimespan     *int64
-	ChangeTimespan     *int64
-	OrigCodingTimespan int64
-	OrigReviewLag      int64
-	OrigReviewTimespan int64
-	OrigDeployTimespan int64
+	AuthorId       string `gorm:"type:varchar(100)"`
+	ParentPrId     string `gorm:"index;type:varchar(100)"`
+	PullRequestKey int
+	CreatedDate    time.Time
+	MergedDate     *time.Time
+	ClosedDate     *time.Time
+	Type           string `gorm:"type:varchar(100)"`
+	Component      string `gorm:"type:varchar(100)"`
+	MergeCommitSha string `gorm:"type:varchar(40)"`
+	HeadRef        string `gorm:"type:varchar(255)"`
+	BaseRef        string `gorm:"type:varchar(255)"`
+	BaseCommitSha  string `gorm:"type:varchar(40)"`
+	HeadCommitSha  string `gorm:"type:varchar(40)"`
 }
 
 func (PullRequest) TableName() string {
diff --git a/models/domainlayer/crossdomain/project_pr_metric.go b/models/domainlayer/crossdomain/project_pr_metric.go
new file mode 100644
index 000000000..f6c43f22b
--- /dev/null
+++ b/models/domainlayer/crossdomain/project_pr_metric.go
@@ -0,0 +1,39 @@
+/*
+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 crossdomain
+
+import (
+	"github.com/apache/incubator-devlake/models/domainlayer"
+)
+
+type ProjectPrMetric struct {
+	domainlayer.DomainEntity
+	ProjectName    string `gorm:"primaryKey;type:varchar(100)"`
+	FirstCommitSha string
+	CodingTimespan *int64
+	FirstReviewId  string
+	ReviewLag      *int64
+	ReviewTimespan *int64
+	DeploymentId   string
+	DeployTimespan *int64
+	ChangeTimespan *int64
+}
+
+func (ProjectPrMetric) TableName() string {
+	return "project_pr_metrics"
+}
diff --git a/models/migrationscripts/20221111_add_project_pr_metric.go b/models/migrationscripts/20221111_add_project_pr_metric.go
new file mode 100644
index 000000000..68f5782e0
--- /dev/null
+++ b/models/migrationscripts/20221111_add_project_pr_metric.go
@@ -0,0 +1,63 @@
+/*
+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 migrationscripts
+
+import (
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/helpers/migrationhelper"
+	"github.com/apache/incubator-devlake/models/migrationscripts/archived"
+	"github.com/apache/incubator-devlake/plugins/core"
+)
+
+type addProjectPrMetric struct{}
+
+func (u *addProjectPrMetric) Up(baseRes core.BasicRes) errors.Error {
+	db := baseRes.GetDal()
+	err := migrationhelper.AutoMigrateTables(
+		baseRes,
+		&archived.ProjectPrMetric{},
+	)
+	if err != nil {
+		return err
+	}
+	prColums := []string{
+		`coding_timespan`,
+		`review_lag`,
+		`review_timespan`,
+		`deploy_timespan`,
+		`change_timespan`,
+		`orig_coding_timespan`,
+		`orig_review_lag`,
+		`orig_review_timespan`,
+		`orig_deploy_timespan`,
+	}
+	err = db.DropColumns(`pull_requests`, prColums...)
+	if err != nil {
+		return err
+	}
+	err = db.DropColumns(`cicd_pipeline_commits`, "repo_url")
+	return err
+}
+
+func (*addProjectPrMetric) Version() uint64 {
+	return 20221111000001
+}
+
+func (*addProjectPrMetric) Name() string {
+	return "add project pr metric tables"
+}
diff --git a/models/migrationscripts/archived/project_pr_metrics.go b/models/migrationscripts/archived/project_pr_metrics.go
new file mode 100644
index 000000000..03e0f5252
--- /dev/null
+++ b/models/migrationscripts/archived/project_pr_metrics.go
@@ -0,0 +1,35 @@
+/*
+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 archived
+
+type ProjectPrMetric struct {
+	DomainEntity
+	ProjectName    string `gorm:"primaryKey;type:varchar(100)"`
+	FirstReviewId  string
+	FirstCommitSha string
+	CodingTimespan *int64
+	ReviewLag      *int64
+	ReviewTimespan *int64
+	DeploymentId   string
+	DeployTimespan *int64
+	ChangeTimespan *int64
+}
+
+func (ProjectPrMetric) TableName() string {
+	return "project_pr_metrics"
+}
diff --git a/models/migrationscripts/register.go b/models/migrationscripts/register.go
index 7e62d2a51..e28c33174 100644
--- a/models/migrationscripts/register.go
+++ b/models/migrationscripts/register.go
@@ -56,6 +56,7 @@ func All() []core.MigrationScript {
 		new(addCicdScope),
 		new(addSkipOnFail),
 		new(modifyCommitsDiffs),
+		new(addProjectPrMetric),
 		new(addProjectTables),
 		new(addProjectToBluePrint),
 	}
diff --git a/plugins/dora/e2e/calculate_change_lead_time_test.go b/plugins/dora/e2e/calculate_change_lead_time_test.go
index 599d8c483..9667a5040 100644
--- a/plugins/dora/e2e/calculate_change_lead_time_test.go
+++ b/plugins/dora/e2e/calculate_change_lead_time_test.go
@@ -18,13 +18,14 @@ limitations under the License.
 package e2e
 
 import (
-	"testing"
-
 	"github.com/apache/incubator-devlake/helpers/e2ehelper"
+	"github.com/apache/incubator-devlake/models/common"
 	"github.com/apache/incubator-devlake/models/domainlayer/code"
+	"github.com/apache/incubator-devlake/models/domainlayer/crossdomain"
 	"github.com/apache/incubator-devlake/models/domainlayer/devops"
 	"github.com/apache/incubator-devlake/plugins/dora/impl"
 	"github.com/apache/incubator-devlake/plugins/dora/tasks"
+	"testing"
 )
 
 func TestCalculateCLTimeDataFlow(t *testing.T) {
@@ -33,7 +34,7 @@ func TestCalculateCLTimeDataFlow(t *testing.T) {
 
 	taskData := &tasks.DoraTaskData{
 		Options: &tasks.DoraOptions{
-			RepoId: "github:GithubRepo:1:384111310",
+			ProjectName: "project1",
 			TransformationRules: tasks.TransformationRules{
 				ProductionPattern: "(?i)deploy",
 			},
@@ -43,52 +44,22 @@ func TestCalculateCLTimeDataFlow(t *testing.T) {
 	dataflowTester.FlushTabler(&code.PullRequest{})
 
 	// import raw data table
-	dataflowTester.ImportCsvIntoTabler("./raw_tables/lake_cicd_tasks.csv", &devops.CICDTask{})
+	dataflowTester.ImportCsvIntoTabler("./raw_tables/cicd_tasks_changeleadtime.csv", &devops.CICDTask{})
 	dataflowTester.ImportCsvIntoTabler("./raw_tables/pull_requests.csv", &code.PullRequest{})
+	dataflowTester.ImportCsvIntoTabler("./raw_tables/commits_diffs.csv", &code.CommitsDiff{})
+	dataflowTester.ImportCsvIntoTabler("./raw_tables/cicd_pipeline_commits.csv", &devops.CiCDPipelineCommit{})
+	dataflowTester.ImportCsvIntoTabler("./raw_tables/project_mapping.csv", &crossdomain.ProjectMapping{})
+	dataflowTester.ImportCsvIntoTabler("./raw_tables/commits.csv", &code.Commit{})
 	dataflowTester.ImportCsvIntoTabler("./raw_tables/pull_request_comments.csv", &code.PullRequestComment{})
 	dataflowTester.ImportCsvIntoTabler("./raw_tables/pull_request_commits.csv", &code.PullRequestCommit{})
-	dataflowTester.ImportCsvIntoTabler("./raw_tables/commits.csv", &code.Commit{})
+	dataflowTester.ImportCsvIntoTabler("./raw_tables/repos.csv", &code.Repo{})
+	dataflowTester.ImportCsvIntoTabler("./raw_tables/cicd_scopes.csv", &devops.CicdScope{})
 
 	// verify converter
+	dataflowTester.FlushTabler(&crossdomain.ProjectPrMetric{})
 	dataflowTester.Subtask(tasks.CalculateChangeLeadTimeMeta, taskData)
-	dataflowTester.VerifyTable(
-		code.PullRequest{},
-		"./snapshot_tables/pull_requests.csv",
-		[]string{
-			"id",
-			"_raw_data_params",
-			"_raw_data_table",
-			"_raw_data_id",
-			"_raw_data_remark",
-			"base_repo_id",
-			"head_repo_id",
-			"status",
-			"title",
-			"description",
-			"url",
-			"author_name",
-			"author_id",
-			"parent_pr_id",
-			"pull_request_key",
-			"created_date",
-			"merged_date",
-			"closed_date",
-			"type",
-			"component",
-			"merge_commit_sha",
-			"head_ref",
-			"base_ref",
-			"base_commit_sha",
-			"head_commit_sha",
-			"coding_timespan",
-			"review_lag",
-			"review_timespan",
-			"deploy_timespan",
-			"change_timespan",
-			"orig_coding_timespan",
-			"orig_review_lag",
-			"orig_review_timespan",
-			"orig_deploy_timespan",
-		},
-	)
+	dataflowTester.VerifyTableWithOptions(&crossdomain.ProjectPrMetric{}, e2ehelper.TableOptions{
+		CSVRelPath:  "./snapshot_tables/project_pr_metrics.csv",
+		IgnoreTypes: []interface{}{common.NoPKModel{}},
+	})
 }
diff --git a/plugins/dora/e2e/raw_tables/cicd_pipeline_commits.csv b/plugins/dora/e2e/raw_tables/cicd_pipeline_commits.csv
new file mode 100644
index 000000000..3c5029696
--- /dev/null
+++ b/plugins/dora/e2e/raw_tables/cicd_pipeline_commits.csv
@@ -0,0 +1,17 @@
+pipeline_id,commit_sha,branch,repo_id,repo_url
+pipeline110,commit300,z5z07j8qDO,repo1,
+pipeline111,commit301,7j3t5IFWb6,repo1,
+pipeline112,commit302,VBSqoF8WIh,repo1,
+pipeline113,commit303,VXyuBwhkIE,repo1,
+pipeline114,commit304,hJlTujPgFq,repo1,
+pipeline115,commit305,I8UmFLEYPO,repo2,
+pipeline116,commit306,oMhYUwGyVX,repo1,
+pipeline117,commit307,SQGBUB075n,repo1,
+pipeline118,commit308,XVxzMFClc5,repo1,
+pipeline119,commit309,Pr9qLgnNPD,repo1,
+pipeline18,commit22,QXxXzo0V6U,repo2,
+pipeline19,commit22,ey6y4W9jdm,repo3,
+pipeline19,commit24,wUYEULZ850,repo2,
+pipeline20,commit31,ptVxHa6jwh,repo1,
+pipeline21,commit32,RcX46MDXRQ,repo1,
+pipeline24,commit22,8LaxgnEhid,repo2,
diff --git a/plugins/dora/e2e/raw_tables/cicd_scopes.csv b/plugins/dora/e2e/raw_tables/cicd_scopes.csv
new file mode 100644
index 000000000..29cbc88d2
--- /dev/null
+++ b/plugins/dora/e2e/raw_tables/cicd_scopes.csv
@@ -0,0 +1,4 @@
+id
+cicd1
+cicd2
+cicd3
diff --git a/plugins/dora/e2e/raw_tables/cicd_tasks_changeleadtime.csv b/plugins/dora/e2e/raw_tables/cicd_tasks_changeleadtime.csv
new file mode 100644
index 000000000..b8acabbae
--- /dev/null
+++ b/plugins/dora/e2e/raw_tables/cicd_tasks_changeleadtime.csv
@@ -0,0 +1,19 @@
+id,name,pipeline_id,status,result,type,environment,duration_sec,started_date,finished_date,cicd_scope_id
+task10,deployxIG,pipeline110,DONE,SUCCESS,DEPLOYMENT,PRODUCTION,,2022-07-19 22:06:28,2022-11-13 22:37:21,cicd1
+task11,deploya,pipeline111,DONE,SUCCESS,DEPLOYMENT,PRODUCTION,,2022-08-06 14:06:50,2022-11-13 00:07:21,cicd1
+task12,deployc,pipeline112,DONE,SUCCESS,DEPLOYMENT,PRODUCTION,,2022-08-23 17:44:05,2022-11-02 07:21:09,cicd2
+task13,deploy,pipeline113,DONE,SUCCESS,DEPLOYMENT,PRODUCTION,,2022-08-30 23:45:29,2022-11-28 00:46:47,cicd1
+task14,deployp0;,pipeline114,DONE,SUCCESS,DEPLOYMENT,PRODUCTION,,2022-09-07 02:49:26,2022-11-16 20:34:01,cicd1
+task15,deployY{,pipeline115,DONE,SUCCESS,DEPLOYMENT,PRODUCTION,,2022-09-27 01:07:50,2022-11-19 07:17:33,cicd2
+task16,deploy8',pipeline116,DONE,SUCCESS,DEPLOYMENT,PRODUCTION,,2022-09-30 21:05:38,2022-11-08 07:56:03,cicd1
+task17,deployKd%,pipeline117,IN_PROGRESS,,DEPLOYMENT,PRODUCTION,,2022-10-09 06:42:02,,cicd1
+task19,deploy1,pipeline119,DONE,FAILURE,DEPLOYMENT,PRODUCTION,,2022-10-24 18:41:04,2022-11-24 04:26:48,cicd1
+task21,deploy^^.,pipeline39,DONE,FAILURE,DEPLOYMENT,STAGING,,2004-01-10 03:31:11,2022-11-28 20:41:59,cicd1
+task22,deploy,pipeline35,DONE,SUCCESS,DEPLOYMENT,TESTING,,2000-10-25 09:57:28,2022-11-28 21:24:02,cicd1
+task23,deploy,pipeline36,IN_PROGRESS,,DEPLOYMENT,TESTING,,2005-02-07 11:03:27,2022-11-05 18:18:03,cicd1
+task24,deploym,pipeline12,IN_PROGRESS,,DEPLOYMENT,STAGING,,2015-08-11 19:58:06,2022-11-01 22:31:56,cicd1
+task25,deploy$p<,pipeline26,DONE,FAILURE,,TESTING,,2014-02-06 13:42:43,2022-11-30 08:01:38,cicd3
+task26,deployb>@,pipeline20,IN_PROGRESS,,DEPLOYMENT,STAGING,,2016-08-26 05:41:49,2022-11-15 07:31:46,cicd2
+task27,deployKfn,pipeline37,DONE,SUCCESS,DEPLOYMENT,STAGING,,2003-12-13 23:19:14,2022-11-11 18:29:31,cicd2
+task28,deployl?,pipeline29,IN_PROGRESS,,DEPLOYMENT,TESTING,,2007-01-19 01:13:39,2022-11-24 05:39:46,cicd3
+task29,deployUb,pipeline27,IN_PROGRESS,,DEPLOYMENT,TESTING,,2006-05-20 18:17:13,2022-11-28 10:13:51,cicd2
diff --git a/plugins/dora/e2e/raw_tables/commits_diffs.csv b/plugins/dora/e2e/raw_tables/commits_diffs.csv
new file mode 100644
index 000000000..f81ed28c3
--- /dev/null
+++ b/plugins/dora/e2e/raw_tables/commits_diffs.csv
@@ -0,0 +1,13 @@
+commit_sha,new_commit_sha,old_commit_sha,sorting_index
+commit200,commit301,commit300,438
+commit201,commit302,commit301,491
+commit202,commit303,commit302,808
+commit203,commit304,commit303,247
+commit204,commit305,commit304,247
+commit3,commit24,commit25,58
+commit4,commit24,commit22,802
+commit5,commit23,commit23,355
+commit6,commit25,commit21,908
+commit7,commit27,commit23,47
+commit8,commit24,commit23,51
+commit9,commit22,commit20,957
diff --git a/plugins/dora/e2e/raw_tables/project_mapping.csv b/plugins/dora/e2e/raw_tables/project_mapping.csv
new file mode 100644
index 000000000..bcc085fc9
--- /dev/null
+++ b/plugins/dora/e2e/raw_tables/project_mapping.csv
@@ -0,0 +1,6 @@
+project_name,table,row_id
+project1,cicd_scopes,cicd1
+project1,cicd_scopes,cicd2
+project1,repos,repo1
+project1,repos,repo2
+project2,cicd_scopes,cicd3
diff --git a/plugins/dora/e2e/raw_tables/pull_requests.csv b/plugins/dora/e2e/raw_tables/pull_requests.csv
index b9c1a6684..e32c680a1 100644
--- a/plugins/dora/e2e/raw_tables/pull_requests.csv
+++ b/plugins/dora/e2e/raw_tables/pull_requests.csv
@@ -1,10 +1,12 @@
-id,created_at,updated_at,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark,base_repo_id,head_repo_id,status,title,description,url,author_name,author_id,parent_pr_id,pull_request_key,created_date,merged_date,closed_date,type,component,merge_commit_sha,head_ref,base_ref,base_commit_sha,head_commit_sha,coding_timespan,review_lag,review_timespan,deploy_timespan,change_timespan,orig_coding_timespan,orig_review_lag,orig_review_timespan,orig_deploy_timespan
-github:GithubPullRequest:1:1043463302,2022-09-15 05:36:39.856,2022-09-15 05:36:39.856,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,13176,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,feat: implement API for plugin `customize`,"""# Summary\r\n\r\nfulfill the requirement \r\n#2880 ([Feature][customize] implement API for plugin customize) \r\n#2985 ([Feature][customize] new sub-task ExtractCustomizedFields for [...]
-github:GithubPullRequest:1:1048233599,2022-09-15 05:36:39.856,2022-09-15 05:36:39.856,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,13211,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,feat: new plugin for gitea,"""# Summary\r\n[Gitea](https://gitea.io/) is an open-source software package for hosting software development version control using Git as well as other collaborative features like bug tracking, code [...]
-github:GithubPullRequest:1:1049191985,2022-09-15 05:36:40.170,2022-09-15 05:36:40.170,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,12680,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,[issue-2908]: Bump lake-builder version to 0.0.8,"""### ⚠️ Pre Checklist\r\n\r\n> Please complete _ALL_ items in this checklist, and remove before submitting\r\n\r\n- [x] I have read through the [Contributing Documentation](htt [...]
-github:GithubPullRequest:1:1051243958,2022-09-15 05:36:40.170,2022-09-15 05:36:40.170,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,12691,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,fix: golangci-lint error,"""# Summary\r\n\r\ngolint-ci was down for a while, this PR try to fix all missed linting errors during the period.\r\n\r\n### Screenshots\r\n![image](https://user-images.githubusercontent.com/61080/189 [...]
-github:GithubPullRequest:1:1051273681,2022-09-15 05:36:40.170,2022-09-15 05:36:40.170,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,12693,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,fix: update pipelineId,"""# Summary\r\n\r\nupdate pipelineId\r\n\r\n### Does this close any open issues?\r\nrelated to #2998 \r\n\r\n### Screenshots\r\n![image](https://user-images.githubusercontent.com/101256042/189307562-88fc [...]
-github:GithubPullRequest:1:1051340471,2022-09-15 05:36:40.170,2022-09-15 05:36:40.170,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,12694,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,Jenkins fix,"""# Summary\r\nAdd TestJenkinsStagesDataFlow.\r\nAdd path to stages collect.\r\n\r\n<!--\r\nThanks for submitting a pull request!\r\n\r\nWe appreciate you spending the time to work on these changes.\r\nPlease fill  [...]
-github:GithubPullRequest:1:1051524882,2022-09-15 05:36:40.170,2022-09-15 05:36:40.170,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,12696,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,fix: task failure caused by 404,"""# Summary\r\nfix #2960 [Bug][gitihub] collect account failed by not found user\r\nIn the case of `err == ErrIgnoreAndContinue`, the `err` should not be wrapped, because on the caller side, the [...]
-github:GithubPullRequest:1:1051574863,2022-09-15 05:36:40.170,2022-09-15 05:36:40.170,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,12697,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,fix(jenkins): update e2e,"""# Summary\r\n\r\nUpdate e2e according to recent changes\r\n\r\n### Does this close any open issues?\r\nrelate to #2854\r\n\r\n\r\n### Screenshots\r\nInclude any relevant screenshots here.\r\n\r\n###  [...]
-github:GithubPullRequest:1:1051637383,2022-09-15 05:36:40.170,2022-09-15 05:36:40.170,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,12699,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,fix(gitlab): update e2e,"""# Summary\r\n\r\nupdate gitlab e2e according to recent changes\r\n\r\n### Does this close any open issues?\r\nrelate to #2871\r\n\r\n\r\n### Screenshots\r\nInclude any relevant screenshots here.\r\n\r [...]
+id,_raw_data_params,base_repo_id,created_date,merged_date,merge_commit_sha,base_commit_sha,head_commit_sha
+github:GithubPullRequest:1:1043463302,,repo1,2022-09-01 09:34:40,2022-09-09 07:52:53,commit200,,
+github:GithubPullRequest:1:1048233599,,repo1,2022-09-07 04:07:39,2022-09-10 02:35:43,commit201,,
+github:GithubPullRequest:1:1049191985,,repo1,2022-09-07 20:30:44,2022-09-09 03:39:50,commit202,,
+github:GithubPullRequest:1:1051112182,,repo1,2022-09-09 12:35:29,2022-09-09 13:32:51,commit203,,
+github:GithubPullRequest:1:1051524992,,repo1,2022-09-09 12:35:29,,commit111,,
+github:GithubPullRequest:1:1051524993,,repo1,2022-09-09 12:35:29,,commit112,,
+github:GithubPullRequest:1:1051524994,,repo1,2022-09-09 12:35:29,,commit113,,
+github:GithubPullRequest:1:1051574863,,repo1,2022-09-09 13:23:37,2022-09-09 15:12:23,commit204,,
+github:GithubPullRequest:1:1051637383,,repo1,2022-09-09 14:17:24,2022-09-09 15:16:28,commit92,,
+github:GithubPullRequest:2:1051123483,,repo4,2022-09-09 14:17:24,2022-09-09 15:16:28,commit109,,
+github:GithubPullRequest:3:1051342463,,repo3,2022-09-09 13:23:37,2022-09-09 15:12:23,commit102,,
diff --git a/plugins/dora/e2e/raw_tables/repos.csv b/plugins/dora/e2e/raw_tables/repos.csv
index 199b4f355..6cf992dec 100644
--- a/plugins/dora/e2e/raw_tables/repos.csv
+++ b/plugins/dora/e2e/raw_tables/repos.csv
@@ -1,2 +1,5 @@
-id,created_at,updated_at,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark,name,url,description,owner_id,language,forked_from,created_date,updated_date,deleted
-github:GithubRepo:1:384111310,2022-09-20 13:47:48.519,2022-09-20 13:47:48.519,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_repositories,1,"",apache/incubator-devlake,https://github.com/apache/incubator-devlake,"Apache DevLake is an open-source dev data platform to ingest, analyze, and visualize the fragmented data from DevOps tools, which can distill insights to improve engineering productivity.","",Go,"",2021-07-08 12:06:09,2022-09-20 03:25:21,0
+id
+github:GithubRepo:1:384111310
+repo1
+repo2
+repo3
diff --git a/plugins/dora/e2e/snapshot_tables/project_pr_metrics.csv b/plugins/dora/e2e/snapshot_tables/project_pr_metrics.csv
new file mode 100644
index 000000000..9391c7c40
--- /dev/null
+++ b/plugins/dora/e2e/snapshot_tables/project_pr_metrics.csv
@@ -0,0 +1,7 @@
+id,project_name,first_commit_sha,coding_timespan,first_review_id,review_lag,review_timespan,deployment_id,deploy_timespan,change_timespan
+github:GithubPullRequest:1:1043463302,project1,75ab753225b5b8acf3bc6e40e463b54b6800e7ed,,github:GithubPrReview:1:1098724785,8558,2859,task11,93134,104551
+github:GithubPullRequest:1:1048233599,project1,4f8cdefc9a9d53af16dd482c61623312eb9e9b5e,,github:GithubPrReview:1:0,194,4033,task12,76605,80832
+github:GithubPullRequest:1:1049191985,project1,4b71faf666833c0c7b915a512811e2c5e746d3de,1,github:GithubPrReview:1:1099918590,156,1712,task13,115026,116895
+github:GithubPullRequest:1:1051112182,project1,,,,,,task14,98341,98341
+github:GithubPullRequest:1:1051574863,project1,,,,,,,,
+github:GithubPullRequest:1:1051637383,project1,9d53fb594958e65456793caa1bfa8d07a7614291,1,,45,13,,,59
diff --git a/plugins/dora/e2e/snapshot_tables/pull_requests.csv b/plugins/dora/e2e/snapshot_tables/pull_requests.csv
deleted file mode 100644
index cab46ef5a..000000000
--- a/plugins/dora/e2e/snapshot_tables/pull_requests.csv
+++ /dev/null
@@ -1,10 +0,0 @@
-id,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark,base_repo_id,head_repo_id,status,title,description,url,author_name,author_id,parent_pr_id,pull_request_key,created_date,merged_date,closed_date,type,component,merge_commit_sha,head_ref,base_ref,base_commit_sha,head_commit_sha,coding_timespan,review_lag,review_timespan,deploy_timespan,change_timespan,orig_coding_timespan,orig_review_lag,orig_review_timespan,orig_deploy_timespan
-github:GithubPullRequest:1:1043463302,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,13176,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,feat: implement API for plugin `customize`,"""# Summary\r\n\r\nfulfill the requirement \r\n#2880 ([Feature][customize] implement API for plugin customize) \r\n#2985 ([Feature][customize] new sub-task ExtractCustomizedFields for plugin customize)\r\n\r\nrelated to #2802 \r\nW [...]
-github:GithubPullRequest:1:1048233599,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,13211,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,feat: new plugin for gitea,"""# Summary\r\n[Gitea](https://gitea.io/) is an open-source software package for hosting software development version control using Git as well as other collaborative features like bug tracking, code review, kanban boards, tickets, and wikis.\r\n< [...]
-github:GithubPullRequest:1:1049191985,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,12680,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,[issue-2908]: Bump lake-builder version to 0.0.8,"""### ⚠️ Pre Checklist\r\n\r\n> Please complete _ALL_ items in this checklist, and remove before submitting\r\n\r\n- [x] I have read through the [Contributing Documentation](https://devlake.apache.org/community/).\r\n- [x] I  [...]
-github:GithubPullRequest:1:1051243958,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,12691,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,fix: golangci-lint error,"""# Summary\r\n\r\ngolint-ci was down for a while, this PR try to fix all missed linting errors during the period.\r\n\r\n### Screenshots\r\n![image](https://user-images.githubusercontent.com/61080/189302830-d54ad54d-a3e6-470c-b517-9d803dbcd248.png) [...]
-github:GithubPullRequest:1:1051273681,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,12693,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,fix: update pipelineId,"""# Summary\r\n\r\nupdate pipelineId\r\n\r\n### Does this close any open issues?\r\nrelated to #2998 \r\n\r\n### Screenshots\r\n![image](https://user-images.githubusercontent.com/101256042/189307562-88fc6b6d-5daf-4b9c-8cec-0493cfe5a42c.png)\r\n\r\n### [...]
-github:GithubPullRequest:1:1051340471,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,12694,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,Jenkins fix,"""# Summary\r\nAdd TestJenkinsStagesDataFlow.\r\nAdd path to stages collect.\r\n\r\n<!--\r\nThanks for submitting a pull request!\r\n\r\nWe appreciate you spending the time to work on these changes.\r\nPlease fill out as many sections below as possible.\r\n-->\r [...]
-github:GithubPullRequest:1:1051524882,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,12696,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,fix: task failure caused by 404,"""# Summary\r\nfix #2960 [Bug][gitihub] collect account failed by not found user\r\nIn the case of `err == ErrIgnoreAndContinue`, the `err` should not be wrapped, because on the caller side, the expression `err == ErrIgnoreAndContinue`  would [...]
-github:GithubPullRequest:1:1051574863,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,12697,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,fix(jenkins): update e2e,"""# Summary\r\n\r\nUpdate e2e according to recent changes\r\n\r\n### Does this close any open issues?\r\nrelate to #2854\r\n\r\n\r\n### Screenshots\r\nInclude any relevant screenshots here.\r\n\r\n### Other Information\r\nAny other information that  [...]
-github:GithubPullRequest:1:1051637383,"{""ConnectionId"":1,""Owner"":""apache"",""Repo"":""incubator-devlake""}",_raw_github_api_pull_requests,12699,,github:GithubRepo:1:384111310,github:GithubRepo:1:384111310,closed,fix(gitlab): update e2e,"""# Summary\r\n\r\nupdate gitlab e2e according to recent changes\r\n\r\n### Does this close any open issues?\r\nrelate to #2871\r\n\r\n\r\n### Screenshots\r\nInclude any relevant screenshots here.\r\n\r\n### Other Information\r\nAny other information [...]
diff --git a/plugins/dora/tasks/change_lead_time_calculator.go b/plugins/dora/tasks/change_lead_time_calculator.go
index d6bcdb4b9..33e80f5da 100644
--- a/plugins/dora/tasks/change_lead_time_calculator.go
+++ b/plugins/dora/tasks/change_lead_time_calculator.go
@@ -19,12 +19,12 @@ package tasks
 
 import (
 	goerror "errors"
+	"github.com/apache/incubator-devlake/models/domainlayer/crossdomain"
 	"reflect"
 	"time"
 
 	"github.com/apache/incubator-devlake/errors"
 	"github.com/apache/incubator-devlake/models/domainlayer/code"
-	"github.com/apache/incubator-devlake/models/domainlayer/devops"
 	"github.com/apache/incubator-devlake/plugins/core"
 	"github.com/apache/incubator-devlake/plugins/core/dal"
 	"github.com/apache/incubator-devlake/plugins/helper"
@@ -34,9 +34,41 @@ import (
 func CalculateChangeLeadTime(taskCtx core.SubTaskContext) errors.Error {
 	db := taskCtx.GetDal()
 	log := taskCtx.GetLogger()
+	data := taskCtx.GetData().(*DoraTaskData)
+	// construct a list of tuple[task, oldPipelineCommitSha, newPipelineCommitSha, taskFinishedDate]
+	pipelineIdClauses := []dal.Clause{
+		dal.Select(`ct.id as task_id, cpc.commit_sha as new_deploy_commit_sha, 
+			ct.finished_date as task_finished_date, cpc.repo_id as repo_id`),
+		dal.From(`cicd_tasks ct`),
+		dal.Join(`left join cicd_pipeline_commits cpc on ct.pipeline_id = cpc.pipeline_id`),
+		dal.Join(`left join project_mapping pm on pm.row_id = ct.cicd_scope_id`),
+		dal.Where(`ct.environment = ? and ct.type = ? and ct.result = ? and pm.project_name = ? and pm.table = ?`,
+			"PRODUCTION", "DEPLOYMENT", "SUCCESS", data.Options.ProjectName, "cicd_scopes"),
+		dal.Orderby(`cpc.repo_id, ct.started_date `),
+	}
+	deploymentPairList := make([]deploymentPair, 0)
+	err := db.All(&deploymentPairList, pipelineIdClauses...)
+	if err != nil {
+		return err
+	}
+	// deploymentPairList[i-1].NewDeployCommitSha is deploymentPairList[i].OldDeployCommitSha
+	oldDeployCommitSha := ""
+	lastRepoId := ""
+	for i := 0; i < len(deploymentPairList); i++ {
+		// if two deployments belong to different repo, let's skip
+		if lastRepoId == deploymentPairList[i].RepoId {
+			deploymentPairList[i].OldDeployCommitSha = oldDeployCommitSha
+		} else {
+			lastRepoId = deploymentPairList[i].RepoId
+		}
+		oldDeployCommitSha = deploymentPairList[i].NewDeployCommitSha
+	}
+
+	// get prs by repo project_name
 	clauses := []dal.Clause{
 		dal.From(&code.PullRequest{}),
-		dal.Where("merged_date IS NOT NULL"),
+		dal.Join(`left join project_mapping pm on pm.row_id = pull_requests.base_repo_id`),
+		dal.Where("pull_requests.merged_date IS NOT NULL and pm.project_name = ? and pm.table = ?", data.Options.ProjectName, "repos"),
 	}
 	cursor, err := db.Cursor(clauses...)
 	if err != nil {
@@ -44,11 +76,11 @@ func CalculateChangeLeadTime(taskCtx core.SubTaskContext) errors.Error {
 	}
 	defer cursor.Close()
 
-	enricher, err := helper.NewDataConverter(helper.DataConverterArgs{
+	converter, err := helper.NewDataConverter(helper.DataConverterArgs{
 		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
-			Ctx:    taskCtx,
+			Ctx: taskCtx,
 			Params: DoraApiParams{
-				// TODO
+				ProjectName: data.Options.ProjectName,
 			},
 			Table: "pull_requests",
 		},
@@ -57,66 +89,74 @@ func CalculateChangeLeadTime(taskCtx core.SubTaskContext) errors.Error {
 		Input:        cursor,
 		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
 			pr := inputRow.(*code.PullRequest)
-			firstCommitDate, err := getFirstCommitTime(pr.Id, db)
+			firstCommit, err := getFirstCommit(pr.Id, db)
+			if err != nil {
+				return nil, err
+			}
+			projectPrMetric := &crossdomain.ProjectPrMetric{}
+			projectPrMetric.Id = pr.Id
+			projectPrMetric.ProjectName = data.Options.ProjectName
 			if err != nil {
 				return nil, err
 			}
-			if firstCommitDate != nil {
-				codingTime := int64(pr.CreatedDate.Sub(*firstCommitDate).Seconds())
+			if firstCommit != nil {
+				codingTime := int64(pr.CreatedDate.Sub(firstCommit.AuthoredDate).Seconds())
 				if codingTime/60 == 0 && codingTime%60 > 0 {
 					codingTime = 1
 				} else {
 					codingTime = codingTime / 60
 				}
-				pr.OrigCodingTimespan = codingTime
+				projectPrMetric.CodingTimespan = processNegativeValue(codingTime)
+				projectPrMetric.FirstCommitSha = firstCommit.Sha
 			}
-			firstReviewTime, err := getFirstReviewTime(pr.Id, pr.AuthorId, db)
+			firstReview, err := getFirstReview(pr.Id, pr.AuthorId, db)
 			if err != nil {
 				return nil, err
 			}
-			if firstReviewTime != nil {
-				pr.OrigReviewLag = int64(firstReviewTime.Sub(pr.CreatedDate).Minutes())
-				pr.OrigReviewTimespan = int64(pr.MergedDate.Sub(*firstReviewTime).Minutes())
+			if firstReview != nil {
+				projectPrMetric.ReviewLag = processNegativeValue(int64(firstReview.CreatedDate.Sub(pr.CreatedDate).Minutes()))
+				projectPrMetric.ReviewTimespan = processNegativeValue(int64(pr.MergedDate.Sub(firstReview.CreatedDate).Minutes()))
+				projectPrMetric.FirstReviewId = firstReview.ReviewId
 			}
-			deployment, err := getDeployment(devops.PRODUCTION, *pr.MergedDate, db)
+			deployment, err := getDeployment(pr.MergeCommitSha, pr.BaseRepoId, deploymentPairList, db)
 			if err != nil {
 				return nil, err
 			}
-			if deployment != nil && deployment.FinishedDate != nil {
-				timespan := deployment.FinishedDate.Sub(*pr.MergedDate)
-				pr.OrigDeployTimespan = int64(timespan.Minutes())
+			if deployment != nil && deployment.TaskFinishedDate != nil {
+				timespan := deployment.TaskFinishedDate.Sub(*pr.MergedDate)
+				projectPrMetric.DeployTimespan = processNegativeValue(int64(timespan.Minutes()))
+				projectPrMetric.DeploymentId = deployment.TaskId
 			} else {
 				log.Debug("deploy time of pr %v is nil\n", pr.PullRequestKey)
 			}
-			processNegativeValue(pr)
-			pr.ChangeTimespan = nil
-			result := int64(0)
-			if pr.CodingTimespan != nil {
-				result += *pr.CodingTimespan
+			projectPrMetric.ChangeTimespan = nil
+			var result int64
+			if projectPrMetric.CodingTimespan != nil {
+				result += *projectPrMetric.CodingTimespan
 			}
-			if pr.ReviewLag != nil {
-				result += *pr.ReviewLag
+			if projectPrMetric.ReviewLag != nil {
+				result += *projectPrMetric.ReviewLag
 			}
-			if pr.ReviewTimespan != nil {
-				result += *pr.ReviewTimespan
+			if projectPrMetric.ReviewTimespan != nil {
+				result += *projectPrMetric.ReviewTimespan
 			}
-			if pr.DeployTimespan != nil {
-				result += *pr.DeployTimespan
+			if projectPrMetric.DeployTimespan != nil {
+				result += *projectPrMetric.DeployTimespan
 			}
 			if result > 0 {
-				pr.ChangeTimespan = &result
+				projectPrMetric.ChangeTimespan = &result
 			}
-			return []interface{}{pr}, nil
+			return []interface{}{projectPrMetric}, nil
 		},
 	})
 	if err != nil {
 		return err
 	}
 
-	return enricher.Execute()
+	return converter.Execute()
 }
 
-func getFirstCommitTime(prId string, db dal.Dal) (*time.Time, errors.Error) {
+func getFirstCommit(prId string, db dal.Dal) (*code.Commit, errors.Error) {
 	commit := &code.Commit{}
 	commitClauses := []dal.Clause{
 		dal.From(&code.Commit{}),
@@ -131,10 +171,10 @@ func getFirstCommitTime(prId string, db dal.Dal) (*time.Time, errors.Error) {
 	if err != nil {
 		return nil, err
 	}
-	return &commit.AuthoredDate, nil
+	return commit, nil
 }
 
-func getFirstReviewTime(prId string, prCreator string, db dal.Dal) (*time.Time, errors.Error) {
+func getFirstReview(prId string, prCreator string, db dal.Dal) (*code.PullRequestComment, errors.Error) {
 	review := &code.PullRequestComment{}
 	commentClauses := []dal.Clause{
 		dal.From(&code.PullRequestComment{}),
@@ -148,57 +188,40 @@ func getFirstReviewTime(prId string, prCreator string, db dal.Dal) (*time.Time,
 	if err != nil {
 		return nil, err
 	}
-	return &review.CreatedDate, nil
+	return review, nil
 }
 
-func getDeployment(environment string, mergeDate time.Time, db dal.Dal) (*devops.CICDTask, errors.Error) {
+func getDeployment(mergeSha string, repoId string, deploymentPairList []deploymentPair, db dal.Dal) (*deploymentPair, errors.Error) {
 	// ignore environment at this point because detecting it by name is obviously not engouh
 	// take https://github.com/apache/incubator-devlake/actions/workflows/build.yml for example
 	// one can not distingush testing/production by looking at the job name solely.
-	cicdTask := &devops.CICDTask{}
-	cicdTaskClauses := []dal.Clause{
-		dal.From(&devops.CICDTask{}),
-		dal.Where(`
-			type = ?
-			AND cicd_tasks.result = ?
-			AND cicd_tasks.started_date > ?`,
-			"DEPLOYMENT",
-			"SUCCESS",
-			mergeDate,
-		),
-		dal.Orderby("cicd_tasks.started_date ASC"),
-		dal.Limit(1),
-	}
-	err := db.First(cicdTask, cicdTaskClauses...)
-	if goerror.Is(err, gorm.ErrRecordNotFound) {
-		return nil, nil
-	}
-	if err != nil {
-		return nil, err
+	commitDiff := &code.CommitsDiff{}
+	// find if tuple[merge_sha, new_commit_sha, old_commit_sha] exist in commits_diffs, if yes, return pair.FinishedDate
+	for _, pair := range deploymentPairList {
+		if repoId != pair.RepoId {
+			continue
+		}
+		err := db.First(commitDiff, dal.Where(`commit_sha = ? and new_commit_sha = ? and old_commit_sha = ?`,
+			mergeSha, pair.NewDeployCommitSha, pair.OldDeployCommitSha))
+		if err == nil {
+			return &pair, nil
+		}
+		if goerror.Is(err, gorm.ErrRecordNotFound) {
+			continue
+		}
+		if err != nil {
+			return nil, err
+		}
+
 	}
-	return cicdTask, nil
+	return nil, nil
 }
 
-func processNegativeValue(pr *code.PullRequest) {
-	if pr.OrigCodingTimespan > 0 {
-		pr.CodingTimespan = &pr.OrigCodingTimespan
+func processNegativeValue(v int64) *int64 {
+	if v > 0 {
+		return &v
 	} else {
-		pr.CodingTimespan = nil
-	}
-	if pr.OrigReviewLag > 0 {
-		pr.ReviewLag = &pr.OrigReviewLag
-	} else {
-		pr.ReviewLag = nil
-	}
-	if pr.OrigReviewTimespan > 0 {
-		pr.ReviewTimespan = &pr.OrigReviewTimespan
-	} else {
-		pr.ReviewTimespan = nil
-	}
-	if pr.OrigDeployTimespan > 0 {
-		pr.DeployTimespan = &pr.OrigDeployTimespan
-	} else {
-		pr.DeployTimespan = nil
+		return nil
 	}
 }
 
@@ -209,3 +232,11 @@ var CalculateChangeLeadTimeMeta = core.SubTaskMeta{
 	Description:      "Calculate change lead time",
 	DomainTypes:      []string{core.DOMAIN_TYPE_CICD, core.DOMAIN_TYPE_CODE},
 }
+
+type deploymentPair struct {
+	TaskId             string
+	RepoId             string
+	NewDeployCommitSha string
+	OldDeployCommitSha string
+	TaskFinishedDate   *time.Time
+}
diff --git a/plugins/dora/tasks/task_data.go b/plugins/dora/tasks/task_data.go
index 9345eaabf..16ce528af 100644
--- a/plugins/dora/tasks/task_data.go
+++ b/plugins/dora/tasks/task_data.go
@@ -23,6 +23,7 @@ import (
 )
 
 type DoraApiParams struct {
+	ProjectName string
 }
 
 type TransformationRules struct {
@@ -36,6 +37,7 @@ type DoraOptions struct {
 	Since               string
 	RepoId              string `json:"repoId"`
 	Prefix              string `json:"prefix"`
+	ProjectName         string `json:"projectName"`
 	TransformationRules `mapstructure:"transformationRules" json:"transformationRules"`
 }