You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@devlake.apache.org by li...@apache.org on 2023/01/06 12:20:20 UTC

[incubator-devlake] branch main updated: Fix bitbucket (#4126)

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

likyh 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 4c82ea058 Fix bitbucket (#4126)
4c82ea058 is described below

commit 4c82ea05882c801b3e2ef2c140a90c2cddbf73d1
Author: Warren Chen <yi...@merico.dev>
AuthorDate: Fri Jan 6 20:20:16 2023 +0800

    Fix bitbucket (#4126)
    
    * feat: add bitbucket pipeline e2e test (#3147)
    
    * feat: add bitbucket pipeline e2e test
    
    * fix: remove create_date and complete_on in bibucket pipeline tool table test
    
    * fix: change time to iso86time in bitbucketApiPipeline
    
    * fix: ignore created_data and finished_data temporarily
    
    * fix: e2e
    
    Co-authored-by: Klesh Wong <zh...@merico.dev>
    
    * fix(bitbucket): update e2e (#4125)
    
    * feat: add bitbucket pullrequest_commits collector&extractor&convertor (#3337)
    
    * feat: add bitbucket pullrequest_commits collector&extractor&convertor
    
    * fix: bitbucket commit response do not have committer
    
    Co-authored-by: Klesh Wong <zh...@merico.dev>
    
    * fix(bitbucket): fix minor bugs
    
    Co-authored-by: tsoc <47...@users.noreply.github.com>
    Co-authored-by: Klesh Wong <zh...@merico.dev>
---
 plugins/bitbucket/e2e/pipelines_test.go            |  87 +++++++++++
 .../raw_tables/_raw_bitbucket_api_pipelines.csv    |  12 ++
 .../snapshot_tables/_tool_bitbucket_pipelines.csv  |  11 ++
 .../e2e/snapshot_tables/_tool_bitbucket_repos.csv  |   2 +-
 .../e2e/snapshot_tables/cicd_pipelines.csv         |  11 ++
 plugins/bitbucket/impl/impl.go                     |  15 +-
 ...egister.go => 20221008_add_prcommits_tables.go} |  24 +++-
 .../{register.go => archived/pr_commit.go}         |  21 +--
 .../bitbucket/models/migrationscripts/register.go  |   1 +
 .../{migrationscripts/register.go => pr_commit.go} |  21 +--
 plugins/bitbucket/tasks/pipeline_convertor.go      |   1 -
 plugins/bitbucket/tasks/pipeline_extractor.go      |  25 ++--
 plugins/bitbucket/tasks/pr_commit_collector.go     |  60 ++++++++
 plugins/bitbucket/tasks/pr_commit_convertor.go     |  86 +++++++++++
 plugins/bitbucket/tasks/pr_commit_extractor.go     | 159 +++++++++++++++++++++
 15 files changed, 492 insertions(+), 44 deletions(-)

diff --git a/plugins/bitbucket/e2e/pipelines_test.go b/plugins/bitbucket/e2e/pipelines_test.go
new file mode 100644
index 000000000..a1a00feea
--- /dev/null
+++ b/plugins/bitbucket/e2e/pipelines_test.go
@@ -0,0 +1,87 @@
+/*
+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 e2e
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-devlake/helpers/e2ehelper"
+	"github.com/apache/incubator-devlake/models/domainlayer/devops"
+	"github.com/apache/incubator-devlake/plugins/bitbucket/impl"
+	"github.com/apache/incubator-devlake/plugins/bitbucket/models"
+	"github.com/apache/incubator-devlake/plugins/bitbucket/tasks"
+)
+
+func TestBitbucketPipelineDataFlow(t *testing.T) {
+
+	var bitbucket impl.Bitbucket
+	dataflowTester := e2ehelper.NewDataFlowTester(t, "bitbucket", bitbucket)
+
+	bitbucketRepository := &models.BitbucketRepo{
+		BitbucketId: "thenicetgp/ptest",
+	}
+	taskData := &tasks.BitbucketTaskData{
+		Options: &tasks.BitbucketOptions{
+			ConnectionId: 1,
+			Owner:        "thenicetgp",
+			Repo:         "ptest",
+		},
+		Repo: bitbucketRepository,
+	}
+	// import raw data table
+	dataflowTester.ImportCsvIntoRawTable("./raw_tables/_raw_bitbucket_api_pipelines.csv", "_raw_bitbucket_api_pipelines")
+
+	// verify extraction
+	dataflowTester.FlushTabler(&models.BitbucketPipeline{})
+	dataflowTester.Subtask(tasks.ExtractApiPipelinesMeta, taskData)
+	dataflowTester.VerifyTable(
+		models.BitbucketPipeline{},
+		"./snapshot_tables/_tool_bitbucket_pipelines.csv",
+		[]string{
+			"connection_id",
+			"bitbucket_id",
+			"status",
+			"result",
+			"ref_name",
+			"web_url",
+			"duration_in_seconds",
+			"_raw_data_params",
+			"_raw_data_table",
+			"_raw_data_id",
+			"_raw_data_remark",
+		},
+	)
+
+	// verify conversion
+	dataflowTester.FlushTabler(&devops.CiCDPipelineCommit{})
+	dataflowTester.FlushTabler(&devops.CICDPipeline{})
+	dataflowTester.Subtask(tasks.ConvertPipelineMeta, taskData)
+	dataflowTester.VerifyTable(
+		devops.CICDPipeline{},
+		"./snapshot_tables/cicd_pipelines.csv",
+		[]string{
+			"id",
+			"name",
+			"result",
+			"status",
+			"type",
+			"duration_sec",
+			"environment",
+		},
+	)
+}
diff --git a/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_pipelines.csv b/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_pipelines.csv
new file mode 100644
index 000000000..97156b371
--- /dev/null
+++ b/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_pipelines.csv
@@ -0,0 +1,12 @@
+id,params,data,url,input,created_at
+41,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}","{""type"": ""pipeline"", ""uuid"": ""{a57ab3dc-2afd-4e23-acd3-7acf1bb0cf28}"", ""repository"": {""type"": ""repository"", ""full_name"": ""thenicetgp/ptest"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/thenicetgp/ptest""}, ""html"": {""href"": ""https://bitbucket.org/thenicetgp/ptest""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Bb304e403-4841-4ebe-9d4e-432025a507d2%7 [...]
+42,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}","{""type"": ""pipeline"", ""uuid"": ""{844365c2-2d8c-4b67-9e27-21c2fcda7bd7}"", ""repository"": {""type"": ""repository"", ""full_name"": ""thenicetgp/ptest"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/thenicetgp/ptest""}, ""html"": {""href"": ""https://bitbucket.org/thenicetgp/ptest""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Bb304e403-4841-4ebe-9d4e-432025a507d2%7 [...]
+43,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}","{""type"": ""pipeline"", ""uuid"": ""{0af285e5-c07d-48eb-b0e9-b579f63f6f54}"", ""repository"": {""type"": ""repository"", ""full_name"": ""thenicetgp/ptest"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/thenicetgp/ptest""}, ""html"": {""href"": ""https://bitbucket.org/thenicetgp/ptest""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Bb304e403-4841-4ebe-9d4e-432025a507d2%7 [...]
+44,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}","{""type"": ""pipeline"", ""uuid"": ""{fc8cfdbd-2e0f-4789-9abb-19bf326f704b}"", ""repository"": {""type"": ""repository"", ""full_name"": ""thenicetgp/ptest"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/thenicetgp/ptest""}, ""html"": {""href"": ""https://bitbucket.org/thenicetgp/ptest""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Bb304e403-4841-4ebe-9d4e-432025a507d2%7 [...]
+45,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}","{""type"": ""pipeline"", ""uuid"": ""{0b0986ff-87ab-4c61-8244-72ee93270992}"", ""repository"": {""type"": ""repository"", ""full_name"": ""thenicetgp/ptest"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/thenicetgp/ptest""}, ""html"": {""href"": ""https://bitbucket.org/thenicetgp/ptest""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Bb304e403-4841-4ebe-9d4e-432025a507d2%7 [...]
+46,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}","{""type"": ""pipeline"", ""uuid"": ""{105b3616-0140-4f17-993e-65d8836cbfd4}"", ""repository"": {""type"": ""repository"", ""full_name"": ""thenicetgp/ptest"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/thenicetgp/ptest""}, ""html"": {""href"": ""https://bitbucket.org/thenicetgp/ptest""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Bb304e403-4841-4ebe-9d4e-432025a507d2%7 [...]
+47,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}","{""type"": ""pipeline"", ""uuid"": ""{76e9c380-bedf-48f8-ad11-9b4a60307dd6}"", ""repository"": {""type"": ""repository"", ""full_name"": ""thenicetgp/ptest"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/thenicetgp/ptest""}, ""html"": {""href"": ""https://bitbucket.org/thenicetgp/ptest""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Bb304e403-4841-4ebe-9d4e-432025a507d2%7 [...]
+48,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}","{""type"": ""pipeline"", ""uuid"": ""{60bd9ab0-57d7-4da6-bf39-3b04e8133223}"", ""repository"": {""type"": ""repository"", ""full_name"": ""thenicetgp/ptest"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/thenicetgp/ptest""}, ""html"": {""href"": ""https://bitbucket.org/thenicetgp/ptest""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Bb304e403-4841-4ebe-9d4e-432025a507d2%7 [...]
+49,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}","{""type"": ""pipeline"", ""uuid"": ""{d676e969-7294-4ca2-9173-4fba9b419fe9}"", ""repository"": {""type"": ""repository"", ""full_name"": ""thenicetgp/ptest"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/thenicetgp/ptest""}, ""html"": {""href"": ""https://bitbucket.org/thenicetgp/ptest""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Bb304e403-4841-4ebe-9d4e-432025a507d2%7 [...]
+50,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}","{""type"": ""pipeline"", ""uuid"": ""{accb6177-eea1-4d13-9806-037645ca3f67}"", ""repository"": {""type"": ""repository"", ""full_name"": ""thenicetgp/ptest"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/thenicetgp/ptest""}, ""html"": {""href"": ""https://bitbucket.org/thenicetgp/ptest""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Bb304e403-4841-4ebe-9d4e-432025a507d2%7 [...]
+
diff --git a/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_pipelines.csv b/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_pipelines.csv
new file mode 100644
index 000000000..929dde8f9
--- /dev/null
+++ b/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_pipelines.csv
@@ -0,0 +1,11 @@
+connection_id,bitbucket_id,status,result,ref_name,web_url,duration_in_seconds,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
+1,{0af285e5-c07d-48eb-b0e9-b579f63f6f54},IN_PROGRESS,PAUSED,main,http://bitbucket-pipelines.prod.public.atl-paas.net//rest/1.0/accounts/{64135f6a-3978-4297-99b1-21827e0faf0b}/repositories/{b304e403-4841-4ebe-9d4e-432025a507d2}/pipelines/{0af285e5-c07d-48eb-b0e9-b579f63f6f54},10,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}",_raw_bitbucket_api_pipelines,43,
+1,{0b0986ff-87ab-4c61-8244-72ee93270992},IN_PROGRESS,PAUSED,main,http://bitbucket-pipelines.prod.public.atl-paas.net//rest/1.0/accounts/{64135f6a-3978-4297-99b1-21827e0faf0b}/repositories/{b304e403-4841-4ebe-9d4e-432025a507d2}/pipelines/{0b0986ff-87ab-4c61-8244-72ee93270992},10,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}",_raw_bitbucket_api_pipelines,45,
+1,{105b3616-0140-4f17-993e-65d8836cbfd4},IN_PROGRESS,PAUSED,pipeline,http://bitbucket-pipelines.prod.public.atl-paas.net//rest/1.0/accounts/{64135f6a-3978-4297-99b1-21827e0faf0b}/repositories/{b304e403-4841-4ebe-9d4e-432025a507d2}/pipelines/{105b3616-0140-4f17-993e-65d8836cbfd4},9,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}",_raw_bitbucket_api_pipelines,46,
+1,{60bd9ab0-57d7-4da6-bf39-3b04e8133223},COMPLETED,FAILED,feature/pipelinetest,http://bitbucket-pipelines.prod.public.atl-paas.net//rest/1.0/accounts/{64135f6a-3978-4297-99b1-21827e0faf0b}/repositories/{b304e403-4841-4ebe-9d4e-432025a507d2}/pipelines/{60bd9ab0-57d7-4da6-bf39-3b04e8133223},0,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}",_raw_bitbucket_api_pipelines,48,
+1,{76e9c380-bedf-48f8-ad11-9b4a60307dd6},COMPLETED,STOPPED,pipeline,http://bitbucket-pipelines.prod.public.atl-paas.net//rest/1.0/accounts/{64135f6a-3978-4297-99b1-21827e0faf0b}/repositories/{b304e403-4841-4ebe-9d4e-432025a507d2}/pipelines/{76e9c380-bedf-48f8-ad11-9b4a60307dd6},0,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}",_raw_bitbucket_api_pipelines,47,
+1,{844365c2-2d8c-4b67-9e27-21c2fcda7bd7},IN_PROGRESS,PAUSED,main,http://bitbucket-pipelines.prod.public.atl-paas.net//rest/1.0/accounts/{64135f6a-3978-4297-99b1-21827e0faf0b}/repositories/{b304e403-4841-4ebe-9d4e-432025a507d2}/pipelines/{844365c2-2d8c-4b67-9e27-21c2fcda7bd7},10,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}",_raw_bitbucket_api_pipelines,42,
+1,{a57ab3dc-2afd-4e23-acd3-7acf1bb0cf28},COMPLETED,SUCCESSFUL,main,http://bitbucket-pipelines.prod.public.atl-paas.net//rest/1.0/accounts/{64135f6a-3978-4297-99b1-21827e0faf0b}/repositories/{b304e403-4841-4ebe-9d4e-432025a507d2}/pipelines/{a57ab3dc-2afd-4e23-acd3-7acf1bb0cf28},14,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}",_raw_bitbucket_api_pipelines,41,
+1,{accb6177-eea1-4d13-9806-037645ca3f67},COMPLETED,FAILED,,http://bitbucket-pipelines.prod.public.atl-paas.net//rest/1.0/accounts/{64135f6a-3978-4297-99b1-21827e0faf0b}/repositories/{b304e403-4841-4ebe-9d4e-432025a507d2}/pipelines/{accb6177-eea1-4d13-9806-037645ca3f67},0,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}",_raw_bitbucket_api_pipelines,50,
+1,{d676e969-7294-4ca2-9173-4fba9b419fe9},COMPLETED,FAILED,pipeline,http://bitbucket-pipelines.prod.public.atl-paas.net//rest/1.0/accounts/{64135f6a-3978-4297-99b1-21827e0faf0b}/repositories/{b304e403-4841-4ebe-9d4e-432025a507d2}/pipelines/{d676e969-7294-4ca2-9173-4fba9b419fe9},0,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}",_raw_bitbucket_api_pipelines,49,
+1,{fc8cfdbd-2e0f-4789-9abb-19bf326f704b},IN_PROGRESS,PAUSED,feature/pipelinetest,http://bitbucket-pipelines.prod.public.atl-paas.net//rest/1.0/accounts/{64135f6a-3978-4297-99b1-21827e0faf0b}/repositories/{b304e403-4841-4ebe-9d4e-432025a507d2}/pipelines/{fc8cfdbd-2e0f-4789-9abb-19bf326f704b},12,"{""ConnectionId"":1,""Owner"":""thenicetgp"",""Repo"":""ptest""}",_raw_bitbucket_api_pipelines,44,
diff --git a/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_repos.csv b/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_repos.csv
index 554457e65..50d7b9bb9 100644
--- a/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_repos.csv
+++ b/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_repos.csv
@@ -1,2 +1,2 @@
 connection_id,bitbucket_id,name,html_url,description,owner_id,language,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-1,panjf2000/ants,panjf2000/ants,https://bitbucket.org/panjf2000/ants,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_repositories,2,
+1,panjf2000/ants,panjf2000/ants,https://bitbucket.org/panjf2000/ants,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_repositories,2,
\ No newline at end of file
diff --git a/plugins/bitbucket/e2e/snapshot_tables/cicd_pipelines.csv b/plugins/bitbucket/e2e/snapshot_tables/cicd_pipelines.csv
new file mode 100644
index 000000000..c7f646737
--- /dev/null
+++ b/plugins/bitbucket/e2e/snapshot_tables/cicd_pipelines.csv
@@ -0,0 +1,11 @@
+id,name,result,status,type,duration_sec,environment
+bitbucket:BitbucketPipeline:1:{0af285e5-c07d-48eb-b0e9-b579f63f6f54},bitbucket:BitbucketPipeline:1:main,SUCCESS,IN_PROGRESS,CI/CD,10,
+bitbucket:BitbucketPipeline:1:{0b0986ff-87ab-4c61-8244-72ee93270992},bitbucket:BitbucketPipeline:1:main,SUCCESS,IN_PROGRESS,CI/CD,10,
+bitbucket:BitbucketPipeline:1:{105b3616-0140-4f17-993e-65d8836cbfd4},bitbucket:BitbucketPipeline:1:pipeline,SUCCESS,IN_PROGRESS,CI/CD,9,
+bitbucket:BitbucketPipeline:1:{60bd9ab0-57d7-4da6-bf39-3b04e8133223},bitbucket:BitbucketPipeline:1:feature/pipelinetest,FAILURE,DONE,CI/CD,0,
+bitbucket:BitbucketPipeline:1:{76e9c380-bedf-48f8-ad11-9b4a60307dd6},bitbucket:BitbucketPipeline:1:pipeline,ABORT,DONE,CI/CD,0,
+bitbucket:BitbucketPipeline:1:{844365c2-2d8c-4b67-9e27-21c2fcda7bd7},bitbucket:BitbucketPipeline:1:main,SUCCESS,IN_PROGRESS,CI/CD,10,
+bitbucket:BitbucketPipeline:1:{a57ab3dc-2afd-4e23-acd3-7acf1bb0cf28},bitbucket:BitbucketPipeline:1:main,SUCCESS,DONE,CI/CD,14,
+bitbucket:BitbucketPipeline:1:{accb6177-eea1-4d13-9806-037645ca3f67},bitbucket:BitbucketPipeline:1:,FAILURE,DONE,CI/CD,0,
+bitbucket:BitbucketPipeline:1:{d676e969-7294-4ca2-9173-4fba9b419fe9},bitbucket:BitbucketPipeline:1:pipeline,FAILURE,DONE,CI/CD,0,
+bitbucket:BitbucketPipeline:1:{fc8cfdbd-2e0f-4789-9abb-19bf326f704b},bitbucket:BitbucketPipeline:1:feature/pipelinetest,SUCCESS,IN_PROGRESS,CI/CD,12,
diff --git a/plugins/bitbucket/impl/impl.go b/plugins/bitbucket/impl/impl.go
index 54da3c035..95c41518d 100644
--- a/plugins/bitbucket/impl/impl.go
+++ b/plugins/bitbucket/impl/impl.go
@@ -69,22 +69,33 @@ func (plugin Bitbucket) SubTaskMetas() []core.SubTaskMeta {
 	return []core.SubTaskMeta{
 		tasks.CollectApiRepoMeta,
 		tasks.ExtractApiRepoMeta,
+
 		tasks.CollectApiPullRequestsMeta,
 		tasks.ExtractApiPullRequestsMeta,
-		tasks.CollectApiIssuesMeta,
-		tasks.ExtractApiIssuesMeta,
+
 		tasks.CollectApiPrCommentsMeta,
 		tasks.ExtractApiPrCommentsMeta,
+
+		tasks.CollectApiPrCommitsMeta,
+		tasks.ExtractApiPrCommitsMeta,
+
+		tasks.CollectApiIssuesMeta,
+		tasks.ExtractApiIssuesMeta,
+
 		tasks.CollectApiIssueCommentsMeta,
 		tasks.ExtractApiIssueCommentsMeta,
+
 		tasks.CollectApiPipelinesMeta,
 		tasks.ExtractApiPipelinesMeta,
+
 		tasks.CollectApiDeploymentsMeta,
 		tasks.ExtractApiDeploymentsMeta,
+
 		tasks.ConvertRepoMeta,
 		tasks.ConvertAccountsMeta,
 		tasks.ConvertPullRequestsMeta,
 		tasks.ConvertPrCommentsMeta,
+		tasks.ConvertPrCommitsMeta,
 		tasks.ConvertIssuesMeta,
 		tasks.ConvertIssueCommentsMeta,
 		tasks.ConvertPipelineMeta,
diff --git a/plugins/bitbucket/models/migrationscripts/register.go b/plugins/bitbucket/models/migrationscripts/20221008_add_prcommits_tables.go
similarity index 60%
copy from plugins/bitbucket/models/migrationscripts/register.go
copy to plugins/bitbucket/models/migrationscripts/20221008_add_prcommits_tables.go
index d6d7fc8eb..ef5951693 100644
--- a/plugins/bitbucket/models/migrationscripts/register.go
+++ b/plugins/bitbucket/models/migrationscripts/20221008_add_prcommits_tables.go
@@ -18,15 +18,25 @@ limitations under the License.
 package migrationscripts
 
 import (
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/plugins/bitbucket/models/migrationscripts/archived"
 	"github.com/apache/incubator-devlake/plugins/core"
 )
 
-// All return all the migration scripts
-func All() []core.MigrationScript {
-	return []core.MigrationScript{
-		new(addInitTables20220803),
-		new(addPipeline20220914),
-		new(addDeployment20221013),
-		new(addRepoIdAndCommitShaField20221014),
+type addPrCommits20221008 struct{}
+
+func (*addPrCommits20221008) Up(basicRes core.BasicRes) errors.Error {
+	err := basicRes.GetDal().AutoMigrate(&archived.BitbucketPrCommit{})
+	if err != nil {
+		return errors.Convert(err)
 	}
+	return nil
+}
+
+func (*addPrCommits20221008) Version() uint64 {
+	return 20221008182354
+}
+
+func (*addPrCommits20221008) Name() string {
+	return "bitbucket add _tool_bitbucket_pull_requests_commits table"
 }
diff --git a/plugins/bitbucket/models/migrationscripts/register.go b/plugins/bitbucket/models/migrationscripts/archived/pr_commit.go
similarity index 66%
copy from plugins/bitbucket/models/migrationscripts/register.go
copy to plugins/bitbucket/models/migrationscripts/archived/pr_commit.go
index d6d7fc8eb..0721a9606 100644
--- a/plugins/bitbucket/models/migrationscripts/register.go
+++ b/plugins/bitbucket/models/migrationscripts/archived/pr_commit.go
@@ -15,18 +15,19 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package migrationscripts
+package archived
 
 import (
-	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/models/migrationscripts/archived"
 )
 
-// All return all the migration scripts
-func All() []core.MigrationScript {
-	return []core.MigrationScript{
-		new(addInitTables20220803),
-		new(addPipeline20220914),
-		new(addDeployment20221013),
-		new(addRepoIdAndCommitShaField20221014),
-	}
+type BitbucketPrCommit struct {
+	ConnectionId  uint64 `gorm:"primaryKey"`
+	CommitSha     string `gorm:"primaryKey;type:varchar(40)"`
+	PullRequestId int    `gorm:"primaryKey;autoIncrement:false"`
+	archived.NoPKModel
+}
+
+func (BitbucketPrCommit) TableName() string {
+	return "_tool_bitbucket_pull_request_commits"
 }
diff --git a/plugins/bitbucket/models/migrationscripts/register.go b/plugins/bitbucket/models/migrationscripts/register.go
index d6d7fc8eb..d5f5374b1 100644
--- a/plugins/bitbucket/models/migrationscripts/register.go
+++ b/plugins/bitbucket/models/migrationscripts/register.go
@@ -26,6 +26,7 @@ func All() []core.MigrationScript {
 	return []core.MigrationScript{
 		new(addInitTables20220803),
 		new(addPipeline20220914),
+		new(addPrCommits20221008),
 		new(addDeployment20221013),
 		new(addRepoIdAndCommitShaField20221014),
 	}
diff --git a/plugins/bitbucket/models/migrationscripts/register.go b/plugins/bitbucket/models/pr_commit.go
similarity index 67%
copy from plugins/bitbucket/models/migrationscripts/register.go
copy to plugins/bitbucket/models/pr_commit.go
index d6d7fc8eb..262093242 100644
--- a/plugins/bitbucket/models/migrationscripts/register.go
+++ b/plugins/bitbucket/models/pr_commit.go
@@ -15,18 +15,19 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package migrationscripts
+package models
 
 import (
-	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/models/common"
 )
 
-// All return all the migration scripts
-func All() []core.MigrationScript {
-	return []core.MigrationScript{
-		new(addInitTables20220803),
-		new(addPipeline20220914),
-		new(addDeployment20221013),
-		new(addRepoIdAndCommitShaField20221014),
-	}
+type BitbucketPrCommit struct {
+	ConnectionId  uint64 `gorm:"primaryKey"`
+	CommitSha     string `gorm:"primaryKey;type:varchar(40)"`
+	PullRequestId int    `gorm:"primaryKey;autoIncrement:false"`
+	common.NoPKModel
+}
+
+func (BitbucketPrCommit) TableName() string {
+	return "_tool_bitbucket_pull_request_commits"
 }
diff --git a/plugins/bitbucket/tasks/pipeline_convertor.go b/plugins/bitbucket/tasks/pipeline_convertor.go
index 7b97916c6..3b3c030f8 100644
--- a/plugins/bitbucket/tasks/pipeline_convertor.go
+++ b/plugins/bitbucket/tasks/pipeline_convertor.go
@@ -77,7 +77,6 @@ func ConvertPipelines(taskCtx core.SubTaskContext) errors.Error {
 					Generate(bitbucketPipeline.ConnectionId, bitbucketPipeline.RepoId),
 				CommitSha: bitbucketPipeline.CommitSha,
 				Branch:    bitbucketPipeline.RefName,
-				Repo:      bitbucketPipeline.WebUrl,
 			}
 			domainPipeline := &devops.CICDPipeline{
 				DomainEntity: domainlayer.DomainEntity{
diff --git a/plugins/bitbucket/tasks/pipeline_extractor.go b/plugins/bitbucket/tasks/pipeline_extractor.go
index 8cb12f6bd..a08c6dc7f 100644
--- a/plugins/bitbucket/tasks/pipeline_extractor.go
+++ b/plugins/bitbucket/tasks/pipeline_extractor.go
@@ -23,7 +23,6 @@ import (
 	"github.com/apache/incubator-devlake/plugins/bitbucket/models"
 	"github.com/apache/incubator-devlake/plugins/core"
 	"github.com/apache/incubator-devlake/plugins/helper"
-	"time"
 )
 
 type bitbucketApiCommit struct {
@@ -72,14 +71,14 @@ type BitbucketApiPipeline struct {
 		Name string `json:"name"`
 		Type string `json:"type"`
 	} `json:"trigger"`
-	CreatedOn         *time.Time `json:"created_on"`
-	CompletedOn       *time.Time `json:"completed_on"`
-	RunNumber         int        `json:"run_number"`
-	DurationInSeconds uint64     `json:"duration_in_seconds"`
-	BuildSecondsUsed  int        `json:"build_seconds_used"`
-	FirstSuccessful   bool       `json:"first_successful"`
-	Expired           bool       `json:"expired"`
-	HasVariables      bool       `json:"has_variables"`
+	CreatedOn         *helper.Iso8601Time `json:"created_on"`
+	CompletedOn       *helper.Iso8601Time `json:"completed_on"`
+	RunNumber         int                 `json:"run_number"`
+	DurationInSeconds uint64              `json:"duration_in_seconds"`
+	BuildSecondsUsed  int                 `json:"build_seconds_used"`
+	FirstSuccessful   bool                `json:"first_successful"`
+	Expired           bool                `json:"expired"`
+	HasVariables      bool                `json:"has_variables"`
 	Links             struct {
 		Self struct {
 			Href string `json:"href"`
@@ -114,14 +113,14 @@ func ExtractApiPipelines(taskCtx core.SubTaskContext) errors.Error {
 			bitbucketPipeline := &models.BitbucketPipeline{
 				ConnectionId:        data.Options.ConnectionId,
 				BitbucketId:         bitbucketApiPipeline.Uuid,
-				WebUrl:              bitbucketApiPipeline.Target.Commit.Links.Html.Href,
+				WebUrl:              bitbucketApiPipeline.Links.Self.Href,
 				Status:              bitbucketApiPipeline.State.Name,
 				RefName:             bitbucketApiPipeline.Target.RefName,
 				CommitSha:           bitbucketApiPipeline.Target.Commit.Hash,
-				RepoId:              bitbucketApiPipeline.Repo.FullName,
+				RepoId:              bitbucketApiPipeline.Repo.BitbucketId,
 				DurationInSeconds:   bitbucketApiPipeline.DurationInSeconds,
-				BitbucketCreatedOn:  bitbucketApiPipeline.CreatedOn,
-				BitbucketCompleteOn: bitbucketApiPipeline.CompletedOn,
+				BitbucketCreatedOn:  helper.Iso8601TimeToTime(bitbucketApiPipeline.CreatedOn),
+				BitbucketCompleteOn: helper.Iso8601TimeToTime(bitbucketApiPipeline.CompletedOn),
 			}
 			if err != nil {
 				return nil, err
diff --git a/plugins/bitbucket/tasks/pr_commit_collector.go b/plugins/bitbucket/tasks/pr_commit_collector.go
new file mode 100644
index 000000000..5adbb91a0
--- /dev/null
+++ b/plugins/bitbucket/tasks/pr_commit_collector.go
@@ -0,0 +1,60 @@
+/*
+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 (
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+const RAW_PULL_REQUEST_COMMITS_TABLE = "bitbucket_api_pull_request_commits"
+
+var CollectApiPrCommitsMeta = core.SubTaskMeta{
+	Name:             "collectApiPullRequestCommits",
+	EntryPoint:       CollectApiPullRequestCommits,
+	EnabledByDefault: true,
+	Description:      "Collect PullRequestCommits data from Bitbucket api",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CODE_REVIEW},
+}
+
+func CollectApiPullRequestCommits(taskCtx core.SubTaskContext) errors.Error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_PULL_REQUEST_COMMITS_TABLE)
+
+	iterator, err := GetPullRequestsIterator(taskCtx)
+	if err != nil {
+		return err
+	}
+	defer iterator.Close()
+
+	collector, err := helper.NewApiCollector(helper.ApiCollectorArgs{
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		ApiClient:          data.ApiClient,
+		PageSize:           100,
+		Incremental:        false,
+		Input:              iterator,
+		UrlTemplate:        "repositories/{{ .Params.Owner }}/{{ .Params.Repo }}/pullrequests/{{ .Input.BitbucketId }}/commits",
+		Query:              GetQuery,
+		ResponseParser:     GetRawMessageFromResponse,
+	})
+
+	if err != nil {
+		return err
+	}
+	return collector.Execute()
+}
diff --git a/plugins/bitbucket/tasks/pr_commit_convertor.go b/plugins/bitbucket/tasks/pr_commit_convertor.go
new file mode 100644
index 000000000..bd428441c
--- /dev/null
+++ b/plugins/bitbucket/tasks/pr_commit_convertor.go
@@ -0,0 +1,86 @@
+/*
+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/models/domainlayer/didgen"
+	bitbucketModels "github.com/apache/incubator-devlake/plugins/bitbucket/models"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+var ConvertPrCommitsMeta = core.SubTaskMeta{
+	Name:             "convertPullRequestCommits",
+	EntryPoint:       ConvertPullRequestCommits,
+	EnabledByDefault: true,
+	Description:      "Convert tool layer table bitbucket_pull_request_commits into  domain layer table pull_request_commits",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CODE_REVIEW},
+}
+
+func ConvertPullRequestCommits(taskCtx core.SubTaskContext) (err errors.Error) {
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*BitbucketTaskData)
+	repoId := data.Repo.BitbucketId
+
+	pullIdGen := didgen.NewDomainIdGenerator(&bitbucketModels.BitbucketPullRequest{})
+
+	cursor, err := db.Cursor(
+		dal.From(&bitbucketModels.BitbucketPrCommit{}),
+		dal.Join(`left join _tool_bitbucket_pull_requests on _tool_bitbucket_pull_requests.bitbucket_id = _tool_bitbucket_pull_request_commits.pull_request_id`),
+		dal.Where("_tool_bitbucket_pull_requests.repo_id = ? and _tool_bitbucket_pull_requests.connection_id = ?", repoId, data.Options.ConnectionId),
+		dal.Orderby("pull_request_id ASC"),
+	)
+	if err != nil {
+		return err
+	}
+	defer cursor.Close()
+
+	converter, err := helper.NewDataConverter(helper.DataConverterArgs{
+		InputRowType: reflect.TypeOf(bitbucketModels.BitbucketPrCommit{}),
+		Input:        cursor,
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: BitbucketApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_PULL_REQUEST_COMMITS_TABLE,
+		},
+		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
+			bitbucketPullRequestCommit := inputRow.(*bitbucketModels.BitbucketPrCommit)
+			domainPrCommit := &code.PullRequestCommit{
+				CommitSha:     bitbucketPullRequestCommit.CommitSha,
+				PullRequestId: pullIdGen.Generate(data.Options.ConnectionId, bitbucketPullRequestCommit.PullRequestId),
+			}
+			return []interface{}{
+				domainPrCommit,
+			}, nil
+		},
+	})
+	if err != nil {
+		return err
+	}
+
+	return converter.Execute()
+}
diff --git a/plugins/bitbucket/tasks/pr_commit_extractor.go b/plugins/bitbucket/tasks/pr_commit_extractor.go
new file mode 100644
index 000000000..0191522b7
--- /dev/null
+++ b/plugins/bitbucket/tasks/pr_commit_extractor.go
@@ -0,0 +1,159 @@
+/*
+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 (
+	"encoding/json"
+	"strings"
+	"time"
+
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/plugins/bitbucket/models"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+var ExtractApiPrCommitsMeta = core.SubTaskMeta{
+	Name:             "extractApiPullRequestCommits",
+	EntryPoint:       ExtractApiPullRequestCommits,
+	EnabledByDefault: true,
+	Description:      "Extract raw PullRequestCommits data into tool layer table bitbucket_commits",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CODE_REVIEW},
+}
+
+type ApiPrCommitsResponse struct {
+	Type   string    `json:"type"`
+	Hash   string    `json:"hash"`
+	Date   time.Time `json:"date"`
+	Author struct {
+		Type string                   `json:"type"`
+		Raw  string                   `json:"raw"`
+		User BitbucketAccountResponse `json:"user"`
+	} `json:"author"`
+	Message string `json:"message"`
+	Summary struct {
+		Type   string `json:"type"`
+		Raw    string `json:"raw"`
+		Markup string `json:"markup"`
+		HTML   string `json:"html"`
+	} `json:"summary"`
+	Links struct {
+		Self struct {
+			Href string `json:"href"`
+		} `json:"self"`
+		HTML struct {
+			Href string `json:"href"`
+		} `json:"html"`
+	} `json:"links"`
+	Parents []struct {
+		Type  string `json:"type"`
+		Hash  string `json:"hash"`
+		Links struct {
+			Self struct {
+				Href string `json:"href"`
+			} `json:"self"`
+			HTML struct {
+				Href string `json:"href"`
+			} `json:"html"`
+		} `json:"links"`
+	} `json:"parents"`
+	Repository BitbucketApiRepo `json:"repository"`
+}
+
+func ExtractApiPullRequestCommits(taskCtx core.SubTaskContext) errors.Error {
+	data := taskCtx.GetData().(*BitbucketTaskData)
+	repoId := data.Repo.BitbucketId
+	extractor, err := helper.NewApiExtractor(helper.ApiExtractorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			/*
+				This struct will be JSONEncoded and stored into database along with raw data itself, to identity minimal
+				set of data to be process, for example, we process JiraIssues by Board
+			*/
+			Params: BitbucketApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			/*
+				Table store raw data
+			*/
+			Table: RAW_PULL_REQUEST_COMMITS_TABLE,
+		},
+		Extract: func(row *helper.RawData) ([]interface{}, errors.Error) {
+			apiPullRequestCommit := &ApiPrCommitsResponse{}
+			if strings.HasPrefix(string(row.Data), "Not Found") {
+				return nil, nil
+			}
+			err := errors.Convert(json.Unmarshal(row.Data, apiPullRequestCommit))
+			if err != nil {
+				return nil, err
+			}
+			pull := &BitbucketInput{}
+			err = errors.Convert(json.Unmarshal(row.Input, pull))
+			if err != nil {
+				return nil, err
+			}
+			// need to extract 2 kinds of entities here
+			results := make([]interface{}, 0, 3)
+			bitbucketRepoCommit := &models.BitbucketRepoCommit{
+				ConnectionId: data.Options.ConnectionId,
+				RepoId:       repoId,
+				CommitSha:    apiPullRequestCommit.Hash,
+			}
+			results = append(results, bitbucketRepoCommit)
+
+			bitbucketCommit, err := convertPullRequestCommit(apiPullRequestCommit, data.Options.ConnectionId)
+			if err != nil {
+				return nil, err
+			}
+			results = append(results, bitbucketCommit)
+
+			bitbucketPullRequestCommit := &models.BitbucketPrCommit{
+				ConnectionId:  data.Options.ConnectionId,
+				CommitSha:     apiPullRequestCommit.Hash,
+				PullRequestId: pull.BitbucketId,
+			}
+			if err != nil {
+				return nil, err
+			}
+			results = append(results, bitbucketPullRequestCommit)
+			return results, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return extractor.Execute()
+}
+
+func convertPullRequestCommit(prCommit *ApiPrCommitsResponse, connId uint64) (*models.BitbucketCommit, errors.Error) {
+	bitbucketCommit := &models.BitbucketCommit{
+		Sha:           prCommit.Hash,
+		Message:       prCommit.Message,
+		AuthorId:      prCommit.Author.User.AccountId,
+		AuthorName:    prCommit.Author.User.UserName,
+		AuthorEmail:   prCommit.Author.Raw,
+		AuthoredDate:  prCommit.Date,
+		CommittedDate: prCommit.Date,
+		Url:           prCommit.Links.Self.Href,
+	}
+	return bitbucketCommit, nil
+}